Re: [DISCUSS] KIP-932: Queues for Kafka

2024-04-03 Thread Jun Rao
Hi, Andrew,

Thanks for the updated KIP. A few more comments.

40. It would be useful to document the flow for serving the initial
ShareGroupHeartbeatRequest. The coordinator needs to (1) write the
ConsumerGroupMetadata record, the ConsumerGroupMemberMetadata record, the
initial ShareGroupPartitionMetadata record, (2) issue the listOffset
request to find the starting offset for each partition, (3) send
InitializeShareGroupState to share coordinator, which will need to write
the ShareCheckpoint record, (4) write the finalized
ShareGroupPartitionMetadata record. When will the group coordinator send
the response to the initial ShareGroupHeartbeatRequest? Until all those
steps are done? Will the group coordinator hold the group lock for all the
steps? If another member wants to join at the same time, how long will it
be blocked?

41. It would be useful to clarify the flow to initialize the
ShareGroupState.
One part of the KIP says
" When a share-partition leader receives its first ShareFetch request for a
topic-partition, it needs to initialize the share-partition state. It finds
the share coordinator using the FindCoordinator RPC using (key:
"groupId:topicId:partition", key_type: SHARE ). Then, it sends the
ReadShareGroupState RPC to the share coordinator. If the share coordinator
has no share-partition state to return, it returns the
UNKNOWN_TOPIC_OR_PARTITION error code. This tells the share-partition
leader to initialize the SPSO based on the group.share.auto.offset.reset
configuration."

Another part of the KIP says
"When a topic-partition is assigned to a member of a share group for the
first time, the group coordinator writes a ShareGroupPartitionMetadata
record to the __consumer_offsets  topic and sends an
InitializeShareGroupState  request to the share coordinator."

So, is it the group coordinator or the share-partition leader that
initializes the ShareGroupState? If it's the latter, it doesn't seem that
the share-partition leader has the group epoch, which is needed in the
InitializeShareGroupState request.

42. Group epoch: Should we bump it up when altering share group offset?
Otherwise, it's not clear how we could handle retries properly.

43. Could we document the process in the group coordinator when a new
partition is added?

44. What's the usage of ShareFetchRequest.ShareSessionEpoch? From the KIP,
it seems it's mostly to distinguish between a full and an incremental fetch?

45. Some of the new value records like ShareGroupPartitionMetadataValue
have "flexibleVersions": "none". Should we make them flexible so that we
could support downgrading in the future?

46. "The share-partition leader clearly has to look within the data
returned from the replica manager in order to understand the record batches
it fetches. This means that records retrieved using a share group are not
able to benefit from the zero-copy optimisation."
In the common case, does the broker use FileRecords or MemoryRecords in
ShareFetchResponse? If it's the former, zero-copy can be used.

47. AcknowledgementBatches:
47.1 Does each batch get acked separately? If a consumer processes multiple
consecutive batches, it's more efficient to have a single  that covers multiple batches.
47.2 It's also possible that the same batch may be acked multiple times.
For example, a batch may include offset 1-7. The first 4 records get acked
first and the last 4 get acked next. In this case, do we send the first
AcknowledgementBatches with =<1, 4> and the second
AcknowledgementBatches with =<5, 7>? If so, it
seems that we should rename BaseOffset to sth like StartOffset.

48. "The share-partition leader does not maintain an explicit cache of
records that it has fetched."
48.1 What's being cached in in-flight records, just the batch headers?
48.2 What's the value of in-flight records? Is it configurable?

49. Could we document the cases when the broker needs to decompress a
batch? For example, this is needed when redelivering a subset of the
records in a batch.

50. In read_uncommitted mode, does the broker return the endMarker record
batches to the client? If so, how does the client ack since the control
records are not returned to the application?

51. "If acknowledgements are being made for a partition and no records
should be fetched, PartitionMaxBytes  should be set to zero."
If we do that, is there a need for a separate ShareAcknowledgeRequest?

52. Could we document how ShareAcknowledgeRequest is served on the broker?
Does it need to wait for the share-partition leader to send
WriteShareGroupStateRequest to the share coordinator and wait for the
commit of the records in __share_group_state topic?

53. In the example Acknowledge 119 (consumer 3), WriteShareGroupState
request contains the following.
{
  "BaseOffset": 111,
  "LastOffset": 118,
  "State": 0 (Available),   "DeliveryCount": 0
},
The state should be acquired, not available, right? Also, I thought we
don't track acquired records.

54. Should we rename 

[PR] MINOR: make ZK migrating to KRaft feature as production ready [kafka-site]

2024-04-03 Thread via GitHub


showuon opened a new pull request, #594:
URL: https://github.com/apache/kafka-site/pull/594

   Ref: https://github.com/apache/kafka/pull/15552
   
   The current document about Zookeeper to kraft migration is depreciated.
   Accord to [KIP-833: Mark KRaft as Production 
Ready](https://cwiki.apache.org/confluence/display/KAFKA/KIP-833%3A+Mark+KRaft+as+Production+Ready),
 the migration from ZK mode supported as GA.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #2780

2024-04-03 Thread Apache Jenkins Server
See 




Re: [DISCUSS] KIP-932: Queues for Kafka

2024-04-03 Thread Tom Bentley
Hi Andrew (and Omnia),

Thanks for the KIP. I hope to provide some feedback on this KIP soon, but I
had a thought on the specific subject of group configs and MM2. If brokers
validate for known groups configs then doesn't this induce an ordering
requirement on upgrading clusters: Wouldn't you have to upgrade a
destination cluster first, in order that it knew about `group.type`,
otherwise it would reject attempts to configure an unknown group config
parameter? A similar issue arises wrt topic configs, but this is the first
instance (that I'm aware of) of a config being added during the MM2 era, so
perhaps this is a minor problem worth thinking about.

Cheers,

Tom

On Wed, 3 Apr 2024 at 05:31, Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:

> Hi Omnia,
> Thanks for your questions.
>
> The DR angle on `group.type` is interesting and I had not considered it.
> The namespace of groups contains
> both consumer groups and share groups, so I was trying to ensure that
> which group type was used was
> deterministic rather than a race to create the first member. There are
> already other uses of the group protocol
> such as Kafka Connect, so it’s all a bit confusing even today.
>
> It is actually KIP-848 which introduces configurations for group resources
> and KIP-932 is just building on
> the idea. I think that MM2 will need to sync these configurations. The
> question of whether `group.type` is
> a sensible configuration I think is separate.
>
> Imagine that we do have `group.type` as a group configuration. How would
> we end up with groups with
> the same ID but different types on the two ends of MM2? Assuming that both
> ends have KIP-932 enabled,
> either the configuration was not set, and a consumer group was made on one
> end while a share group was
> made on the other, OR, the configuration was set but its value changed,
> and again we get a divergence.
>
> I think that on balance, having `group.type` as a configuration does at
> least mean there’s a better chance that
> the two ends of MM2 do agree on the type of group. I’m happy to consider
> other ways to do this better. The
> fact that we have different kinds of group in the same namespace is the
> tricky thing. I think this was possible
> before this KIP, but it’s much more likely now.
>
>
> Onto the question of memory. There are several different parts to this,
> all of which are distributed across
> the cluster.
>
> * For the group coordinator, the memory consumption will be affected by
> the number of groups,
> the number of members and the number of topic-partitions to be assigned to
> the members. The
> group coordinator is concerned with membership and assignment, so the
> memory per topic-partition
> will be small.
> * For the share coordinator, the memory consumption will be affected by
> the number of groups, the
> number of topic-partitions being consumed in the group, and the number of
> in-flight records, but not
> the number of members. We should be talking about no more than kilobytes
> per topic-partition.
> * For the share-partition leader, the memory consumption will be affected
> by the number of share group
> members assigned the topic-partition and the number of in-flight records.
> Again, we should be talking
> about no more than kilobytes per topic-partition.
>
> Of these, the factor that is not directly under control is the number of
> topic-partitions. The reason is that
> I wanted to avoid a situation where the number of partitions in a topic
> was increased and suddenly
> consumption in a share group hit a limit that was not anticipated.
>
> I could introduce a configuration for controlling the number of topics
> allowed to be subscribed in a share
> group. Personally, I think 1 would be a good starting point.
>
> Let me know what you think.
>
> Thanks,
> Andrew
>
>
> > On 2 Apr 2024, at 15:39, Omnia Ibrahim  wrote:
> >
> > Hi Andrew,
> > Thanks for the KIP it is definitely an interesting read. I have few
> questions
> > As the KIP proposing extending `AdminClient.incrementalAlterConfigs` to
> add an explicit `group.type` what would this means for DR feature in MM2
> offering?
> > Right now MM2 sync consumer group offsets from source to destination
> cluster. And it also offer sync ACLs which contribute to DR feature. Would
> this KIP means MM2 needs to also sync the type of groups to destination?
> > As `AdminClient.incrementalAlterConfigs` means "when a new group is
> created with this name, it must have this type”. What will happened if
> clusters on both ends of MM2 has same group id but with different types?
> > If this concern is out of the scope we might need to call this out
> somewhere in the KIP.
> > While the number of share-group and the number of consumers in
> share-group is limited by `group.share.max.groups`and
> `group.share.max.size` the total number of share-group state records that
> might need to be loaded in-memeory has another factor which is the number
> of partitions. In cases where 

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

2024-04-03 Thread Justine Olshan
Find and replace has failed me :(

Group version seems a little vague, but we can update it. Hopefully find
and replace won't fail me again, otherwise I will get another email on this.

Justine

On Wed, Apr 3, 2024 at 12:15 PM David Jacot 
wrote:

> Thanks, Justine.
>
> * Should we also use `group.version` (GV) as I suggested in my previous
> message in order to be consistent?
> * Should we add both names to the `Public Interfaces` section?
> * There is still at least one usage of `transaction.protocol.verison` in
> the KIP too.
>
> Best,
> David
>
> On Wed, Apr 3, 2024 at 6:29 PM Justine Olshan  >
> wrote:
>
> > I had missed the David's message yesterday about the naming for
> transaction
> > version vs transaction protocol version.
> >
> > After some offline discussion with Jun, Artem, and David, we agreed that
> > transaction version is simpler and conveys more than just protocol
> changes
> > (flexible records for example)
> >
> > I will update the KIP as well as KIP-890
> >
> > Thanks,
> > Justine
> >
> > On Tue, Apr 2, 2024 at 2:50 PM Justine Olshan 
> > wrote:
> >
> > > Updated!
> > >
> > > Justine
> > >
> > > On Tue, Apr 2, 2024 at 2:40 PM Jun Rao 
> wrote:
> > >
> > >> Hi, Justine,
> > >>
> > >> Thanks for the reply.
> > >>
> > >> 21. Sounds good. It would be useful to document that.
> > >>
> > >> 22. Should we add the IV in "metadata.version=17 has no dependencies"
> > too?
> > >>
> > >> Jun
> > >>
> > >>
> > >> On Tue, Apr 2, 2024 at 11:31 AM Justine Olshan
> > >> 
> > >> wrote:
> > >>
> > >> > Jun,
> > >> >
> > >> > 21. Next producer ID field doesn't need to be populated for TV 1. We
> > >> don't
> > >> > have the same need to retain this since it is written directly to
> the
> > >> > transaction log in InitProducerId. It is only needed for KIP-890
> part
> > 2
> > >> /
> > >> > TV 2.
> > >> >
> > >> > 22. We can do that.
> > >> >
> > >> > Justine
> > >> >
> > >> > On Tue, Apr 2, 2024 at 10:41 AM Jun Rao 
> > >> wrote:
> > >> >
> > >> > > Hi, Justine,
> > >> > >
> > >> > > Thanks for the reply.
> > >> > >
> > >> > > 21. What about the new NextProducerId field? Will that be
> populated
> > >> with
> > >> > TV
> > >> > > 1?
> > >> > >
> > >> > > 22. In the dependencies output, should we show both IV and level
> for
> > >> > > metadata.version too?
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > > On Mon, Apr 1, 2024 at 4:43 PM Justine Olshan
> > >> >  > >> > > >
> > >> > > wrote:
> > >> > >
> > >> > > > Hi Jun,
> > >> > > >
> > >> > > > 20. I can update the KIP.
> > >> > > >
> > >> > > > 21. This is used to complete some of the work with KIP-360. (We
> > use
> > >> > > > previous producer ID there, but never persisted it which was in
> > the
> > >> KIP
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89068820
> > >> )
> > >> > > > The KIP also mentions including previous epoch but we explained
> in
> > >> this
> > >> > > KIP
> > >> > > > how we can figure this out.
> > >> > > >
> > >> > > > Justine
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Mon, Apr 1, 2024 at 3:56 PM Jun Rao  >
> > >> > wrote:
> > >> > > >
> > >> > > > > Hi, Justine,
> > >> > > > >
> > >> > > > > Thanks for the updated KIP. A couple of more comments.
> > >> > > > >
> > >> > > > > 20. Could we show the output of version-mapping?
> > >> > > > >
> > >> > > > > 21. "Transaction version 1 will include the flexible fields in
> > the
> > >> > > > > transaction state log, and transaction version 2 will include
> > the
> > >> > > changes
> > >> > > > > to the transactional protocol as described by KIP-890 (epoch
> > bumps
> > >> > and
> > >> > > > > implicit add partitions.)"
> > >> > > > >   So TV 1 enables the writing of new tagged fields like
> > >> > PrevProducerId?
> > >> > > > But
> > >> > > > > those fields are only usable after the epoch bump, right? What
> > >> > > > > functionality does TV 1 achieve?
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > >
> > >> > > > > On Mon, Apr 1, 2024 at 2:06 PM Justine Olshan
> > >> > > >  > >> > > > > >
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > > > I have also updated the KIP to mention the feature tool's
> > >> > --metadata
> > >> > > > flag
> > >> > > > > > will be deprecated.
> > >> > > > > > It will still work for users as they learn the new flag,
> but a
> > >> > > warning
> > >> > > > > > indicating the alternatives will be shown.
> > >> > > > > >
> > >> > > > > > Justine
> > >> > > > > >
> > >> > > > > > On Thu, Mar 28, 2024 at 11:03 AM Justine Olshan <
> > >> > > jols...@confluent.io>
> > >> > > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Hi Jun,
> > >> > > > > > >
> > >> > > > > > > For both transaction state and group coordinator state,
> > there
> > >> are
> > >> > > > only
> > >> > > > > > > version 0 records.
> > >> > > > > > > KIP-915 introduced flexible versions, but it was never put
> > to
> > >> > use.
> > >> > > MV
> > >> > > > > has
> > >> > > > > > > never gated these. This 

Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Justine Olshan
Thanks for clarifying!
I took a look at the documentation.html file in there, and it said 3.4. Is
that expected?

There are some files that request fullDot version and that seemed closer to
what was expected: "fullDotVersion": "3.6.2-SNAPSHOT"
The upgrade.html file also looked ok.

Thanks for running the release and answering my questions!
Justine

On Wed, Apr 3, 2024 at 10:21 AM Manikumar  wrote:

> Hi Justine,
>
> Yes, it is intended. For bug fix releases website docs will be updated
> during the final release process.
> We can verify the site-docs artifacts here:
>
> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/kafka_2.12-3.6.2-site-docs.tgz
> These site-docs artifacts will be used to update website docs.
>
>
> Thanks,
>
> On Wed, Apr 3, 2024 at 10:30 PM Justine Olshan
> 
> wrote:
>
> > Hi Manikumar,
> >
> > I've verified the keys, scanned the artifacts, and other docs.
> > I built from source and ran with a ZK cluster (since I saw that we
> updated
> > ZK version in this release)
> > I ran a few tests on this cluster.
> >
> > I also ran the 2.12 binary.
> >
> > I noticed the docs link (https://kafka.apache.org/36/documentation.html)
> > mentions 3.6.1 as the latest. Is that intended?
> > I will give my final vote when we figure this out.
> >
> > Thanks,
> > Justine
> >
> > On Wed, Apr 3, 2024 at 7:25 AM Lianet M.  wrote:
> >
> > > Hi Manikumar, I did the following checks:
> > >
> > > - downloaded and built from src
> > > - ran all unit test and integration test for clients
> > > - ran quickstart with Kraft mode
> > > - ran simple workloads with the console consumer/producer
> > > - checked all links
> > >
> > > All looks good to me with this.
> > >
> > > +1 (non-binding)
> > >
> > > Thanks!
> > >
> > >
> > >
> > > On Wed, Apr 3, 2024, 2:19 a.m. Manikumar  wrote:
> > >
> > > > Gentle reminder. Please download, test and vote for the release.
> > > >
> > > > Thanks,
> > > >
> > > > On Fri, Mar 29, 2024 at 4:57 PM Manikumar 
> > wrote:
> > > >
> > > > > Hi All,
> > > > >
> > > > > System test runs are green. There were 13 test failures in the
> first
> > > run.
> > > > > All the failed tests passed in the second run.
> > > > >
> > > > > System test results:
> > > > > https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
> > > > >
> > > > > Thanks,
> > > > >
> > > > > On Thu, Mar 28, 2024 at 3:21 PM Manikumar 
> > > wrote:
> > > > >
> > > > >> Hello Kafka users, developers and client-developers,
> > > > >>
> > > > >> This is the second candidate we are considering for the release of
> > > > Apache
> > > > >> Kafka 3.6.2.
> > > > >>
> > > > >> This is a bugfix release with several fixes, including dependency
> > > > >> version bumps for CVEs.
> > > > >>
> > > > >> Release notes for the 3.6.2 release:
> > > > >>
> > https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
> > > > >>
> > > > >> *** Please download, test and vote by by Wednesday, April 3rd
> > > > >>
> > > > >> 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/~manikumar/kafka-3.6.2-rc2/
> > > > >>
> > > > >>
> > > > >> * Maven artifacts to be voted upon:
> > > > >>
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > >>
> > > > >> * Javadoc:
> > > > >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
> > > > >>
> > > > >> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
> > > > >> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
> > > > >>
> > > > >> * Documentation:
> > > > >> https://kafka.apache.org/36/documentation.html
> > > > >>
> > > > >> * Protocol:
> > > > >> https://kafka.apache.org/36/protocol.html
> > > > >>
> > > > >> * Successful Jenkins builds for the 3.6 branch:
> > > > >> Unit/integration tests:
> > > > >> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/
> (test
> > > > >> failures passed in local runs)
> > > > >> System tests: I will update system test results soon.
> > > > >>
> > > > >>
> > > > >> Thanks,
> > > > >> Manikumar
> > > > >>
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] KIP-1020 Move `window.size.ms` and `windowed.inner.serde.class` from `StreamsConfig` to TimeWindowedDe/Serializer class

2024-04-03 Thread Sophie Blee-Goldman
+1 (binding)

Thanks Lucia!

On Tue, Apr 2, 2024 at 12:23 AM Matthias J. Sax  wrote:

> +1 (binding)
>
>
> -Matthias
>
> On 4/1/24 7:44 PM, Lucia Cerchie wrote:
> > Hello everyone,
> >
> > I'd like to call a vote on KIP-1020
> > <
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=290982804
> >.
> > It has been under discussion since Feb 15, and has received edits to the
> > KIP and approval by discussion participants.
> >
> > Best,
> > Lucia Cerchie
> >
>


Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.5 #107

2024-04-03 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 566445 lines...]
[2024-04-03T20:38:11.915Z] > Task :raft:testClasses UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task :connect:json:testClasses UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task :group-coordinator:compileTestJava UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task :group-coordinator:testClasses UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task :connect:json:testJar
[2024-04-03T20:38:11.915Z] > Task :connect:json:testSrcJar
[2024-04-03T20:38:11.915Z] > Task :metadata:compileTestJava UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task :metadata:testClasses UP-TO-DATE
[2024-04-03T20:38:11.915Z] > Task 
:clients:generateMetadataFileForMavenJavaPublication
[2024-04-03T20:38:14.767Z] 
[2024-04-03T20:38:14.767Z] > Task :connect:api:javadoc
[2024-04-03T20:38:14.768Z] 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_3.5/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java:44:
 warning - Tag @link: reference not found: org.apache.kafka.connect.data
[2024-04-03T20:38:15.870Z] 1 warning
[2024-04-03T20:38:15.870Z] 
[2024-04-03T20:38:15.870Z] > Task :connect:api:copyDependantLibs UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task :connect:api:jar UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task 
:connect:api:generateMetadataFileForMavenJavaPublication
[2024-04-03T20:38:15.870Z] > Task :connect:json:copyDependantLibs UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task :connect:json:jar UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task 
:connect:json:generateMetadataFileForMavenJavaPublication
[2024-04-03T20:38:15.870Z] > Task :connect:api:javadocJar
[2024-04-03T20:38:15.870Z] > Task :connect:api:compileTestJava UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task :connect:api:testClasses UP-TO-DATE
[2024-04-03T20:38:15.870Z] > Task 
:connect:json:publishMavenJavaPublicationToMavenLocal
[2024-04-03T20:38:15.870Z] > Task :connect:json:publishToMavenLocal
[2024-04-03T20:38:15.870Z] > Task :connect:api:testJar
[2024-04-03T20:38:15.870Z] > Task :connect:api:testSrcJar
[2024-04-03T20:38:15.870Z] > Task 
:connect:api:publishMavenJavaPublicationToMavenLocal
[2024-04-03T20:38:15.870Z] > Task :connect:api:publishToMavenLocal
[2024-04-03T20:38:18.385Z] > Task :streams:javadoc
[2024-04-03T20:38:18.385Z] > Task :streams:javadocJar
[2024-04-03T20:38:19.745Z] 
[2024-04-03T20:38:19.745Z] > Task :clients:javadoc
[2024-04-03T20:38:19.745Z] 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_3.5/clients/src/main/java/org/apache/kafka/clients/admin/ScramMechanism.java:32:
 warning - Tag @see: missing final '>': "https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API;>KIP-554:
 Add Broker-side SCRAM Config API
[2024-04-03T20:38:19.745Z] 
[2024-04-03T20:38:19.745Z]  This code is duplicated in 
org.apache.kafka.common.security.scram.internals.ScramMechanism.
[2024-04-03T20:38:19.745Z]  The type field in both files must match and must 
not change. The type field
[2024-04-03T20:38:19.745Z]  is used both for passing ScramCredentialUpsertion 
and for the internal
[2024-04-03T20:38:19.745Z]  UserScramCredentialRecord. Do not change the type 
field."
[2024-04-03T20:38:20.847Z] 
/home/jenkins/jenkins-agent/workspace/Kafka_kafka_3.5/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/secured/package-info.java:21:
 warning - Tag @link: reference not found: 
org.apache.kafka.common.security.oauthbearer
[2024-04-03T20:38:20.847Z] 2 warnings
[2024-04-03T20:38:22.207Z] 
[2024-04-03T20:38:22.207Z] > Task :clients:javadocJar
[2024-04-03T20:38:22.207Z] > Task :clients:srcJar
[2024-04-03T20:38:22.207Z] > Task :clients:testJar
[2024-04-03T20:38:23.308Z] > Task :clients:testSrcJar
[2024-04-03T20:38:23.308Z] > Task 
:clients:publishMavenJavaPublicationToMavenLocal
[2024-04-03T20:38:23.308Z] > Task :clients:publishToMavenLocal
[2024-04-03T20:38:43.034Z] > Task :core:compileScala
[2024-04-03T20:39:44.773Z] > Task :core:classes
[2024-04-03T20:39:44.773Z] > Task :core:compileTestJava NO-SOURCE
[2024-04-03T20:40:10.433Z] > Task :core:compileTestScala
[2024-04-03T20:41:03.546Z] > Task :core:testClasses
[2024-04-03T20:41:03.546Z] > Task :streams:compileTestJava UP-TO-DATE
[2024-04-03T20:41:03.546Z] > Task :streams:testClasses UP-TO-DATE
[2024-04-03T20:41:03.546Z] > Task :streams:testJar
[2024-04-03T20:41:03.546Z] > Task :streams:testSrcJar
[2024-04-03T20:41:03.546Z] > Task 
:streams:publishMavenJavaPublicationToMavenLocal
[2024-04-03T20:41:03.546Z] > Task :streams:publishToMavenLocal
[2024-04-03T20:41:03.546Z] 
[2024-04-03T20:41:03.546Z] Deprecated Gradle features were used in this build, 
making it incompatible with Gradle 9.0.
[2024-04-03T20:41:03.546Z] 
[2024-04-03T20:41:03.546Z] You can use '--warning-mode all' to show the 
individual deprecation warnings and determine if they come from your own 
scripts or plugins.
[2024-04-03T20:41:03.546Z] 
[2024-04-03T20:41:03.546Z] See 

[jira] [Created] (KAFKA-16469) Metadata Schema Checker

2024-04-03 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-16469:


 Summary: Metadata Schema Checker
 Key: KAFKA-16469
 URL: https://issues.apache.org/jira/browse/KAFKA-16469
 Project: Kafka
  Issue Type: New Feature
Reporter: Colin McCabe






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


Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.7 #125

2024-04-03 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 345162 lines...]
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseIntegerPartitionValue PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseIntegerOffsetValue STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseIntegerOffsetValue PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseStringOffsetValue STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseStringOffsetValue PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseEmptyPartitionOffsetMap STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseEmptyPartitionOffsetMap PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseStringPartitionValue STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseStringPartitionValue PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testConsumerGroupOffsetsToConnectorOffsets STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testConsumerGroupOffsetsToConnectorOffsets PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseInvalidOffset STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseInvalidOffset PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullOffset 
STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullOffset 
PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullPartition 
STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullPartition 
PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullTopic 
STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > testNullTopic PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseInvalidPartition STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.SinkUtilsTest > 
testValidateAndParseInvalidPartition PASSED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.TopicAdminTest > 
verifyingTopicCleanupPolicyShouldReturnFalseWhenBrokerVersionIsUnsupported 
STARTED
[2024-04-03T19:56:52.128Z] 
[2024-04-03T19:56:52.128Z] Gradle Test Run :connect:runtime:test > Gradle Test 
Executor 50 > org.apache.kafka.connect.util.TopicAdminTest > 

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

2024-04-03 Thread David Jacot
Thanks, Justine.

* Should we also use `group.version` (GV) as I suggested in my previous
message in order to be consistent?
* Should we add both names to the `Public Interfaces` section?
* There is still at least one usage of `transaction.protocol.verison` in
the KIP too.

Best,
David

On Wed, Apr 3, 2024 at 6:29 PM Justine Olshan 
wrote:

> I had missed the David's message yesterday about the naming for transaction
> version vs transaction protocol version.
>
> After some offline discussion with Jun, Artem, and David, we agreed that
> transaction version is simpler and conveys more than just protocol changes
> (flexible records for example)
>
> I will update the KIP as well as KIP-890
>
> Thanks,
> Justine
>
> On Tue, Apr 2, 2024 at 2:50 PM Justine Olshan 
> wrote:
>
> > Updated!
> >
> > Justine
> >
> > On Tue, Apr 2, 2024 at 2:40 PM Jun Rao  wrote:
> >
> >> Hi, Justine,
> >>
> >> Thanks for the reply.
> >>
> >> 21. Sounds good. It would be useful to document that.
> >>
> >> 22. Should we add the IV in "metadata.version=17 has no dependencies"
> too?
> >>
> >> Jun
> >>
> >>
> >> On Tue, Apr 2, 2024 at 11:31 AM Justine Olshan
> >> 
> >> wrote:
> >>
> >> > Jun,
> >> >
> >> > 21. Next producer ID field doesn't need to be populated for TV 1. We
> >> don't
> >> > have the same need to retain this since it is written directly to the
> >> > transaction log in InitProducerId. It is only needed for KIP-890 part
> 2
> >> /
> >> > TV 2.
> >> >
> >> > 22. We can do that.
> >> >
> >> > Justine
> >> >
> >> > On Tue, Apr 2, 2024 at 10:41 AM Jun Rao 
> >> wrote:
> >> >
> >> > > Hi, Justine,
> >> > >
> >> > > Thanks for the reply.
> >> > >
> >> > > 21. What about the new NextProducerId field? Will that be populated
> >> with
> >> > TV
> >> > > 1?
> >> > >
> >> > > 22. In the dependencies output, should we show both IV and level for
> >> > > metadata.version too?
> >> > >
> >> > > Jun
> >> > >
> >> > > On Mon, Apr 1, 2024 at 4:43 PM Justine Olshan
> >> >  >> > > >
> >> > > wrote:
> >> > >
> >> > > > Hi Jun,
> >> > > >
> >> > > > 20. I can update the KIP.
> >> > > >
> >> > > > 21. This is used to complete some of the work with KIP-360. (We
> use
> >> > > > previous producer ID there, but never persisted it which was in
> the
> >> KIP
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89068820
> >> )
> >> > > > The KIP also mentions including previous epoch but we explained in
> >> this
> >> > > KIP
> >> > > > how we can figure this out.
> >> > > >
> >> > > > Justine
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Mon, Apr 1, 2024 at 3:56 PM Jun Rao 
> >> > wrote:
> >> > > >
> >> > > > > Hi, Justine,
> >> > > > >
> >> > > > > Thanks for the updated KIP. A couple of more comments.
> >> > > > >
> >> > > > > 20. Could we show the output of version-mapping?
> >> > > > >
> >> > > > > 21. "Transaction version 1 will include the flexible fields in
> the
> >> > > > > transaction state log, and transaction version 2 will include
> the
> >> > > changes
> >> > > > > to the transactional protocol as described by KIP-890 (epoch
> bumps
> >> > and
> >> > > > > implicit add partitions.)"
> >> > > > >   So TV 1 enables the writing of new tagged fields like
> >> > PrevProducerId?
> >> > > > But
> >> > > > > those fields are only usable after the epoch bump, right? What
> >> > > > > functionality does TV 1 achieve?
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > >
> >> > > > > On Mon, Apr 1, 2024 at 2:06 PM Justine Olshan
> >> > > >  >> > > > > >
> >> > > > > wrote:
> >> > > > >
> >> > > > > > I have also updated the KIP to mention the feature tool's
> >> > --metadata
> >> > > > flag
> >> > > > > > will be deprecated.
> >> > > > > > It will still work for users as they learn the new flag, but a
> >> > > warning
> >> > > > > > indicating the alternatives will be shown.
> >> > > > > >
> >> > > > > > Justine
> >> > > > > >
> >> > > > > > On Thu, Mar 28, 2024 at 11:03 AM Justine Olshan <
> >> > > jols...@confluent.io>
> >> > > > > > wrote:
> >> > > > > >
> >> > > > > > > Hi Jun,
> >> > > > > > >
> >> > > > > > > For both transaction state and group coordinator state,
> there
> >> are
> >> > > > only
> >> > > > > > > version 0 records.
> >> > > > > > > KIP-915 introduced flexible versions, but it was never put
> to
> >> > use.
> >> > > MV
> >> > > > > has
> >> > > > > > > never gated these. This KIP will do that. I can include this
> >> > > context
> >> > > > in
> >> > > > > > the
> >> > > > > > > KIP.
> >> > > > > > >
> >> > > > > > > I'm happy to modify his 1 and 2 to 0 and 1.
> >> > > > > > >
> >> > > > > > > Justine
> >> > > > > > >
> >> > > > > > > On Thu, Mar 28, 2024 at 10:57 AM Jun Rao
> >> >  >> > > >
> >> > > > > > wrote:
> >> > > > > > >
> >> > > > > > >> Hi, David,
> >> > > > > > >>
> >> > > > > > >> Thanks for the reply.
> >> > > > > > >>
> >> > > > > > >> Historically, the format of all records were controlled by
> >> MV.
> >> > > Now,
> >> > > > > > >> records
> >> > > > > > >> 

Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Manikumar
Hi Justine,

Yes, it is intended. For bug fix releases website docs will be updated
during the final release process.
We can verify the site-docs artifacts here:
https://home.apache.org/~manikumar/kafka-3.6.2-rc2/kafka_2.12-3.6.2-site-docs.tgz
These site-docs artifacts will be used to update website docs.


Thanks,

On Wed, Apr 3, 2024 at 10:30 PM Justine Olshan 
wrote:

> Hi Manikumar,
>
> I've verified the keys, scanned the artifacts, and other docs.
> I built from source and ran with a ZK cluster (since I saw that we updated
> ZK version in this release)
> I ran a few tests on this cluster.
>
> I also ran the 2.12 binary.
>
> I noticed the docs link (https://kafka.apache.org/36/documentation.html)
> mentions 3.6.1 as the latest. Is that intended?
> I will give my final vote when we figure this out.
>
> Thanks,
> Justine
>
> On Wed, Apr 3, 2024 at 7:25 AM Lianet M.  wrote:
>
> > Hi Manikumar, I did the following checks:
> >
> > - downloaded and built from src
> > - ran all unit test and integration test for clients
> > - ran quickstart with Kraft mode
> > - ran simple workloads with the console consumer/producer
> > - checked all links
> >
> > All looks good to me with this.
> >
> > +1 (non-binding)
> >
> > Thanks!
> >
> >
> >
> > On Wed, Apr 3, 2024, 2:19 a.m. Manikumar  wrote:
> >
> > > Gentle reminder. Please download, test and vote for the release.
> > >
> > > Thanks,
> > >
> > > On Fri, Mar 29, 2024 at 4:57 PM Manikumar 
> wrote:
> > >
> > > > Hi All,
> > > >
> > > > System test runs are green. There were 13 test failures in the first
> > run.
> > > > All the failed tests passed in the second run.
> > > >
> > > > System test results:
> > > > https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
> > > >
> > > > Thanks,
> > > >
> > > > On Thu, Mar 28, 2024 at 3:21 PM Manikumar 
> > wrote:
> > > >
> > > >> Hello Kafka users, developers and client-developers,
> > > >>
> > > >> This is the second candidate we are considering for the release of
> > > Apache
> > > >> Kafka 3.6.2.
> > > >>
> > > >> This is a bugfix release with several fixes, including dependency
> > > >> version bumps for CVEs.
> > > >>
> > > >> Release notes for the 3.6.2 release:
> > > >>
> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
> > > >>
> > > >> *** Please download, test and vote by by Wednesday, April 3rd
> > > >>
> > > >> 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/~manikumar/kafka-3.6.2-rc2/
> > > >>
> > > >>
> > > >> * Maven artifacts to be voted upon:
> > > >>
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > >>
> > > >> * Javadoc:
> > > >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
> > > >>
> > > >> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
> > > >> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
> > > >>
> > > >> * Documentation:
> > > >> https://kafka.apache.org/36/documentation.html
> > > >>
> > > >> * Protocol:
> > > >> https://kafka.apache.org/36/protocol.html
> > > >>
> > > >> * Successful Jenkins builds for the 3.6 branch:
> > > >> Unit/integration tests:
> > > >> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
> > > >> failures passed in local runs)
> > > >> System tests: I will update system test results soon.
> > > >>
> > > >>
> > > >> Thanks,
> > > >> Manikumar
> > > >>
> > > >
> > >
> >
>


Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Justine Olshan
Hi Manikumar,

I've verified the keys, scanned the artifacts, and other docs.
I built from source and ran with a ZK cluster (since I saw that we updated
ZK version in this release)
I ran a few tests on this cluster.

I also ran the 2.12 binary.

I noticed the docs link (https://kafka.apache.org/36/documentation.html)
mentions 3.6.1 as the latest. Is that intended?
I will give my final vote when we figure this out.

Thanks,
Justine

On Wed, Apr 3, 2024 at 7:25 AM Lianet M.  wrote:

> Hi Manikumar, I did the following checks:
>
> - downloaded and built from src
> - ran all unit test and integration test for clients
> - ran quickstart with Kraft mode
> - ran simple workloads with the console consumer/producer
> - checked all links
>
> All looks good to me with this.
>
> +1 (non-binding)
>
> Thanks!
>
>
>
> On Wed, Apr 3, 2024, 2:19 a.m. Manikumar  wrote:
>
> > Gentle reminder. Please download, test and vote for the release.
> >
> > Thanks,
> >
> > On Fri, Mar 29, 2024 at 4:57 PM Manikumar  wrote:
> >
> > > Hi All,
> > >
> > > System test runs are green. There were 13 test failures in the first
> run.
> > > All the failed tests passed in the second run.
> > >
> > > System test results:
> > > https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
> > >
> > > Thanks,
> > >
> > > On Thu, Mar 28, 2024 at 3:21 PM Manikumar 
> wrote:
> > >
> > >> Hello Kafka users, developers and client-developers,
> > >>
> > >> This is the second candidate we are considering for the release of
> > Apache
> > >> Kafka 3.6.2.
> > >>
> > >> This is a bugfix release with several fixes, including dependency
> > >> version bumps for CVEs.
> > >>
> > >> Release notes for the 3.6.2 release:
> > >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
> > >>
> > >> *** Please download, test and vote by by Wednesday, April 3rd
> > >>
> > >> 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/~manikumar/kafka-3.6.2-rc2/
> > >>
> > >>
> > >> * Maven artifacts to be voted upon:
> > >>
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >>
> > >> * Javadoc:
> > >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
> > >>
> > >> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
> > >> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
> > >>
> > >> * Documentation:
> > >> https://kafka.apache.org/36/documentation.html
> > >>
> > >> * Protocol:
> > >> https://kafka.apache.org/36/protocol.html
> > >>
> > >> * Successful Jenkins builds for the 3.6 branch:
> > >> Unit/integration tests:
> > >> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
> > >> failures passed in local runs)
> > >> System tests: I will update system test results soon.
> > >>
> > >>
> > >> Thanks,
> > >> Manikumar
> > >>
> > >
> >
>


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

2024-04-03 Thread Justine Olshan
I had missed the David's message yesterday about the naming for transaction
version vs transaction protocol version.

After some offline discussion with Jun, Artem, and David, we agreed that
transaction version is simpler and conveys more than just protocol changes
(flexible records for example)

I will update the KIP as well as KIP-890

Thanks,
Justine

On Tue, Apr 2, 2024 at 2:50 PM Justine Olshan  wrote:

> Updated!
>
> Justine
>
> On Tue, Apr 2, 2024 at 2:40 PM Jun Rao  wrote:
>
>> Hi, Justine,
>>
>> Thanks for the reply.
>>
>> 21. Sounds good. It would be useful to document that.
>>
>> 22. Should we add the IV in "metadata.version=17 has no dependencies" too?
>>
>> Jun
>>
>>
>> On Tue, Apr 2, 2024 at 11:31 AM Justine Olshan
>> 
>> wrote:
>>
>> > Jun,
>> >
>> > 21. Next producer ID field doesn't need to be populated for TV 1. We
>> don't
>> > have the same need to retain this since it is written directly to the
>> > transaction log in InitProducerId. It is only needed for KIP-890 part 2
>> /
>> > TV 2.
>> >
>> > 22. We can do that.
>> >
>> > Justine
>> >
>> > On Tue, Apr 2, 2024 at 10:41 AM Jun Rao 
>> wrote:
>> >
>> > > Hi, Justine,
>> > >
>> > > Thanks for the reply.
>> > >
>> > > 21. What about the new NextProducerId field? Will that be populated
>> with
>> > TV
>> > > 1?
>> > >
>> > > 22. In the dependencies output, should we show both IV and level for
>> > > metadata.version too?
>> > >
>> > > Jun
>> > >
>> > > On Mon, Apr 1, 2024 at 4:43 PM Justine Olshan
>> > > > > >
>> > > wrote:
>> > >
>> > > > Hi Jun,
>> > > >
>> > > > 20. I can update the KIP.
>> > > >
>> > > > 21. This is used to complete some of the work with KIP-360. (We use
>> > > > previous producer ID there, but never persisted it which was in the
>> KIP
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89068820
>> )
>> > > > The KIP also mentions including previous epoch but we explained in
>> this
>> > > KIP
>> > > > how we can figure this out.
>> > > >
>> > > > Justine
>> > > >
>> > > >
>> > > >
>> > > > On Mon, Apr 1, 2024 at 3:56 PM Jun Rao 
>> > wrote:
>> > > >
>> > > > > Hi, Justine,
>> > > > >
>> > > > > Thanks for the updated KIP. A couple of more comments.
>> > > > >
>> > > > > 20. Could we show the output of version-mapping?
>> > > > >
>> > > > > 21. "Transaction version 1 will include the flexible fields in the
>> > > > > transaction state log, and transaction version 2 will include the
>> > > changes
>> > > > > to the transactional protocol as described by KIP-890 (epoch bumps
>> > and
>> > > > > implicit add partitions.)"
>> > > > >   So TV 1 enables the writing of new tagged fields like
>> > PrevProducerId?
>> > > > But
>> > > > > those fields are only usable after the epoch bump, right? What
>> > > > > functionality does TV 1 achieve?
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > >
>> > > > > On Mon, Apr 1, 2024 at 2:06 PM Justine Olshan
>> > > > > > > > > >
>> > > > > wrote:
>> > > > >
>> > > > > > I have also updated the KIP to mention the feature tool's
>> > --metadata
>> > > > flag
>> > > > > > will be deprecated.
>> > > > > > It will still work for users as they learn the new flag, but a
>> > > warning
>> > > > > > indicating the alternatives will be shown.
>> > > > > >
>> > > > > > Justine
>> > > > > >
>> > > > > > On Thu, Mar 28, 2024 at 11:03 AM Justine Olshan <
>> > > jols...@confluent.io>
>> > > > > > wrote:
>> > > > > >
>> > > > > > > Hi Jun,
>> > > > > > >
>> > > > > > > For both transaction state and group coordinator state, there
>> are
>> > > > only
>> > > > > > > version 0 records.
>> > > > > > > KIP-915 introduced flexible versions, but it was never put to
>> > use.
>> > > MV
>> > > > > has
>> > > > > > > never gated these. This KIP will do that. I can include this
>> > > context
>> > > > in
>> > > > > > the
>> > > > > > > KIP.
>> > > > > > >
>> > > > > > > I'm happy to modify his 1 and 2 to 0 and 1.
>> > > > > > >
>> > > > > > > Justine
>> > > > > > >
>> > > > > > > On Thu, Mar 28, 2024 at 10:57 AM Jun Rao
>> > > > > >
>> > > > > > wrote:
>> > > > > > >
>> > > > > > >> Hi, David,
>> > > > > > >>
>> > > > > > >> Thanks for the reply.
>> > > > > > >>
>> > > > > > >> Historically, the format of all records were controlled by
>> MV.
>> > > Now,
>> > > > > > >> records
>> > > > > > >> in _offset_commit will be controlled by
>> > > `group.coordinator.version`,
>> > > > > is
>> > > > > > >> that right? It would be useful to document that.
>> > > > > > >>
>> > > > > > >> Also, we should align on the version numbering.
>> "kafka-feature
>> > > > > disable"
>> > > > > > >> says "Disable one or more feature flags. This is the same as
>> > > > > downgrading
>> > > > > > >> the version to zero". So, in the `group.coordinator.version'
>> > case,
>> > > > we
>> > > > > > >> probably should use version 0 for the old consumer protocol.
>> > > > > > >>
>> > > > > > >> Jun
>> > > > > > >>
>> > > > > > >> On Thu, Mar 28, 2024 at 2:13 AM Andrew Schofield <
>> > > > > > >> 

Re: [DISCUSS] KIP-1026: Handling producer snapshot when upgrading from < v2.8.0 for Tiered Storage

2024-04-03 Thread Arpit Goyal
Thanks @Kamal Chandraprakash   Greg Harris
I currently do not have detailed understanding on the behaviour when empty
producer snapshot  restored. I will try to test out the behaviour.Meanwhile
I would request other community members if they can chime in and assist if
they are already aware of the behaviour mentioned above.
Thanks and Regards
Arpit Goyal
8861094754


On Tue, Mar 26, 2024 at 4:04 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Hi,
>
> Sorry for the late reply. Greg has raised some good points:
>
> > Does an empty producer snapshot have the same behavior as a
> non-existent snapshot when restored?
>
> Producer snapshots maintain the status about the ongoing txn to either
> COMMIT/ABORT the transaction. In the older version (<2.8), we maintain
> the producer snapshots only for the recent segments. If such a topic gets
> onboarded to tiered storage and the recently built replica becomes then
> leader,
> then it might break the producer.
>
> Assume there are 100 segments for a partition, and the producer snapshots
> are available only for the recent 2 segments. Then, tiered storage is
> enabled
> for that topic, 90/100 segments are uploaded and local-log-segments are
> cleared upto segment 50. If a new follower builds the state from remote, it
> will
> have the empty producer snapshots and start reading the data from the
> leader
> from segment-51. If a transaction is started at segment-40, then it will
> break the
> client.
>
> We also have to check the impact of expiring producer-ids before the
> default
> expiration time of 7 days: *transactional.id.expiration.ms
> *
>
> > 2. Why were empty producer snapshots not backfilled for older data
> when clusters were upgraded from 2.8?
>
> https://github.com/apache/kafka/pull/7929 -- It was not required at that
> time.
> With tiered storage, we need the snapshot file for each segment to reliably
> build the follower state from remote storage.
>
> > 3. Do producer snapshots need to be available contiguously, or can
> earlier snapshots be empty while later segments do not exist?
>
> I assume you refer to "while later segments do exist". Each snapshot file
> will contain
> the cumulative/complete data of all the previous segments. So, a recent
> segment
> snapshot is enough to build the producer state. We need to figure out a
> solution to
> build the complete producer state for replicas that built the state using
> the remote.
>
> Arpit,
> We have to deep dive into each of them to come up with the proper solution.
>
> --
> Kamal
>
>
> On Tue, Mar 26, 2024 at 3:55 AM Greg Harris 
> wrote:
>
> > Hi Arpit,
> >
> > I think creating empty producer snapshots would be
> > backwards-compatible for the tiered storage plugins, but I'm not aware
> > of what the other compatibility/design concerns might be. Maybe you or
> > another reviewer can answer these questions:
> > 1. Does an empty producer snapshot have the same behavior as a
> > non-existent snapshot when restored?
> > 2. Why were empty producer snapshots not backfilled for older data
> > when clusters were upgraded from 2.8?
> > 3. Do producer snapshots need to be available contiguously, or can
> > earlier snapshots be empty while later segments do not exist?
> >
> > Thanks,
> > Greg
> >
> > On Sat, Mar 23, 2024 at 3:24 AM Arpit Goyal 
> > wrote:
> > >
> > > Yes Luke,
> > > I am also in favour of creating producer snapshot at run time if
> > > foundEmpty  as this would only be required for topics migrated from <
> 2.8
> > > version. This will not break the existing contract with the plugin.
> Yes,
> > > metrics do not make sense here as of now.
> > > Greg, @Kamal Chandraprakash   WDYT ?
> > > Arpit Goyal
> > > 8861094754
> > >
> > >
> > > On Sat, Mar 23, 2024 at 3:05 PM Luke Chen  wrote:
> > >
> > > > Hi Arpit,
> > > >
> > > > I'm in favor of creating an empty producer snapshot since it's only
> for
> > > > topics <= v2.8.
> > > > About the metric, I don't know what we expect users to know.
> > > > I think we can implement with the empty producer snapshot method,
> > without
> > > > the metric.
> > > > And add them if users are requested it.
> > > > WDYT?
> > > >
> > > > Thank you.
> > > > Luke
> > > >
> > > > On Sat, Mar 23, 2024 at 1:24 PM Arpit Goyal <
> goyal.arpit...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Team,
> > > > > Any further comments or suggestions on the possible approaches
> > discussed
> > > > > above.
> > > > >
> > > > > On Tue, Mar 19, 2024, 09:55 Arpit Goyal 
> > > > wrote:
> > > > >
> > > > > > @Luke Chen @Kamal Chandraprakash  >
> > > > @Greg
> > > > > > Harris Any suggestion on the above two possible approaches.
> > > > > > On Sun, Mar 17, 2024, 13:36 Arpit Goyal <
> goyal.arpit...@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > >>
> > > > >   In summary , There are two possible solution to handle the
> > above
> > > > > >> scenario when producer snapshot file found to be null
> > > > > >>
> > > > > >> 1. 

Community over Code EU 2024: Start planning your trip!

2024-04-03 Thread Ryan Skraba
[Note: You're receiving this email because you are subscribed to one
or more project dev@ mailing lists at the Apache Software Foundation.]

Dear community,

We hope you are doing great, are you ready for Community Over Code EU?
Check out the featured sessions, get your tickets with special
discounts and start planning your trip.

Save your spot! Take a look at our lineup of sessions, panelists and
featured speakers and make your final choice:

* EU policies and regulations affecting open source specialists working in OSPOs

The panel will discuss how EU legislation affects the daily work of
open source operations. Panelists will cover some recent policy
updates, the challenges of staying compliant when managing open source
contribution and usage within organizations, and their personal
experiences in adapting to the changing European regulatory
environment.

* Doing for sustainability, what open source did for software

In this keynote Asim Hussain will explain the history of Impact
Framework, a coalition of hundreds of software practitioners with
tangible solutions that directly foster meaningful change by measuring
the environmental impacts of a piece of software.

Don’t forget that we have special discounts for groups, students and
Apache committers. Visit the website to discover more about these
rates.[1]

It's time for you to start planning your trip. Remember that we have
prepared a “How to get there” guide that will be helpful to find out
the best transportation, either train, bus, flight or boat to
Bratislava from wherever you are coming from. Take a look at the
different options and please reach out to us if you have any
questions.

We have available rooms -with a special rate- at the Radisson Blu
Carlton Hotel, where the event will take place and at the Park Inn
Hotel which is only 5 minutes walking from the venue. [2] However, you
are free to choose any other accommodation options around the city.

See you in Bratislava,
Community Over Code EU Team

[1]: https://eu.communityovercode.org/tickets/ "Register"
[2]: https://eu.communityovercode.org/venue/ "Where to stay"


Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Lianet M.
Hi Manikumar, I did the following checks:

- downloaded and built from src
- ran all unit test and integration test for clients
- ran quickstart with Kraft mode
- ran simple workloads with the console consumer/producer
- checked all links

All looks good to me with this.

+1 (non-binding)

Thanks!



On Wed, Apr 3, 2024, 2:19 a.m. Manikumar  wrote:

> Gentle reminder. Please download, test and vote for the release.
>
> Thanks,
>
> On Fri, Mar 29, 2024 at 4:57 PM Manikumar  wrote:
>
> > Hi All,
> >
> > System test runs are green. There were 13 test failures in the first run.
> > All the failed tests passed in the second run.
> >
> > System test results:
> > https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
> >
> > Thanks,
> >
> > On Thu, Mar 28, 2024 at 3:21 PM Manikumar  wrote:
> >
> >> Hello Kafka users, developers and client-developers,
> >>
> >> This is the second candidate we are considering for the release of
> Apache
> >> Kafka 3.6.2.
> >>
> >> This is a bugfix release with several fixes, including dependency
> >> version bumps for CVEs.
> >>
> >> Release notes for the 3.6.2 release:
> >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
> >>
> >> *** Please download, test and vote by by Wednesday, April 3rd
> >>
> >> 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/~manikumar/kafka-3.6.2-rc2/
> >>
> >>
> >> * Maven artifacts to be voted upon:
> >> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >>
> >> * Javadoc:
> >> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
> >>
> >> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
> >> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
> >>
> >> * Documentation:
> >> https://kafka.apache.org/36/documentation.html
> >>
> >> * Protocol:
> >> https://kafka.apache.org/36/protocol.html
> >>
> >> * Successful Jenkins builds for the 3.6 branch:
> >> Unit/integration tests:
> >> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
> >> failures passed in local runs)
> >> System tests: I will update system test results soon.
> >>
> >>
> >> Thanks,
> >> Manikumar
> >>
> >
>


Re: [ANNOUNCE] New committer: Christo Lolov

2024-04-03 Thread Bill Bejeck
Congrats Christo!

-Bill

On Wed, Mar 27, 2024 at 4:03 AM Matthias J. Sax  wrote:

> Congrats!
>
> On 3/26/24 9:39 PM, Christo Lolov wrote:
> > Thank you everyone!
> >
> > It wouldn't have been possible without quite a lot of reviews and
> extremely
> > helpful inputs from you and the rest of the community! I am looking
> forward
> > to working more closely with you going forward :)
> >
> > On Tue, 26 Mar 2024 at 14:31, Kirk True  wrote:
> >
> >> Congratulations Christo!
> >>
> >>> On Mar 26, 2024, at 7:27 AM, Satish Duggana 
> >> wrote:
> >>>
> >>> Congratulations Christo!
> >>>
> >>> On Tue, 26 Mar 2024 at 19:20, Ivan Yurchenko  wrote:
> 
>  Congrats!
> 
>  On Tue, Mar 26, 2024, at 14:48, Lucas Brutschy wrote:
> > Congrats!
> >
> > On Tue, Mar 26, 2024 at 2:44 PM Federico Valeri <
> fedeval...@gmail.com>
> >> wrote:
> >>
> >> Congrats!
> >>
> >> On Tue, Mar 26, 2024 at 2:27 PM Mickael Maison <
> >> mickael.mai...@gmail.com> wrote:
> >>>
> >>> Congratulations Christo!
> >>>
> >>> On Tue, Mar 26, 2024 at 2:26 PM Chia-Ping Tsai  >
> >> wrote:
> 
>  Congrats Christo!
> 
>  Chia-Ping
> >
> >>
> >>
> >
>


Participate in the ASF 25th Anniversary Campaign

2024-04-03 Thread Brian Proffitt
Hi everyone,

As part of The ASF’s 25th anniversary campaign[1], we will be celebrating
projects and communities in multiple ways.

We invite all projects and contributors to participate in the following
ways:

* Individuals - submit your first contribution:
https://news.apache.org/foundation/entry/the-asf-launches-firstasfcontribution-campaign
* Projects - share your public good story:
https://docs.google.com/forms/d/1vuN-tUnBwpTgOE5xj3Z5AG1hsOoDNLBmGIqQHwQT6k8/viewform?edit_requested=true
* Projects - submit a project spotlight for the blog:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278466116
* Projects - contact the Voice of Apache podcast (formerly Feathercast) to
be featured: https://feathercast.apache.org/help/
*  Projects - use the 25th anniversary template and the #ASF25Years hashtag
on social media:
https://docs.google.com/presentation/d/1oDbMol3F_XQuCmttPYxBIOIjRuRBksUjDApjd8Ve3L8/edit#slide=id.g26b0919956e_0_13

If you have questions, email the Marketing & Publicity team at
mark...@apache.org.

Peace,
BKP

[1] https://apache.org/asf25years/

[NOTE: You are receiving this message because you are a contributor to an
Apache Software Foundation project. The ASF will very occasionally send out
messages relating to the Foundation to contributors and members, such as
this one.]

Brian Proffitt
VP, Marketing & Publicity
VP, Conferences


[jira] [Created] (KAFKA-16468) Listener not found error in SendRPCsToBrokersEvent

2024-04-03 Thread David Arthur (Jira)
David Arthur created KAFKA-16468:


 Summary: Listener not found error in SendRPCsToBrokersEvent
 Key: KAFKA-16468
 URL: https://issues.apache.org/jira/browse/KAFKA-16468
 Project: Kafka
  Issue Type: Bug
  Components: controller, migration
Reporter: David Arthur
 Fix For: 3.8.0


During the ZK to KRaft migration, the controller will send RPCs using the 
configured "control.plane.listener.name" or more commonly, the 
"inter.broker.listener.name". If a ZK broker did not register with this 
listener, we get a error at the time of sending the first RPC to a broker.

{code}
[2024-04-03 09:28:59,043] ERROR Encountered nonFatalFaultHandler fault: 
Unhandled error in SendRPCsToBrokersEvent 
(org.apache.kafka.server.fault.MockFaultHandler:44)
kafka.common.BrokerEndPointNotAvailableException: End point with listener name 
EXTERNAL not found for broker 0
at kafka.cluster.Broker.$anonfun$node$1(Broker.scala:94)
at scala.Option.getOrElse(Option.scala:201)
at kafka.cluster.Broker.node(Broker.scala:93)
at 
kafka.controller.ControllerChannelManager.addNewBroker(ControllerChannelManager.scala:122)
at 
kafka.controller.ControllerChannelManager.addBroker(ControllerChannelManager.scala:105)
at 
kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2(MigrationPropagator.scala:98)
at 
kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2$adapted(MigrationPropagator.scala:98)
at scala.collection.immutable.Set$Set3.foreach(Set.scala:261)
at 
kafka.migration.MigrationPropagator.publishMetadata(MigrationPropagator.scala:98)
at 
kafka.migration.MigrationPropagator.sendRPCsToBrokersFromMetadataImage(MigrationPropagator.scala:219)
at 
org.apache.kafka.metadata.migration.KRaftMigrationDriver$SendRPCsToBrokersEvent.run(KRaftMigrationDriver.java:777)
at 
org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:128)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:211)
at 
org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:182)
at java.base/java.lang.Thread.run(Thread.java:833)
{code}

At this point, the KRaft controller has already migrated the metadata. Recovery 
at this point is possible by restarting the brokers with the correct listener 
names, but we can catch this much sooner in the process.

When a ZK broker registers with the KRaft controller, we should reject the 
registration if the expected listener name is not present. This will prevent 
the migration from starting.



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


[jira] [Resolved] (KAFKA-16413) Add tests for FileLock

2024-04-03 Thread Chia-Ping Tsai (Jira)


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

Chia-Ping Tsai resolved KAFKA-16413.

Fix Version/s: 3.8.0
   Resolution: Fixed

> Add tests for FileLock
> --
>
> Key: KAFKA-16413
> URL: https://issues.apache.org/jira/browse/KAFKA-16413
> Project: Kafka
>  Issue Type: Test
>Reporter: PoAn Yang
>Assignee: PoAn Yang
>Priority: Minor
> Fix For: 3.8.0
>
>
> Ref: [https://github.com/apache/kafka/pull/15568#pullrequestreview-1950676267]



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


Re: [DISCUSS] KIP-1031: Control offset translation in MirrorSourceConnector

2024-04-03 Thread Chris Egerton
Hi Omnia,

Thanks for the KIP! Two small things come to mind:

1. It'd be nice to mention that increasing the max offset lag to INT_MAX
could work as a partial workaround for users on existing versions (though
of course this wouldn't prevent creation of the syncs topic).

2. Will it be illegal to disable offset syncs if other features that rely
on offset syncs are explicitly enabled in the connector config? If they're
not explicitly enabled then it should probably be fine to silently disable
them, but I'd be interested in your thoughts.

Cheers,

Chris

On Wed, Apr 3, 2024, 20:41 Luke Chen  wrote:

> Hi Omnia,
>
> Thanks for the KIP!
> It LGTM!
> But I'm not an expert of MM2, it would be good to see if there is any other
> comment from MM2 experts.
>
> Thanks.
> Luke
>
> On Thu, Mar 14, 2024 at 6:08 PM Omnia Ibrahim 
> wrote:
>
> > Hi everyone, I would like to start a discussion thread for KIP-1031:
> > Control offset translation in MirrorSourceConnector
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1031%3A+Control+offset+translation+in+MirrorSourceConnector
> >
> > Thanks
> > Omnia
> >
>


Re: [DISCUSS] KIP-1031: Control offset translation in MirrorSourceConnector

2024-04-03 Thread Luke Chen
Hi Omnia,

Thanks for the KIP!
It LGTM!
But I'm not an expert of MM2, it would be good to see if there is any other
comment from MM2 experts.

Thanks.
Luke

On Thu, Mar 14, 2024 at 6:08 PM Omnia Ibrahim 
wrote:

> Hi everyone, I would like to start a discussion thread for KIP-1031:
> Control offset translation in MirrorSourceConnector
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1031%3A+Control+offset+translation+in+MirrorSourceConnector
>
> Thanks
> Omnia
>


Re: [DISCUSS] KIP-971 Expose replication-offset-lag MirrorMaker2 metric

2024-04-03 Thread Elxan Eminov
Hi Mickael,
Thanks for your response and apologies for a huge delay in mine.

My thinking is that any partition could go stale if there are no records
being produced into it. If enough of such partitions are present and are
owned by a single MM task, an OOM could happen.

Regarding the scenario where the TTL value is lower than the refresh
interval - I believe that this is an edge that we need to document and
prevent against, for example either failing to start on such a combination
or resorting to a default value that would satisfy the constraint and
logging an error.

Thanks,
Elkhan

On Thu, 8 Feb 2024 at 14:17, Mickael Maison 
wrote:

> Hi,
>
> Thanks for the updates.
> I'm wondering whether we really need the ttl eviction mechanism. The
> motivation is to "avoid storing stale LRO entries which can cause an
> eventual OOM error". How could it contain stake entries? I would
> expect its cache to only contain entries for partitions assigned to
> the task that owns it. Also what is the expected behavior if there's
> no available LRO in the cache? If we keep this mechanism what happens
> if its value is lower than
> replication.record.lag.metric.refresh.interval?
>
> Thanks,
> Mickael
>
> On Mon, Feb 5, 2024 at 5:23 PM Elxan Eminov 
> wrote:
> >
> > Hi Mickael!
> > Any further thoughts on this?
> >
> > Thanks,
> > Elkhan
> >
> > On Thu, 18 Jan 2024 at 11:53, Mickael Maison 
> > wrote:
> >
> > > Hi Elxan,
> > >
> > > Thanks for the updates.
> > >
> > > We used dots to separate words in configuration names, so I think
> > > replication.offset.lag.metric.last-replicated-offset.ttl should be
> > > named replication.offset.lag.metric.last.replicated.offset.ttl
> > > instead.
> > >
> > > About the names of the metrics, fair enough if you prefer keeping the
> > > replication prefix. Out of the alternatives you mentioned, I think I
> > > prefer replication-record-lag. I think the metrics and configuration
> > > names should match too. Let's see what the others think about it.
> > >
> > > Thanks,
> > > Mickael
> > >
> > > On Mon, Jan 15, 2024 at 9:50 PM Elxan Eminov 
> > > wrote:
> > > >
> > > > Apologies, forgot to reply on your last comment about the metric
> name.
> > > > I believe both replication-lag and record-lag are a little too
> abstract -
> > > > what do you think about either leaving it as replication-offset-lag
> or
> > > > renaming to replication-record-lag?
> > > >
> > > > Thanks
> > > >
> > > > On Wed, 10 Jan 2024 at 15:31, Mickael Maison <
> mickael.mai...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Elxan,
> > > > >
> > > > > Thanks for the KIP, it looks like a useful addition.
> > > > >
> > > > > Can you add to the KIP the default value you propose for
> > > > > replication.lag.metric.refresh.interval? In MirrorMaker most
> interval
> > > > > configs can be set to -1 to disable them, will it be the case for
> this
> > > > > new feature or will this setting only accept positive values?
> > > > > I also wonder if replication-lag, or record-lag would be clearer
> names
> > > > > instead of replication-offset-lag, WDYT?
> > > > >
> > > > > Thanks,
> > > > > Mickael
> > > > >
> > > > > On Wed, Jan 3, 2024 at 6:15 PM Elxan Eminov <
> elxanemino...@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > Hi all,
> > > > > > Here is the vote thread:
> > > > > > https://lists.apache.org/thread/ftlnolcrh858dry89sjg06mdcdj9mrqv
> > > > > >
> > > > > > Cheers!
> > > > > >
> > > > > > On Wed, 27 Dec 2023 at 11:23, Elxan Eminov <
> elxanemino...@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > > I've updated the KIP with the details we discussed in this
> thread.
> > > > > > > I'll call in a vote after the holidays if everything looks
> good.
> > > > > > > Thanks!
> > > > > > >
> > > > > > > On Sat, 26 Aug 2023 at 15:49, Elxan Eminov <
> > > elxanemino...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Relatively minor change with a new metric for MM2
> > > > > > >>
> > > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-971%3A+Expose+replication-offset-lag+MirrorMaker2+metric
> > > > > > >>
> > > > > > >
> > > > >
> > >
>


Kafka capabilities

2024-04-03 Thread Kafka Life
Hi Kafka users

Does any one have a document or ppt that showcases the capabilities of
Kafka along with any cost management capability?
i have a customer who is still using IBM MQM and rabbit MQ. I want the
client to consider kafka for messaging and data streaming. I wanted to seek
your expert help if you have any document or ppt i can propose it as an
example. could you pls help.

thanks and regards
KrisG


Re: [DISCUSS] KIP-1028: Docker Official Image for Apache Kafka

2024-04-03 Thread Luke Chen
Hi Krishna,

I also have the same question as Manikumar raised:
1. Will the Docker inventory files/etc are the same for OSS Image and
Docker Official Images?
If no, then why not? Could we make them identical so that we don't have to
build 2 images for each release?

Thank you.
Luke

On Wed, Apr 3, 2024 at 12:41 AM Manikumar  wrote:

> Hi Krishna,
>
> Thanks for the KIP.
>
> I think Docker Official Images will be beneficial to the Kafka community.
> Few queries below.
>
> 1. Will the Docker inventory files/etc are the same for OSS Image and
> Docker Official Images
> 2. I am a bit worried about the new steps to the release process. Maybe we
> should consider Docker Official Images release as Post-Release activity.
>
> Thanks,
>
> On Fri, Mar 22, 2024 at 3:29 PM Krish Vora  wrote:
>
> > Hi Hector,
> >
> > Thanks for reaching out. This KIP builds on top of KIP-975
> > <
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-975%3A+Docker+Image+for+Apache+Kafka
> > >
> > and
> > aims to introduce a JVM-based Docker Official Image (DOI
> > ) for Apache
> > Kafka that will be visible under Docker Official Images
> > . Once
> implemented
> > for Apache Kafka, for each release, there will be one more JVM-based
> Docker
> > image available to users.
> >
> > Currently, we already have an OSS sponsored image, which was introduced
> via
> > KIP-975 (apache/kafka )
> which
> > comes under The Apache Software Foundation <
> > https://hub.docker.com/u/apache> in
> > Docker Hub. The new Docker Image is the Docker Official Image (DOI),
> which
> > will be built and maintained by Docker Community.
> >
> > For example, for a release version like 3.8.0 we will have two JVM based
> > docker images:-
> >
> >- apache/kafka:3.8.0 (OSS sponsored image)
> >- kafka:3.8.0 (Docker Official image)
> >
> >
> > I have added the same in the KIP too for everyone's reference.
> > Thanks,
> > Krish.
> >
> > On Fri, Mar 22, 2024 at 2:50 AM Hector Geraldino (BLOOMBERG/ 919 3RD A) <
> > hgerald...@bloomberg.net> wrote:
> >
> > > Hi,
> > >
> > > What is the difference between this KIP and KIP-975: Docker Image for
> > > Apache Kafka?
> > >
> > > From: dev@kafka.apache.org At: 03/21/24 07:30:07 UTC-4:00To:
> > > dev@kafka.apache.org
> > > Subject: [DISCUSS] KIP-1028: Docker Official Image for Apache Kafka
> > >
> > > Hi everyone,
> > >
> > > I would like to start the discussion on
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1028%3A+Docker+Official+Im
> > > age+for+Apache+Kafka
> > >  .
> > >
> > > This KIP aims to introduce JVM based Docker Official Image (DOI) for
> > Apache
> > > Kafka.
> > >
> > > Regards,
> > > Krish.
> > >
> > >
> > >
> >
>


Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Jakub Scholz
+1 (non-binding). I used the staged Scala 2.13 binaries and the staging
Maven repo for my testing. All seems to work fine. Thanks for doing the
release.

Jakub

On Thu, Mar 28, 2024 at 10:51 AM Manikumar  wrote:

> Hello Kafka users, developers and client-developers,
>
> This is the second candidate we are considering for the release of Apache
> Kafka 3.6.2.
>
> This is a bugfix release with several fixes, including dependency
> version bumps for CVEs.
>
> Release notes for the 3.6.2 release:
> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
>
> *** Please download, test and vote by by Wednesday, April 3rd
>
> 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/~manikumar/kafka-3.6.2-rc2/
>
>
> * Maven artifacts to be voted upon:
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>
> * Javadoc:
> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
>
> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
>
> * Documentation:
> https://kafka.apache.org/36/documentation.html
>
> * Protocol:
> https://kafka.apache.org/36/protocol.html
>
> * Successful Jenkins builds for the 3.6 branch:
> Unit/integration tests:
> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
> failures passed in local runs)
> System tests: I will update system test results soon.
>
>
> Thanks,
> Manikumar
>


Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Andrew Schofield
Hi,
I downloaded the binary release and upgraded a broker from 3.6.0 to 3.6.2. 
Documentation and protocol links
look fine.

+1 (non-binding)

Thanks,
Andrew

> On 3 Apr 2024, at 07:17, Manikumar  wrote:
>
> Gentle reminder. Please download, test and vote for the release.
>
> Thanks,
>
> On Fri, Mar 29, 2024 at 4:57 PM Manikumar  wrote:
>
>> Hi All,
>>
>> System test runs are green. There were 13 test failures in the first run.
>> All the failed tests passed in the second run.
>>
>> System test results:
>> https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
>>
>> Thanks,
>>
>> On Thu, Mar 28, 2024 at 3:21 PM Manikumar  wrote:
>>
>>> Hello Kafka users, developers and client-developers,
>>>
>>> This is the second candidate we are considering for the release of Apache
>>> Kafka 3.6.2.
>>>
>>> This is a bugfix release with several fixes, including dependency
>>> version bumps for CVEs.
>>>
>>> Release notes for the 3.6.2 release:
>>> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
>>>
>>> *** Please download, test and vote by by Wednesday, April 3rd
>>>
>>> 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/~manikumar/kafka-3.6.2-rc2/
>>>
>>>
>>> * Maven artifacts to be voted upon:
>>> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>>>
>>> * Javadoc:
>>> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
>>>
>>> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
>>> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
>>>
>>> * Documentation:
>>> https://kafka.apache.org/36/documentation.html
>>>
>>> * Protocol:
>>> https://kafka.apache.org/36/protocol.html
>>>
>>> * Successful Jenkins builds for the 3.6 branch:
>>> Unit/integration tests:
>>> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
>>> failures passed in local runs)
>>> System tests: I will update system test results soon.
>>>
>>>
>>> Thanks,
>>> Manikumar
>>>
>>



Re: [DISCUSS] KIP-950: Tiered Storage Disablement

2024-04-03 Thread Luke Chen
Hi Christo,

1. I agree with Doguscan that in KRaft mode, the controller won't send RPCs
to the brokers (except in the migration path).
So, I think we could adopt the similar way we did to `AlterReplicaLogDirs` (
KIP-858
)
that let the broker notify controller any update, instead of controller to
broker. And once the controller receives all the complete requests from
brokers, it'll enter "Disabled" state. WDYT?

2. Why should we wait until all brokers to respond before moving to
"Disabled" state in "KRaft mode"?
Currently, only the leader node does the remote log upload/fetch tasks, so
does that mean the controller only need to make sure the leader completes
the stopPartition?
If during the leader node stopPartition process triggered leadership
change, then the new leader should receive and apply the configRecord
update before the leadership change record based on the KRaft design, which
means there will be no gap that the follower node becomes the leader and
starting doing unexpected upload/fetch tasks, right?
I agree we should make sure in ZK mode, all brokers are completed the
stopPartitions before moving to "Disabled" state because ZK node watcher is
working in a separate thread. But not sure about KRaft mode.

Thanks.
Luke


On Fri, Mar 29, 2024 at 4:14 PM Christo Lolov 
wrote:

> Heya everyone!
>
> re: Doguscan
>
> I believe the answer to 101 needs a bit more discussion. As far as I know,
> tiered storage today has methods to update a metadata of a segment to say
> "hey, I would like this deleted", but actual deletion is left to plugin
> implementations (or any background cleaners). In other words, there is no
> "immediate" deletion. In this KIP, we would like to continue doing the same
> if the retention policy is set to delete. So I believe the answer is
> actually that a) we will update the metadata of the segments to mark them
> as deleted and b) we will advance the log start offset. Any deletion of
> actual files will still be delegated to plugin implementations. I believe
> this is further supported by "*remote.log.disable.policy=delete:* Logs that
> are archived in the remote storage will not be part of the contiguous
> "active" log and will be deleted asynchronously as part of the disablement
> process"
>
> Following from the above, I believe for 102 it is fine to allow setting of
> remote.log.disable.policy on a disabled topic in much the same way we allow
> other remote-related configurations to be set on a topic (i.e.
> local.retention.*) - it just won't have an effect. Granted, I do believe we
> should restrict the policy being changed while a disablement is ongoing.
>
> re: Satish and Kamal
>
> 104, 1 and 2 are fair asks, I will work with Doguscan to update the KIP
> with the information!
>
> Best,
> Christo
>
> On Thu, 28 Mar 2024 at 10:31, Doğuşcan Namal 
> wrote:
>
> > Hi Satish, I will try to answer as much as I can and the others could
> chime
> > in with further details.
> >
> >
> >
> >
> >
> > *101. For remote.log.disable.policy=delete: Does it delete the remote log
> > data immediately and the data in remote storage will not be taken into
> > account by any replica? That means log-start-offset is moved to the
> earlier
> > local-log-start-offset.*
> > *Exactly. RemoteLogData will be deleted immediately. *
> > *So before the deletion starts we move LogStart offset to
> > LocalLogStartOffset to ensure that no RemoteLog will be accessed after
> that
> > point.*
> >
> >
> > * 102. Can we update the remote.log.disable.policy after tiered storage
> is
> > disabled on a topic?*
> >
> > *This is a good point. I think we should not allow modifying this
> > configuration*
> > *because changing the policy from Deletion to Retain when there is an
> > ongoing Deletion will result in an undefined behaviour and where we
> retain
> > half of the remote log and delete the other half.*
> >
> > * 103. Do we plan to add any metrics related to this feature?*
> >
> >
> >
> > *Any recommendations?*
> > *We may emit a gauge showing the enablement state of a topic but we could
> > gather that info from the logs as well.*
> > *The total duration for remote topic deletion could be added as well but
> > this is more of a metric for the RemotePartitionRemover itself.*
> >
> >
> >
> > *104. Please add configuration details about copier thread pool,
> expiration
> > thread pool and the migration of the existing
> > RemoteLogManagerScheduledThreadPool.*
> >
> > *Will add the details.*
> >
> > 105. How is the behaviour with topic or partition deletion request
> > handled when tiered storage disablement request is still being
> > processed on a topic?
> >
> > *If the disablement policy is Delete then a successive topic deletion
> > request is going to be a NOOP because RemoteLogs is already being
> deleted.*
> > *If the disablement policy is 

Re: [VOTE] 3.6.2 RC2

2024-04-03 Thread Manikumar
Gentle reminder. Please download, test and vote for the release.

Thanks,

On Fri, Mar 29, 2024 at 4:57 PM Manikumar  wrote:

> Hi All,
>
> System test runs are green. There were 13 test failures in the first run.
> All the failed tests passed in the second run.
>
> System test results:
> https://gist.github.com/omkreddy/17d23d3eb36ef840011f2494d65bbd4f
>
> Thanks,
>
> On Thu, Mar 28, 2024 at 3:21 PM Manikumar  wrote:
>
>> Hello Kafka users, developers and client-developers,
>>
>> This is the second candidate we are considering for the release of Apache
>> Kafka 3.6.2.
>>
>> This is a bugfix release with several fixes, including dependency
>> version bumps for CVEs.
>>
>> Release notes for the 3.6.2 release:
>> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/RELEASE_NOTES.html
>>
>> *** Please download, test and vote by by Wednesday, April 3rd
>>
>> 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/~manikumar/kafka-3.6.2-rc2/
>>
>>
>> * Maven artifacts to be voted upon:
>> https://repository.apache.org/content/groups/staging/org/apache/kafka/
>>
>> * Javadoc:
>> https://home.apache.org/~manikumar/kafka-3.6.2-rc2/javadoc/
>>
>> * Tag to be voted upon (off 3.6 branch) is the 3.6.2 tag:
>> https://github.com/apache/kafka/releases/tag/3.6.2-rc2
>>
>> * Documentation:
>> https://kafka.apache.org/36/documentation.html
>>
>> * Protocol:
>> https://kafka.apache.org/36/protocol.html
>>
>> * Successful Jenkins builds for the 3.6 branch:
>> Unit/integration tests:
>> https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/167/ (test
>> failures passed in local runs)
>> System tests: I will update system test results soon.
>>
>>
>> Thanks,
>> Manikumar
>>
>