Re: [DISCUSS] Apache Kafka 3.8.0 release

2024-06-12 Thread José Armando García Sancio
Hi Josep,

See my comment below.

On Wed, Jun 12, 2024 at 1:17 PM Josep Prat  wrote:
> How long do you think it will take to bring KIP-853 to completion?

We are still missing a few issues/jiras that need to get implemented
for the feature to be usable. I would say a few more weeks. May be
early July or mid July.

Thanks,
-- 
-José


Re: [DISCUSS] Apache Kafka 3.8.0 release

2024-06-01 Thread José Armando García Sancio
Hi Josep,

See my comments below.

On Wed, May 29, 2024 at 10:52 AM Josep Prat  wrote:
> So I would propose to leave the deadlines as they are and manually cherry
> pick the commits related to KIP-853 and KIP-966.

Your proposal sounds good to me. I suspect that will be doing feature
development for KIP-853 past the feature freeze and code freeze date.
Maybe feature development will be finished around the end of June.

I'll make sure to cherry pick commits for KIP-853 to the 3.8 branch
once we have one.

Thanks,
-- 
-José


Re: [VOTE] KIP-909: DNS Resolution Fallures Should Not Fail the Client

2024-04-29 Thread José Armando García Sancio
Thanks for the KIP Philip.

+1 binding.

On Mon, Apr 29, 2024 at 9:55 AM Philip Nee  wrote:
>
> Thanks all.  Will find time to get the patch out.
>
> On Mon, Apr 29, 2024 at 7:42 AM Federico Valeri 
> wrote:
>
> > +1 (non binding)
> > Thanks
> >
> > On Mon, Apr 29, 2024 at 10:18 AM Rajini Sivaram 
> > wrote:
> > >
> > > Hi Philip,
> > >
> > > +1 (binding). Thanks for the KIP!
> > >
> > > Regards,
> > >
> > > Rajini
> > >
> > >
> > > On Tue, Apr 25, 2023 at 8:52 PM Philip Nee  wrote:
> > >
> > > > Thanks for the vote.  We've decided to make a minor change to the
> > default
> > > > timeout from 5min to 2min.
> > > >
> > > > On Tue, Apr 25, 2023 at 11:42 AM David Jacot 
> > > > wrote:
> > > >
> > > > > +1 (binding) Thanks for the KIP, Philip!
> > > > >
> > > > > Le mar. 25 avr. 2023 à 20:23, José Armando García Sancio
> > > > >  a écrit :
> > > > >
> > > > > > +1. Thanks for the design. Looking forward to the implementation.
> > > > > >
> > > > > > On Tue, Apr 25, 2023 at 10:49 AM Jason Gustafson
> > > > > >  wrote:
> > > > > > >
> > > > > > > +1 Thanks Philip!
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Apr 13, 2023 at 7:49 AM Kirk True 
> > wrote:
> > > > > > >
> > > > > > > > +1 (non-binding)
> > > > > > > >
> > > > > > > > > On Apr 10, 2023, at 1:53 PM, Philip Nee  > >
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > Hey everyone!
> > > > > > > > >
> > > > > > > > > I'm starting a vote for KIP-909: DNS Resolution Fallures
> > Should
> > > > Not
> > > > > > Fail
> > > > > > > > > the Client <http://DNS Resolution Failure Should Not Fail
> > the
> > > > > > Clients>
> > > > > > > > >
> > > > > > > > > Please refer to the discussion thread here:
> > > > > > > > >
> > https://lists.apache.org/thread/st84zzwnq5m3pkzd1r7jk9lmqdt9m98s
> > > > > > > > >
> > > > > > > > > Thanks!
> > > > > > > > > P
> > > > > > > >
> > > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -José
> > > > > >
> > > > >
> > > >
> >



-- 
-José


Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-23 Thread José Armando García Sancio
Hi all,

I am closing the voting. The KIP passed with:
Jun Rao +1 binding
Jason Gustafson +1 binding
José Armando García Sancio +1 binding

Thank you all,

On Mon, Apr 22, 2024 at 11:57 AM José Armando García Sancio
 wrote:
>
> I am going to close the vote tomorrow morning (PST).
>
> On Mon, Apr 22, 2024 at 10:06 AM José Armando García Sancio
>  wrote:
> >
> > +1 binding.
> >
> > On Mon, Apr 22, 2024 at 9:28 AM Jason Gustafson
> >  wrote:
> > >
> > > Thanks Jose. +1. Great KIP!
> > >
> > > On Fri, Mar 29, 2024 at 11:16 AM Jun Rao  
> > > wrote:
> > >
> > > > Hi, Jose,
> > > >
> > > > Thanks for the KIP. +1
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio
> > > >  wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I would like to call a vote to adopt KIP-853.
> > > > >
> > > > > KIP: https://cwiki.apache.org/confluence/x/nyH1D
> > > > > Discussion thread:
> > > > > https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46
> > > > >
> > > > > Thanks,
> > > > > --
> > > > > -José
> > > > >
> > > >
> >
> >
> >
> > --
> > -José
>
>
>
> --
> -José



-- 
-José


Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread José Armando García Sancio
I am going to close the vote tomorrow morning (PST).

On Mon, Apr 22, 2024 at 10:06 AM José Armando García Sancio
 wrote:
>
> +1 binding.
>
> On Mon, Apr 22, 2024 at 9:28 AM Jason Gustafson
>  wrote:
> >
> > Thanks Jose. +1. Great KIP!
> >
> > On Fri, Mar 29, 2024 at 11:16 AM Jun Rao  wrote:
> >
> > > Hi, Jose,
> > >
> > > Thanks for the KIP. +1
> > >
> > > Jun
> > >
> > > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio
> > >  wrote:
> > >
> > > > Hi all,
> > > >
> > > > I would like to call a vote to adopt KIP-853.
> > > >
> > > > KIP: https://cwiki.apache.org/confluence/x/nyH1D
> > > > Discussion thread:
> > > > https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46
> > > >
> > > > Thanks,
> > > > --
> > > > -José
> > > >
> > >
>
>
>
> --
> -José



-- 
-José


Re: [VOTE] KIP-853: KRaft Controller Membership Changes

2024-04-22 Thread José Armando García Sancio
+1 binding.

On Mon, Apr 22, 2024 at 9:28 AM Jason Gustafson
 wrote:
>
> Thanks Jose. +1. Great KIP!
>
> On Fri, Mar 29, 2024 at 11:16 AM Jun Rao  wrote:
>
> > Hi, Jose,
> >
> > Thanks for the KIP. +1
> >
> > Jun
> >
> > On Fri, Mar 29, 2024 at 9:55 AM José Armando García Sancio
> >  wrote:
> >
> > > Hi all,
> > >
> > > I would like to call a vote to adopt KIP-853.
> > >
> > > KIP: https://cwiki.apache.org/confluence/x/nyH1D
> > > Discussion thread:
> > > https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46
> > >
> > > Thanks,
> > > --
> > > -José
> > >
> >



-- 
-José


Re: [VOTE] KIP-1022 Formatting and Updating Features

2024-04-10 Thread José Armando García Sancio
Hi Justine,

+1 (binding)

Thanks for the improvement.
-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-04-10 Thread José Armando García Sancio
Hi Justine,

On Tue, Apr 9, 2024 at 4:19 PM Justine Olshan
 wrote:
> As for the validation criteria. It seems like one bit of code that
> validates whether a version is allowed is in the method
> `reasonNotSupported` which checks the range of features available for the
> given feature.
> For metadata.version we have a method to do "additional checks" and we
> could have those for the various other features as well. I have an
> (internal) FeatureVersion interface in mind that would work well for this.
> For any of these validations, we return the same error
> `INVALID_UPDATE_VERSION`. I would think continuing to use this error
> follows naturally, but if we think it is necessary to specify the error
> code, I can do so in my KIP.

Thanks for looking into this. The updates to the KIP look good to me.

-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-04-09 Thread José Armando García Sancio
Hi Justine,

Thanks for the KIP. I see that the KIP doesn't make any updates to the
UpdateFeatures RPC. I was trying to understand how errors will be
communicated to the client.

Are you planning to use the INVALID_UPDATE_VERSION error and overwrite
the ErrorMessage field for all of the validations you mentioned in the
KIP? I see that INVALID_UPDATE_VERSION is in the code for Apache Kafka
but I couldn't find the KIP that adds this error. It is not in KIP-584
or KIP-778. If you agree, do you think we should document this error
in this KIP?

It is also not clear to me when the UpdateFeaturesResponse will return
an error per feature versus an error for the entire RPC. KIP-584
defines this relationship but it doesn't specify when exactly a top
level error will be returned versus when a feature level error will be
returned. I think that most users wouldn't want partial failures. They
instead would like to be guaranteed that all of the feature updates
succeeded or none did. Do you agree? Should we update the KIP to make
this clear?

Thanks!
-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-04-02 Thread José Armando García Sancio
Hi Justine,

See my comments below.

On Mon, Apr 1, 2024 at 4:43 PM Justine Olshan
 wrote:
> 20. I can update the KIP.

I took a look at the latest KIP.

* Should the output of this command "bin/kafka-features.sh
version-mapping --release-version 3.6-IV1" be something like this:
metadata.version=13
transaction.protocol.version=0
group.coordinator.version=0
kraft.version=0

I added the kraft.version to show the changes that are coming from
KIP-853. I find this formatting much easier to parse by scripts/tools.
They can even use Java's Properties parser if they want.

* Maybe I missed the discussion but can you talk about why both
"kafka-storage" and "kafka-features" are going to implement the
"version-mapping" and dependencies"? I assume that it is sufficient
for only one (kafka-features) to implement these new commands.

* For the command "dependencies" in the "kafka-features" command, it
is probably obvious that the dependencies listed are feature version
dependencies. This is not obvious when the user uses "kafka-storage
dependencies".  This reads as the dependencies of kafka-storage.

* Should we state that Kafka will not allow cycles in the feature
version dependency definition? Meaning the user is not allowed to
define a feature version X which depends on Y which depends on Z which
depends on X.

* Some of the example output start with the characters "> ". Will the
CLI print those characters or is that just part of the KIP's
formating?

Thanks,
-- 
-José


[VOTE] KIP-853: KRaft Controller Membership Changes

2024-03-29 Thread José Armando García Sancio
Hi all,

I would like to call a vote to adopt KIP-853.

KIP: https://cwiki.apache.org/confluence/x/nyH1D
Discussion thread:
https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-28 Thread José Armando García Sancio
Jun, thanks a lot for your help. I feel that the KIP is much better
after your detailed input.

If there is no more feedback, I'll start a voting thread tomorrow
morning. I'll monitor KIP-1022's discussion thread and update this KIP
with anything that affects the KIP's specification.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-28 Thread José Armando García Sancio
Hi Jun,

See my comments below.

On Thu, Mar 28, 2024 at 11:09 AM Jun Rao  wrote:
> If I am adding a new voter and it takes a long time (because the new voter
> is catching up), I'd want to know if the request is indeed being processed.
> I thought that's the usage of uncommitted-voter-change.

They can get related information by using the 'kafka-metadata describe
--replication" command (or the log-end-offset metric from KIP-595).
That command (and metric) displays the LEO of all of the replicas
(voters and observers), according to the leader. They can use that
output to discover if the observer they are trying to add is lagging
or is not replicating at all.

When the user runs the command above, they don't know the exact offset
that the new controller needs to reach but they can do some rough
estimation of how far behind it is. What do you think? Is this good
enough?

> Also, I am still not sure about having multiple brokers reporting the same
> metric. For example, if they don't report the same value (e.g. because one
> broker is catching up), how does a user know which value is correct?

They are all correct according to the local view. Here are two
examples of monitors that the user can write:

1. Is there a voter that I need to remove from the quorum? They can
create a monitor that fires, if the number-of-offline-voters metric
has been greater than 0 for the past hour.
2. Is there a cluster that doesn't have 3 voters? They can create a
monitor that fires, if any replica doesn't report three for
number-of-voters for the past hour.

Is there a specific metric that you have in mind that should only be
reported by the KRaft leader?

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-28 Thread José Armando García Sancio
Hi Jun,

On Thu, Mar 28, 2024 at 10:35 AM Jun Rao  wrote:
> The following are the steps of AddVoter. The bulk of the time is probably
> in step 5, but the updated VotersRecord won't be written until step 6. So,
> ideally, the controller leader should report the pending voter as soon as
> step 1. The brokers and non-leader controllers can't do that until after
> step 6. Having multiple brokers report the same metric can be confusing
> when there is inconsistency.

First, the replicas (leader, following voters and observers) will
compute the metrics the same way. In other words, in the leader the
uncommitted-voter-change metric will be true (1) from step 7 to after
step 8.

I added the metric to indicate to the operator if there are replicas
that have updated their voters set to a value that is uncommitted
value. The leader doesn't update its voters set until step 7.

I don't think that we should add metrics to track the state of a
specific RPC. Or if we do, it should be a seperate KIP where we have a
mechanism for consistently tracking this state across all admin RPCs.
What do you think?

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-27 Thread José Armando García Sancio
Hi Jun,

On Wed, Mar 27, 2024 at 2:26 PM Jun Rao  wrote:
> 55.1 How does the broker and non-leader controller know the pending voters?

They are in the log. Pending voter sets are VotersRecords between the
HWM and the LEO. The leader will make sure that there is at most one
VoterRecord that is uncommitted (between the HWM and LEO).

Maybe uncommitted-voter-change is a better name. Updated the KIP to
use this name.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-03-27 Thread José Armando García Sancio
Hi Justine,

See my comment below.

On Wed, Mar 27, 2024 at 1:31 PM Justine Olshan
 wrote:
> The feature command includes the upgrade or downgrade command along with
> the --release-version flag. If some features are not moving in the
> direction mentioned (upgrade or downgrade) the command will fail -- perhaps
> with an error of which features were going in the wrong direction.

How about updating the KIP to show and document this behavior?

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-27 Thread José Armando García Sancio
Hi Jun,

Thanks for the feedback. See my comments below.

On Mon, Mar 25, 2024 at 2:21 PM Jun Rao  wrote:
> 54. Yes, we could include SecurityProtocol in DescribeQuorumResponse. Then,
> we could include it in the output of kafka-metadata-quorum --describe.

Yes, I updated the DescribeQuorumResponse to include the
SecurityProtocol and I also updated the example output for
"kafka-metadata-quorum describe --status".

> 55.1 Could number-of-observers and pending-voter-change be reported by all
> brokers and controllers? I thought only the controller leader tracks those.

These metrics are reported by all of the KRaft replicas (broker and
controller). I think this makes it easier to monitor since metrics
collectors can collect the same metrics from all of the nodes
irrespective of their role (broker or controller). The main exception
that Kafka has right now is type=KafkaController vs
type=broker-metadata-metrics but I would favor a KIP that unified
these two sets of metrics to something like type=metadata-metrics.

> 55.2 So, IgnoredStaticVoters and IsObserver are Yammer metrics and the rest
> are KafkaMetric. It would be useful to document the metric names clearer.
> For Yammer metrics, we need to specify group, type, name and tags. For
> KafkaMetrics, we need to specify just name and tags.

Yeah. I always struggle with the MBean specification. I connected
jconsole to Kafka and updated the KIP to be more accurate. Please take
a look.

> 57. Could we remove --release-version 3.8 in the upgrade example?

Done. I also removed wording about deprecating --metadata from
kafka-features.sh. I'll let KIP-1022 and the discussion there make
that decision.

Thanks,
--
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-03-26 Thread José Armando García Sancio
Hi Justine,

On Mon, Mar 25, 2024 at 5:09 PM Justine Olshan
 wrote:
> The reason it is not removed is purely for backwards
> compatibility. Colin had strong feelings about not removing any flags.

We are not saying that we should remove that flag. That would break
backward compatibility of 3.8 with 3.7. We are suggesting to deprecate
the flag in the next release.

Thanks,
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-03-25 Thread José Armando García Sancio
Hi Justine,

Thanks for the update. See my comments below.

On Mon, Mar 25, 2024 at 2:51 PM Justine Olshan
 wrote:
> I've updated the KIP with the changes I mentioned earlier. I have not yet
> removed the --feature-version flag from the upgrade tool.

What's the "--feature-version" flag? This is the first time I see it
mentioned and I don't see it in the KIP. Did you mean
"--release-version"?

> Please take a look at the API to get the versions for a given
> metadata version. Right now I'm using ApiVersions request and specifying a
> metadata version. The supported versions are then supplied in the
> ApiVersions response.
> There were tradeoffs with this approach. It is a pretty minimal change, but
> reusing the API means that it could be confusing (ie, the ApiKeys will be
> supplied in the response but not needed.) I considered making a whole new
> API, but that didn't seem necessary for this use.

I agree that this is extremely confusing and we shouldn't overload the
ApiVersions RPC to return this information. The KIP doesn't mention
how it is going to use this API. Do you need to update the Admin
client to include this information?

Having said this, as you mentioned in the KIP the kafka-storage tool
needs this information and that tool cannot assume that there is a
running server it can query (send an RPC). Can the kafka-features use
the same mechanism used by kafka-storage without calling into a
broker?

re: "It will work just like the storage tool and upgrade all the
features to a version"

Does this mean that --release-version cannot be used with
"kafka-features downgrade"?

re: Consistency with KIP-853

Jun and I have been having a similar conversation in the discussion
thread for KIP-853. From what I can tell both changes are compatible.
Do you mind taking a look at these two sections and confirming that
they don't contradict your KIP?
1. 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-853%3A+KRaft+Controller+Membership+Changes#KIP853:KRaftControllerMembershipChanges-kafka-storage
2. 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-853%3A+KRaft+Controller+Membership+Changes#KIP853:KRaftControllerMembershipChanges-kafka-features

re: nit "For MVs that existed before these features, we map the new
features to version 0 (no feature enabled)."

To me version 0 doesn't necessarily mean that the feature is not
enabled. For example, for kraft.version, version 0 means the protocol
prior to KIP-853. Version 0 is the currently implemented version of
the KRaft protocol.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-25 Thread José Armando García Sancio
Hi Jun,

See my comments below.

On Fri, Mar 22, 2024 at 1:30 PM Jun Rao  wrote:
> 54. Admin.addMetadataVoter: It seems that Endpoint shouldn't include
> securityProtocol since it's not in DescribeQuorumResponse.

Yeah. I noticed that when I made the Admin changes. We either use a
different type in the Admin client or add SecurityProtocol to the
DescribeQuorumResponse. I was originally leaning towards adding
SecurityProtocol to the DescribeQuorumResponse.

Does the user want to see the security protocol in the response and
CLI output? Listener name is not very useful unless the user also has
access to the controller's configuration.

I can go either way, what do you think?

> 55. Metrics:
> 55.1 It would be useful to be clear whether they are reported by the
> controller leader, all controllers or all controllers and brokers.

Done. I also noticed that I was missing one metric in the controller
process role.

> 55.2 IsObserver, type=KafkaController: Should we use the dash convention to
> be consistent with the rest of the metrics?

I would like to but I had to do this for backward compatibility. The
existing controller metrics are all scoped under the KafkaController
type. We had a similar discussion for "KIP-835: Monitor KRaft
Controller Quorum Health."

> 56. kafka-storage : "If the --release-version flag is not specified, the
> IBP in the configuration is used."
>   kafka-storage takes controller.properties as the input parameter and IBP
> is not a controller property, right?

I was documenting the current code. I suspect that the developer that
implemented kafka-storage wanted it to work with a configuration that
had an IBP.

> 57. To be consistent with kafka-storage, should we make the
> --release-version flag in kafka-features optional too? If this is not
> specified, the default associated with the tool will be used.

Sounds good. I updated that section to define this behavior for both
the upgrade and downgrade commands.

> 58. Typo: when the voter ID and UUID doesn't match
>   doesn't => don't

Fixed.

Thanks, I already updated the KIP to match my comments above and
include your feedback.
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-22 Thread José Armando García Sancio
Hi Claude,

On Fri, Mar 22, 2024 at 4:36 AM Claude Warren  wrote:
> Is there test code, or initial POC code for this KIP somewhere?  I would like 
> to help move this forward but need a few pointers to associated resources.  I 
> have read KIP-853 and it is beginning to sink in, but code would be nice.

Thanks for your interest and I would appreciate the help with the
implementation. I don't have a lot of code to show at the moment. The
existing KRaft implementation is in the "raft" Java module in the
apache/kafka repo.

I am planning to create a set of sub-tasks under KAFKA-14094 soon, to
give a rough outline of what needs implementing.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-21 Thread José Armando García Sancio
Hi Jun,

On Thu, Mar 14, 2024 at 3:38 PM Jun Rao  wrote:
> 52. Admin client: Could you provide a bit more details on the changes?

I updated the KIP to include the API changes to the Admin client.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-20 Thread José Armando García Sancio
Hi Jun,

See my comments below.

On Thu, Mar 14, 2024 at 3:38 PM Jun Rao  wrote:
> 37. Have you updated the wiki? It seems that LeaderIdAndEpoch and
> NodeEpoint are still two separate structs.

It is updated now. Apologies for the delayed wiki updates but I was
dealing with other issues in the past couple of weeks.

> 45. kafka-storage format --cluster-id  --release-version 3.8
> --standalone --config controller.properties
> It seems that --release-version is optional and the default value of this
> initial metadata.version is statically defined by the binary used to run
> the tool. So, in the common case, it seems that we could just omit
> --release-version.

That's correct. I removed it from the user explanation section and
updated the reference explanation section for the kafka-storage tool.

> 46. The KIP says "The leader will learn the range of supported version from
> the UpdateVoter RPC.
> 46.1 Then "KRaft will use the information in the controller registration,
> broker registration and add voter records to determine if the new version
> is compatible." is no longer accurate.

Good catch. I updated it to "KRaft will use the information in the
broker registration, voters records and UpdateVoters RPC to determine
if the new kraft.version are compatible. The new metadata.version will
be handled by the controller (QuorumController) like it is today."

> 46.2 Do we have a bit of a chicken-and-egg problem? A voter can't send
> UpdateVoter RPC until kraft.version is upgraded to 1. But to upgrade
> kraft.version, the leader needs to receive UpdateVoter RPCs.

Yes we do. It is a little subtle but this is why I relaxed UpdateVoter
so that it is sent irrespective of the kraft.version. The voters that
are following the leader will send a UpdateVoter request as long as
the "ApiKey" is in the ApiVersion response. I updated this sentence in
the UpdateVoter RPC section to make this clear:

"The voter will always send the UpdateVoter RPC whenever it starts and
whenever the leader changes irrespective of the kraft.version"

The KIP already has this wording in the handling section:

"5. Append the updated VotersRecord to the log if the finalized
kraft.version is greater than 0."

> 46.3 If the leader always learns the range of supported kraft.version from
> UpdateVoter RPC, does the VotersRecord need to include KRaftVersionFeature?

Yes. Storing it in VotersRecord allows the feature (kraft.version) to
be upgraded while K voters are offline where K is the number of
failures tolerated by the voter set. Upgrading from version
kraft.version 0 to kraft.version 1 is special. To upgrade from version
0 to version 1 all of the voters needed to be online at some point
while the current leader has been leader.

> 47. When a voter is started, in what order does it send the UpdateVoter and
> ControllerRegistration RPC?

They are independent of one another and can be sent in any order. Why
do you ask?

> 48. Voters set: "If the partition doesn't contain a VotersRecord control
> record then the value stored in the controller.quorum.voters property will
> be used."
>   Hmm, I thought controller.quorum.voters is only the fallback
> for controller.quorum.bootstrap.servers?

My observation is that the current controller.quorum.voters
configuration plays two roles. 1) It is the voters set for "voter
replicas". 2) It is the bootstrapping endpoint for "observer
replicas".

For 1) the resolution is going to be VotersRecord first,
controller.quorum.voters second.
For 2) the resolution is going to be
controller.quorum.bootstrap.servers first, controller.quorum.voters
second.

This is mainly there for backwards compatibility with configurations
that are valid in 3.7 and before.

> 49. "If the local replica is getting added to the voters set, it will allow
> the transition to prospective candidate when the fetch timer expires."
>   Could you explain why this is needed?

This is just saying that when a replica sees a VotersRecord which
includes itself it means that this local replica has become a voter.
Only voters are allowed to transition to prospective with KIP-996
(pre-vote) or transition to candidate (without KIP-996).

> 50. Quorum state: AppliedOffset will get removed in version 1.
>   This is the original description of AppliedOffset: Reflects the maximum
> offset that has been applied to this quorum state. This is used for log
> recovery. The broker must scan from this point on initialization to detect
> updates to this file. If we remove this field, how do we reason about the
> consistency between the quorum state and the metadata log?

With this KIP, these are the only fields that are present in the
lastest quorum data:
leader id, leader epoch, voted id and voted uuid. None of them are
related to the voters set that is stored in the log and snapshot.

For example, it is correct for the leader id to be in the voters set
and it is also correct for the leader id to not be in the voter set.
This second case can happen when the leader 

Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-03-15 Thread José Armando García Sancio
Hi Justine,

Thanks for the update. Some comments below.

On Wed, Mar 13, 2024 at 10:53 AM Justine Olshan
 wrote:
> 4. Include an API to list the features for a given metadata version

I am not opposed to designing and implementing this. I am just
wondering if this is strictly required?

Would having auto-generated documentation address the issue of not
knowing which feature versions are associated with a given release
version?

Does it need to be a Kafka API (RPC)? Or can this be strictly
implemented in the tool? The latest tool, which is needed to parse
release version to feature version, can print this mapping. Is this
what you mean by API?

> 5. I'm going back and forth on whether we should support the
> --release-version flag still. If we do, we need to include validation so we
> only upgrade on upgrade.

I am not sure how this is different from supporting multiple --feature
flags. The user can run an upgrade command where some of the features
specified are greater than what the cluster has finalized and some of
the features specified are less than what the cluster has finalized.

In other words, the KRaft controller and kafka-storage tool are going
to have to implement this validation even if you don't implement
--release-version in the tools.
Thanks,
-- 
-José


Re: [DISCUSS] Personal branches under apache/kafka

2024-03-13 Thread José Armando García Sancio
On Wed, Mar 13, 2024 at 11:02 AM Mickael Maison
 wrote:
> What do you think?

I agree. I wouldn't be surprised if these branches (not trunk or
release branches) were created by mistake by the committer.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-11 Thread José Armando García Sancio
Hi Jun

Thanks for the feedback. See my comments below.

On Wed, Mar 6, 2024 at 4:47 PM Jun Rao  wrote:
> 20.1. It seems that the PreferredSuccessors field didn't get fixed. It's
> still there together with PreferredCandidates.
> +{ "name": "PreferredSuccessors", "type": "[]int32", "versions":
> "0",
> +  "about": "A sorted list of preferred successors to start the
> election" },
> +{ "name": "PreferredCandidates", "type": "[]ReplicaInfo",
> "versions": "1+",
> +  "about": "A sorted list of preferred candidates to start the
> election", "fields": [

Notice that the PreferredSuccessors field is only for version 0 while
the PreferredCandidate field is for version 1 or greater. I had to
create a new field because arrays of int32 ([]int32) are not
compatible with arrays of structs because of tagged fields in
sub-structs.

> 37. If we don't support batching in AddVoterResponse, RemoveVoterResponse
> and UpdateVoterResponse, should we combine CurrentLeader and NodeEndpoint
> into a single field?

Yes. I replaced the LeaderIdAndEpoch and NodeEpoint structs into a
single struct that contains the leader id, epoch, host and port.

> 42. We include VoterId and VoterUuid for the receiver in Vote and
> BeginQuorumEpoch requests, but not in EndQuorumEpoch, Fetch and
> FetchSnapshot. Could you explain how they are used?

For the Vote request and BeginQuorumEpoch request the replica
(candidate for Vote and leader for BeginQuorumEpoch) sending the
request needs to make sure that it is sending the request to the
correct node. This is needed for correctness. The most important case
that I wanted to make sure that replicas handle correctly is the
following:
1. Voter set is A, B, C  and the leader is A. The voter A is both the
voter id and voter uuid
2. Assume that A crashes and loses its disk. When it recovers it will
come back as A'. A' means a replica with the same id but with a
different replica uuid.

Replica A' will most likely be accepting connection and handling
requests (e.g. Vote and BeginQuorumEpoch) on the same endpoints as A.
There can be inconsistency in the state, if for example replica B
sends a Vote request to A' but A' handles it as if it was A. This is
the reason the sender sends the remote replica's id and uuid (VoterId
and VoterUuid) in the request. The same analysis applies to
BeginEpochQuorum.

For the Fetch and FetchSnapshot request the closest equivalent would
be leader id and leader epoch. Those RPCs only have leader epochs. You
can argue that they don't need the leader id because a leader epoch
can have at most one leader id. In other words, the leader epoch also
uniquely identifies the leader id if there is one. I am reluctant to
change the Fetch RPC unless it is strictly required because that RPC
is also used for regular topic partition replication. I tried to make
the FetchSnapshot RPC as consistent to the Fetch RPC as possible since
they have similar access patterns.

EndQuorumEpoch is not needed for correctness. It is there for improved
availability; to speedup leadership change when the nodes are
cooperating (controlled shutdown and resignation). The sending replica
(leader) doesn't need to wait for the response or to check that the
RPC was handled correctly.

I'll reread the KIP and update it to better explain the need for
VoteId and VoteUuid in the Vote and BeginQuorumEpoch requests.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-06 Thread José Armando García Sancio
Hi Jun,

See my comments below.

On Tue, Mar 5, 2024 at 2:57 PM Jun Rao  wrote:
> 37. We don't batch multiple topic partitions in AddVoter, RemoveVoter and
> UpdateVoter requests while other requests like Vote and BeginQuorumEpoch
> support batching. Should we make them consistent?

Originally I had them as batched RPCs but decided to make them only
operate on one KRaft topic partition. I made this change primarily
because this is false sharing and batching. Topic partitions in KRaft
are independent; those operations will be handled independently and
committed independently but because of the batching the kraft node
will be required to wait on all the batched operations before it can
send the response.

I know that this is inconsistent with the other RPCs but I am hesitant
to propagate this incorrect batching to new RPCs.

> 38. BeginQuorumEpochRequest: It seems that we need to replace the name
> field with a nodeId field in LeaderEndpoints?

The schema looks good to me. This struct has a different meaning from
the struct in the response of other RPCs. The BeginQuorumEpoch request
is sent by the leader so the expectation is that the sending
node/replica is the leader for all of the partitions sent. This also
means that the endpoints sent in LeaderEndpoints are all for the same
leader (or replica). The reason that BeginQuorumEpoch sends multiple
endpoints is because the leader may be binding to multiple listeners.
The leader sends a tuple (listener name, host name, host port) for
each of its own advertised controller listeners.

> 39. VoteRequest: Will the Voter ever be different from the Candidate? I
> thought that in all the VoteRequests, the voter just votes for itself.

Yes. This is a naming that always confuses me but makes sense to me
after further analysis. The voter (VoterId and VoterUuid) is the
replica receiving the Vote request and potentially voting for the
sender (candidate). The candidate (CandidateId and CandidateUuid) is
the replica sending the Vote request and asking for votes from the
receivers (voters). I tried to better document it in their respective
"about" schema fields.

> 40. EndQuorumEpochRequest: Should we add a replicaUUID field to pair with
> LeaderId?

I don't think so. We can add it for consistency and to help debugging
but I don't think it is needed for correctness. A leader cannot
(should not) change replica uuid and remain leader. In theory the only
way for a replica to change uuid is to lose their disk. If this
happens the expectation is that they will also lose their
QuorumStateData.

> 41. Regarding including replica UUID to identify a voter: It adds a bit of
> complexity. Could you explain whether it is truly needed? Before this KIP,
> KRaft already supports replacing a disk on the voter node, right?

Yes. This KIP attempts to solve two general problems. 1) How to
proactively change the voters set by increasing or decreasing the
replication factor; or replace voters in the voters set. 2) Identify
disk failures and recover from them safely. This is what I have in the
Motivation section:
"Consensus on the cluster metadata partition was achieved by the
voters (controllers). If the operator of a KRaft cluster wanted to
make changes to the set of voters, they would have to shutdown all of
the controllers nodes and manually make changes to the on-disk state
of the old controllers and new controllers. If the operator wanted to
replace an existing voter because of a disk failure or general
hardware failure, they would have to make sure that the new voter node
has a superset of the previous voter's on-disk state. Both of these
solutions are manual and error prone."

directory.id (replica uuid) is needed to identify and resolve disk
failures in a voter. The section "Proposed change / User explanation /
Common scenarios / Disk failure recovery" documents this use case in
more detail.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-04 Thread José Armando García Sancio
Hi Jun,

Thanks for the feedback. See my comments below.

On Fri, Mar 1, 2024 at 11:36 AM Jun Rao  wrote:
> 30. Historically, we used MV to gate the version of Fetch request. Are you
> saying that voters will ignore MV and only depend on raft.version when
> choosing the version of Fetch request?

Between Kafka servers/nodes (brokers and controllers) there are two
implementations for the Fetch RPC.

One, is the one traditionally used between brokers to replicate ISR
based topic partitions. As you point out Kafka negotiates those
versions using the IBP for ZK-based clusters and MV for KRaft-based
clusters. This KIP doesn't change that. There have been offline
conversations of potentially using the ApiVersions to negotiate those
RPC versions but that is outside the scope of this KIP.

Two, is the KRaft implementation. As of today only the controller
listeners  (controller.listener.names) implement the request handlers
for this version of the Fetch RPC. KafkaRaftClient implements the
client side of this RPC. This version of the Fetch RPC is negotiated
using ApiVersions.

I hope that clarifies the two implementations. On a similar note,
Jason and I did have a brief conversation regarding if KRaft should
use a different RPC from Fetch to replicate the log of KRaft topic
partition. This could be a long term option to make these two
implementations clearer and allow them to diverge. I am not ready to
tackle that problem in this KIP.

> 35. Upgrading the controller listeners.
> 35.1 So, the protocol is that each controller will pick the first listener
> in controller.listener.names to initiate a connection?

Yes. The negative of this solution is that it requires 3 rolls of
voters (controllers) and 1 roll of observers (brokers) to replace a
voter endpoint. In the future, we can have a solution that initiates
the connection based on the state of the VotersRecord for voters RPCs.
That solution can replace an endpoint with 2 rolls of voters and 1
roll of observers.

> 35.2 Should we include the new listeners in the section "Change the
> controller listener in the brokers"?

Yes. We need to. The observers (brokers) need to know what security
protocol to use to connect to the endpoint(s) in
controller.quorum.bootstrap.servers. This is also how connections to
controller.quorum.voters work today.

> 35.3 For every RPC that returns the controller leader, do we need to
> return multiple endpoints?

KRaft only needs to return the endpoint associated with the listener
used to send the RPC request. This is similar to how the Metadata RPC
works. The Brokers field in the Metadata response only returns the
endpoints that match the listener used to receive the Metadata
request.

This is the main reason why KRaft needs to initiate connections using
a security protocol (listener name) that is supported by all of the
replicas. All of the clients (voters and observers) need to know
(security protocol) how to connect to the redirection endpoint. All of
the voters need to be listening on that listener name so that
redirection works no matter the leader.

> 35.4 The controller/observer can now get the endpoint from both records and
> RPCs. Which one takes precedence? For example, suppose that a voter is down
> for a while. It's started and gets the latest listener for the leader from
> the initial fetch response. When fetching the records, it could see an
> outdated listener. If it picks up this listener, it may not be able to
> connect to the leader.

Yeah. This is where connection and endpoint management gets tricky.
This is my implementation strategy:

1. For the RPCs Vote, BeginQuorumEpoch and EndQuorumEpoch the replicas
(votes) will always initiate connections using the endpoints described
in the VotersRecord (or controller.quorum.voters for kraft.version 0).
2. For the Fetch RPC when the leader is not known, the replicas will
use the endpoints in controller.quorum.bootstrap.servers (or
controller.quorum.voters for kraft.version 0). This is how the
replicas (observers) normally discover the latest leader.
2. For the Fetch and FetchSnapshot RPC when the leader is known, the
replicas use the endpoint that was discovered through previous RPC
response(s) or the endpoint in the BeginQuorumEpoch request.

I have been thinking a lot about this and this is the most consistent
and deterministic algorithm that I can think of. We should be able to
implement a different algorithm in the future without changing the
protocol or KIP.

> 36. Bootstrapping with multiple voters: How does a user get the replica
> uuid? In that case, do we use the specified replica uuid instead of a
> randomly generated one in the meta.properties file in metadata.log.dir?

There are two options:
1. They generate the directory.id for all of the voters using
something like "kafka-storage random-uuid" and specify those in
"kafka-storage format --controller-quorum-voters". This is the safest
option as it can detect disk replacement from bootstrap.

2. They only specify the 

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-01 Thread José Armando García Sancio
Hi Luke,

Thanks for the feedback. See my comments below.

I have also updated the KIP to reflect this discussion.

On Wed, Feb 28, 2024 at 4:34 AM Luke Chen  wrote:
>
> > 2. After "RemoveVoter", what is the role of the node?
> > It looks like after the voter got removed from the voter set, it is not a
> > voter anymore. But I think it can still fetch with the leader. So it
> should
> > be an observer, with a "process.role=controller"? And if the node was
> > originally "process.role=controller,broker", it'll become a broker-only
> > node?
>
> > Kafka nodes need to allow for controllers that are not voters. I don't
> expect too many issues from an implementation point of view. Most of
> it may just be aggressive validation in KafkaConfig. I think the
> easier way to explain this state is that there will be controllers
> that will never become active controllers. If we want, we can have a
> monitor that turns on (1) if a node is in this state. What do you
> think?
>
> I agree we have a way for users to monitor the node state, like when does
> the controller completed the voter removal ( so that it is safe to be
> shutdown), or when does the controller completed the voter addition (so
> that users can start to add another controller), etc.

We have the following metrics in the monitoring section:
name=IsObserver;type=KafkaController - 1 if a controller node is not a
voter for the KRaft cluster metadata partition, 0 otherwise.
name=pending-voter-change;type=raft-metrics - 1 if there is a pending
voter change that has not been committed, 0 otherwise. All of the
Kafka nodes will report this metric.

In addition to the metrics, the user can also use
"kafka-metadata-quorum describe --status" command to query for the
current voters set and the last committed voters set.

The KRaft leader will check that it is safe to add or remove voters.
The KRaft only allows for one uncommitted VotersRecord and an
uncommitted VotersRecord can only change the previously committed
VotersRecord by at most one change (either one addition or one
removal).

> 10. controller.quorum.voters:
> This is an existing configuration. This configuration describes the state
> of the quorum and will only be used if the kraft.version feature is 0.
> > From the discussion, it looks like even if the kraft.version is 1, we
> still first check the `controller.quorum.voters` if
> `controller.quorum.bootstrap.servers` is not set. Is that correct? If so,
> maybe we need to update the description?

That's correct. In the current KRaft implementation
controller.quorum.voters plays two roles: 1) describes the static
voters set 2) the endpoint(s) that can be used to discover the leader.

This KIP replaces 1) with the VotersRecord written to the log and
snapshot and replaces 2) with the controller.quorum.bootstrap.servers
configuration property.

I updated the description for these configurations to make this clearer.

> 11. When a controller starts up before joining as a voter, it'll be an
> observer. In this case, will it be shown in the observer field of
> `kafka-metadata-quorum describe --status`? Same question to a controller
> after getting removed.

Correct. The name=IsObserver;type=KafkaController metric for that node
will also be 1.

Note that the KRaft leader's reporting of observers is best effort. As
long as the observer is fetching, the KRaft leader will report their
state. If an observer doesn't Fetch for some time (I forget the value
we use) the KRaft leader will remove the observer's replication state
from its in-memory bookkeeping.

> 12. What will happen if there's only 1 voter and user still tries to remove
> the voter? Any error returned?

Good point. This should probably be an error. If they are interested
in decommissioning a cluster, they should instead just destroy the
controller node.

For this case, I updated the KIP to return the INVALID_REQUEST error.

Thanks,
--
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-03-01 Thread José Armando García Sancio
Hi Jun,

Thanks for the feedback. See my comments below.

On Tue, Feb 27, 2024 at 11:27 AM Jun Rao  wrote:
> 30. Who controls RPCs like Fetch, FetchSnapshot, DescribeQuorum RPC? They
> are shared between voters and observers.

For Fetch and FetchSnapshot, this KIP adds the tagged field
ReplicaUuid to the request. This means that if the sender supports the
latest version it can always add the replica uuid to the request. If
the receiver supports the new tagged field it is included in the
appropriate FetchRequestData and FetchSnapshotRequestData field. If it
doesn't support the new tagged field it will be in the unknown tagged
fields.

For DescribeQuorum, this KIP only changes the response. The KRaft
leader will inspect the RequestHeader::apiVersion to determine what
information to include in the response.

> Does the client use supported ApiKeys or kraft.version feature in
> ApiVersions response for deciding whether to send AddVoter requests?

That's a good point. The Admin client has access to the finalized
kraft.version in the ApiVersions response. I was thinking of only
using the ApiKeys since we don't have a precedence of using the
finalized features in the Admin client. The receiver of the request
still needs to validate the kraft.version for those requests and
return an UNSUPPORTED_VERSION error in those cases.

Do you mind if we define this in a separate KIP?

Thanks,
-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-02-29 Thread José Armando García Sancio
Thanks for the reply Justine. See my comments below:

On Thu, Feb 29, 2024 at 3:39 PM Justine Olshan
 wrote:
> I wanted to include multiple features in one command, so it seems like
> features is a better name. I discuss more below about why I think we should
> allow setting multiple features at once.

--feature in kafka-features allows for multiple features to be
specified. Please see the implementation of the tool and
UpdateFeatures RPC.

For example, you can execute this command kafka-features.sh upgrade
--feature metadata.version=15 --feature kraft.version=1. You should be
able to support the same UI in kafka-storage.sh. It is what KIP-853
suggests in the interest of making it consistent across CLI.

> For the storage tool, we are setting all the features for the cluster. By
> default, all are set. For the upgrade tool, the default is to set one
> feature. In the storage tool, it is natural for the --release-version to
> set the remaining features that --features didn't cover since otherwise we
> would need to set them all
>
> If we use the flag. In the feature upgrade case, it is less necessary for
> all the features to be set at once and the tool can be run multiple times.
> I'm not opposed to allowing both flags, but it is less necessary in my
> opinion.

I was thinking of making them mutually exclusive in both cases. Much
easier to explain and document. If the user wants to use --feature in
kafka-storage, they need to specify all of the supported features.

For the "kafka-feature upgrade" case, they can enumerate all of the
--feature versions that they want to upgrade in one command.

> See my note above (Jun's comment 12) about deprecating old commands. I
> would think as the KIP stands now, we would not accept both commands.

If you are not going to deprecate or alias --metadata what happens if
the user uses these flags in one command: "kafka-features upgrade
--metadata 3.8 --feature kraft.version=1"

One of the problems I am having is that I can't seem to find the KIP
that introduces --metadata so I can only speculate as to the intent of
this flag from the CLI help and implementation. Do you know which KIP
added that flag?

> I sort of already implemented it as such, so I don't think it is too
> tricky. I'm not sure of an alternative. Kafka features currently only
> supports one feature at a time.
> I would like to support more than one for the storage tool. Do you have
> another suggestion for multiple features in the storage tool?

"kafka-features.sh upgrade" supports multiple --feature flags. Please
see the tools implementation and the UpdateFeatures RPC.

You can specify multiple features in the storage tool with:
"kafka-storage format --feature kraft.version=1 --feature
metadata.version=15". The command line parser used by both tools
support flags that append values to a list. This is how the
kafka-features CLI works already.

> The plan so far is documentation. The idea is that this is an advanced
> feature, so I think it is reasonable to ask folks use documentation

Are you going to generate the documentation of these dependencies
automatically from the released code like we do for Kafka
configuration properties? Or do Kafka developers need to remember to
update the documentation with these dependencies?

> The idea I had was that the cluster will check if all the versions are
> supported. If any version is not supported the tool will throw an error. We
> can also issue a warning if the given command (if supported by the cluster)
> will downgrade a feature. One option is also to require a yes/no prompt or
> flag to allow downgrades. The downgrade tool would allow the same.
> Alternatively we could just fail the command if a feature is downgraded on
> upgrade command or vice versa. I don't have a strong preference.

"kafka-features upgrade" should only upgrade and "kafka-features
downgrade" command should only downgrade. It should be an error if any
of the UpdateFeatures requests violates this. What we need to do is
define if an error in one feature results in an error for the entire
request. The UpdateFeatures RPC seems to allow individual errors but
that gets confusing and difficult to enforce once you introduce
dependencies between feature versions.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-29 Thread José Armando García Sancio
Hi Jun and Luke,

Jun,

I added a section the documents the process for upgrading the
controller listeners endpoints:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-853%3A+KRaft+Controller+Membership+Changes#KIP853:KRaftControllerMembershipChanges-Upgradingcontrollerlistener

Jun and Luke,

I am going to address your latest comments next.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-1022 Formatting and Updating Features

2024-02-29 Thread José Armando García Sancio
Hi Justine and Jun,

Thanks for the KIP Justine. See my comments below.

On Wed, Feb 28, 2024 at 3:09 PM Jun Rao  wrote:
> 13. KIP-853 also extends the tools to support a new feature kraft.version.
> It would be useful to have alignment between that KIP and this one.

I agree. I took a look at this KIP and these are the differences that
I can spot.

1. KIP-853 uses --feature for kafka-storage instead of --features.
This is consistent with the use of --feature in the "kafka-feature.sh
upgrade" command.

2. I find it a bit inconsistent that --feature and --release-version
are mutually exclusive in the kafka-feature CLI but not in the
kafka-storage CLI. What is the reason for this decision?

3. KIP-853 deprecates --metadata in the kafka-features and makes it an
alias for --release-version. In KIP-1022, what happens if the user
specified both --metadata and --feature?

4. There is an example: "kafka-storage format --features
metadata.version=16,transaction.version=2,group.coordinator.version=1".
This is different from the --feature flag in kafka-features which is
an append flag. Meaning that multiple features are specified as
"--feature metadata.version=16 --feature transaction.version=2
--feature group.coordinator.version=1". I would suggest keeping this
consistent with kafka-features. It would avoid having to implement one
more parser in Kafka.

5. In the section "A Note on Feature Interdependence", you mention "an
error should be thrown if trying to format with X version 13 and Y
version 12". How would the user discover (or describe) these
dependencies? As currently described, trial and error seem to be the
only mechanism. Should the Kafka documentation describe these
dependencies? Is that good enough?

6. In "3.8-IV2 that could map to TV1, 3.8-IV3 could also be TV1, and
3.8-IV3 could be TV2" did you mean that 3.8-IV4 would map to TV2? If
not, 3.8-IV3 would map to two different TV values.

7. For --release-version in "kafka-features upgrade", does
--release-version mean that all of the feature versions will either
upgrade or stay the same? Meaning that downgrades will be rejected by
the Kafka controller. How about when --release-version is used with
"kafka-features downgrade"?

-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-27 Thread José Armando García Sancio
Hi Jun,

Thanks for the comments. See my comments below.

On Mon, Feb 26, 2024 at 4:48 PM Jun Rao  wrote:
> 15.1 It would be useful to document the process of changing the controller
> listener. For example, what configs need to be changed in what order and
> how a voter/observer selects the listener when multiple listeners are
> returned.

Yes. I'll do that. I'll add a section to the "User Explanation" section.

> 15.2 Thinking a bit more. The primary usage of multiple listeners is for
> changing the listeners online. So, we may not need to support multiple
> listeners in the initial setup phase.

That is true. I'll revert those changes. We can create another KIP and
add them if there is a use case for supporting multiple endpoints
during bootstrapping.

> 30. Well, VotersRecords and LeaderChangeRecord depend on RPCs like
> AddVoter, UpdateVoter and Vote, etc. Are those RPCs controlled by
> metadata.version? What happens if metadata.version doesn't enable those
> RPCs, but kraft.version requires them.

Those RPCs and their versions are not controlled by the
metadata.version. They will be controlled by the ApiVersions reported
by the replicas and the kraft.version of the sender and receiver.

For AddVoter and RemoveVoter, the receiving replica will return an
UNSUPPORTED_VERSION error if they receive a request and the
kraft.version is not greater than 1. The admin client will send the
AddVoter and RemoveVoter requests if they are reported by ApiVersions.
The receiver will return an UNSUPPORTED_VERSION if the kraft.version
is not greater than 1. Voters will send the AddVoter, RemoveVoter RPCs
if their local kraft.version is greater than 1 and the remote replica
supports those RPCs (through ApiVersions).

Vote is a bit different. Since a replica's behavior is based on
uncommitted state it is possible for a voter in kraft.version 1 to
send a Vote request to a voter that is not in kraft.version 1. In that
case the receiving voter will handle the Vote request as normal but
not write the CandidateUuid that was sent in the request to the
QuorumStateData file. This is not ideal but I think it is okay since
this can only happen while the kraft.version is getting upgraded.

UpdateVoter is also a bit different. Voters will always send this RPC
if the leader supports it through ApiVersion. Once the leader receives
the RPC it will persist the data sent to the log, if the kraft.version
is greater than 1. If the kraft.version is 0, the leader will handle
the RPC and store the information sent in memory but it will not
persist it to the log.

This should be documented at different points in the KIP but I'll
create a section that summarizes this.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-26 Thread José Armando García Sancio
Hi Jun, see my comments below.

I need to make some changes to the KIP to reflect our conversation.

On Fri, Feb 23, 2024 at 1:39 PM Jun Rao  wrote:
> 15.1 "In this configuration, the local replica needs to use
> CONTROLLER_PLAINTEXT because that is what is supported by all of the
> voters."
>Hmm, my understanding is that what listener to use can be made pairwise
> between the sender and the receiver. There is no need to pick the same
> listener across all voters, right?

Yes, this is true. Thinking about it some more, there are a couple of issues.

In the current implementation, the Selector we use for the
NetworkClient only supports one ChannelBuilder and that channel
builder can only use one security protocol. Until we fix that in the
implementation the preferred (first) listener needs to be a listener
that is supported by all of the voters. I'll make sure to document
that in the Kafka documentation until we fix the implementation.

The other issue is that when the leader sends its endpoint in the
BeginQuorumEpoch it needs to send all of its endpoints. The leader
needs to do that because replicas may connect to each other using any
of the configured listeners. For example, let's assume that the voters
(controllers) have been upgraded to support the listeners
CONTROLLER_SSL and CONTROLLER_PLAINTEXT with CONTROLLER_SSL being the
preferred listener. It is possible that the observers (brokers) have
yet to be reconfigured to use CONTROLLER_SSL and are instead using
CONTROLLER_PLAINTEXT. That means that the voters need to know all of
the leader's listeners because observers discover the leader by
sending a Fetch request to a random endpoint in
controller.quorum.bootstrap.servers using the security protocol of the
first controller listener. I'll make that change to the KIP.

> 15.2  When bootstrapping with just one controller, we could register a
> single voter with multiple listeners from controller.listener.names and
> listeners. However, when bootstrapping with multiple controllers,
> kafka-storage only has the option to specify one listener
> (--controller-quorum-voters -@:) per
> voter. Should we make them consistent?

This is true. I was trying to avoid this but this configuration can
get quite complicated. The user will have a better experience if they
can express it as a JSON file. The configuration should just be a
VotersRecord but in JSON. Let me update the KIP.

> 18.3 : "It will not use the metadata layer (QuorumController) to update and
> persist the finalized kraft.version."
> So, we depend on KRaftVersionRecord to propagate finalized kraft.version to
> brokers/controllers? It would be useful to document that.

Yes. I'll make that clear in the KIP.

> 27. "My plan is to rely on KIP-996: Pre-vote:".
> Hmm, Diego's paper says "Unfortunately, the Pre-Vote phase does not solve
> the problem of disruptive servers". Is that a real issue?

An active leader gets disrupted if a replica increases their epoch
past the active leader's epoch.

Diego's Raft allows the replica to increase their epoch and become a
candidate once it wins a pre-vote. Diego defines pre-vote as follows:
"A candidate would first ask other servers whether its log was
up-to-date enough to get their vote. Only if the candidate believed it
could get votes from a majority of the cluster would it increment its
term and start a normal election.

Unfortunately, the Pre-Vote phase does not solve the problem of
disruptive servers: there are situations where the disruptive server’s
log is sufficiently up-to-date, but starting an election would still
be disruptive."

In other words, Diego's Raft only compares log lengths during the
pre-vote phase. That is why Diego then mentions:
"We modify the RequestVote RPC to achieve this: if a server receives a
RequestVote request within the minimum election timeout of hearing
from a current leader, it does not update its term or grant its vote."

Diego can do this for his definition of Raft because he is only
concerned with voters. There is no support for observers. Diego's Raft
nodes are either voters or they are not part of the cluster.

We cannot do that for KRaft because we have observers. When a replica
is removed from the voter set it becomes an observer. I want that
replica to be able to continue participating in the cluster. That
means that we need to stop observers from increasing their epoch even
if they think they are in the voter set.

For KRaft our definition of pre-vote is different. Voters are going to
perform both checks during the pre-vote phase. A voter rejects a Vote
request if either:
1. The sender's log is shorter than the receiver's log
2. The receiver has contacted the leader either through Fetch,
FetchSnapshot or BeginQuorumEpoch during the Fetch timeout.

Check 2 should stop an observer that thinks they are a voter from
increasing their epoch and disrupting the leader. Let me give an
example.

1. Assume a voter set of 1, 2, 3, 4 and it has been replicated to all
of the voters 

Re: [DISCUSS] KIP-966: Eligible Leader Replicas

2024-02-23 Thread José Armando García Sancio
Hi Calvin

On Fri, Feb 23, 2024 at 9:23 AM Calvin Liu  wrote:
> As we agreed to implement the pagination for the new API
> DescribeTopicPartitions, the client side must also add a proper interface
> to handle the pagination.
> The current KafkaAdminClient.describeTopics returns
> the DescribeTopicsResult which is the future for querying all the topics.
> It is awkward to fit the pagination into it because

I suggest taking a look at Java's Flow API:
https://docs.oracle.com/en/java/javase/21/docs/api/java.base/java/util/concurrent/Flow.html
It was design for this specific use case and many libraries integrate with it.

If the Kafka client cannot be upgraded to support the Java 9 which
introduced that API, you can copy the same interface and semantics.
This would allow users to easily integrate with reactive libraries
since they all integrate with Java Flow.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-23 Thread José Armando García Sancio
Jun, I updated one of the rejected ideas to better explain why KRaft
can't rely on information stored by the metadata layer.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-22 Thread José Armando García Sancio
Thanks for the additional feedback Jun. Comments below.

On Fri, Feb 16, 2024 at 4:09 PM Jun Rao  wrote:
> 10. "The controller state machine will instead push the brokers'
> kraft.version information to the KRaft client". If we do that, why do we
> need KRaftVersionRecord?

I am doing this as a reasonable compromise. Let me better explain why
we need KRaftVersionRecord and VotersRecord for voters as control
records.

First the controller state machine (QuorumController in the code)
operates on committed data (records that have an offset smaller than
the HWM). That means that to read committed data the HWM needs to be
established. The HWM is discovered by the KRaft leader. To establish
the KRaft leader the voters need to send RPCs to other voters. To be
able to send RPCs to other voters the replicas need to be able to read
and process the locally uncommitted KRaftVersionRecord and
VotersRecord.

In short, the metadata layer (quorum controller) reads and processes
committed data while the KRaft layer reads and processes uncommitted
data. KRaft needs to read and process uncommitted data because that
data is required to establish a majority (consensus) and a leader.

I am relaxing this for observers (brokers) for two reasons:
1. Observers are dynamic and unknown to the voters (leader). Voters
only need to handle Fetch and FetchSnapshot requests from observers
(brokers). Their information is not persisted to disk and it is only
tracked in-memory for reporting purposes (DescribeQuorum) while they
continue to Fetch from the leader.
2. The voters don't need to read the uncommitted information about the
brokers (observers) to establish a majority and the leader. So there
is not strict requirement to include this information as control
record in the log and snapshot.

> 15. Hmm, I thought controller.listener.names already provides the listener
> name. It's a list so that we could support changing security protocols.

Not sure if I fully understand the comment but here is an example that
maybe illustrates why we need all of the information included in the
KIP (VotersRecord). Let's assume the following local configuration:
controller.listener.names=CONTROLLER_SSL,CONTROLLER_PLAINTEXT

With this configuration the voter (controller) prefers connecting
through CONTROLLER_SSL first and CONTROLLER_PLAINTEXT second. To
establish consensus and leadership the voters need to send the Vote
request to other voters. Which host and endpoint should the voter use?
Let's assume the follow VotersRecord:

{ "VoterId": 0, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_SSL", "host": "controller-0", "port": 1234}, {"name":
"CONTROLLER_PLAINTEXT", ... } ]
{ "VoterId": 1, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_SSL", "host": "controller-1", "port": 1234}, {"name":
"CONTROLLER_PLAINTEXT", ... } ]
{ "VoterId": 2, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_SSL", "host": "controller-2", "port": 1234}, {"name":
"CONTROLLER_PLAINTEXT", ... } ]

In this configuration, the local replica can use CONTROLLER_SSL and
lookup the host and port because that is the preferred (first)
listener and it is supported by all of the voters.

Now let's assume the following VotersRecord:

{ "VoterId": 0, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_SSL", "host": "controller-0", "port": 1234}, {"name":
"CONTROLLER_PLAINTEXT", ... } ]
{ "VoterId": 1, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_PLAINTEXT", ... } ]
{ "VoterId": 2, "VoterUuid": "...", "Endpoints": [ {"name":
"CONTROLLER_SSL", "host": "controller-2", "port": 1234}, {"name":
"CONTROLLER_PLAINTEXT", ... } ]

In this configuration, the local replica needs to use
CONTROLLER_PLAINTEXT because that is what is supported by all of the
voters.

> 17.1 "1. They are implemented at two different layers of the protocol. The
> Kafka controller is an application of the KRaft protocol. I wanted to
> keep this distinction in this design. The controller API is going to
> forward ControllerRegistrationRequest to the QuorumController and it
> is going to forward UpdateVoter to the KafkaRaftClient."
> Hmm, but the controller already pushes the brokers' kraft.version
> information to the KRaft client.

Right but only for brokers (observers). Voters are different in that
their information is required to establish consensus. KRaft needs to
read this information as uncommitted data because committed data (HWM)
can only be established after leadership has been established.

> "2. If the voter getting updated is not part of the
> voter set the leader will reject the update."
> Would it be simpler to just relax that? The KIP already relaxed some of the
> checks during the vote.

We could but in this KIP replicas only stores information about
voters. If we relax that, replicas will start storing information
about observers (or brokers). Now we would have to add a mechanism for
deleting this data similar to UnregisterBrokerRequest.

> "3. The other semantic difference is 

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-16 Thread José Armando García Sancio
Hi Jason and Jun,

I have updated the KIP to incorporate your feedback. Some of the changes are:

1. Removed security protocol from the listener definition. The local
replica is going to assume that the remote replica has the same
listener name to security group mapping.
2. Reverted most of the changes to QuorumStateData. In summary the KIP
only adds VotedUuid to this persistent data.
3. Explicitly mentioned in a few places that downgrades from
kraft.version 1 to kraft.version 0 is not supported.
4. Added an ErrorMessage field to all of the admin client RPCs used by
KRaft: AddVoter, RemoveVoter and DescribeQuorum.
5. Quoted the KIP-996: Pre-vote with respect to fencing voters that
were removed from the voter set affecting election.
6. Updated the admin client section to mention that AddVoter,
RemoveVoter and DescribeQuorum will be sent directly to the active
controller when using a controller bootstrap server.
7. Added why we rejected using controller mechanisms to the rejected
alternative section. This came up a few times so it is worth
documenting why it was rejected.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-16 Thread José Armando García Sancio
Thanks for the feedback Jason. See my comments below.

On Mon, Feb 5, 2024 at 3:11 PM Jason Gustafson
 wrote:
> 1. When adding a voter, the KIP proposes to return a timeout error if the
> voter cannot catch up in time. It might be more useful to return a more
> specific error so that an operator can understand why the timeout occurred.
> Alternatively, perhaps we could keep the generic error but add an
> ErrorMessage field in the response to explain the issue. (At the same time,
> perhaps we can add ErrorMessage to the other new RPCs. I think this is
> becoming more or less standard in Kafka RPCs.)

I added the ErrorMessage field to the 3 admin client RPCs: AddVoter,
RemoveVoter and DescribeQuorrum. I didn't add it to the other,
internal, RPCs since I didn't think it would be useful in practice.
Let me know if you think we should add that field to the "internal"
RPCs.

> 2. Say that the voters are A, B, and C, and that C is offline. What error
> would be returned from RemoveVoters if we try to remove B before C is back
> online?

In this example, the case below would never succeed:
"5. Wait for the VotersRecord to commit using the majority of new
configuration."

I am planning to return a REQUEST_TIMED_OUT error for this case. A
future RemoveVoter request for the same replica id and uuid would also
return a REQUEST_TIMED_OUT if the VotersRecord
stays uncommitted because of:
"1. Wait until there are no uncommitted VotersRecord. Note that the
implementation may just return a REQUEST_TIMED_OUT error if there are
pending operations."

Note that this is after the leader (A or B) wrote VotersRecord that
removed B, so the cluster is unavailable. If C doesn't come back
online, eventually "check quorum" will fail and the leader (A or B)
will resign.

This is also an example of the HWM decreasing because of removing the voter.

We could add a heuristic to lower the impact of this example but I am
not sure if it is worth the implementation cost at the moment. For
example, the leader can check the LEO and lastFetchTimestamp of the
replicas (C in this example). What are these replicas in a bigger
voter set?

> 3. There is a note about a voter accepting a Vote request from a voter that
> is not in their voter set. I guess the same comment applies to
> BeginQuorumEpoch?

Yes. The other extension to the BeginQuorumEpoch request is that the
leader will resend the request (even if it was previously
acknowledged), if the voters fails to send a Fetch request within the
"check quorum" timeout. I had to change this because BeginQuorumEpoch
also includes the leader's endpoint information. Yet the receiving
replica doesn't persist the leader endpoint information into the
quorum-state file like it does for leader id, uuid and epoch.

> In other words, a voter should accept a new leader even
> if it is not in its local voter set.

Yes. There are two cases for this. 1. A voter has become leader before
another replica (observer or voter) has replicated the voter set. 2.
The active leader is the replica that is getting removed from the
voter set.

> The note in the reference says: "Thus,
> servers process incoming RPC requests without consulting their current
> configurations." On the other hand, when it comes to election, voters will
> become candidates based on the latest voter set from the log (regardless
> whether it is committed), and they will seek out votes only from the same
> voter set. Is that about right?

Yes. That's correct. When transitioning to prospective (pre-vote KIP),
when transitioning to candidate, or when deciding to send RPCs like
Vote (and BeginQuorumEpoch); the replica will do so based on the
latest voter set (VotersRecord) in the log.

Thanks
--
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-14 Thread José Armando García Sancio
Hi Jun,

Thanks for the feedback. Excuse the delay, it took me a while to
properly address your detailed feedback. See my comments below.

I am going to update the KIP as outlined in this email. I will send
another email when I have made all of the changes.

On Fri, Feb 2, 2024 at 10:54 AM Jun Rao  wrote:
> 10. kraft.version: Functionality wise, this seems very similar to
> metadata.version, which is to make sure that all brokers/controllers are on
> a supported version before enabling a new feature. Could you explain why we
> need a new one instead of just relying on metadata.version?

Yes, they are trying to solve similar problems but they are doing them
at a different abstraction layer. "metadata.version" uses
FeatureLevelRecord which is a "data" record. I am trying to design the
KRaft protocol (KafkaRaftClient implementation) to not assume a
specific application (metadata and the controller). To support
multiple applications possibly using KRaft means that we need to use
control records to define KRaft's behavior. Instead of using
FeatureLevelRecord which is a cluster metadata record, this KIP uses
KRaftVersionRecord which will be a control record.

The reason why we need to consider observers (brokers) when
determining the kraft.version is because of snapshot (compaction).
Since snapshots need to include all logical records in the log
segments and observers (brokers) maintain the log on disk, this means
that they need to be able to decode and encode both KRaftVersionRecord
and VotersRecord before the leader is allowed to write them.

> 11. Both the quorum-state file and controller.quorum.bootstrap.servers
> contain endpoints. Which one takes precedence?

I updated the KIP (section "Reference explanation / Endpoints
information") after Jack's email. KRaft is going to keep two sets of
endpoints.

1. Voters set. This set of endpoints is used by voters to establish
leadership with the Vote, BeginQuorumEpoch and EndQuorumEpoch RPCs.
The precedence order is the VotersRecord in the snapshot and log. If
there are no VotersRecords in the snapshot or log, it will use the
configuration in controller.quorum.voters.

2. Bootstrap servers. This is mainly used by observers (brokers) to
discover the leader through Fetch RPCs. The precedence order is
controller.quorum.bootstrap.servers first, controller.quorum.voters
second. Voters won't use this property as they discover the leader and
its endpoint from the BeginQuorumEpoch RPC from the leader.

I believe that the original intent of the voters in quorum state file
was as a cache of the controller.quorum.voters configuration and to
identify invalid changes in the configuration. I was trying to keep
this functionality in the new version of the quorum state file.

The more I think about the implementation, I don't think this is
useful or even implementable. KRaft needs to keep all voters sets from
the latest snapshot to the LEO so that it could include the correct
voters set when generating a snapshot. I am going to update the KIP to
remove voter information from the quorum state file.

> 12. It seems that downgrading from this KIP is not supported? Could we have
> a section to make it explicit?

Yes. Downgrades will not be supported. I updated the "Compatibility,
deprecation and migration plan". There is a sentence about this in the
"Public interfaces / Command line interface / kafka-features" section.
I'll also update the "Proposed changes / Reference explanation /
Supported features" and "Public interfaces / RPCs / UpdateFeatures /
Handling" sections.

> 13. controller.quorum.auto.join.enable: If this is set true, when does the
> controller issue the addVoter RPC? Does it need to wait until it's caught
> up? Does it issue the addVoter RPC on every restart?

The controller will monitor the voters set. If the controller finds
that the voters set doesn't contain itself, it will send an AddVoter
RPC to the leader. To avoid the quorum adding a node and becoming
unavailable, I will change the handling of the AddVoter RPC to not
allow duplicate replica id. If there is any replica with the same id
but a different uuid, the old uuid must be removed first (with the
RemoveVoter RPC) before the new uuid can be added.

Here is an example that shows how a KRaft partition can become
unavailable if we allow automatically adding duplicate replica ids.
Assume that the partition starts with voters v1 and v2. Operator
starts controller 3 as v3 (replica id 3, some generated replica uuid)
and it attempts to automatically join the cluster by sending the
AddVoter RPC. Assume that the voter set (v1, v2, v3) gets committed by
v1 and v2. Controller 3 restarts with a new disk as v3' and sends an
AddVoter RPC. Because (v1, v2, v3) was committed the leader is able to
change the quorum to (v1, v2, v3, v3') but won't be able to commit it
if the controller 3 restarts again and comes back with a new disk as
v3''.

This case is avoided if the KRaft leader rejects any AddVoter RPC that
duplicates the 

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-11 Thread José Armando García Sancio
I updated the KIP to include the information discussed in this thread.
I updated the sections "Reference explanation / Endpoints information"
and "Reference explanation / Leader election".

On Sun, Feb 11, 2024 at 1:06 PM José Armando García Sancio
 wrote:
> Great examples. The short answer is that yes, the leader needs to keep
> sending a BeingQuorumEpoch to a voter until the voter acknowledges it.
> In the current KRaft implementation, the KRaft leader already does
> this (the set of unacknowledged voters is tracked in
> LeaderState::nonAcknowledgeVoters and it used to send the
> BeginQuorumEpoch RPC). The leader continues to send the
> BeginQuorumEpoch RPC until a voter has acknowledged it. When the voter
> handles the BeginQuorumEpoch, it first persists the leader id, leader
> uuid (new in this KIP) and leader epoch to the quorum-state file
> before replying to the RPC. Since the leader's state is persisted
> before replying to the BeginQuorumEpoch RPC, one RPC and
> acknowledgement is enough. Note that if a replica loses its disk and
> quorum state it will come back with a different replica uuid and won't
> be part of the quorum.

This explanation is not entirely correct with this KIP since the
leader's endpoint is not persisted when handling the BeginQuorumEpoch
RPC.

You are correct that the leader needs to send BeginQuorumEpoch
requests to any voter that is not sending Fetch and FetchSnapshot
requests. I included this information in the "Reference explanation /
Leader election".

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-02-11 Thread José Armando García Sancio
Hi Jack, see my comments below.

On Thu, Feb 1, 2024 at 7:26 AM Jack Vanlightly  wrote:
> After thinking it through, it occurs to me that in examples 1 and 2, the
> leader (of the latest configuration) should be sending BeginQuorumEpoch
> requests to r3 after a certain timeout? r3 can start elections (based on
> its stale configuration) which will go nowhere, until it eventually
> receives a BeginQuorumEpoch from the leader and it will learn of the leader
> and resume fetching.
>
> In the case of an observer, I suppose it must fallback to
> 'controller.quorum.voters' or  'controller.quorum.bootstrap.servers' to
> learn of the leader?

Great examples. The short answer is that yes, the leader needs to keep
sending a BeingQuorumEpoch to a voter until the voter acknowledges it.
In the current KRaft implementation, the KRaft leader already does
this (the set of unacknowledged voters is tracked in
LeaderState::nonAcknowledgeVoters and it used to send the
BeginQuorumEpoch RPC). The leader continues to send the
BeginQuorumEpoch RPC until a voter has acknowledged it. When the voter
handles the BeginQuorumEpoch, it first persists the leader id, leader
uuid (new in this KIP) and leader epoch to the quorum-state file
before replying to the RPC. Since the leader's state is persisted
before replying to the BeginQuorumEpoch RPC, one RPC and
acknowledgement is enough. Note that if a replica loses its disk and
quorum state it will come back with a different replica uuid and won't
be part of the quorum.

I think that the other important observation is that, in this KIP,
KRaft will have two sets of endpoints: 1. voters set and 2. bootstrap
servers.

1. The voters set can come from the log as you described or from
controller.quorum.voters if the log doesn't contain any voters sets.
2. The bootstrap servers can come from
controller.quorum.bootstrap.servers or controller.quorum.voters in
that order of preference.

The Fetch RPC will always be sent to the leader's endpoint if it is
known. If the leader is not known, the Fetch RPC will be sent to one
of the endpoints in the bootstrap server set.

The Vote, BeginQuorumEpoch and EndQuorumEpoch will always be sent
using the replicas and endpoints specified in the voters set.

Your example also highlights the importance of "KIP-996: Pre-Vote" to
avoid disruption to the quorum while lagging replicas catch up.

Thanks for your feedback Jack. I'll update the KIP to make this clear.

Thanks,
-- 
-José


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

2024-01-29 Thread José Armando García Sancio
Hi all,

This DISCUSS thread was continue in a new thread at:

[DISCUSS] KIP-853: KRaft Controller Membership Changes:
https://lists.apache.org/thread/6o3sjvcb8dx1ozqfpltb7p0w76b4nd46

Thanks!
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-25 Thread José Armando García Sancio
Hi all,

I have updated the KIP to include information on how KRaft controller
automatic joining will work.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-25 Thread José Armando García Sancio
Hi Jason, Colin and Luke,

I updated the KIP based on your feedback and my comments. Here is what
has changed: 
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=217391519=54=52

Here is an outline of all of the protocol changes:
1. Removed the AddVoterRecord and RemoveVoterRecord control records
and replaced their functionality with VotersRecord instead.
2. VoterUuid in LeaderChangeMessage is now a uuid instead of a int32.
3. Added TimeoutMs field to the AddVoterRequest.
4. Added ClusterId field to the RemoveVoterRequest.
5. Made the ReplicaUuid field in FetchRequest a tagged field.

All of the protocol changes are also documented in this GitHub diff:
https://github.com/apache/kafka/compare/trunk...jsancio:kafka:kraft-membership-protocol

I haven't updated the KIP for adding support for controllers
auto-joining the voters set. I'll do that in the next revision to the
KIP.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-24 Thread José Armando García Sancio
Thanks for the feedback Luke. See my comments below:

On Wed, Jan 24, 2024 at 4:20 AM Luke Chen  wrote:
> 1. About "VotersRecord":
>
> > When a KRaft voter becomes leader it will write a KRaftVersionRecord and
> VotersRecord to the log if the log or the latest snapshot doesn't contain
> any VotersRecord. This is done to make sure that the voter set in the
> bootstrap snapshot gets replicated to all of the voters and to not rely on
> all of the voters being configured with the same bootstrapped voter set.
>
> > This record will also be written to the log if it has never been written
> to the log in the past. This semantic is nice to have to consistently
> replicate the bootstrapping snapshot, at
> -00.checkpoint, of the leader to all of the
> voters.
>
> If the `VotersRecord` has written into
> -00.checkpoint,
> later, a new voter added. Will we write a new checkpoint to the file?
> If so, does that mean the `metadata.log.max.snapshot.interval.ms` will be
> ignored?

KRaft (KafkaRaftClient) won't initiate the snapshot generation. The
snapshot generation will be initiated by the state machine (controller
or broker) using the RaftClient::createSnapshot method. When the state
machine calls into RaftClient::createSnapshot the KafkaRaftClient will
compute the set of voters at the provided offset and epoch, and write
the VotersRecord after the SnapshotHeaderRecord. This does mean that
the KafkaRaftClient needs to store in memory all of the voter set
configurations between the RaftClient::latestSnapshotId and the LEO
for the KRaft partition.

> If not, then how could we make sure the voter set in the bootstrap snapshot
> gets replicated to all of the voters and to not rely on all of the voters
> being configured with the same bootstrapped voter set?

I think my answer above should answer your question. VoterRecord-s
will be in the log (log segments) and the snapshots so they will be
replicated by Fetch and FetchSnapshot. When the voter set is changed
or bootstrapped, the leader will write the VotersRecord to the log
(active log segment). When the state machine (controller or broker)
asks to create a snapshot, KRaft will write the VotersRecord at the
start to the snapshot after the SnapshotHeaderRecord.

> 2. After "RemoveVoter", what is the role of the node?
> It looks like after the voter got removed from the voter set, it is not a
> voter anymore. But I think it can still fetch with the leader. So it should
> be an observer, with a "process.role=controller"? And if the node was
> originally "process.role=controller,broker", it'll become a broker-only
> node?

Kafka nodes need to allow for controllers that are not voters. I don't
expect too many issues from an implementation point of view. Most of
it may just be aggressive validation in KafkaConfig. I think the
easier way to explain this state is that there will be controllers
that will never become active controllers. If we want, we can have a
monitor that turns on (1) if a node is in this state. What do you
think?

> 3. "UpdateVoter" RPC V.S. "ControllerRegistration" RPC?
> To me, the purpose of "UpdateVoter" is basically identical with
> "ControllerRegistration", to register the voter info in the active
> controller/metadata. Should we re-use the existing "ControllerRegistration"
> one?

Yeah. I thought about this option. There are two issues here.
1) The semantics are a bit different. I want the UpdateVoter RPC to
only update the endpoints and kraft.version. I don't want the RPC to
add the controller to the voter set. ControllerRegistration RPC is an
upsert. It creates a registration if it doesn't exist or it updates a
registration if one already exists.

2) These are two different layers. Metadata is the application layers.
KRaft is the consensus (control) layer. To give a precise example, the
controller is determining if it should send a
ControllerRegistrationRequest based on the state of the cluster
metadata (ControllerRegistrationRecord) while KRaft needs to send this
information based on the state of the VotesRecord control record. It
is beneficial to keep these layers separate even if they duplicate
information as it leads to a better implementation that we can test,
simulate and verify.

> 4. > When the kraft.version is upgraded to a version greater 0 and the
> version is supported by the voters, the leader will write a control record
> batch that will include the kraft.version record and all of the AddVoter
> records for all of the voters...
>
> When can't we just write a "VoterRecord" including all voters?

Yes. I need to update the KIP to remove the AddVoterRecord and
RemoveVoterRecord control records. Jason had a similar question and I
provided this answer just to give some context on the current state:

"Yes. We can remove them and I'll remove them. For context, I
originally only had AddVoterRecord and RemoveVoterRecord. When
fleshing out the details of the bootstrapping process, I realized 

Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-22 Thread José Armando García Sancio
Thanks Jason. Comments below.

On Wed, Jan 10, 2024 at 9:06 AM Jason Gustafson
 wrote:
> One additional thought. It would be helpful to have an example to justify
> the need for this:
>
> > Wait for the fetch offset of the replica (ID, UUID) to catch up to the
> log end offset of the leader.
>
> It is helpful also to explain how this affects the AddVoter RPC. Do we wait
> indefinitely? Or do we give up and return a timeout error if the new voter
> cannot catch up? Probably the latter makes the most sense.

Yes. I will have more details here. Jason and I discussed this offline
but waiting for the new replica to catch (to the LEO) is a heuristic
that would minimize the amount of time where the leader cannot
increase the HWM because the new replica is needed to form the
majority. A example that shows this is:

Current Voter Set:
A: offset = 100
B: offset = 100
C: offset = 0

In this configuration the leader can continue to advance the HWM since
the majority A, B is at the HWM/LEO.

If the user now adds a voter to the voter set:
A: offset = 100
B: offset = 100
C: offset = 0
D: offset = 0

The leader cannot advance the HWM until either C or D catches up to
the HWM because the majority has to include one of either C, D or
both.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-22 Thread José Armando García Sancio
Thanks for your feedback Jason and excuse the delayed response.

See comments below.

On Tue, Jan 9, 2024 at 5:08 PM Jason Gustafson
 wrote:
>
> Hi Jose,
>
> Thanks for the KIP! A few initial questions below:
>
> 1. In the user experience section, the user is expected to provide supply
> the UUID for each voter. I'm assuming this is the directory.id coming from
> KIP-858. I thought it was generated by the format comand automatically? It
> seems like we will need a way to specify it explicitly.

Yes. It is currently using a random uuid. I was planning to update
that code to instead use the uuid provided in the
--controller-quorum-voters flag. For example, if the node.id in the
--config file matches one of the replica-id in
--controller-quorum-voters, use the specified replica-uuid for the
directory.id of the metadata.log.dir.

> 2. Do we need the AddVoters and RemoveVoters control records? Perhaps the
> VotersRecord is sufficient since changes to the voter set will be rare.

Yes. We can remove them and I'll remove them. For context, I
originally only had AddVoterRecord and RemoveVoterRecord. When
fleshing out the details of the bootstrapping process, I realized that
I needed a control record that would override the voter set and not do
an incremental change. I needed this functionality because there is no
guarantee that the content of the bootstrap checkpoint
(...-....checkpoint) matches in all of the voters.

After I added the VotersRecord, my thinking for keeping AddVoterRecord
and RemoveVoterRecord was to make it explicitly that the protocol only
allows changing one voter at a time. I can instead write a comparison
function that KRaft can use whenever it attempts to write or read a
VotersRecord. The comparison function would fail if all of the
possible majorities of the old voter set don't intersect with all of
the possible majority of the new voter set.

What do you think?

> 3. Why does UpdateVoter need to commit after every leader change?

My thinking is that this algorithm is easier to implement.
Technically, the following (fetching) voter only needs to send an
UpdateVoter RPC when the endpoints known by the leader don't match the
latest endpoint for the voter. This is not something that the follower
can know reliably. This is why I prefered to add an idempotent RPC
like UpdateVoter RPC that the follower voter can perform aggressively
against the leader when the voter discovers a leader.

> 4. Should ReplicaUuid in FetchRequest be a tagged field? It seems like a
> lot of overhead for all consumer fetches.

Yes. I'll make it a tagged field. For now it will only be used by
KRaft. In the future, I can see the broker wanting to use this field
to implement JBOD "reassignment". I don't think consumers will ever
use this field.

> 5. I was looking for the correctness conditions when changing the voter
> set. I recalled something about always taking the voter set from the log
> even if it is not yet known to be committed. Do you have those details
> documented?

Yes. That's correct.

The section Reference Explanation / Voter Changes / Adding Voters has:
"If the new leader supports the current kraft.version, it will write a
AddVoterRecord to the log and immediately update its in-memory quorum
state to include this voter as part of the quorum."

The section Public Interface / Log and Snapshot control records /
AddVoterRecord has:
"KRaft replicas will read all of the control records in the snapshot
and the log irrespective of the commit state and HWM. When a replica
encounters an AddVoterRecord it will add the replica ID and UUID to
its voter set."

The section Public Interface / RPCs / AddVoter / Handling has:
"6. Append the AddVoterRecord to the log.
7. The KRaft internal listener will read this record from the log and
add the voter to the voter set.
8. Wait for the AddVoterRecord to commit using the majority of new voter set."

Thanks,
-- 
-José


Re: [DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-22 Thread José Armando García Sancio
Thanks for the feedback Colin. Comments below.

On Tue, Jan 9, 2024 at 4:58 PM Colin McCabe  wrote:
> 1. restarting a controller with an empty storage directory
>
> The controller can contact the quorum to get the cluster ID and current MV. 
> If the MV doesn't support quorum reconfiguration, it can bail out. Otherwise, 
> it can re-format itself with a random directory ID. It can then remove (ID, 
> OLD_DIR_ID) from the quorum, and add (ID, NEW_DIR_ID) to the quorum.
>
> I think this can all be done automatically without user intervention. If the 
> remove / add steps fail (because the quorum is down, for example), then of 
> course we can just log an exception and bail out.
>

Yes. We should be able to implement this. I'll update the KIP and add
another configuration: controller.quorum.auto.join.enabled=true

The high-level algorithm is something like this:
1. The controllers will fetch the latest quorum state from the Leader
2. The controller will remove any voter that matches its replica id
but doesn't match its directory id (replica uuid).
3. If the controller (replica id and replica uuid) is not in the voter
set it sends a AddVoter RPC to the controller until it sees itself in
the voter set.

> 2. restarting a broker with an empty storage directory
>
> The broker can contact the quorum to get the cluster ID and current MV. If 
> the MV doesn't support directory IDs, we can bail out. Otherwise, it can 
> reformat itself with a random directory ID and start up. Its old replicas 
> will be correctly treated as gone due to the JBOD logic.

This feature seems reasonable to me. I don't think we should make this
part of this KIP. It should be a seperate KIP as it is not related to
controller dynamic membership changes.

> 4. Bringing up a totally new cluster
>
> I think we need at least one controller node to be formatted, so that we can 
> decide what metadata version to use. Perhaps we should even require a quorum 
> of controller nodes to be explicitly formatted (aka, in practice, people just 
> format them all).

Yes. When I document this feature my recommended process would be:
1. One of the controllers needs to be formatted in --standalone
(kafka-storage format --cluster-id  --release-version 3.8
--standalone --config controller.properties). This needs to be an
explicit operation as it violates one of the invariants enumerated in
the KIP.
"To make changes to the voter set safe it is required that the
majority of the competing voter sets commit the voter changes. In this
design the competing voter sets are the current voter set and new
voter set. Since this design only allows one voter change at a time
the majority of the new configuration always overlaps (intercepts) the
majority of the old configuration. This is done by the leader
committing the current epoch when it becomes leader and committing
single voter changes with the new voter set before accepting another
voter change."

An easy example that shows the issue with auto formatting is:
1. Voter set is (1) by running --standalone.
2. Voter set to (1, 2, 3) after two AddVoter RPCs (either manually but
by auto joining).
3. Voter 1 loses its disk reformats back to (1). Now it is possible to
have two quorums one with just the replica 1 and one with the replicas
2 and 3.


Thanks. I'll update the KIP shortly to reflect my comments above,
--
-José


[DISCUSS] KIP-853: KRaft Controller Membership Changes

2024-01-08 Thread José Armando García Sancio
Hi all,

KIP-853: KRaft Controller Membership Changes is ready for another
round of discussion.

There was a previous discussion thread at
https://lists.apache.org/thread/zb5l1fsqw9vj25zkmtnrk6xm7q3dkm1v

I have changed the KIP quite a bit since that discussion. The core
idea is still the same. I changed some of the details to be consistent
with some of the protocol changes to Kafka since the original KIP. I
also added a section that better describes the feature's UX.

KIP: https://cwiki.apache.org/confluence/x/nyH1D

Thanks. Your feedback is greatly appreciated!
-- 
-José


Re: [ANNOUNCE] New Kafka PMC Member: Divij Vaidya

2023-12-28 Thread José Armando García Sancio
Congratulations Divij!

On Thu, Dec 28, 2023 at 10:48 AM Divij Vaidya  wrote:
>
> Thank you everyone for your warm wishes 
>
> --
> Divij Vaidya
>
>
>
> On Thu, Dec 28, 2023 at 2:37 PM Yash Mayya  wrote:
>
> > Congratulations Divij!
> >
> > On Wed, Dec 27, 2023 at 5:15 PM Luke Chen  wrote:
> >
> > > Hi, Everyone,
> > >
> > > Divij has been a Kafka committer since June, 2023. He has remained very
> > > active and instructive in the community since becoming a committer. It's
> > my
> > > pleasure to announce that Divij is now a member of Kafka PMC.
> > >
> > > Congratulations Divij!
> > >
> > > Luke
> > > on behalf of Apache Kafka PMC
> > >
> >



-- 
-José


Re: [DISCUSS] Road to Kafka 4.0

2023-12-26 Thread José Armando García Sancio
Hi Divij,

Thanks for the feedback. I agree that having a 3.8 release is
beneficial but some of the comments in this message are inaccurate and
could mislead the community and users.

On Thu, Dec 21, 2023 at 7:00 AM Divij Vaidya  wrote:
> 1\ Durability/availability bugs in kraft - Even though kraft has been
> around for a while, we keep finding bugs that impact availability and data
> durability in it almost with every release [1] [2]. It's a complex feature
> and such bugs are expected during the stabilization phase. But we can't
> remove the alternative until we see stabilization in kraft i.e. no new
> stability/durability bugs for at least 2 releases.

I took a look at both of these issues and neither of them are bugs
that affect KRaft's durability and availability.

> [1] https://issues.apache.org/jira/browse/KAFKA-15495

This issue is not specific to KRaft and has been an issue in Apache
Kafka since the ISR leader election and replication algorithm was
added to Apache Kafka. I acknowledge that this misunderstanding is
partially due to the Jira description which insinuates that this only
applies to KRaft which is not true.

> [2] https://issues.apache.org/jira/browse/KAFKA-15489

First, technically this issue was not first discovered in some recent
release. This issue was identified by me back in January of 2022:
https://issues.apache.org/jira/browse/KAFKA-13621. I decided to lower
the priority as it requires a very specific network partition where
the controllers are partitioned from the current leader but the
brokers are not.

This is not a durability bug as the KRaft cluster metadata partition
leader will not be able to advance the HWM and hence commit records.

Regarding availability, The KRaft's cluster metadata partition favors
consistency and partition tolerance versus availability from CAP. This
is by design and not a bug in the protocol or implementation.

> 2\ Parity with Zk - There are also pending bugs [3] which are in the
> category of Zk parity. Removing Zk from Kafka without having full feature
> parity with Zk will leave some Kafka users with no upgrade path.
> 3\ Test coverage - We also don't have sufficient test coverage for kraft
> since quite a few tests are Zk only at this stage.
>
> Given these concerns, I believe we need to reach 100% Zk parity and allow
> new feature stabilisation (such as scram, JBOD) for at least 1 version
> (maybe more if we find bugs in that feature) before we remove Zk. I also
> agree with the point of view that we can't delay 4.0 indefinitely and we
> need a clear cut line.

There seems to be some misunderstanding regarding Apache Kafka
versioning scheme. Minor versions (e.g. 3.x) are needed for feature
releases like new RPCs and configurations. They are not needed for bug
fixes. Bug fixes can and should be done in patch releases (e.g.
3.7.x).

This means that you don't need a 3.8 or 3.9 release to fix a bug in Kafka.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-1013: Drop broker and tools support for Java 11 in Kafka 4.0 (deprecate in 3.7)

2023-12-26 Thread José Armando García Sancio
Hi Ismael,

Looks good to me. Looking forward to programming using features and
types included in JDK17 in 4.0 and not having to program using a 10
year old programming language and library.

Thanks!
-- 
-José


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

2023-12-20 Thread José Armando García Sancio
Yes. I have been working on the KIP. I don't expect to change too much
from what is currently documented. I am mainly focusing on improving
the UX and readability of the KIP.

I am hoping to be able to restart the discussion first thing next year.

Thanks

On Wed, Dec 20, 2023 at 6:11 AM Josep Prat  wrote:
>
> Hi Jose,
> Did you have any time to update the KIP? Let me know if we can help in any
> way.
>
> Thanks in advance,
>
> On Mon, Nov 27, 2023 at 7:04 PM José Armando García Sancio
>  wrote:
>
> > On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> > > I wanted to revive this thread and see if there is anything preventing
> > it to be voted on. Happy to help unblocking anything that might be holding
> > this back.
> >
> > Hi Josep,
> >
> > Thanks for reviving the thread. I need to make some changes to the
> > KIP. My thinking has changed a bit since I wrote this KIP. The core of
> > the design still holds. I just want to improve the wording and
> > usability.
> >
> > I should have an updated KIP this week. I'll restart the discussion
> > thread at that point.
> >
> > Thanks,
> > --
> > -José
> >
>
>
> --
> [image: Aiven] <https://www.aiven.io>
>
> *Josep Prat*
> Open Source Engineering Director, *Aiven*
> josep.p...@aiven.io   |   +491715557497
> aiven.io <https://www.aiven.io>   |   <https://www.facebook.com/aivencloud>
>   <https://www.linkedin.com/company/aiven/>   <https://twitter.com/aiven_io>
> *Aiven Deutschland GmbH*
> Alexanderufer 3-7, 10117 Berlin
> Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> Amtsgericht Charlottenburg, HRB 209739 B



-- 
-José


Re: [VOTE] KIP-996: Pre-Vote

2023-12-08 Thread José Armando García Sancio
+1.

Thanks for the KIP. Looking forward to the implementation!

-- 
-José


Re: [DISCUSS] KIP-996: Pre-Vote

2023-12-07 Thread José Armando García Sancio
Hi Alyssa,

Thanks for the answers and the updates to the KIP. I took a look at
the latest version and it looks good to me.

-- 
-José


Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-29 Thread José Armando García Sancio
Hi Alyssa,

1. In the schema for VoteRequest and VoteResponse, you are using
"boolean" as the type keyword. The correct keyword should be "bool"
instead.

2. In the states and state transaction table you have the following entry:
>  * Candidate transitions to:
> *...
> *Prospective: After expiration of the election timeout

Can you explain the reason a candidate would transition back to
prospective? If a voter transitions to the candidate state it is
because the voters don't support KIP-996 or the replica was able to
win the majority of the votes at some point in the past. Are we
concerned that the network partition might have occurred after the
replica has become a candidate? If so, I think we should state this
explicitly in the KIP.

3. In the proposed section and state transition section, I think it
would be helpful to explicitly state that we have an invariant that
only the prospective state can transition to the candidate state. This
transition to the candidate state from the prospective state can only
happen because the replica won the majority of the votes or there is
at least one remote voter that doesn't support pre-vote.

4. I am a bit confused by this paragraph
> A candidate will now send a VoteRequest with the PreVote field set to true 
> and CandidateEpoch set to its [epoch + 1] when its election timeout expires. 
> If [majority - 1] of VoteResponse grant the vote, the candidate will then 
> bump its epoch up and send a VoteRequest with PreVote set to false which is 
> our standard vote that will cause state changes for servers receiving the 
> request.

I am assuming that "candidate" refers to the states enumerated on the
table above this quote. If so, I think you mean "prospective" for the
first candidate.

CandidateEpoch should be ReplicaEpoch.

[epoch + 1] should just be epoch. I thought we agreed that replicas
will always send their current epoch to the remote replicas.

5. I am a bit confused by this bullet section
> true if the server receives less than [majority] VoteResponse with 
> VoteGranted set to false within [election.timeout.ms + a little randomness] 
> and the first bullet point does not apply
 Explanation for why we don't send a standard vote at this point
is explained in rejected alternatives.

Can we explain this case in plain english? I assume that this case is
trying to cover the scenario where the election timer expired but the
prospective candidate hasn't received enough votes (granted or
rejected) to make a decision if it could win an election.

6.
> Yes. If a leader is unable to receive fetch responses from a majority of 
> servers, it can impede followers that are able to communicate with it from 
> voting in an eligible leader that can communicate with a majority of the 
> cluster.

In general, leaders don't receive fetch responses. They receive FETCH
requests. Did you mean "if a leader is able to send FETCH responses to
the majority - 1 of the voters, it can impede fetching voters
(followers) from granting their vote to prospective candidates. This
should stop prospective candidates from getting enough votes to
transition to the candidate state and increase their epoch".

7.
> Check Quorum ensures a leader steps down if it is unable to receive fetch 
> responses from a majority of servers.

I think you mean "... if it is unable to receive FETCH requests from
the majority - 1 of the voters".

8. At the end of the Proposed changes section you have the following:
> The logic now looks like the following for servers receiving VoteRequests 
> with PreVote set to true:
>
> When servers receive VoteRequests with the PreVote field set to true, they 
> will respond with VoteGranted set to
>
> * true if they are not a Follower and the epoch and offsets in the Pre-Vote 
> request satisfy the same requirements as a standard vote
> * false if they are a Follower or the epoch and end offsets in the Pre-Vote 
> request do not satisfy the requirements

This seems to duplicate the same algorithm that was stated earlier in
the section.

9. I don't understand this rejected idea: Sending Standard Votes after
failure to win Pre-Vote

In your example in the "Disruptive server scenarios" voters 4 and 5
are partitioned from the majority of the voters. We don't want voters
4 and 5 increasing their epoch and transitioning to the candidate
state else they would disrupt the quorum established by voters 1, 2
and 3.


Thanks,
-- 
-José


Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-28 Thread José Armando García Sancio
Hi Alyssa

On Mon, Nov 27, 2023 at 1:40 PM Jason Gustafson
 wrote:
> 2. Do you think the pretend epoch bump is necessary? Would it be simpler to
> change the prevote acceptance check to assert a greater than or equal epoch?

I agree with Jason it would be better if all of the requests always
sent the current epoch. For the VoterRequest, it should be correct for
the prospective node to not increase the epoch and send the current
epoch and id. Since there are two states (prospective and candidate)
that can send a VoteRequest, maybe we can change the field name to
just ReplicaEpoch and ReplicaId.

Thanks,
-- 
-José


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

2023-11-27 Thread José Armando García Sancio
On Mon, Nov 27, 2023 at 2:32 AM Josep Prat  wrote:
> I wanted to revive this thread and see if there is anything preventing it to 
> be voted on. Happy to help unblocking anything that might be holding this 
> back.

Hi Josep,

Thanks for reviving the thread. I need to make some changes to the
KIP. My thinking has changed a bit since I wrote this KIP. The core of
the design still holds. I just want to improve the wording and
usability.

I should have an updated KIP this week. I'll restart the discussion
thread at that point.

Thanks,
-- 
-José


Re: [VOTE] KIP-896: Remove old client protocol API versions in Kafka 4.0

2023-11-21 Thread José Armando García Sancio
Thanks. LGTM. +1.

On Tue, Nov 21, 2023 at 2:54 PM Jun Rao  wrote:
>
> Hi, Ismael,
>
> Thanks for the KIP. +1
>
> It would be useful to clarify in the KIP that the new metric and new
> request log attribute will be added in Apache 3.7.
>
> Jun
>
> On Tue, Nov 21, 2023 at 1:57 PM Colin McCabe  wrote:
>
> > Ah. I forget that KIP-724 not only deprecated, but proposed a removal in
> > 4.0. Great.
> >
> > +1 (binding) for KIP-896
> >
> > best,
> > Colin
> >
> > On Tue, Nov 21, 2023, at 12:36, Ismael Juma wrote:
> > > Hi Colin,
> > >
> > > That change was proposed and approved via KIP-724:
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support+for+message+formats+v0+and+v1
> > >
> > > Ismael
> > >
> > > On Tue, Nov 21, 2023, 12:21 PM Colin McCabe  wrote:
> > >
> > >> Hi Ismael,
> > >>
> > >> Can we state somewhere that the message.format.version configuration
> > will
> > >> be gone in 4.0? We only will support one message format version (for
> > now,
> > >> at least). If we do want more versions later, I don't think we'll want
> > to
> > >> configure them via a static config.
> > >>
> > >> best,
> > >> Colin
> > >>
> > >>
> > >> On Tue, Nov 21, 2023, at 12:06, Ismael Juma wrote:
> > >> > Hi all,
> > >> >
> > >> > I would like to start a vote on KIP-896. Please take a look and let us
> > >> know
> > >> > what you think.
> > >> >
> > >> > Even though most of the changes in this KIP will be done for Apache
> > Kafka
> > >> > 4.0, I would like to introduce a new metric and new request log
> > attribute
> > >> > in Apache 3.7 to help users identify usage of deprecated protocol api
> > >> > versions.
> > >> >
> > >> > Link:
> > >> >
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-896%3A+Remove+old+client+protocol+API+versions+in+Kafka+4.0
> > >> >
> > >> > Thanks,
> > >> > Ismael
> > >>
> >



-- 
-José


Re: [VOTE] KIP-1001; CurrentControllerId Metric

2023-11-21 Thread José Armando García Sancio
LGTM. +1 binding.

On Mon, Nov 20, 2023 at 1:48 PM Jason Gustafson
 wrote:
>
> The KIP makes sense. +1
>
> On Mon, Nov 20, 2023 at 12:37 PM David Arthur
>  wrote:
>
> > Thanks Colin,
> >
> > +1 from me
> >
> > -David
> >
> > On Tue, Nov 14, 2023 at 3:53 PM Colin McCabe  wrote:
> >
> > > Hi all,
> > >
> > > I'd like to call a vote for KIP-1001: Add CurrentControllerId metric.
> > >
> > > Take a look here:
> > > https://cwiki.apache.org/confluence/x/egyZE
> > >
> > > best,
> > > Colin
> > >
> >
> >
> > --
> > -David
> >



-- 
-José


Re: [DISCUSS] Road to Kafka 4.0

2023-11-20 Thread José Armando García Sancio
Hi all,

If we do a 3.8 release before 4.0 and we implement KIP-853 in 3.8, the
user will be able to migrate to a KRaft cluster that supports
dynamically changing the set of voters and has better support for disk
failures.

What are the disadvantages of adding the 3.8 release before 4.0? This
would push the 4.0 release by 3-4 months. From what I can tell, it
would also delay when KIP-896 can be implemented and extend how long
the community needs to maintain the code used by ZK mode. Is there
anything else?

Thanks
-- 
-José


Re: UncleanLeaderElectionsPerSec metric and Raft

2023-10-24 Thread José Armando García Sancio
Hi Neil,

You are correct. I don't think we have implemented the unclean
election configuration.

The kafka-leader-election CLI works with KRaft. Have you looked at
that option for performing unclean leader elections with the CLI? Does
it meet your requirements?

In general using the unclean leader election config is extremely
unsafe and I recommend using the CLI if it meets your requirements.

Thanks,
José

On Wed, Oct 18, 2023 at 10:28 AM Neil Buesing  wrote:
>
> Development,
>
> with Raft controllers, is the unclean leader election / sec metric supose
> to be available?
>
> kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec
>
> Nothing in documentation indicates that it isn’t as well as in code
> navigation nothing indicates to me that it wouldn’t show up, but even added
> unclean leader election to true for both brokers and controllers and
> nothing.
>
> (set this for all controllers and brokers)
>   KAFKA_UNCLEAN_LEADER_ELECTION_ENABLE: true
>
> Happy to report a Jira, but wanted to figure out if the bug was in the
> documentation or the metric not being available?
>
> Thanks,
>
> Neil
>
> P.S. I did confirm that others have seen and wondered about this,
> https://github.com/strimzi/strimzi-kafka-operator/issues/8169, but that is
> about the only other report on this I have found.



-- 
-José


Re: [kafka-clients] [VOTE] 3.6.0 RC1

2023-09-25 Thread José Armando García Sancio
On Sat, Sep 23, 2023 at 3:08 AM Luke Chen  wrote:
>
> Hi Satish,
>
> I found the current KRaft implementation will have "split brain" issue when
> network partition happens, which will cause inconsistent metadata returned
> from the controller.
> Filed KAFKA-15489  for
> this issue, and PR  is ready
> for review.
>
> Even though this is not a regression issue (this has already existed since
> the 1st release of KRaft feature), I think this is an important issue since
> KRaft is announced production ready.
> Not sure what other people's thoughts are.

Thanks for the report and PR Luke. This looks related to this issue:
https://issues.apache.org/jira/browse/KAFKA-13621

Do you agree? We can move our conversation to those issues but I also
agree that I don't think this issue should be a release blocker.

Thanks!
-José


Re: Apache Kafka 3.6.0 release

2023-09-07 Thread José Armando García Sancio
Hi Satish,

On Wed, Sep 6, 2023 at 4:58 PM Satish Duggana  wrote:
>
> Hi Greg,
> It seems https://issues.apache.org/jira/browse/KAFKA-14273 has been
> there in 3.5.x too.

I also agree that it should be a blocker for 3.6.0. It should have
been a blocker for those previous releases. I didn't fix it because,
unfortunately, I wasn't aware of the issue and jira.
I'll create a PR with a fix in case the original author doesn't respond in time.

Satish, do you agree?

Thanks!
-- 
-José


Re: Unable to start the Kafka with Kraft in Windows 11

2023-09-07 Thread José Armando García Sancio
Thanks for bringing this to my attention. I agree that it should be a blocker.

On Wed, Sep 6, 2023 at 9:41 AM Greg Harris  wrote:
>
> Hi Ziming,
>
> Thanks for finding that! I've mentioned that in the 3.6.0 release
> thread as a potential blocker since this appears to have a pretty
> substantial impact.
>
> Hey Sumanshu,
>
> Thank you so much for bringing this issue to our attention! It appears
> that your issue is caused by a bug in Kafka, so you shouldn't feel
> obligated to answer my questions from earlier.
> We'll see about trying to get a fix for this issue in the upcoming
> release. I apologize that the released versions of KRaft don't work on
> windows, and are preventing you from evaluating it. You will need to
> use Zookeeper clusters, or run custom builds of Kafka until the fix is
> released.
>
> Thanks,
> Greg
>
> On Tue, Sep 5, 2023 at 7:44 PM ziming deng  wrote:
> >
> > It seems this is related to KAFKA-14273, there is already a pr for this 
> > problem, but it’s not merged.
> >  https://github.com/apache/kafka/pull/12763
> >
> > --
> > Ziming
> >
> > > On Sep 6, 2023, at 07:25, Greg Harris  
> > > wrote:
> > >
> > > Hey Sumanshu,
> > >
> > > Thanks for trying out Kraft! I hope that you can get it working :)
> > >
> > > I am not familiar with Kraft or Windows, but the error appears to
> > > mention that the file is already in use by another process so maybe we
> > > can start there.
> > >
> > > 1. Have you verified that no other Kafka processes are running, such
> > > as in the background or in another terminal?
> > > 2. Are you setting up multiple Kafka brokers on the same machine in your 
> > > test?
> > > 3. Do you see the error if you restart your machine before starting Kafka?
> > > 4. Do you see the error if you delete the log directory and format it
> > > again before starting Kafka?
> > > 5. Have you made any changes to the `server.properties`, such as
> > > changing the log directories? (I see that the default is
> > > `/tmp/kraft-combined-logs`, I don't know if that is a valid path for
> > > Windows).
> > >
> > > Thanks,
> > > Greg
> > >
> > > On Mon, Sep 4, 2023 at 2:21 PM Sumanshu Nankana
> > >  wrote:
> > >>
> > >> Hi Team,
> > >>
> > >> I am following the steps mentioned here 
> > >> https://kafka.apache.org/quickstart to Install the Kafka.
> > >>
> > >> Windows 11
> > >> Kafka Version 
> > >> https://www.apache.org/dyn/closer.cgi?path=/kafka/3.5.0/kafka_2.13-3.5.0.tgz
> > >> 64 Bit Operating System
> > >>
> > >>
> > >> Step1: Generate the Cluster UUID
> > >>
> > >> $KAFKA_CLUSTER_ID=.\bin\windows\kafka-storage.bat random-uuid
> > >>
> > >> Step2: Format Log Directories
> > >>
> > >> .\bin\windows\kafka-storage.bat format -t $KAFKA_CLUSTER_ID -c 
> > >> .\config\kraft\server.properties
> > >>
> > >> Step3: Start the Kafka Server
> > >>
> > >> .\bin\windows\kafka-server-start.bat .\config\kraft\server.properties
> > >>
> > >> I am getting the error. Logs are attached
> > >>
> > >> Could you please help me to sort this error.
> > >>
> > >> Kindly let me know, if you need any more information.
> > >>
> > >> -
> > >> Best
> > >> Sumanshu Nankana
> > >>
> > >>



-- 
-José


Re: [DISCUSS] Apache Kafka 3.4.1 release

2023-08-08 Thread José Armando García Sancio
Hey Luke,

Thanks for working on the release for 3.4.1. I was working on some
cherry picks and I noticed that branch 3.4 doesn't contain the
commit/tag for 3.4.1. I think we are supposed to merge the tag back to
the 3.4 branch. E.g.:

> Merge the last version change / rc tag into the release branch and bump the 
> version to 0.10.0.1-SNAPSHOT
>
> git checkout 0.10.0
> git merge 0.10.0.0-rc6

from: https://cwiki.apache.org/confluence/display/KAFKA/Release+Process

Did we forget to do that part?

Thanks!
-- 
-José


Re: [VOTE] KIP-951: Leader discovery optimisations for the client

2023-07-27 Thread José Armando García Sancio
The KIP LGTM. Thanks for the design. I am looking forward to the implementation.

+1 (binding).

Thanks!
-- 
-José


Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-24 Thread José Armando García Sancio
Hi Mayank,

On Mon, Jul 24, 2023 at 8:21 AM Mayank Shekhar Narula
 wrote:
>
> Thanks Jose/David/Ismael for your inputs.
>
> Not bumping the version, would require both broker & client to backport
> changes. Especially for FetchResponse, as backporting would have to be done
> all the way back to 3.1, so this effort isn't trivial, and was originally
> underestimated.
>
> Considering backporting effort, and given it does make it easier to reason
> about the client-server interaction, I will bump the version. For Produce &
> Fetch, new bumped versions are going to be 10 & 16 respectively.
>
> Do note that FetchResponse has currentLeader introduced at version 12, but
> only Java client at version 16 will support it. But that's the best we can
> do.
>

Makes sense to me. Thanks for the changes!


Re: Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-24 Thread José Armando García Sancio
Hey Mayank,

It is probably binary compatible to have the NodeEndponts fielld at
taggedVersion 12+ but I think it is misleading as a code reviewer. The
Java Kafka client at version 12 will never be able to handle those
fields. Or are you planning to backport these improvements to those
clients and brokers? If not, can we set the version and taggedVersion
to 15+?

Thanks,
-- 
-José


Re: KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum

2023-07-21 Thread José Armando García Sancio
Thanks for the KIP Colin. Apologies if some of these points have
already been made. I have not followed the discussion closely:

1. Re: Periodically, each controller will check that the controller
registration for its ID is as expected

Does this need to be periodic? Can't the controller schedule this RPC,
retry etc, when it finds that the incarnation ID doesn't match its
own?

2. Did you consider including the active controller's epoch in the
ControllerRegistrationRequest?

This would allow the active controller to reject registration from
controllers that are not part of the active quorum and don't know the
latest controller epoch. This should mitigate some of the concerns you
raised in bullet point 1.

3. Which endpoint will the inactive controllers use to send the
ControllerRegistrationRequest?

Will it use the first endpoint described in the cluster metadata
controller registration record? Or would it use the endpoint described
in the server configuration at controller.quorum.voters?

4. Re: Raft integration in the rejected alternatives

Yes, The KRaft layer needs to solve a similar problem like endpoint
discovery to support dynamic controller membership change. As you
point out the requirements are different and the set of information
that needs to be tracked is different. I think it is okay to use a
different solution for each of these problems.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-17 Thread José Armando García Sancio
Hi Krik,

On Fri, Jul 14, 2023 at 10:59 AM Kirk True  wrote:
> Is the requested restructuring of the response “simply” to preserve bytes, or 
> is it possible that the fetch response could/should/would return leadership 
> changes for partitions that we’re specifically requested?

Both. My reasoning for the restructuring is that embedding the node
endpoint in the partition response would lead to duplicate information
being returned and as you point out the node endpoint information is
orthogonal to the partition leader.

> > 3. In the future, I may use this information in the KRaft/Metadata
> > implementation of FETCH. In that implementation not all of the
> > replicas are brokers.
>
> Side point: any references to the change you’re referring to? The idea of 
> non-brokers serving as replicas is blowing my mind a bit :)

I am especially referring to the Draft KIP for KRaft Controller
Membership Change (https://cwiki.apache.org/confluence/x/nyH1D). The
Fetch RPC is used by KRaft's cluster metadata partition which
implements a different consensus protocol that is used by both
Controllers and Brokers.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-14 Thread José Armando García Sancio
Hi Mayank,

On Thu, Jul 13, 2023 at 10:03 AM Mayank Shekhar Narula
 wrote:
> 3. If I understood this correctly, certain replicas "aren't" brokers, what
> are they then?

In a Kafka KRaft cluster they can be either brokers, controllers or
both. The term we use is node. A Kafka node can be either a broker,
controller or both. For example, we have the following configuration
documentation:

node.id
The node ID associated with the roles this process is playing when
`process.roles` is non-empty. This is required configuration when
running in KRaft mode.

>From https://kafka.apache.org/documentation/#brokerconfigs_node.id

> Also how about replacing "Replica" with "Leader", this is more readable on
> the client. so, how about this?

I vote for "Node". Note that this array is a mapping of replica/node
id to their endpoint. For example, in the future we may use it to send
endpoint information when sending the replica id to fetch from a
follower.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-13 Thread José Armando García Sancio
Hi Andrew,

On Thu, Jul 13, 2023 at 8:35 AM Andrew Schofield
 wrote:
> I have a question about José’s comment (2). I can see that it’s possible for 
> multiple
> partitions to change leadership to the same broker/node and it’s wasteful to 
> repeat
> all of the connection information for each topic-partition. But, I think it’s 
> important to
> know which partitions are now lead by which node. That information at least 
> needs to be
> per-partition I think. I may have misunderstood, but it sounded like your 
> comment
> suggestion lost that relationship.

Each partition in both the FETCH response and the PRODUCE response
will have the CurrentLeader, the tuple leader id and leader epoch.
Clients can use this information to update their partition to leader
id and leader epoch mapping.

They can also use the NodeEndpoints to update their mapping from
replica id to the tuple host, port and rack so that they can connect
to the correct node for future FETCH requests and PRODUCE requests.

Thanks,
-- 
-José


Re: [DISCUSS] KIP-951: Leader discovery optimisations for the client

2023-07-13 Thread José Armando García Sancio
Hi Mayank, thanks for the KIP. I look forward to this improvement for
new clients.

Some comments below.

On Thu, Jul 13, 2023 at 7:15 AM Mayank Shekhar Narula
 wrote:
> Following KIP is up for discussion. Thanks for your feedback

Regarding the FETCH response changes:
1. Tagged field 2 already exists. Looks like you can use tagged field 3.

2. The CurrentLeaderBroker should not be within PartitionData. It is
possible to have multiple partitions that would change leadership to
the same broker/node. We should instead move that information to a
top-level field that is an array of (replica id, host, port, rack).

3. In the future, I may use this information in the KRaft/Metadata
implementation of FETCH. In that implementation not all of the
replicas are brokers. Do you mind removing all references to the word
broker in the description and field name. Maybe you can use the word
replica instead. How about something like this:
{ "name": "NodeEndpoints", "type": "[]NodeEndpoint", "versions":
"15+", "taggedVersions": "15+", "tag": 3,
  "about": "Endpoint information for all current leaders
enumerated in PartitionData.", "fields": [
  { "name": "ReplicaId", "type": "int32", "versions": "15+",
"mapKey": true, "entityType": "brokerId",
"about": "The ID of the associated replica"},
  { "name": "Host", "type": "string", "versions": "15+",
"about": "The replica's hostname." },
  { "name": "Port", "type": "int32", "versions": "15+",
"about": "The replica's port." },
  { "name": "Rack", "type": "string", "versions": "15+",
"ignorable": true, "default": "null",
"about": "The rack of the replica, or null if it has not
been assigned to a rack." }
  ]},

Regarding the PRODUCE response changes:
4. Can we make similar changes as the ones mentioned in bullet points
2. and 3 above?.

5. If you make the changes enumerated in bullet point 4., you'll
probably want to change the tag so that NodeEpoint has tag 0 while
CurrentLeader has tag 1.

Thanks!
-- 
-José


Re: [VOTE] KIP-909: DNS Resolution Fallures Should Not Fail the Client

2023-04-25 Thread José Armando García Sancio
+1. Thanks for the design. Looking forward to the implementation.

On Tue, Apr 25, 2023 at 10:49 AM Jason Gustafson
 wrote:
>
> +1 Thanks Philip!
>
>
> On Thu, Apr 13, 2023 at 7:49 AM Kirk True  wrote:
>
> > +1 (non-binding)
> >
> > > On Apr 10, 2023, at 1:53 PM, Philip Nee  wrote:
> > >
> > > Hey everyone!
> > >
> > > I'm starting a vote for KIP-909: DNS Resolution Fallures Should Not Fail
> > > the Client 
> > >
> > > Please refer to the discussion thread here:
> > > https://lists.apache.org/thread/st84zzwnq5m3pkzd1r7jk9lmqdt9m98s
> > >
> > > Thanks!
> > > P
> >
> >



-- 
-José


Re: [ANNOUNCE] New PMC chair: Mickael Maison

2023-04-24 Thread José Armando García Sancio
Congratulations Mickael and thank you Jun for performing this role for
the past 10 years!

On Mon, Apr 24, 2023 at 10:15 AM Yash Mayya  wrote:
>
> Congratulations Mickael!
>
> On Fri, Apr 21, 2023 at 8:39 PM Jun Rao  wrote:
>
> > Hi, everyone,
> >
> > After more than 10 years, I am stepping down as the PMC chair of Apache
> > Kafka. We now have a new chair Mickael Maison, who has been a PMC member
> > since 2020. I plan to continue to contribute to Apache Kafka myself.
> >
> > Congratulations, Mickael!
> >
> > Jun
> >



-- 
-José


Re: [ANNOUNCE] New Kafka PMC Member: Chris Egerton

2023-03-09 Thread José Armando García Sancio
Congrats Chris.

On Thu, Mar 9, 2023 at 2:01 PM Kowshik Prakasam  wrote:
>
> Congrats Chris!
>
> On Thu, Mar 9, 2023 at 1:33 PM Divij Vaidya  wrote:
>
> > Congratulations Chris! I am in awe with the amount of effort you put in
> > code reviews and helping out the community members. Very well deserved.
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Thu, Mar 9, 2023 at 9:49 PM Jorge Esteban Quilcate Otoya <
> > quilcate.jo...@gmail.com> wrote:
> >
> > > So well deserved! Congratulations Chris!!!
> > >
> > > On Thu, 9 Mar 2023 at 22:09, Lucas Brutschy  > > .invalid>
> > > wrote:
> > >
> > > > Congratulations!
> > > >
> > > > On Thu, Mar 9, 2023 at 8:48 PM Roman Schmitz 
> > > > wrote:
> > > > >
> > > > > Congratulations Chris!
> > > > >
> > > > > Am Do., 9. März 2023 um 20:33 Uhr schrieb Chia-Ping Tsai <
> > > > chia7...@gmail.com
> > > > > >:
> > > > >
> > > > > > Congratulations Chris!
> > > > > >
> > > > > > > Mickael Maison  於 2023年3月10日 上午2:21
> > 寫道:
> > > > > > >
> > > > > > > Congratulations Chris!
> > > > > > >
> > > > > > >> On Thu, Mar 9, 2023 at 7:17 PM Bill Bejeck 
> > > > wrote:
> > > > > > >>
> > > > > > >> Congratulations Chris!
> > > > > > >>
> > > > > > >>> On Thu, Mar 9, 2023 at 1:12 PM Jun Rao
> >  > > >
> > > > > > wrote:
> > > > > > >>>
> > > > > > >>> Hi, Everyone,
> > > > > > >>>
> > > > > > >>> Chris Egerton has been a Kafka committer since July 2022. He
> > has
> > > > been
> > > > > > very
> > > > > > >>> instrumental to the community since becoming a committer. It's
> > my
> > > > > > pleasure
> > > > > > >>> to announce that Chris is now a member of Kafka PMC.
> > > > > > >>>
> > > > > > >>> Congratulations Chris!
> > > > > > >>>
> > > > > > >>> Jun
> > > > > > >>> on behalf of Apache Kafka PMC
> > > > > > >>>
> > > > > >
> > > >
> > >
> >



-- 
-José


Re: [ANNOUNCE] New Kafka PMC Member: David Arthur

2023-03-09 Thread José Armando García Sancio
Congrats David!

On Thu, Mar 9, 2023 at 2:00 PM Kowshik Prakasam  wrote:
>
> Congrats David!
>
> On Thu, Mar 9, 2023 at 12:09 PM Lucas Brutschy
>  wrote:
>
> > Congratulations!
> >
> > On Thu, Mar 9, 2023 at 8:37 PM Manikumar 
> > wrote:
> > >
> > > Congrats David!
> > >
> > >
> > > On Fri, Mar 10, 2023 at 12:24 AM Josep Prat  > >
> > > wrote:
> > > >
> > > > Congrats David!
> > > >
> > > > ———
> > > > Josep Prat
> > > >
> > > > Aiven Deutschland GmbH
> > > >
> > > > Alexanderufer 3-7, 10117 Berlin
> > > >
> > > > Amtsgericht Charlottenburg, HRB 209739 B
> > > >
> > > > Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen
> > > >
> > > > m: +491715557497
> > > >
> > > > w: aiven.io
> > > >
> > > > e: josep.p...@aiven.io
> > > >
> > > > On Thu, Mar 9, 2023, 19:22 Mickael Maison 
> > > wrote:
> > > >
> > > > > Congratulations David!
> > > > >
> > > > > On Thu, Mar 9, 2023 at 7:20 PM Chris Egerton  > >
> > > > > wrote:
> > > > > >
> > > > > > Congrats David!
> > > > > >
> > > > > > On Thu, Mar 9, 2023 at 1:17 PM Bill Bejeck 
> > wrote:
> > > > > >
> > > > > > > Congratulations David!
> > > > > > >
> > > > > > > On Thu, Mar 9, 2023 at 1:12 PM Jun Rao  > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hi, Everyone,
> > > > > > > >
> > > > > > > > David Arthur has been a Kafka committer since 2013. He has been
> > > very
> > > > > > > > instrumental to the community since becoming a committer. It's
> > my
> > > > > > > pleasure
> > > > > > > > to announce that David is now a member of Kafka PMC.
> > > > > > > >
> > > > > > > > Congratulations David!
> > > > > > > >
> > > > > > > > Jun
> > > > > > > > on behalf of Apache Kafka PMC
> > > > > > > >
> > > > > > >
> > > > >
> >



-- 
-José


Re: [VOTE] KIP-903: Replicas with stale broker epoch should not be allowed to join the ISR

2023-02-21 Thread José Armando García Sancio
Hi Calvin,

Thanks for the improvement.

1. In the KIP, you suggest changing the Fetch request to "Rename the
ReplicaId to BrokerId" and "Add a new Field BrokerEpoch". The Fetch
RPC is used by replicas that are not brokers, for example controllers
in KRaft.
Can we keep the name "ReplicaId" and use "ReplicaEpoch". Both KRaft
and ISR partitions have the concept of replica id and replica epoch
but not necessarily the concept of a broker.

2. Since the new field "BrokerEpoch '' is ignorable, should it also
have a default value? How about -1 since that is what you use in
AlterPartittion RPC.

-- 
-José


Re: [VOTE] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM for Kafka Brokers

2023-02-21 Thread José Armando García Sancio
LGTM Proven. Thanks for the improvements. +1 (binding)

-- 
-José


Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-02-21 Thread José Armando García Sancio
Hi Proven,

On Tue, Feb 21, 2023 at 1:37 PM Proven Provenzano
 wrote:
>
> Hi Jose,
>
> 1. The SCRAM in SCRAM-SHA-256 is required as the mechanism name is
> SCRAM-SHA-256.
> I do realize there is a bit of redundancy here.
>
> 2. I'll add documentation for all the possible values. They are
> SCRAM-SHA-256 and SCRAM-SHA-512.
>
> 3. I'd like to keep it with a capital letter as it is a specific type of
> message we want to insert. I do agree that -A
> is not the correct choice so I'll suggest -S for SCRAM and that leaves
> -A for --add-acl and -a for a generic
> --add-config in the future.
>

It makes sense to me. Thanks for the changes.

-- 
-José


Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-02-17 Thread José Armando García Sancio
Hi Proven,

Thanks for the changes to KIP-900. It looks good to me in general.
Here are some suggestions and questions.

1. In the KIP you give the following example:
--add-scram SCRAM-SHA-256=[user=alice,password=alice-secret]

Is "SCRAM-" required as a prefix? The flag already has the suffix
"-scram". Can the value to the flag be
SHA-256=[user=alice,password=alice-secret]?

2. Should the CLI document all possible values for the --add-scram? Is
SCRAM-SHA-256 the only supported algorithm?

3. Should the short version of the flag --add-scram be -s? I suspect
that in the future we may want to add more options like --add-acl and
--add-config.

Thanks!
-- 
-José


Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-02-13 Thread José Armando García Sancio
Comments below.

On Wed, Feb 1, 2023 at 2:24 PM Proven Provenzano
 wrote:
> The following are also acceptable from your example. I changed the ordering
> because it does't matter.
>
> --add-config entity-type brokers entity-name 0 foo=bar
> --add-config default-entity entity-type broker baaz=quux
>
> The --add-config sub argument parsing knows about the possible sub fields
> entity-type, entity-name, default-entity and the key=value indicates what
> the config is applied to.

Are you saying that --add-config will have 3 or 4 arguments. The 3
argument form is for the default entity and the 4 argument form is for
a specific entity.
Are you planning to allow the user to configure any entity and
key-value? Does this mean that you need to implement the dynamic
configuration validation logic in the storage-tool?

-- 
-José


Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-02-13 Thread José Armando García Sancio
Comments below.

On Mon, Feb 13, 2023 at 11:44 AM Proven Provenzano
 wrote:
>
> Hi Jose
>
> I want to clarify that the file parsing that Argparse4j provides is just a
> mechanism for
> taking command line args and putting them in a file. It doesn't
> actually change what the
> command line args are for processing the file. So I can add any
> kafka-storage command
> line arg into the file including say the storage UUID. I see that the file
> parsing will be useful
> in the future as we add more record types to add for the bootstrap process.

Understood.

> I'm against adding specific parsing for a list of configs vs. a separate
> JSON file as it is adding
> more surface area that needs testing for a feature that is used
> infrequently. One config method
> should be sufficient for one or more SCRAM records that a customer wants to
> bootstrap with.

Does this mean that the storage tool won't parse and validate SCRAM
configuration? How will the user know that their SCRAM configuration
is correct? Do they need to start the cluster to discover if their
SCRAM configuration is correct?

Thanks,
-- 
-José


Re: [DISCUSS] KIP-900: KRaft kafka-storage.sh API additions to support SCRAM

2023-02-13 Thread José Armando García Sancio
Thanks for the discussion Colin and Proven.

CLIs can be difficult to design when there are complicated use cases.
Did we consider having CLI flags only for the common case? I would
think that the common case is SCRAM for one user.

For the complicated and less common cases they have to provide a
"configuration file" that contains all of the SCRAM users and their
passwords.

The common case could be:
--scram-user alice
--scram-algo SHA-256
--scram-password ...
--scram-salt ...
--scram-iterations ...

If the user wants to provide multiple scram configurations they need
to use --scram-config . The file would be human readable
(maybe JSON or .properties) and contains the scram configuration for
multiple users.

-- 
-José


Re: [DISCUSS] Apache Kafka 3.5.0 release

2023-02-08 Thread José Armando García Sancio
Thanks for volunteering Mickael.

-- 
-José


Re: [DISCUSS] KIP-896: Remove old client protocol API versions in Kafka 4.0

2023-01-11 Thread José Armando García Sancio
Thanks Ismael.

> The following metrics are used to determine both questions:
> >
> >- Client name and version:
> >
> > kafka.server:clientSoftwareName=(client-software-name),clientSoftwareVersion=(client-software-version),listener=(listener),networkProcessor=(processor-index),type=(type)
> >- Request name and version:
> >
> > kafka.network:type=RequestMetrics,name=RequestsPerSec,request=(api-name),version=(api-version)}
> >
> >
> Are you suggesting that this is too complicated and hence we should add a
> metric that tracks AK 4.0 support explicitly?

Correct. It doesn't look trivial for the users to implement this check
against the RequestMetrics. I was wondering if it is worth it for
Kafka to implement this for them and expose a simple metric that they
can check.

-- 
-José


Re: [DISCUSS] KIP-896: Remove old client protocol API versions in Kafka 4.0

2023-01-10 Thread José Armando García Sancio
Hi Ismael,

Thanks for the improvement.

I haven't been following the discussion in detail so it is possible
that this was already discussed.

If a user upgrades to Apache Kafka 4.0 it is possible for some of
their clients to stop working because the request's version would not
be a version that Kafka 4.0 supports. Should we add metrics or some
other mechanism that the user can monitor to determine if it is safe
to upgrade Kafka to 4.0. For example, the metrics could report if a
Kafka broker received a request or response in the past 7 days that
would not be supported by Kafka 4.0.

Thanks
-- 
-José


Re: [VOTE] 3.3.2 RC1

2023-01-10 Thread José Armando García Sancio
Hey Chris,

Here are the results:
http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/system-test-kafka-branch-builder--1673314598--apache--HEAD--b66af662e6/2023-01-09--001./2023-01-09--001./report.html

It looks like all of the failures are when trying to upgrade to
3.3.2-SNAPSHOT. I saw a similar error in my PR here but I am not sure
if it is related: https://github.com/apache/kafka/pull/13077

Maybe someone familiar with Kafka Streams can help.

Thanks,
-- 
-José


Re: [VOTE] 3.3.2 RC1

2023-01-09 Thread José Armando García Sancio
Hey Chris,

I started a run for the system tests. The commit is b66af662e6. I'll
message you again with a link to the results when it is done. Probably
at the end of the day today or tomorrow morning.

Thanks,
-- 
-José


Re: bug with kraft recreating topics with periods in the name

2022-12-20 Thread José Armando García Sancio
Hi David,

It is likely that you are running into this issue:
https://issues.apache.org/jira/browse/KAFKA-14337

Can you try with trunk or the fixed versions enumerated in that jira?

Thanks,
-- 
-José


[DISCUSS] Suppressing style checks

2022-12-20 Thread José Armando García Sancio
Hi all,

I added the following modules to the code style checker:
https://github.com/apache/kafka/blob/44b3177a087ff809a9d95a27b63b10e00aa4da7d/checkstyle/checkstyle.xml#L147-L156

I think this feature has a few benefits.
1) It should allow us to move style check suppressions to the code
using the @SuppressWarnings annotation. Here is an example:
https://github.com/apache/kafka/blob/44b3177a087ff809a9d95a27b63b10e00aa4da7d/metadata/src/main/java/org/apache/kafka/controller/ControllerMetricsManager.java#L110

2) Have more targeted warning suppression instead of suppressing
coding style for the entire file.

3) Incrementally remove some of the clutter from the
checkstyle/suppressions.xml file.

Thanks!
-- 
-José


Re: [DISCUSS] Apache Kafka 3.4.0 release

2022-12-16 Thread José Armando García Sancio
Hi Sophie,

I am interested in including a bug fix for
https://issues.apache.org/jira/browse/KAFKA-14457 in the 3.4.0
release. The fix is here: https://github.com/apache/kafka/pull/12994.

I think it is important to include this fix because some of the
controller metrics are inaccurate without this fix. This could impact
some users' ability to monitor the cluster.

What do you think?
-- 
-José


Re: [DISCUSS] Apache Kafka 3.3.2

2022-12-14 Thread José Armando García Sancio
Hey Chris,

Here are the results of just running the upgrade system tests on the
latest 3.3 branch:
http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/system-test-kafka-branch-builder--1671061000--apache--3.3--69fbaf2457/2022-12-14--001./2022-12-14--001./report.html

Looks like the commit is now 69fbaf2457...

Thanks
-- 
-José


Re: [DISCUSS] Apache Kafka 3.3.2

2022-12-14 Thread José Armando García Sancio
Hey all,

Here are the system test results for the 3.3 branch:
http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/system-test-kafka-branch-builder--1670984851--apache--3.3--22af3f29ce/2022-12-13--001./2022-12-13--001./report.html

The commit for that run is 22af3f29ced9e969c254004f592c06c8a45b84cc.
It is also encoded in the URL above.

Thanks
-- 
-José


Re: [DISCUSS] Apache Kafka 3.3.2

2022-12-13 Thread José Armando García Sancio
Thanks Chris!

I started a build for the system tests against the 3.3 branch using
Confluent's infrastructure. I'll reply here when the results are
avaialbe.

-- 
-José


Re: [DISCUSS] Apache Kafka 3.3.2

2022-11-16 Thread José Armando García Sancio
+1. Thanks for volunteering.

-- 
-José


Re: [ANNOUNCE] New Kafka PMC Member: Bruno Cadonna

2022-11-02 Thread José Armando García Sancio
Congratulations Bruno!

On Wed, Nov 2, 2022 at 1:54 PM Matthias J. Sax  wrote:
>
> Congrats!
>
> On 11/1/22 7:08 PM, Luke Chen wrote:
> > Congrats Bruno!
> > Well deserved!
> >
> > Luke
> >
> > On Wed, Nov 2, 2022 at 10:07 AM John Roesler  wrote:
> >
> >> Congratulations, Bruno!!!
> >>
> >> On Tue, Nov 1, 2022, at 15:16, Lucas Brutschy wrote:
> >>> Wow, congratulations!
> >>>
> >>> On Tue, Nov 1, 2022 at 8:55 PM Chris Egerton 
> >> wrote:
> 
>  Congrats!
> 
>  On Tue, Nov 1, 2022, 15:44 Bill Bejeck 
> >> wrote:
> 
> > Congrats Bruno! Well deserved.
> >
> > -Bill
> >
> > On Tue, Nov 1, 2022 at 3:36 PM Guozhang Wang 
> >> wrote:
> >
> >> Hi everyone,
> >>
> >> I'd like to introduce our new Kafka PMC member, Bruno.
> >>
> >> Bruno has been a committer since April. 2021 and has been very
> >> active in
> >> the community. He's a key contributor to Kafka Streams, and also
> >> helped
> >> review a lot of horizontal improvements such as Mockito. It is my
> > pleasure
> >> to announce that Bruno has agreed to join the Kafka PMC.
> >>
> >> Congratulations, Bruno!
> >>
> >> -- Guozhang Wang, on behalf of Apache Kafka PMC
> >>
> >
> >>
> >



-- 
-José


  1   2   3   >