Hey all,
For the KIP-500 work for 3.0 we would like to propose the following
Jiras as blockers:
1. https://issues.apache.org/jira/browse/KAFKA-13168
2. https://issues.apache.org/jira/browse/KAFKA-13165
3. https://issues.apache.org/jira/browse/KAFKA-13161
The description for each Jira should
On Tue, Oct 12, 2021 at 10:57 AM Colin McCabe wrote:
> > 11. For downgrades, it would be useful to describe how to determine the
> > downgrade process (generating new snapshot, propagating the snapshot, etc)
> > has completed. We could block the UpdateFeature request until the process
> > is
On Tue, Oct 5, 2021 at 8:53 AM David Arthur wrote:
> 2. Generate snapshot on downgrade
> > > Metadata snapshot is generated and sent to the other inactive
> > controllers and to brokers (this snapshot may be lossy!)
> > Why do we need to send this downgraded snapshot to the brokers? The
> >
On Thu, Oct 7, 2021 at 5:20 PM Jun Rao wrote:
> 7. Jose, what control records were you referring?
>
Hey Jun, in KRaft we have 3 control records.
- LeaderChangeMessage - this is persistent in the replica log when a
new leader gets elected and the epoch increases. We never included
this record in
On Fri, Oct 15, 2021 at 7:24 AM David Arthur wrote:
> Hmm. So I think you are proposing the following flow:
> > 1. Cluster metadata partition replicas establish a quorum using
> > ApiVersions and the KRaft protocol.
> > 2. Inactive controllers send a registration RPC to the active controller.
> >
During the development of KIP-630 we made some minor changes to the
KIP to better match the implementation details. Here is a summary of
the changes we made to the KIP:
1. Added control records at the begin and end of the snapshots. The
control records are versioned. The snapshot header record
During the development of KIP-630 we made some minor changes to the
KIP to better match the implementation details. Here is a summary of
the changes we made to the KIP:
1. Added control records at the begin and end of the snapshots. The
control records are versioned. The snapshot header record
Thank you David for the detailed KIP.
1. Generate snapshot on upgrade
> Metadata snapshot is generated and sent to the other nodes
Why does the Active Controller need to generate a new snapshot and
force a snapshot fetch from the replicas (inactive controller and
brokers) on an upgrade? Isn't
One more comment.
7.Downgrade records
I think we should explicitly mention that the downgrade process will
downgrade both metadata records and controller records. In KIP-630 we
introduced two control records for snapshots.
Hi David Jacot,
I cherry picked this commit https://github.com/apache/kafka/pull/11511
to the 3.1 branch to fix the kafka.metrics.MetricsTest integration
tests for that branch.
Thanks,
-Jose
On Tue, Dec 14, 2021 at 8:09 AM David Jacot wrote:
>
> Hi Jason,
>
> Thanks for bringing this up. I do
Congrats David!
On Fri, Dec 17, 2021 at 3:09 PM Gwen Shapira wrote:
>
> Hi everyone,
>
> David Jacot has been an Apache Kafka committer since Oct 2020 and has been
> contributing to the community consistently this entire time - especially
> notable the fact that he reviewed around 150 PRs in
Hey David,
Thanks for the KIP! LGTM. +1.
I noticed the following minor issues while reading the KIP:
1. Should the versions of the "AllowDowngrade" be changed to "0-1"?
2. I think you meant --unsafe here:
> The controller validates that the cluster can be safely downgraded to this
> version
Thanks for the additional information Colin.
On Mon, Dec 13, 2021 at 4:43 PM Colin McCabe wrote:
>
> Hi José,
>
> I think these are good questions. We have a few situations like this where
> there is something brokers have to know before they can contact the
> controller quorum -- or something
Hi Colin,
Thanks for the KIP.
1. Can you talk about how the set of ACLs needed to authorize
controllers and brokers will get bootstrapped? I am particularly
interested in how we are going to configure a new cluster so that the
controllers nodes can authorize each other's requests to establish
Hi all,
I would like to open the discussion on implementing "KIP-704: Send a
hint to broker if it is an unclean leader." See this wiki page for
details: https://cwiki.apache.org/confluence/x/kAZRCg
Thanks!
--
-Jose
Thanks for the feedback David Jacot
David Jacot wrote:
> I have one question regarding how fetch from followers will
> work when the leader is recovering. My understanding is that
> the leader will reject any produce and fetch requests with a
> NOT_LEADER_OR_FOLLOWER error while the followers
>
David Jacot wrote:
> The behavior of the leader is clear. However, the part which is
> not clear to me is how followers which could get fetch requests
> from consumers as well will handle them. Sorry if I was not clear.
Got it. I updated the KIP to add more information regarding how the
topic
Hi Jason,
Jason wrote:
> Thanks for the updates. I noticed that `LeaderRecoveryState` is marked as
> ignorable in the `AlterPartition` request. It would be helpful to
> understand the motivation for that.
I think it is fine for this property to be marked as ignorable because
the property is
Hi all,
Jason and I discussed this offline. At a high-level I have made the
following changes to the KIP.
1. IBP will be used to enable this feature and to determine which
version of LeaderAndIsr and AlterPartition will be used.
2. The LeaderRecoveryState field for LeaderAndIsr and
You are correct Raman. I updated the KIP to reflect your observations
and corrections.
Raman wrote:
> I think the line #2 here is wrong with regard to AlterPartitionRequest.
Yes. Bullet 2. Now reads: 2. The LeaderRecoveryState is changing from
RECOVERED to RECOVERING.
Raman wrote:
> Also, in
Thanks for the vote Jason.
This is a friendly reminder to vote on this KIP.
--
-José
Hi all,
With 4 binding votes (Jason, David Jacot, Colin and David Arthur) and
2 non-binding votes (Raman and Luke) the KIP passes.
Thanks to all that participated in the discussion and voting,
--
-José
Thanks Jason and David for your feedback. See my comments below.
David wrote:
> 1) Does recovering from an unclean state bump the leader epoch?
Looking at the controller code, the leader epoch is only increased if
the leader id changes.
David wrote:
> 2) The name of "NewIsUnclean" field in
Jose wrote:
> I'll update the KIP with this information. The leader will return
> "NOT_LEADER_OR_FOLLOWER" for any partition that is still recovering
> for Fetch, Produce, OffsetsForLeaderEpoch and DeleteRecords requests.
> This error type is retriable by the clients.
I forgot to include
Hi all,
The following suggestions are not strictly required to implement this
KIP but what do we think about:
1. Changing the name of the AlterIsr RPC to AlterPartition RPC.
2. Change the name of the field "CurrentIsrVersion" to
"PartitionEpoch". This is the name that we use in the KRaft
Thanks Raman and Colin for your feedback.
Raman wrote:
> - Could you please explain the following about backward compatibility.
> If a leader has been elected unclean. And we decide to roll the
> cluster back when the leader is in the middle of recovery, leader will
> simply not be able to
Thanks for the feedback Colin and Luke.
Colin wrote:
> The KIP talks a bit about "recovery," which is a new term (as far as I
> know). If I understand correctly, this is a state that the partition enters
> into after an unclean leader election. I would suggest using a different
> term for this,
Hi all,
I made the following changes to the KIP:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=173082256=12=11
Some of the highlights are:
1. Changed the field from IsUnclean to IsLeaderRecovering
2. Added a few more sentences explaining why this KIP is backward
Thanks for the additional context regarding AlterIsrResponse.
Jason wrote:
> In regard to the naming of `IsLeaderRecovering`, I agree it still seems a
> bit awkward. I kind of liked the idea of turning it into a `PartitionState`
> field instead. That would also address the inconsistent type in
Thanks for the feedback Colin.
Colin wrote:
> We already have many classes that are called "partition state." For example,
> PartitionStates.java on the client side, PartitionStateMachine.scala and
> TopicPartitionStateZNode in the old controller,
> RemotePartitionDeleteState.java under
Hi all,
I'd like to start a vote on KIP-704: Send a hint to the partition
leader to recover the partition.
KIP wiki: https://cwiki.apache.org/confluence/x/kAZRCg
Discussion thread:
https://lists.apache.org/thread/ld2t2xkby7rpgrggqo1h344goddfdnxb
Thanks,
-José
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
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
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 Guozhang, thanks for the feedback.
Guozhang wrote:
> Could you elaborate a bit on what does "load-processing-time-us" measure? I
> looked through the discussion thread and the KIP / JIRA but cannot find its
> definitions.
Yes. I updated the KIP. This is what I documented:
1.
Hi Colin,
Thanks for the KIP.
>The rationale for deprecating ZK in the 3.4 release is so that we can remove
>it in the 4.0 release. (In general, Kafka requires features to be deprecated
>for at least one release before they can be removed in the following major
>release.) During the
Hi Jun,
Jun wrote:
> 20. For the metric type and name, we use the camel names in some cases and
> dashed lower names in some other cases. Should we make them consistent?
For the metrics group `type=KafkaController`, I am using camel names
like `MetadataLastAppliedRecordOffset` because it matches
Thanks everyone for your feedback and help. KIP-835 was approved with
3 binding votes from Guozhang, Luke and David.
On Thu, May 19, 2022 at 10:21 AM Guozhang Wang wrote:
>
> That makes sense. Thanks!
>
> +1 (binding).
>
> On Thu, May 19, 2022 at 8:46 AM José Armando Garc
Thanks for the changes to KIP-836 Niket.
KIP-836 has the following output for "--describe replication":
> bin/kafka-metadata-quorum.sh --describe replication
ReplicaId LogEndOffsetLag LastFetchTimeMs
LastCaughtUpTimeMsStatus
0 234134 0 tnow
Guozhang Wang wrote:
>
> Thanks José! For 1/2 above, just checking if we would record the
> corresponding sensors only during broker bootstrap time, or whenever there
> are new metadata records being committed by the controller quorum (since
> there are always a short period of time, between when
Thanks for the updates to the KIP.
I like enumerating invariants. Is it safe to say that if
`InControlledShutdown` is true then `Fenced` must be false.
Thanks for all of the feedback. Some comments below:
Luke wrote:
> 1. Jason has asked but you didn't answer: What is the default value for `
> metadata.monitor.write.interval.ms`?
Thanks for asking again. Looks like I missed this in my previous
reply. In the implementation I am currently working
:
> >
> > +1 (binding)
> >
> > Thanks for the KIP!
> >
> > David
> >
> > Le ven. 20 mai 2022 à 21:08, David Arthur a écrit :
> >
> >> Hey Niket, +1 (binding) from me.
> >>
> >> -David
> >>
> >> On Thu, May 19, 2
On Fri, May 20, 2022 at 4:04 PM Niket Goel wrote:
>
> So Ideally the LastCatchUpTimeMs is supposed to be a proxy for lag. We could
> report this field as lag, but I am personally not in favor of doing that as
> the LastCaughtUpTimeMs is just an approximation of lag, but not actual lag.
>
> I
Hey Niket,
I took a look at the latest KIP. It looks like QuorumInfo.ReplicaState
is missing the RPC fields added by this PR. Is the plan to return them
to the Admin Client? E.g. it is missing LastFetchTimestamp and
LastCaughtUpTimestamp.
For those fields what will the admin client return when
Hi all,
Thanks for your feedback. I started a voting thread here:
https://lists.apache.org/thread/x1cy5otpf7mj9ytghnktr5hog27hdf7k
Hi all,
I would like to start a vote for KIP-835:
https://cwiki.apache.org/confluence/x/0xShD
Thanks you,
-José
Thanks for the Kafka improvement Niket.
1. For the fields `LastFetchTime` and `LastCaughtUpTime`, Kafka tends
to use the suffix "Timestamp" when the value is an absolute wall clock
value.
2. The method `result()` for the type `DescribeQuorumResult` returns
the type `DescribeQuorumResponseData`.
On Thu, Jul 7, 2022 at 7:20 AM Sagar wrote:
> Pr for KIP-843 also got merged almost 24 days ago. There’s a follow up pr
> open for documentation changes. Could this also be added plz?
Thank you. Added KIP-843 to the list of KIPs for 3.3.0.
I will create a release branch for 3.3 next Monday,
Hi all,
Friendly reminder that Feature Freeze was yesterday July 6th. The Code
Freeze date is the 20th of July.
On Wed, Jun 22, 2022 at 7:45 AM Viktor Somogyi-Vass
wrote:
> KIP-373 just went in yesterday (and likely add some docs too today), would
> you please add it to the list?
I added
On Fri, Jul 8, 2022 at 6:37 AM Rajini Sivaram wrote:
> We have a PR for batched offset fetch API, which is part of KIP-709 (
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258)
> that was opened a year ago, but didn't get merged because we didn't have
> time to merge
Hi all,
I created the branch for 3.3
(https://github.com/apache/kafka/tree/3.3). If you have bug fixes for
the 3.3.0 release please make sure to cherry pick them to that branch.
Thanks
> Anyway, I can help you re-run the jenkins build.
> Just point me the jenkins build link.
> Or you can actually merge with the latest trunk to trigger the build.
>
> Thank you.
> Luke
>
> On Sat, Jun 4, 2022 at 5:52 AM José Armando García Sancio
> wrote:
>
> &g
Hi all,
This is a friendly reminder that the KIP freeze date is today, June 15th, 2022.
The feature freeze date is July 6th, 2022.
Thanks,
-José
Hi Divij,
On Thu, Jun 16, 2022 at 1:37 AM Divij Vaidya wrote:
> *Question#1*: Do we only track the KIPs over here that are blockers for
> release or do we track the non-KIP JIRA tickets as well?
This page documents the KIPs and Jira issues I am tracking for the
3.3.0 release.
Thanks for the updates everyone. I added KIP-618, KIP-841, KIP-827 and
KIP-834 to the planned KIPs for 3.3.0.
David Jacot, it looks like KIP-841 is not linked from the "Kafka
Improvement Proposals" page:
https://cwiki.apache.org/confluence/x/4QwIAw
--
-José
Hey all,
I am trying to re-run a Jenkins build. It looks like my Apache login
doesn't work with Jenkins. Do I need to ask a Kafka PMC to add me to
the Jenkins infrastructure? I see the following from an Apache wiki
page.
How do I get an account?
Jenkins uses the Apache LDAP servers for
David Jacot wrote:
> At the moment, the KIP stipulates that the broker remains in
> InControlledShutdown state until it is re-registered with a new
> incarnation id. This implies that a broker can be both fenced and in
> controlled shutdown state. We could make them mutually exclusive but I
>
Thanks for proposing this improvement David Jacot. I think it is going
to make the graceful shutdown process much more efficient.
+1 (binding) from me.
Hi Colin,
Thank you for the KIP. +1 (binding).
--
-José
Hi David,
Thanks for the KIP. In the "Compatibility, Deprecation, and Migration
Plan", you have:
> The change is backward compatible.
I assume that we don't need to increase the metadata.version/IBP for
AlterPartition because AlterPartitionManager uses ApiVersions for that
channel. Should we
Thanks for the feedback Colin. I updated the KIP with your suggestions
and replied to your comments below.
Colin McCabe wrote:
> 1. It seems like the proposal is to have a UUID per partition directory on
> the voter. If I understand correctly, this is sometimes referred to as
> "VoterUUID" and
+1. Thanks for volunteering David.
--
-José
Divij Vaidya wrote:
> 1. As an aside, would we be open to accept other alternative forms of
> proofs such as property based testing (semi-formal methods) in future?
The Apache Kafka repository already has support for property based
testing using jqwik. So feel free to add more property based
Hi all,
I would like to start the discussion on my design to support
dynamically changing the set of voters in the KRaft cluster metadata
topic partition.
KIP URL: https://cwiki.apache.org/confluence/x/nyH1D
Thanks!
-José
Thanks Niket for your feedback. I have made changes to the KIP and
replied to your comments below.
Niket Goel wrote:
> > This UUID will be generated once and persisted as part of the quorum state
> > for the topic partition
> Do we mean that it will be generated every time the disk on the
Tom Bentley wrote:
> Thanks for the KIP. As Justine mentioned, this KIP currently lacks a
> motivation, and nor does the JIRA provide any context. Please could you
> provide this context, otherwise it's impossible for people on this list to
> understand the problem you're trying to solve here.
Hi all,
I would like to start the discussion on my design to allow KRaft to
detect and recover from disk failures in the minority of voters. For
those following the discussion on KIP-853, this is a subset of that
KIP with only the mechanisms required to solve the problem described
in the
Hi all,
Community members Jason Gustafson, Colin P. McCabe and I have been
having some offline conversations.
At a high-level KIP-853 solves the problems:
1) How can KRaft detect and recover from disk failures on the minority
of the voters?
2) How can KRaft support a changing set of voter nodes?
Hi Igor,
Thanks for the KIP. Looking forward to this improvement. I'll review your KIP.
I should mention that I started a discussion thread on KIP-856: KRaft
Disk Failure Recovery at
https://lists.apache.org/thread/ytv0t18cplwwwqcp77h6vry7on378jzj
Both keep introducing similar concepts. For
Jack Vanlightly wrote:
> - Regarding the removal of voters, when a leader appends a
> RemoveVoterRecord to its log, it immediately switches to the new
> configuration. There are two cases here:
> 1. The voter being removed is the leader itself. The KIP documents that
> the followers will
Hi all,
I created a KIP for adding a mechanism to monitor the health of the
KRaft Controller quorum through metrics. See KIP-835:
https://cwiki.apache.org/confluence/x/0xShD
Thanks for your feedback,
-José
Hi all,
I would like to volunteer for the release of Apache Kafka 3.3.0. If
people agree, I'll start working on the release plan and update this
thread.
Thanks,
-José
Thanks for your feedback Jason, much appreciated.
Here are the changes to the KIP:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=211883219=5=4
On Tue, May 10, 2022 at 1:34 PM Jason Gustafson
wrote:
> The approach sounds reasonable. By the way, I think one of the
Great.
I went ahead and created a release page for 3.3.0:
https://cwiki.apache.org/confluence/x/-xahD
The planned KIP content is based on the list of KIPs targeting 3.3.0
in https://cwiki.apache.org/confluence/x/4QwIAw. Please take a look at
the list and let me know if I missed your KIP.
The
Hello Kafka users, developers and client-developers,
This is the first candidate for release of Apache Kafka 3.3.0.
Release notes for the 3.3.0 release:
https://home.apache.org/~jsancio/kafka-3.3.0-rc1/RELEASE_NOTES.html
Please download and test.
Kafka's KEYS file containing PGP keys we use to
The documentation and protocol links are not working. Looking into it.
https://kafka.apache.org/33/documentation.html
https://kafka.apache.org/33/protocol.html
Thanks,
-José
Hello Kafka users, developers and client-developers,
This is the first candidate for the release of Apache Kafka 3.3.0.
There are some issues that we still have to resolve before we can make
a final release. Those issues are documented here:
Thanks Artem and Colin for identifying and fixing the issues
KAFKA-14156 and KAFKA-14187. I have marked both of them as blocker for
this release.
I also don't think that these issues should block testing other parts
of the release.
Thanks
José
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é
Thanks for your feedback Luke.
On Thu, Oct 13, 2022 at 11:58 PM Luke Chen wrote:
> The only thing I'd like to point out is the compatibility section.
> Since this new config is default to 1 hour, which means if users explicitly
> set the config `metadata.log.max.record.bytes.between.snapshots`
+1 from me.
Thanks for your help David. Much appreciated.
On Mon, Sep 19, 2022 at 10:29 AM David Arthur
wrote:
>
> Hey folks, José has asked me to help push the release along this week while
> he's out of the office.
>
> -David
>
> On Tue, Aug 30, 2022 at 12:01 PM José
Thanks for the votes John, Bill and David.
Here are the system test results:
http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/3.3/2022-09-30--001.system-test-kafka-3.3--1664605767--confluentinc--3.3--eefe867118/report.html
There are two failures. I reran those tests. I will
Hello Kafka users, developers and client-developers,
This is the first candidate for release of Apache Kafka 3.3.1. This
release fixes two issues with the 3.3.0 release.
* [KAFKA-14259] - BrokerRegistration#toString throws an exception,
terminating metadata replay
* [KAFKA-14265] - Prefix ACLs
On Thu, Sep 29, 2022 at 2:39 PM José Armando García Sancio
wrote:
> Please download, test and vote by Tuesday, October 4, 9am PT.
The vote will be open for 72 hours. Please vote by Sunday, October 2nd, 3 PM PT.
Thanks!
--
-José
Hey Kafka committers,
Can anyone help me with these commands:
Upload the new release and kafka-stream-x.x.x-test.jar (can be found
in the .release_work_dir created by the release.py script) to the S3
bucket "kafka-packages".
Use the AWS console to upload the files in the bucket or the CLI if
you
I talked to Manikumar offline. He explained to me that this is a
bucket owned by Confluent. I was unblocked and was able to upload the
artifacts to the S3 bucket. I am updating the release process page to
better document this.
On Tue, Oct 4, 2022 at 10:38 AM José Armando García Sancio
wrote
Armando García Sancio,
jparag, Justine Olshan, K8sCat, Kirk True, Konstantine Karantasis,
Kvicii, Lee Dongjin, Levani Kokhreidze, Liam Clarke-Hutchinson, Lucas
Bradstreet, Lucas Wang, Luke Chen, Manikumar Reddy, Marco Aurelio
Lotz, Matthew de Detrich, Matthias J. Sax, Mickael Maison, Mike
Lothian
On Wed, Oct 12, 2022 at 3:02 PM Niket Goel wrote:
> 1. Do we need this value to be of the order of `ms`. Is it better off being
> tunable to a minute granularity?
Hmm. The most common unit for time intervals in Kafka is milliseconds.
Very rarely does Kafka express time intervals using another
On Wed, Oct 12, 2022 at 1:27 AM David Jacot wrote:
> I would name the
> new property `metadata.log.snapshot.interval.ms` as `between` is
> implied by the `interval`.
I agree. I updated the KIP to use your suggestions for naming the
property `metadata.log.snapshot.interval.ms`.
--
-José
On Wed, Oct 12, 2022 at 3:02 PM Colin McCabe wrote:
> Given that we already have metadata.log.max.record.bytes.between.snapshots,
> we need to define how the two properties interact. I would expect that the
> time-based property would take effect only if the bytes-based property did
> not
Hello all,
I would like to start voting for "KIP-876: Time based cluster metadata
snapshots."
KIP: https://cwiki.apache.org/confluence/x/MY3GDQ
Discussion thread:
https://lists.apache.org/thread/ww67h9d4xvgw1f7jn4zxwydmt8x1mq72
Thanks!
--
-José
Thanks for your feedback David Jacot, Colin McCabe and Niket Goel.
I started the vote thread at
https://lists.apache.org/thread/yzzhbvdqxg9shttgbzooc2f42l1cv2sj
--
-José
Thanks for volunteering Sophie.
On Wed, Oct 5, 2022 at 3:01 PM Sophie Blee-Goldman
wrote:
>
> Hey all,
>
> I'd like to volunteer as release manager for the next feature release,
> which will be Apache
> Kafka 3.4.0. If that sounds good to everyone I'll update this thread with
> the release plan
Congratulations Ziming. Well deserved and I much appreciate your
contributions to the project.
--
-José
Hey all,
I am interested in allowing brokers and controllers in KRaft to
generate snapshots for the cluster metadata partition on a timely
basis. This would better allow Kafka users to use cluster metadata
snapshots as a solution for backing up the cluster's metadata.
Let's use this thread to
On Mon, Oct 3, 2022 at 2:00 PM Igor Soarez wrote:
>
> Thanks Jose and David for running this patch release. Congratulations to all!
>
> I don't see the tag or the usual commit sequence in the 3.3 branch for this
> release. I'd expect a `3.3.1` and a commit moving the version to
>
-builder--1664643010--apache--3.3--cdb25e10dc/2022-10-01--001./2022-10-01--001./report.html
On Sat, Oct 1, 2022 at 9:14 AM José Armando García Sancio
wrote:
>
> Thanks for the votes John, Bill and David.
>
> Here are the system test results:
> http://confluent-kafka-system-test-resul
Hi all,
All of the system tests for 3.3 passed.
http://confluent-kafka-system-test-results.s3-us-west-2.amazonaws.com/3.3/2022-09-30--001.system-test-kafka-3.3--1664605767--confluentinc--3.3--eefe867118/report.html
This build ran all of the tests and there were two failures:
1 - 100 of 210 matches
Mail list logo