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
> >
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).
>
>
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,
+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
Hi Justine,
+1 (binding)
Thanks for the improvement.
--
-José
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
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
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:
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é
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,
--
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.
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.
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
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
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
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
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
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
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
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é
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
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
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é
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",
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
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?
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
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
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
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,
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
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
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."
>
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
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é
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
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
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
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
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 exa
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
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é
Hi all,
I have updated the KIP to include information on how KRaft controller
automatic joining will work.
Thanks,
--
-José
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
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
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.
>
>
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
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
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
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 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
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é
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:
+1.
Thanks for the KIP. Looking forward to the implementation!
--
-José
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é
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
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
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
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
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,
>
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
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
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
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
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
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
The KIP LGTM. Thanks for the design. I am looking forward to the implementation.
+1 (binding).
Thanks!
--
-José
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
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
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
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
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
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
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
+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
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
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
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
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
LGTM Proven. Thanks for the improvements. +1 (binding)
--
-José
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
>
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
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
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
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
Thanks for volunteering Mickael.
--
-José
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)
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
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
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é
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é
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
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
Hey Chris,
Here are the results of just running the upgrade system tests on the
latest 3.3 branch:
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
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é
+1. Thanks for volunteering.
--
-José
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
Hi all,
During the implementation of this KIP and some offline discussion with
Jason Gustafson, I realized now that Kafka is generating a snapshot
every hour and the default metadata retention is to delete snapshots
after 7 days, every cluster metadata partition will have 168 (1
snapshot per hour
Thanks for all of the votes. I am going to go ahead and close the voting.
5 binding +1: Deng Ziming, David Jacot, David Arthur, Colin McCabe, Luke Chen
--
-José
1 - 100 of 210 matches
Mail list logo