Oops sorry I got confused between https://issues.apache.org/jira/browse/KAFKA-16120 (migration issue) and https://issues.apache.org/jira/browse/KAFKA-14616 (not migration issue)
However, both do not seem related to JBOD based on the jira and PRs Justine On Tue, Jan 23, 2024 at 1:51 PM Justine Olshan <jols...@confluent.io> wrote: > Hey Stan, > > Just wanted to clarify -- KAFKA-14616 is not particularly related to JBOD > but to ZK -> KRaft migration. > > There were some other related migration bugs like > https://issues.apache.org/jira/browse/KAFKA-16180. > > This may or may not influence decisions, but wanted to paint the full > picture of the blocker bugs and their causes. > > Justine > > > On Tue, Jan 23, 2024 at 12:17 PM Stanislav Kozlovski > <stanis...@confluent.io.invalid> wrote: > >> Hey all, I figured I'd give an update about what known blockers we have >> right now: >> >> - KAFKA-16101: KRaft migration rollback documentation is incorrect - >> https://github.com/apache/kafka/pull/15193; This need not block RC >> creation, but we need the docs updated so that people can test properly >> - KAFKA-14616: Topic recreation with offline broker causes permanent URPs >> - >> https://github.com/apache/kafka/pull/15230 ; I am of the understanding >> that >> this is blocking JBOD for 3.7 >> - KAFKA-16162: New created topics are unavailable after upgrading to 3.7 - >> a strict blocker with an open PR >> https://github.com/apache/kafka/pull/15232 >> - although I understand Proveen is out of office >> - KAFKA-16082: JBOD: Possible dataloss when moving leader partition - I am >> hearing mixed opinions on whether this is a blocker ( >> https://github.com/apache/kafka/pull/15136) >> >> Given that there are 3 JBOD blocker bugs, and I am not confident they will >> all be merged this week - I am on the edge of voting to revert JBOD from >> this release, or mark it early access. >> >> By all accounts, it seems that if we keep with JBOD the release will have >> to spill into February, which is a month extra from the time-based release >> plan we had of start of January. >> >> Can I ask others for an opinion? >> >> Best, >> Stan >> >> On Thu, Jan 18, 2024 at 1:21 PM Luke Chen <show...@gmail.com> wrote: >> >> > Hi all, >> > >> > I think I've found another blocker issue: KAFKA-16162 >> > <https://issues.apache.org/jira/browse/KAFKA-16162> . >> > The impact is after upgrading to 3.7.0, any new created >> topics/partitions >> > will be unavailable. >> > I've put my findings in the JIRA. >> > >> > Thanks. >> > Luke >> > >> > On Thu, Jan 18, 2024 at 9:50 AM Matthias J. Sax <mj...@apache.org> >> wrote: >> > >> > > Stan, thanks for driving this all forward! Excellent job. >> > > >> > > About >> > > >> > > > StreamsStandbyTask - >> https://issues.apache.org/jira/browse/KAFKA-16141 >> > > > StreamsUpgradeTest - >> https://issues.apache.org/jira/browse/KAFKA-16139 >> > > >> > > For `StreamsUpgradeTest` it was a test setup issue and should be fixed >> > > now in trunk and 3.7 (and actually also in 3.6...) >> > > >> > > For `StreamsStandbyTask` the failing test exposes a regression bug, so >> > > it's a blocker. I updated the ticket accordingly. We already have an >> > > open PR that reverts the code introducing the regression. >> > > >> > > >> > > -Matthias >> > > >> > > On 1/17/24 9:44 AM, Proven Provenzano wrote: >> > > > We have another blocking issue for the RC : >> > > > https://issues.apache.org/jira/browse/KAFKA-16157. This bug is >> similar >> > > to >> > > > https://issues.apache.org/jira/browse/KAFKA-14616. The new issue >> > however >> > > > can lead to the new topic having partitions that a producer cannot >> > write >> > > to. >> > > > >> > > > --Proven >> > > > >> > > > On Tue, Jan 16, 2024 at 12:04 PM Proven Provenzano < >> > > pprovenz...@confluent.io> >> > > > wrote: >> > > > >> > > >> >> > > >> I have a PR https://github.com/apache/kafka/pull/15197 for >> > > >> https://issues.apache.org/jira/browse/KAFKA-16131 that is building >> > now. >> > > >> --Proven >> > > >> >> > > >> On Mon, Jan 15, 2024 at 5:03 AM Jakub Scholz <ja...@scholz.cz> >> wrote: >> > > >> >> > > >>> *> Hi Jakub,> > Thanks for trying the RC. I think what you found >> is a >> > > >>> blocker bug because it * >> > > >>> *> will generate huge amount of logspam. I guess we didn't find >> it in >> > > >>> junit >> > > >>> tests * >> > > >>> *> since logspam doesn't fail the automated tests. But certainly >> it's >> > > not >> > > >>> suitable * >> > > >>> *> for production. Did you file a JIRA yet?* >> > > >>> >> > > >>> Hi Colin, >> > > >>> >> > > >>> I opened https://issues.apache.org/jira/browse/KAFKA-16131. >> > > >>> >> > > >>> Thanks & Regards >> > > >>> Jakub >> > > >>> >> > > >>> On Mon, Jan 15, 2024 at 8:57 AM Colin McCabe <cmcc...@apache.org> >> > > wrote: >> > > >>> >> > > >>>> Hi Stanislav, >> > > >>>> >> > > >>>> Thanks for making the first RC. The fact that it's titled RC2 is >> > > messing >> > > >>>> with my mind a bit. I hope this doesn't make people think that >> we're >> > > >>>> farther along than we are, heh. >> > > >>>> >> > > >>>> On Sun, Jan 14, 2024, at 13:54, Jakub Scholz wrote: >> > > >>>>> *> Nice catch! It does seem like we should have gated this >> behind >> > the >> > > >>>>> metadata> version as KIP-858 implies. Is the cluster configured >> > with >> > > >>>>> multiple log> dirs? What is the impact of the error messages?* >> > > >>>>> >> > > >>>>> I did not observe any obvious impact. I was able to send and >> > receive >> > > >>>>> messages as normally. But to be honest, I have no idea what else >> > > >>>>> this might impact, so I did not try anything special. >> > > >>>>> >> > > >>>>> I think everyone upgrading an existing KRaft cluster will go >> > through >> > > >>> this >> > > >>>>> stage (running Kafka 3.7 with an older metadata version for at >> > least >> > > a >> > > >>>>> while). So even if it is just a logged exception without any >> other >> > > >>>> impact I >> > > >>>>> wonder if it might scare users from upgrading. But I leave it to >> > > >>> others >> > > >>>> to >> > > >>>>> decide if this is a blocker or not. >> > > >>>>> >> > > >>>> >> > > >>>> Hi Jakub, >> > > >>>> >> > > >>>> Thanks for trying the RC. I think what you found is a blocker bug >> > > >>> because >> > > >>>> it will generate huge amount of logspam. I guess we didn't find >> it >> > in >> > > >>> junit >> > > >>>> tests since logspam doesn't fail the automated tests. But >> certainly >> > > it's >> > > >>>> not suitable for production. Did you file a JIRA yet? >> > > >>>> >> > > >>>>> On Sun, Jan 14, 2024 at 10:17 PM Stanislav Kozlovski >> > > >>>>> <stanis...@confluent.io.invalid> wrote: >> > > >>>>> >> > > >>>>>> Hey Luke, >> > > >>>>>> >> > > >>>>>> This is an interesting problem. Given the fact that the KIP for >> > > >>> having a >> > > >>>>>> 3.8 release passed, I think it weights the scale towards not >> > calling >> > > >>>> this a >> > > >>>>>> blocker and expecting it to be solved in 3.7.1. >> > > >>>>>> >> > > >>>>>> It is unfortunate that it would not seem safe to migrate to >> KRaft >> > in >> > > >>>> 3.7.0 >> > > >>>>>> (given the inability to rollback safely), but if that's true - >> the >> > > >>> same >> > > >>>>>> case would apply for 3.6.0. So in any case users w\ould be >> > expected >> > > >>> to >> > > >>>> use a >> > > >>>>>> patch release for this. >> > > >>>> >> > > >>>> Hi Luke, >> > > >>>> >> > > >>>> Thanks for testing rollback. I think this is a case where the >> > > >>>> documentation is wrong. The intention was to for the steps to >> > > basically >> > > >>> be: >> > > >>>> >> > > >>>> 1. roll all the brokers into zk mode, but with migration enabled >> > > >>>> 2. take down the kraft quorum >> > > >>>> 3. rmr /controller, allowing a hybrid broker to take over. >> > > >>>> 4. roll all the brokers into zk mode without migration enabled >> (if >> > > >>> desired) >> > > >>>> >> > > >>>> With these steps, there isn't really unavailability since a ZK >> > > >>> controller >> > > >>>> can be elected quickly after the kraft quorum is gone. >> > > >>>> >> > > >>>>>> Further, since we will have a 3.8 release - it is >> > > >>>>>> likely we will ultimately recommend users upgrade from that >> > version >> > > >>>> given >> > > >>>>>> its aim is to have strategic KRaft feature parity with ZK. >> > > >>>>>> That being said, I am not 100% on this. Let me know whether you >> > > think >> > > >>>> this >> > > >>>>>> should block the release, Luke. I am also tagging Colin and >> David >> > to >> > > >>>> weigh >> > > >>>>>> in with their opinions, as they worked on the migration logic. >> > > >>>> >> > > >>>> The rollback docs are new in 3.7 so the fact that they're wrong >> is a >> > > >>> clear >> > > >>>> blocker, I think. But easy to fix, I believe. I will create a PR. >> > > >>>> >> > > >>>> best, >> > > >>>> Colin >> > > >>>> >> > > >>>>>> >> > > >>>>>> Hey Kirk and Chris, >> > > >>>>>> >> > > >>>>>> Unless I'm missing something - KAFKALESS-16029 is simply a bad >> log >> > > >>> due >> > > >>>> to >> > > >>>>>> improper closing. And the PR description implies this has been >> > > >>> present >> > > >>>>>> since 3.5. While annoying, I don't see a strong reason for >> this to >> > > >>> block >> > > >>>>>> the release. >> > > >>>>>> >> > > >>>>>> Hey Jakub, >> > > >>>>>> >> > > >>>>>> Nice catch! It does seem like we should have gated this behind >> the >> > > >>>> metadata >> > > >>>>>> version as KIP-858 implies. Is the cluster configured with >> > multiple >> > > >>> log >> > > >>>>>> dirs? What is the impact of the error messages? >> > > >>>>>> >> > > >>>>>> Tagging Igor (the author of the KIP) to weigh in. >> > > >>>>>> >> > > >>>>>> Best, >> > > >>>>>> Stanislav >> > > >>>>>> >> > > >>>>>> On Sat, Jan 13, 2024 at 7:22 PM Jakub Scholz <ja...@scholz.cz> >> > > >>> wrote: >> > > >>>>>> >> > > >>>>>>> Hi, >> > > >>>>>>> >> > > >>>>>>> I was trying the RC2 and run into the following issue ... >> when I >> > > >>> run >> > > >>>>>>> 3.7.0-RC2 KRaft cluster with metadata version set to 3.6-IV2 >> > > >>> metadata >> > > >>>>>>> version, I seem to be getting repeated errors like this in the >> > > >>>> controller >> > > >>>>>>> logs: >> > > >>>>>>> >> > > >>>>>>> 2024-01-13 16:58:01,197 INFO [QuorumController id=0] >> > > >>>>>> assignReplicasToDirs: >> > > >>>>>>> event failed with UnsupportedVersionException in 15 >> microseconds. >> > > >>>>>>> (org.apache.kafka.controller.QuorumController) >> > > >>>>>>> [quorum-controller-0-event-handler] >> > > >>>>>>> 2024-01-13 16:58:01,197 ERROR [ControllerApis nodeId=0] >> > Unexpected >> > > >>>> error >> > > >>>>>>> handling request RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, >> > > >>>>>>> apiVersion=0, clientId=1000, correlationId=14, >> headerVersion=2) >> > -- >> > > >>>>>>> AssignReplicasToDirsRequestData(brokerId=1000, brokerEpoch=5, >> > > >>>>>>> directories=[DirectoryData(id=w_uxN7pwQ6eXSMrOKceYIQ, >> > > >>>>>>> topics=[TopicData(topicId=bvAKLSwmR7iJoKv2yZgygQ, >> > > >>>>>>> partitions=[PartitionData(partitionIndex=2), >> > > >>>>>>> PartitionData(partitionIndex=1)]), >> > > >>>>>>> TopicData(topicId=uNe7f5VrQgO0zST6yH1jDQ, >> > > >>>>>>> partitions=[PartitionData(partitionIndex=0)])])]) with context >> > > >>>>>>> >> > RequestContext(header=RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, >> > > >>>>>>> apiVersion=0, clientId=1000, correlationId=14, >> headerVersion=2), >> > > >>>>>>> connectionId='172.16.14.219:9090-172.16.14.217:53590-7', >> > > >>>> clientAddress=/ >> > > >>>>>>> 172.16.14.217, >> principal=User:CN=my-cluster-kafka,O=io.strimzi, >> > > >>>>>>> listenerName=ListenerName(CONTROLPLANE-9090), >> > securityProtocol=SSL, >> > > >>>>>>> >> > clientInformation=ClientInformation(softwareName=apache-kafka-java, >> > > >>>>>>> softwareVersion=3.7.0), fromPrivilegedListener=false, >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@71004ad2 >> > > >>>>>>> ]) >> > > >>>>>>> (kafka.server.ControllerApis) >> [quorum-controller-0-event-handler] >> > > >>>>>>> java.util.concurrent.CompletionException: >> > > >>>>>>> org.apache.kafka.common.errors.UnsupportedVersionException: >> > > >>> Directory >> > > >>>>>>> assignment is not supported yet. >> > > >>>>>>> >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:332) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:347) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:636) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.controller.QuorumController$ControllerWriteEvent.complete(QuorumController.java:880) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.controller.QuorumController$ControllerWriteEvent.handleException(QuorumController.java:871) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.queue.KafkaEventQueue$EventContext.completeWithException(KafkaEventQueue.java:148) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:137) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210) >> > > >>>>>>> at >> > > >>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181) >> > > >>>>>>> at java.base/java.lang.Thread.run(Thread.java:840) >> > > >>>>>>> >> > > >>>>>>> Caused by: >> > > >>> org.apache.kafka.common.errors.UnsupportedVersionException: >> > > >>>>>>> Directory assignment is not supported yet. >> > > >>>>>>> >> > > >>>>>>> Is that expected? I guess with the metadata version set to >> > > >>> 3.6-IV2, it >> > > >>>>>>> makes sense that the request is not supported. But shouldn't >> then >> > > >>> the >> > > >>>>>>> request not be sent at all by the brokers? (I did not opened a >> > JIRA >> > > >>>> for >> > > >>>>>> it, >> > > >>>>>>> but I can open one if you agree this is not expected) >> > > >>>>>>> >> > > >>>>>>> Thanks & Regards >> > > >>>>>>> Jakub >> > > >>>>>>> >> > > >>>>>>> On Sat, Jan 13, 2024 at 8:03 AM Luke Chen <show...@gmail.com> >> > > >>> wrote: >> > > >>>>>>> >> > > >>>>>>>> Hi Stanislav, >> > > >>>>>>>> >> > > >>>>>>>> I commented in the "Apache Kafka 3.7.0 Release" thread, but >> > maybe >> > > >>>> you >> > > >>>>>>>> missed it. >> > > >>>>>>>> cross-posting here: >> > > >>>>>>>> >> > > >>>>>>>> There is a bug KAFKA-16101 >> > > >>>>>>>> <https://issues.apache.org/jira/browse/KAFKA-16101> >> reporting >> > > >>> that >> > > >>>>>>> "Kafka >> > > >>>>>>>> cluster will be unavailable during KRaft migration rollback". >> > > >>>>>>>> The impact for this issue is that if brokers try to rollback >> to >> > > >>> ZK >> > > >>>> mode >> > > >>>>>>>> during KRaft migration process, there will be a period of >> time >> > > >>> the >> > > >>>>>>> cluster >> > > >>>>>>>> is unavailable. >> > > >>>>>>>> Since ZK migrating to KRaft feature is a production ready >> > > >>> feature, I >> > > >>>>>>> think >> > > >>>>>>>> this should be addressed soon. >> > > >>>>>>>> Do you think this is a blocker for v3.7.0? >> > > >>>>>>>> >> > > >>>>>>>> Thanks. >> > > >>>>>>>> Luke >> > > >>>>>>>> >> > > >>>>>>>> On Sat, Jan 13, 2024 at 8:36 AM Chris Egerton < >> > > >>>> fearthecel...@gmail.com >> > > >>>>>>> >> > > >>>>>>>> wrote: >> > > >>>>>>>> >> > > >>>>>>>>> Thanks, Kirk! >> > > >>>>>>>>> >> > > >>>>>>>>> @Stanislav--do you believe that this warrants a new RC? >> > > >>>>>>>>> >> > > >>>>>>>>> On Fri, Jan 12, 2024, 19:08 Kirk True <k...@kirktrue.pro> >> > > >>> wrote: >> > > >>>>>>>>> >> > > >>>>>>>>>> Hi Chris/Stanislav, >> > > >>>>>>>>>> >> > > >>>>>>>>>> I'm working on the 'Unable to find FetchSessionHandler' log >> > > >>>> problem >> > > >>>>>>>>>> (KAFKA-16029) and have put out a draft PR ( >> > > >>>>>>>>>> https://github.com/apache/kafka/pull/15186). I will use >> the >> > > >>>>>>> quickstart >> > > >>>>>>>>>> approach as a second means to reproduce/verify while I wait >> > > >>> for >> > > >>>> the >> > > >>>>>>>> PR's >> > > >>>>>>>>>> Jenkins job to finish. >> > > >>>>>>>>>> >> > > >>>>>>>>>> Thanks, >> > > >>>>>>>>>> Kirk >> > > >>>>>>>>>> >> > > >>>>>>>>>> On Fri, Jan 12, 2024, at 11:31 AM, Chris Egerton wrote: >> > > >>>>>>>>>>> Hi Stanislav, >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> Thanks for running this release! >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> To verify, I: >> > > >>>>>>>>>>> - Built from source using Java 11 with both: >> > > >>>>>>>>>>> - - the 3.7.0-rc2 tag on GitHub >> > > >>>>>>>>>>> - - the kafka-3.7.0-src.tgz artifact from >> > > >>>>>>>>>>> >> > > >>> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc2/ >> > > >>>>>>>>>>> - Checked signatures and checksums >> > > >>>>>>>>>>> - Ran the quickstart using both: >> > > >>>>>>>>>>> - - The kafka_2.13-3.7.0.tgz artifact from >> > > >>>>>>>>>>> >> > > >>> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc2/ >> > > >>>>>> with >> > > >>>>>>>> Java >> > > >>>>>>>>>> 11 >> > > >>>>>>>>>>> and Scala 13 in KRaft mode >> > > >>>>>>>>>>> - - Our shiny new broker Docker image, >> > > >>> apache/kafka:3.7.0-rc2 >> > > >>>>>>>>>>> - Ran all unit tests >> > > >>>>>>>>>>> - Ran all integration tests for Connect and MM2 >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> I found two minor areas for concern: >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> 1. (Possibly a blocker) >> > > >>>>>>>>>>> When running the quickstart, I noticed this ERROR-level >> log >> > > >>>>>> message >> > > >>>>>>>>> being >> > > >>>>>>>>>>> emitted frequently (not not every time) when I killed my >> > > >>>> console >> > > >>>>>>>>> consumer >> > > >>>>>>>>>>> via ctrl-C: >> > > >>>>>>>>>>> >> > > >>>>>>>>>>>> [2024-01-12 11:00:31,088] ERROR [Consumer >> > > >>>>>>>> clientId=console-consumer, >> > > >>>>>>>>>>> groupId=console-consumer-74388] Unable to find >> > > >>>>>> FetchSessionHandler >> > > >>>>>>>> for >> > > >>>>>>>>>> node >> > > >>>>>>>>>>> 1. Ignoring fetch response >> > > >>>>>>>>>>> >> (org.apache.kafka.clients.consumer.internals.AbstractFetch) >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> I see that this error message is already reported in >> > > >>>>>>>>>>> https://issues.apache.org/jira/browse/KAFKA-16029. I >> > > >>> think we >> > > >>>>>>> should >> > > >>>>>>>>>>> prioritize fixing it for this release. I know it's >> probably >> > > >>>>>> benign >> > > >>>>>>>> but >> > > >>>>>>>>>> it's >> > > >>>>>>>>>>> really not a good look for us when basic operations log >> > > >>> error >> > > >>>>>>>> messages, >> > > >>>>>>>>>> and >> > > >>>>>>>>>>> it may give new users some headaches. >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> 2. (Probably not a blocker) >> > > >>>>>>>>>>> The following unit tests failed the first time around, and >> > > >>>> all of >> > > >>>>>>>> them >> > > >>>>>>>>>>> passed the second time I ran them: >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> - (clients) >> > > >>>>>>>>>> >> > > >>> ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup() >> > > >>>>>>>>>>> - (clients) SelectorTest.testConnectionsByClientMetric() >> > > >>>>>>>>>>> - (clients) >> > > >>> Tls13SelectorTest.testConnectionsByClientMetric() >> > > >>>>>>>>>>> - (connect) >> > > >>>>>>>> TopicAdminTest.retryEndOffsetsShouldRetryWhenTopicNotFound >> > > >>>>>>>>> (I >> > > >>>>>>>>>>> thought I fixed this one! 🤬🤬) >> > > >>>>>>>>>>> - (core) >> > > >>>> ProducerIdManagerTest.testUnrecoverableErrors(Errors)[2] >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> Thanks again for your work on this release, and >> > > >>>> congratulations >> > > >>>>>> to >> > > >>>>>>>>> Kafka >> > > >>>>>>>>>>> Streams for having zero flaky unit tests during my >> > > >>>>>>>> highly-experimental >> > > >>>>>>>>>>> single laptop run! >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> Cheers, >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> Chris >> > > >>>>>>>>>>> >> > > >>>>>>>>>>> On Thu, Jan 11, 2024 at 1:33 PM Stanislav Kozlovski >> > > >>>>>>>>>>> <stanis...@confluent.io.invalid> wrote: >> > > >>>>>>>>>>> >> > > >>>>>>>>>>>> Hello Kafka users, developers, and client-developers, >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> This is the first candidate for release of Apache Kafka >> > > >>>> 3.7.0. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Note it's named "RC2" because I had a few "failed" RCs >> > > >>> that >> > > >>>> I >> > > >>>>>> had >> > > >>>>>>>>>>>> cut/uploaded but ultimately had to scrap prior to >> > > >>> announcing >> > > >>>>>> due >> > > >>>>>>> to >> > > >>>>>>>>> new >> > > >>>>>>>>>>>> blockers arriving before I could even announce them. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Further - I haven't yet been able to set up the system >> > > >>> tests >> > > >>>>>>>>>> successfully. >> > > >>>>>>>>>>>> And the integration/unit tests do have a few failures >> > > >>> that I >> > > >>>>>> have >> > > >>>>>>>> to >> > > >>>>>>>>>> spend >> > > >>>>>>>>>>>> time triaging. I would appreciate any help in case anyone >> > > >>>>>> notices >> > > >>>>>>>> any >> > > >>>>>>>>>> tests >> > > >>>>>>>>>>>> failing that they're subject matters experts in. Expect >> > > >>> me >> > > >>>> to >> > > >>>>>>>> follow >> > > >>>>>>>>>> up in >> > > >>>>>>>>>>>> a day or two with more detailed analysis. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Major changes include: >> > > >>>>>>>>>>>> - Early Access to KIP-848 - the next generation of the >> > > >>>> consumer >> > > >>>>>>>>>> rebalance >> > > >>>>>>>>>>>> protocol >> > > >>>>>>>>>>>> - KIP-858: Adding JBOD support to KRaft >> > > >>>>>>>>>>>> - KIP-714: Observability into Client metrics via a >> > > >>>> standardized >> > > >>>>>>>>>> interface >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Check more information in the WIP blog post: >> > > >>>>>>>>>>>> https://github.com/apache/kafka-site/pull/578 >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Release notes for the 3.7.0 release: >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> >> > > >>>>>>>>>> >> > > >>>>>>>>> >> > > >>>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc2/RELEASE_NOTES.html >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> *** Please download, test and vote by Thursday, January >> > > >>> 18, >> > > >>>> 9am >> > > >>>>>>> PT >> > > >>>>>>>>> *** >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Usually these deadlines tend to be 2-3 days, but due to >> > > >>> this >> > > >>>>>>> being >> > > >>>>>>>>> the >> > > >>>>>>>>>>>> first RC and the tests not having ran yet, I am giving >> > > >>> it a >> > > >>>> bit >> > > >>>>>>>> more >> > > >>>>>>>>>> time. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Kafka's KEYS file containing PGP keys we use to sign the >> > > >>>>>> release: >> > > >>>>>>>>>>>> https://kafka.apache.org/KEYS >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Release artifacts to be voted upon (source and binary): >> > > >>>>>>>>>>>> >> > > >>>> https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc2/ >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Docker release artifact to be voted upon: >> > > >>>>>>>>>>>> apache/kafka:3.7.0-rc2 >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Maven artifacts to be voted upon: >> > > >>>>>>>>>>>> >> > > >>>>>>>>> >> > > >>>>>> >> > > >>> >> > https://repository.apache.org/content/groups/staging/org/apache/kafka/ >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Javadoc: >> > > >>>>>>>>>>>> >> > > >>>>>>>> >> > > >>>> >> > https://home.apache.org/~stanislavkozlovski/kafka-3.7.0-rc2/javadoc/ >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Tag to be voted upon (off 3.7 branch) is the 3.7.0 tag: >> > > >>>>>>>>>>>> https://github.com/apache/kafka/releases/tag/3.7.0-rc2 >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Documentation: >> > > >>>>>>>>>>>> https://kafka.apache.org/37/documentation.html >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Protocol: >> > > >>>>>>>>>>>> https://kafka.apache.org/37/protocol.html >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Successful Jenkins builds for the 3.7 branch: >> > > >>>>>>>>>>>> Unit/integration tests: >> > > >>>>>>>>>>>> >> > > >>>> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.7/58/ >> > > >>>>>>>>>>>> There are failing tests here. I have to follow up with >> > > >>>> triaging >> > > >>>>>>>> some >> > > >>>>>>>>> of >> > > >>>>>>>>>>>> the failures and figuring out if they're actual problems >> > > >>> or >> > > >>>>>>> simply >> > > >>>>>>>>>> flakes. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> System tests: >> > > >>>>>>>>>> >> https://jenkins.confluent.io/job/system-test-kafka/job/3.7/ >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> No successful system test runs yet. I am working on >> > > >>> getting >> > > >>>> the >> > > >>>>>>> job >> > > >>>>>>>>> to >> > > >>>>>>>>>> run. >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> * Successful Docker Image Github Actions Pipeline for 3.7 >> > > >>>>>> branch: >> > > >>>>>>>>>>>> Attached are the scan_report and report_jvm output files >> > > >>>> from >> > > >>>>>> the >> > > >>>>>>>>>> Docker >> > > >>>>>>>>>>>> Build run: >> > > >>>>>>>>>>>> >> > > >>>>>>>>>> >> > > >>>>>>>> >> > > >>>>>> >> > > >>> >> > > >> https://github.com/apache/kafka/actions/runs/7486094960/job/20375761673 >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> And the final docker image build job - Docker Build Test >> > > >>>>>>> Pipeline: >> > > >>>>>>>>>>>> https://github.com/apache/kafka/actions/runs/7486178277 >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> The image is apache/kafka:3.7.0-rc2 - >> > > >>>>>>>>>>>> >> > > >>>>>>>>>> >> > > >>>>>>>>> >> > > >>>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> > >> https://hub.docker.com/layers/apache/kafka/3.7.0-rc2/images/sha256-5b4707c08170d39549fbb6e2a3dbb83936a50f987c0c097f23cb26b4c210c226?context=explore >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> /************************************** >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>>> Thanks, >> > > >>>>>>>>>>>> Stanislav Kozlovski >> > > >>>>>>>>>>>> >> > > >>>>>>>>>>> >> > > >>>>>>>>>> >> > > >>>>>>>>> >> > > >>>>>>>> >> > > >>>>>>> >> > > >>>>>> >> > > >>>>>> >> > > >>>>>> -- >> > > >>>>>> Best, >> > > >>>>>> Stanislav >> > > >>>>>> >> > > >>>> >> > > >>> >> > > >> >> > > > >> > > >> > >> >> >> -- >> Best, >> Stanislav >> >