Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-07 Thread Luke Chen
Hi Colin, Thanks for the response. I have no more comments. +1 (binding) Luke On Thu, Jun 8, 2023 at 6:02 AM Colin McCabe wrote: > > Hi Luke, > > Thanks for the review and the suggestion. > > I think we will add more "handling time" metrics later, but for now I don't > want to make this KIP

[jira] [Created] (KAFKA-15074) offset out of range for partition xxx, resetting offset

2023-06-07 Thread YaYun Wang (Jira)
YaYun Wang created KAFKA-15074: -- Summary: offset out of range for partition xxx, resetting offset Key: KAFKA-15074 URL: https://issues.apache.org/jira/browse/KAFKA-15074 Project: Kafka Issue

Re: [DISCUSS] Regarding Old PRs

2023-06-07 Thread David Arthur
I filed KAFKA-15073 for this. Here is a patch https://github.com/apache/kafka/pull/13827. This simply adds a "stale" label to PRs with no activity in the last 90 days. I figure that's a good starting point. As for developer workflow, the "stale" action is quite flexible in how it finds candidate

[jira] [Created] (KAFKA-15073) Automation for old/inactive PRs

2023-06-07 Thread David Arthur (Jira)
David Arthur created KAFKA-15073: Summary: Automation for old/inactive PRs Key: KAFKA-15073 URL: https://issues.apache.org/jira/browse/KAFKA-15073 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-935: Extend AlterConfigPolicy with existing configurations

2023-06-07 Thread Jorge Esteban Quilcate Otoya
Thank Colin. I've took a closer look on how configs are passed to the policy when delete configs are requested, and either null (KRaft) or empty values (ZkAdminManager) are passed: - ZkAdminManager passes empty values: - Config Entries definition:

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #1908

2023-06-07 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-07 Thread Colin McCabe
Hi Luke, Thanks for the review and the suggestion. I think we will add more "handling time" metrics later, but for now I don't want to make this KIP any bigger than it is already... best, Colin On Wed, Jun 7, 2023, at 03:12, Luke Chen wrote: > Hi Colin, > > One comment: > Should we add a

Re: [DISCUSS] KIP-937 Improve Message Timestamp Validation

2023-06-07 Thread Beyene, Mehari
> Although it's more verbose, splitting the configuration into explicit ‘past’ > and ‘future’ would provide the appropriate tradeoff between constraint and > flexibility, right? +1

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

2023-06-07 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-15072) Flaky test MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition

2023-06-07 Thread Josep Prat (Jira)
Josep Prat created KAFKA-15072: -- Summary: Flaky test MirrorConnectorsIntegrationExactlyOnceTest.testReplicationWithEmptyPartition Key: KAFKA-15072 URL: https://issues.apache.org/jira/browse/KAFKA-15072

Re: [DISCUSS] KIP-937 Improve Message Timestamp Validation

2023-06-07 Thread Beyene, Mehari
Luke, thank you for the suggestion of introducing the two configurations. I have discussed this option internally with Divij, and your suggestion does have its merits. I will update the KIP with your suggestions and will revert back in a day or two. Kirk, thank you for participating in the KIP

[jira] [Resolved] (KAFKA-14539) Simplify StreamsMetadataState by replacing the Cluster metadata with partition info map

2023-06-07 Thread Bill Bejeck (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck resolved KAFKA-14539. - Resolution: Fixed > Simplify StreamsMetadataState by replacing the Cluster metadata with >

[jira] [Created] (KAFKA-15071) Flaky test kafka.admin.LeaderElectionCommandTest.testPreferredReplicaElection for Type=ZK, MetadataVersion=3.5-IV2, Security=PLAINTEXT

2023-06-07 Thread Josep Prat (Jira)
Josep Prat created KAFKA-15071: -- Summary: Flaky test kafka.admin.LeaderElectionCommandTest.testPreferredReplicaElection for Type=ZK, MetadataVersion=3.5-IV2, Security=PLAINTEXT Key: KAFKA-15071 URL:

[jira] [Created] (KAFKA-15070) Flaky test kafka.log.LogCleanerParameterizedIntegrationTest.testCleansCombinedCompactAndDeleteTopic for codec zstd

2023-06-07 Thread Josep Prat (Jira)
Josep Prat created KAFKA-15070: -- Summary: Flaky test kafka.log.LogCleanerParameterizedIntegrationTest.testCleansCombinedCompactAndDeleteTopic for codec zstd Key: KAFKA-15070 URL:

Re: [DISCUSS] Regarding Old PRs

2023-06-07 Thread Josep Prat
Thanks David! ——— Josep Prat Aiven Deutschland GmbH Alexanderufer 3-7, 10117 Berlin Amtsgericht Charlottenburg, HRB 209739 B Geschäftsführer: Oskari Saarenmaa & Hannu Valtonen m: +491715557497 w: aiven.io e: josep.p...@aiven.io On Wed, Jun 7, 2023, 20:28 David Arthur wrote: > Hey all, I

Re: [DISCUSS] Regarding Old PRs

2023-06-07 Thread David Arthur
Hey all, I started poking around at Github actions on my fork. https://github.com/mumrah/kafka/actions I'll post a PR if I get it working and we can discuss what kind of settings we want (or if we want it all) -David On Tue, Jun 6, 2023 at 1:18 PM Chris Egerton wrote: > Hi Josep, > > Thanks

Re: oauthbearer client -- override ssl.property (unable to do so)

2023-06-07 Thread Kirk True
Hi Neil, > On Jun 7, 2023, at 10:35 AM, Neil Buesing wrote: > > The code "AccessTokenRetrieverFactory" uses the "jaasConfig"'s for the > properties used for building the socket factory. > > Shouldn't "jou.createSSLSockerFactor()" use the kafka configs for sasl/ssl > overrides? > > I am

Re: [DISCUSS] KIP-917: Additional custom metadata for remote log segment

2023-06-07 Thread Ivan Yurchenko
Hi Satish, Thank you for your feedback. I've nothing against going from Map to byte[]. Serialization should not be a problem for RSM implementations: `Struct`, `Schema` and other useful serde classes are distributed as a part of the kafka-clients library. Also a good idea to add the size

oauthbearer client -- override ssl.property (unable to do so)

2023-06-07 Thread Neil Buesing
The code "AccessTokenRetrieverFactory" uses the "jaasConfig"'s for the properties used for building the socket factory. Shouldn't "jou.createSSLSockerFactor()" use the kafka configs for sasl/ssl overrides? I am looking to do "oauthbearer.ssl.protocol=TLSv1.2" -- but no luck - I have tried many

Re: [DISCUSS] KIP-937 Improve Message Timestamp Validation

2023-06-07 Thread Kirk True
Hi Mehari, Thanks for the KIP and keeping it up-to-date with the discussions here! Question: 1. Is it possible to check for invalid timestamps in the client? Suppose we were to develop a means to determine with high confidence that the user had provided a timestamp in nanoseconds vs.

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #1906

2023-06-07 Thread Apache Jenkins Server
See Changes: -- [...truncated 575654 lines...] [2023-06-07T17:09:44.913Z] [2023-06-07T17:09:44.913Z] Gradle Test Run :core:integrationTest > Gradle Test Executor 178 >

Re: [DISCUSS] KIP-937 Improve Message Timestamp Validation

2023-06-07 Thread Kirk True
Hi Divij/all, > On Jun 6, 2023, at 5:42 AM, Divij Vaidya wrote: > > Hi Luke > > Thank you for your participation in reviewing this KIP. > > #1 Updated the KIP with correct configuration names and hyperlinks. > > #2 Yes, the semantics change from a perspective that the difference is > always

Re: [VOTE] 3.5.0 RC1

2023-06-07 Thread John Roesler
Thanks for running this release, Mickael! I've verified: * the signature * that I can compile the project * that I can run the tests. I saw one flaky test failure, but I don't think it should block us. Reported as https://issues.apache.org/jira/browse/KAFKA-13531?focusedCommentId=17730190 * the

Re: [DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-07 Thread Николай Ижиков
Hello. As author of one of related KIPs I’m +1 for this change. Long waited feature. > 7 июня 2023 г., в 19:02, Edoardo Comar написал(а): > > Dear all, > Adrian and I would like to start a discussion thread on > > KIP-940: Broker extension point for validating record contents at produce time

[jira] [Created] (KAFKA-15069) Refactor scanning hierarchy out of DelegatingClassLoader

2023-06-07 Thread Greg Harris (Jira)
Greg Harris created KAFKA-15069: --- Summary: Refactor scanning hierarchy out of DelegatingClassLoader Key: KAFKA-15069 URL: https://issues.apache.org/jira/browse/KAFKA-15069 Project: Kafka Issue

[DISCUSS] KIP-940: Broker extension point for validating record contents at produce time

2023-06-07 Thread Edoardo Comar
Dear all, Adrian and I would like to start a discussion thread on KIP-940: Broker extension point for validating record contents at produce time https://cwiki.apache.org/confluence/display/KAFKA/KIP-940%3A+Broker+extension+point+for+validating+record+contents+at+produce+time This KIP proposes a

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

2023-06-07 Thread Andrew Schofield
Hi Daniel, True, I see your point. It’s analogous to a KafkaConsumer fetching uncommitted records but not delivering them to the application. Thanks, Andrew > On 7 Jun 2023, at 16:38, Dániel Urbán wrote: > > Hi Andrew, > > I think the "pending" state could be the solution for reading beyond the

Re: [DISCUSS] Adding non-committers as Github collaborators

2023-06-07 Thread John Roesler
Hello again, all, FYI, I've just opened a request for clarification on the ability to trigger builds: https://issues.apache.org/jira/browse/INFRA-24673 Thanks, -John On Tue, Jun 6, 2023, at 19:11, Hao Li wrote: > Thanks John for looking into this! > > Hao > > On Tue, Jun 6, 2023 at 8:32 AM

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

2023-06-07 Thread Dániel Urbán
Hi Andrew, I think the "pending" state could be the solution for reading beyond the LSO. Pending could indicate that a message is not yet available for consumption (so they won't be offered for consumers), but with transactions ending, they can become "available". With a pending state, records

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

2023-06-07 Thread Andrew Schofield
Hi Daniel, Kind of. I don’t want a transaction abort to cause disappearance of records which are already in-flight. A “pending” state doesn’t seem helpful for read_committed. There’s no such disappearance problem for read_uncommitted. Thanks, Andrew > On 7 Jun 2023, at 16:19, Dániel Urbán

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

2023-06-07 Thread Dániel Urbán
Hi Andrew, I agree with having a single isolation.level for the whole group, it makes sense. As for: "b) The default isolation level for a share group is read_committed, in which case the SPSO and SPEO cannot move past the LSO." With this limitation (SPEO not moving beyond LSO), are you trying

[GitHub] [kafka-site] C0urante opened a new pull request, #520: KAFKA-15051: add missing GET plugin/config endpoint

2023-06-07 Thread via GitHub
C0urante opened a new pull request, #520: URL: https://github.com/apache/kafka-site/pull/520 Ports the changes from https://github.com/apache/kafka/pull/13803 back through 3.2, the the version that originally added this endpoint. -- This is an automated message from the Apache Git

[jira] [Resolved] (KAFKA-10337) Wait for pending async commits in commitSync() even if no offsets are specified

2023-06-07 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-10337. - Fix Version/s: 3.6.0 Resolution: Fixed > Wait for pending async commits in commitSync()

Re: [DISCUSS] Partial CI builds - Reducing flakiness with fewer tests

2023-06-07 Thread Chris Egerton
Hi Greg, I can see the point about enabling partial runs as a temporary measure to fight flakiness, and it does carry some merit. In that case, though, we should have an idea of what the desired end state is once we've stopped relying on any temporary measures. Do you think we should aim to

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

2023-06-07 Thread Andrew Schofield
HI Daniel, I’ve been thinking about this question and I think this area is a bit tricky. If there are some consumers in a share group with isolation level read_uncommitted and other consumers with read_committed, they have different expectations with regards to which messages are visible when

Re: [ANNOUNCE] Apache Kafka 3.4.1

2023-06-07 Thread Bill Bejeck
Thanks for running the release Luke! On Wed, Jun 7, 2023 at 4:29 AM Tom Bentley wrote: > Thanks Luke! > > On Wed, 7 Jun 2023 at 09:11, Mickael Maison > wrote: > > > Thanks for running the release! > > > > On Wed, Jun 7, 2023 at 9:11 AM Bruno Cadonna wrote: > > > > > > Thanks Luke! > > > > > >

Re: [DISCUSS] KIP-928: Making Kafka resilient to log directories becoming full

2023-06-07 Thread Christo Lolov
Hey Colin, I tried the following setup: * Create 3 EC2 machines. * EC2 machine named A acts as a KRaft Controller. * EC2 machine named B acts as a KRaft Broker. (The only configurations different to the default values: log.retention.ms=3, log.segment.bytes=1048576,

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #1905

2023-06-07 Thread Apache Jenkins Server
See

[DISCUSS] KIP-842: Add richer group offset reset mechanisms

2023-06-07 Thread hudeqi
Is there any more attention to this KIP? :) bump this thread. Best, hudeqi -原始邮件- 发件人: hudeqi 16120...@bjtu.edu.cn 发送时间: 2023-03-26 17:42:31 (星期日) 收件人: dev@kafka.apache.org 抄送: 主题: Re: Re: Re: [DISCUSS] KIP-842: Add richer group offset reset mechanisms

[jira] [Created] (KAFKA-15068) Incorrect replication Latency may be calculated when the timestamp of the record is of type CREATE_TIME

2023-06-07 Thread hudeqi (Jira)
hudeqi created KAFKA-15068: -- Summary: Incorrect replication Latency may be calculated when the timestamp of the record is of type CREATE_TIME Key: KAFKA-15068 URL: https://issues.apache.org/jira/browse/KAFKA-15068

Re: [DISCUSS] KIP-916: MM2 distributed mode flow log context

2023-06-07 Thread Dániel Urbán
Hi Chris, Thank you for your comments! I updated the KIP. I still need to add the example before/after log lines, will do that soon, but I addressed all the other points. 1. Added more details on thread renaming under Public Interfaces, removed pseudo code. 2. Removed the stale header -

Re: [VOTE] 3.5.0 RC1

2023-06-07 Thread Josep Prat
Hi MIckael, Apparently you did it in this PR already :) : https://github.com/apache/kafka/pull/13749 (this PR among other things removes classgraph. Without being a lawyer, I think I agree with you as stating we depend on something we don't would be less problematic than the other way around.

Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-07 Thread Divij Vaidya
"Yes, I am referring to the feature level. I changed the description of CurrentMetadataVersion to reference the feature level specifically." Thanks Colin. I have reviewed the KIP after the latest changes (including addition of two new metrics). It looks good to me. +1 (non-binding) -- Divij

Re: [VOTE] 3.5.0 RC1

2023-06-07 Thread Mickael Maison
Hi Josep, Thanks for spotting this. If not already done, can you open a ticket/PR to fix this on trunk? It looks like the last couple of releases already had that issue. Since we're including a license for a dependency we don't ship, I think we can consider this non blocking. The other way around

Re: [VOTE] KIP-938: Add more metrics for measuring KRaft performance

2023-06-07 Thread Luke Chen
Hi Colin, One comment: Should we add a metric to record the snapshot handling time? Since we know the snapshot loading might take long if the size is huge. We might want to know how much time it is processed. WDYT? No matter you think we need it or not, the KIP LGTM. +1 from me. Thank you.

[jira] [Created] (KAFKA-15067) kafka SSL support with differnt ssl providers

2023-06-07 Thread Aldan Brito (Jira)
Aldan Brito created KAFKA-15067: --- Summary: kafka SSL support with differnt ssl providers Key: KAFKA-15067 URL: https://issues.apache.org/jira/browse/KAFKA-15067 Project: Kafka Issue Type: Test

RE: [DISCUSS] Partial CI builds - Reducing flakiness with fewer tests

2023-06-07 Thread Gaurav Narula
Hey Greg, Thanks for sharing this idea! The idea of building and testing a relevant subset of code certainly seems interesting. Perhaps this is a good fit for Bazel [1] where target-determinator [2] can be used to to find a subset of targets that have changed between two commits. Even

[jira] [Created] (KAFKA-15066) passing listener name config into TopicBasedRemoteLogMetadataManagerConfig

2023-06-07 Thread Luke Chen (Jira)
Luke Chen created KAFKA-15066: - Summary: passing listener name config into TopicBasedRemoteLogMetadataManagerConfig Key: KAFKA-15066 URL: https://issues.apache.org/jira/browse/KAFKA-15066 Project: Kafka

Re: [ANNOUNCE] Apache Kafka 3.4.1

2023-06-07 Thread Tom Bentley
Thanks Luke! On Wed, 7 Jun 2023 at 09:11, Mickael Maison wrote: > Thanks for running the release! > > On Wed, Jun 7, 2023 at 9:11 AM Bruno Cadonna wrote: > > > > Thanks Luke! > > > > On 07.06.23 07:55, Federico Valeri wrote: > > > Thanks Luke! > > > > > > On Wed, Jun 7, 2023 at 5:56 AM Kamal

[jira] [Created] (KAFKA-15065) ApiVersionRequest is not properly handled in Sasl ControllerServer

2023-06-07 Thread Deng Ziming (Jira)
Deng Ziming created KAFKA-15065: --- Summary: ApiVersionRequest is not properly handled in Sasl ControllerServer Key: KAFKA-15065 URL: https://issues.apache.org/jira/browse/KAFKA-15065 Project: Kafka

Re: [ANNOUNCE] Apache Kafka 3.4.1

2023-06-07 Thread Mickael Maison
Thanks for running the release! On Wed, Jun 7, 2023 at 9:11 AM Bruno Cadonna wrote: > > Thanks Luke! > > On 07.06.23 07:55, Federico Valeri wrote: > > Thanks Luke! > > > > On Wed, Jun 7, 2023 at 5:56 AM Kamal Chandraprakash > > wrote: > >> > >> Thanks Luke for running this release! > >> > >> On

Re: [ANNOUNCE] Apache Kafka 3.4.1

2023-06-07 Thread Bruno Cadonna
Thanks Luke! On 07.06.23 07:55, Federico Valeri wrote: Thanks Luke! On Wed, Jun 7, 2023 at 5:56 AM Kamal Chandraprakash wrote: Thanks Luke for running this release! On Wed, Jun 7, 2023 at 8:08 AM Chia-Ping Tsai wrote: Thank Luke for this hard work!!! Chris Egerton 於 2023年6月7日 上午10:35

[jira] [Created] (KAFKA-15064) Use KafkaTemplate to send message with below exception - IllegalMonitorStateException

2023-06-07 Thread Xuguang zhan (Jira)
Xuguang zhan created KAFKA-15064: Summary: Use KafkaTemplate to send message with below exception - IllegalMonitorStateException Key: KAFKA-15064 URL: https://issues.apache.org/jira/browse/KAFKA-15064