[DISCUSS] KIP-995: Allow users to specify initial offsets while creating connectors

2023-11-29 Thread Ashwin
Hi all, I'd like to begin discussion on KIP-995 which proposes to allow users to specify initial offset as part of the request to create a connector https://cwiki.apache.org/confluence/display/KAFKA/KIP-995%3A+Allow+users+to+specify+initial+offsets+while+creating+connectors During the

Re: [DISCUSS] KIP-977: Partition-Level Throughput Metrics

2023-11-29 Thread Qichao Chu
Hi Vahid, Thank you so much for the review and voting! Best, Qichao On Wed, Nov 29, 2023 at 2:55 PM Vahid Hashemian wrote: > Hi Qichao, > > Thanks for answering my questions and updating the KIP accordingly. > > It looks good to me. > > --Vahid > > > On Tue, Nov 28, 2023, 7:22 PM Qichao Chu

Re: [DISCUSS] KIP-997 Support fetch(fromKey, toKey, from, to) to WindowRangeQuery and unify WindowKeyQuery and WindowRangeQuery

2023-11-29 Thread Hanyu (Peter) Zheng
Thank you Bruno, 1. Thank you for the notification. I have updated the ticket link accordingly. 2. Certainly, I'll update the KIP name. Should I initiate a new discussion for it, because if I change the name, the link will change. 3. Understood, I will add that to the KIP. 4. I propose we accept

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.6 #121

2023-11-29 Thread Apache Jenkins Server
See Changes: -- [...truncated 413481 lines...] Gradle Test Run :streams:test > Gradle Test Executor 88 > DefaultStateUpdaterTest > shouldRemovePausedTask() STARTED Gradle

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

2023-11-29 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-994: Minor Enhancements to ListTransactions and DescribeTransactions APIs

2023-11-29 Thread Justine Olshan
Hey Raman, Thanks for the KIP! I think this will be super useful. Given https://issues.apache.org/jira/browse/KAFKA-15546 -- do you think it would be useful to specify the duration of the completed transaction rather than the time since the start in the describe output? We would probably want to

Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-29 Thread José Armando García Sancio
Hi Alyssa, 1. In the schema for VoteRequest and VoteResponse, you are using "boolean" as the type keyword. The correct keyword should be "bool" instead. 2. In the states and state transaction table you have the following entry: > * Candidate transitions to: > *... > *Prospective: After

Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-29 Thread Jun Rao
Hi, Alyssa, Thanks for the KIP. A few comments below. 10. "If a server happens to receive multiple VoteResponses from another server for a particular VoteRequest, it can take the first and ignore the rest.": Could you explain why a server would receive multiple responses for the same request?

[jira] [Created] (KAFKA-15950) CommunicationEvent should be scheduled with EarliestDeadlineFunction

2023-11-29 Thread Jun Rao (Jira)
Jun Rao created KAFKA-15950: --- Summary: CommunicationEvent should be scheduled with EarliestDeadlineFunction Key: KAFKA-15950 URL: https://issues.apache.org/jira/browse/KAFKA-15950 Project: Kafka

[jira] [Resolved] (KAFKA-15311) Fix docs about reverting to ZooKeeper mode during KRaft migration

2023-11-29 Thread Colin McCabe (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin McCabe resolved KAFKA-15311. -- Fix Version/s: 3.7.0 Resolution: Fixed > Fix docs about reverting to ZooKeeper mode

[jira] [Created] (KAFKA-15949) Improve the KRaft metadata version related messages

2023-11-29 Thread Jakub Scholz (Jira)
Jakub Scholz created KAFKA-15949: Summary: Improve the KRaft metadata version related messages Key: KAFKA-15949 URL: https://issues.apache.org/jira/browse/KAFKA-15949 Project: Kafka Issue

Re: [VOTE] 3.6.1 RC0

2023-11-29 Thread Justine Olshan
I built from source and ran a simple transactional produce bench. I ran a handful of unit tests as well. I scanned the docs and everything looked reasonable. I was wondering if we got the system test results mentioned > System tests: Still running I'll post an update once they complete. Justine

[jira] [Resolved] (KAFKA-15947) Null pointer on LZ4 compression since Kafka 3.6

2023-11-29 Thread Ludo (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ludo resolved KAFKA-15947. -- Fix Version/s: 3.6.1 Resolution: Duplicate > Null pointer on LZ4 compression since Kafka 3.6 >

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

2023-11-29 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-15948) Refactor AsyncKafkaConsumer shutdown

2023-11-29 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15948: -- Summary: Refactor AsyncKafkaConsumer shutdown Key: KAFKA-15948 URL: https://issues.apache.org/jira/browse/KAFKA-15948 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-15947) Null pointer on LZ4 compression since Kafka 3.6

2023-11-29 Thread Ludo (Jira)
Ludo created KAFKA-15947: Summary: Null pointer on LZ4 compression since Kafka 3.6 Key: KAFKA-15947 URL: https://issues.apache.org/jira/browse/KAFKA-15947 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-15946) AsyncKafkaConsumer should retry commits on the application thread instead of autoretry

2023-11-29 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15946: -- Summary: AsyncKafkaConsumer should retry commits on the application thread instead of autoretry Key: KAFKA-15946 URL: https://issues.apache.org/jira/browse/KAFKA-15946

[jira] [Resolved] (KAFKA-5046) Support file rotation in FileStreamSource Connector

2023-11-29 Thread Konstantine Karantasis (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-5046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Konstantine Karantasis resolved KAFKA-5046. --- Resolution: Won't Fix > Support file rotation in FileStreamSource Connector

[jira] [Resolved] (KAFKA-15046) Produce performance issue under high disk load

2023-11-29 Thread Jun Rao (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15046?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-15046. - Fix Version/s: 3.7.0 Resolution: Fixed merged the PR to trunk. > Produce performance issue

[jira] [Created] (KAFKA-15945) Flaky test - testSyncTopicConfigs() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest

2023-11-29 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-15945: Summary: Flaky test - testSyncTopicConfigs() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationBaseTest Key: KAFKA-15945 URL:

[jira] [Created] (KAFKA-15944) Flaky test - verifyStore[cache=false, log=true, supplier=ROCKS_KV, kind=DSL] – org.apache.kafka.streams.integration.PositionRestartIntegrationTest

2023-11-29 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-15944: Summary: Flaky test - verifyStore[cache=false, log=true, supplier=ROCKS_KV, kind=DSL] – org.apache.kafka.streams.integration.PositionRestartIntegrationTest Key: KAFKA-15944

[jira] [Created] (KAFKA-15943) Flaky test - testHighWaterMarkAfterPartitionReassignment(String).quorum=kraft – org.apache.kafka.tools.reassign.ReassignPartitionsIntegrationTest

2023-11-29 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-15943: Summary: Flaky test - testHighWaterMarkAfterPartitionReassignment(String).quorum=kraft – org.apache.kafka.tools.reassign.ReassignPartitionsIntegrationTest Key: KAFKA-15943

[jira] [Created] (KAFKA-15942) Implement ConsumerInterceptor

2023-11-29 Thread Philip Nee (Jira)
Philip Nee created KAFKA-15942: -- Summary: Implement ConsumerInterceptor Key: KAFKA-15942 URL: https://issues.apache.org/jira/browse/KAFKA-15942 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-15941) Flaky test: shouldRestoreNullRecord() – org.apache.kafka.streams.integration.RestoreIntegrationTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15941: - Summary: Flaky test: shouldRestoreNullRecord() – org.apache.kafka.streams.integration.RestoreIntegrationTest Key: KAFKA-15941 URL:

[jira] [Created] (KAFKA-15940) Flaky test: testLogCleanerConfig(String).quorum=kraft – kafka.server.DynamicBrokerReconfigurationTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15940: - Summary: Flaky test: testLogCleanerConfig(String).quorum=kraft – kafka.server.DynamicBrokerReconfigurationTest Key: KAFKA-15940 URL:

[jira] [Created] (KAFKA-15939) Flaky test: testInvalidAlterConfigs(String).quorum=kraft – kafka.api.AdminClientWithPoliciesIntegrationTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15939: - Summary: Flaky test: testInvalidAlterConfigs(String).quorum=kraft – kafka.api.AdminClientWithPoliciesIntegrationTest Key: KAFKA-15939 URL:

[jira] [Created] (KAFKA-15938) Flaky test: testCreateRemoteTopicWithValidRetentionTime(String).quorum=kraft – kafka.admin.RemoteTopicCrudTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15938: - Summary: Flaky test: testCreateRemoteTopicWithValidRetentionTime(String).quorum=kraft – kafka.admin.RemoteTopicCrudTest Key: KAFKA-15938 URL:

[jira] [Created] (KAFKA-15937) Flaky test: testTopicDeletion(String).quorum=kraft – kafka.admin.RemoteTopicCrudTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15937: - Summary: Flaky test: testTopicDeletion(String).quorum=kraft – kafka.admin.RemoteTopicCrudTest Key: KAFKA-15937 URL: https://issues.apache.org/jira/browse/KAFKA-15937

[jira] [Created] (KAFKA-15936) Flaky tests for class: ConnectorTopicsIntegrationTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15936: - Summary: Flaky tests for class: ConnectorTopicsIntegrationTest Key: KAFKA-15936 URL: https://issues.apache.org/jira/browse/KAFKA-15936 Project: Kafka

[jira] [Created] (KAFKA-15935) Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15935: - Summary: Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.IdentityReplicationIntegrationTest Key: KAFKA-15935 URL:

[jira] [Created] (KAFKA-15934) Flaky test: testMultiConsumerStickyAssignor(String, String).quorum=kraft+kip848.groupProtocol=generic – kafka.api.PlaintextConsumerTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15934: - Summary: Flaky test: testMultiConsumerStickyAssignor(String, String).quorum=kraft+kip848.groupProtocol=generic – kafka.api.PlaintextConsumerTest Key: KAFKA-15934 URL:

[jira] [Created] (KAFKA-15933) Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest

2023-11-29 Thread Apoorv Mittal (Jira)
Apoorv Mittal created KAFKA-15933: - Summary: Flaky test: testRestartReplication() – org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTransactionsTest Key: KAFKA-15933 URL:

[jira] [Reopened] (KAFKA-14089) Flaky ExactlyOnceSourceIntegrationTest.testSeparateOffsetsTopic

2023-11-29 Thread Apoorv Mittal (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14089?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apoorv Mittal reopened KAFKA-14089: --- Assignee: (was: Chris Egerton) Failure occurred on PR build:

Re: [VOTE] 3.6.1 RC0

2023-11-29 Thread Mickael Maison
Hi Josep, Good catch! If it's the only issue we find, I don't think we should block the release just to fix that. If we find another issue, I'll backport it before running another RC, otherwise I'll backport it once 3.6.1 is released. Thanks, Mickael On Wed, Nov 29, 2023 at 11:55 AM Josep Prat

Re: [DISCUSS] KIP-997 Support fetch(fromKey, toKey, from, to) to WindowRangeQuery and unify WindowKeyQuery and WindowRangeQuery

2023-11-29 Thread Bruno Cadonna
Hi, Thanks for the updates! 1. Could you please link the correct ticket in the KIP? 2. Could you please adapt the motivation section and the title to the updated goal of the KIP? There is no fetch() or fetchAll() method in the query class. 3. Could you please add the "// newly added"

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

2023-11-29 Thread Apache Jenkins Server
See Changes: -- [...truncated 424064 lines...] Gradle Test Run :connect:runtime:test > Gradle Test Executor 47 > org.apache.kafka.connect.util.TopicAdminTest >

Re: [DISCUSS] KIP-967: Support custom SSL configuration for Kafka Connect RestServer

2023-11-29 Thread Taras Ledkov
Hi team, Ping for review / vote for KIP-967 [1]. Voting thread is here [2] [1]. https://cwiki.apache.org/confluence/display/KAFKA/KIP-967%3A+Support+custom+SSL+configuration+for+Kafka+Connect+RestServer [2]. https://github.com/apache/kafka/pull/14203 [2].

[jira] [Created] (KAFKA-15932) Flaky test - PlaintextConsumerTest.testSeek("kraft+kip-848","consumer")

2023-11-29 Thread Andrew Schofield (Jira)
Andrew Schofield created KAFKA-15932: Summary: Flaky test - PlaintextConsumerTest.testSeek("kraft+kip-848","consumer") Key: KAFKA-15932 URL: https://issues.apache.org/jira/browse/KAFKA-15932

[jira] [Created] (KAFKA-15931) Cached transaction index gets closed if tiered storage read is interrupted

2023-11-29 Thread Ivan Yurchenko (Jira)
Ivan Yurchenko created KAFKA-15931: -- Summary: Cached transaction index gets closed if tiered storage read is interrupted Key: KAFKA-15931 URL: https://issues.apache.org/jira/browse/KAFKA-15931

Re: [VOTE] 3.6.1 RC0

2023-11-29 Thread Josep Prat
Hi Mickael, This PR[1] made me realize NOTICE-binary is missing the notice for commons-io. I don't know if it's a blocker or not. I can cherry pick the commit to the 3.6 branch if you want. Best, [1]: https://github.com/apache/kafka/pull/14865 On Tue, Nov 28, 2023 at 10:25 AM Josep Prat

[jira] [Resolved] (KAFKA-15887) Autocommit during close consistently fails with exception in background thread

2023-11-29 Thread Lucas Brutschy (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15887?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lucas Brutschy resolved KAFKA-15887. Resolution: Fixed > Autocommit during close consistently fails with exception in

[jira] [Created] (KAFKA-15930) Flaky test - testWithGroupId - TransactionsBounceTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15930: -- Summary: Flaky test - testWithGroupId - TransactionsBounceTest Key: KAFKA-15930 URL: https://issues.apache.org/jira/browse/KAFKA-15930 Project: Kafka

Re: [DISCUSS] KIP-996: Pre-Vote

2023-11-29 Thread Luke Chen
Hi Alyssa, Thanks for the KIP! This is an important improvement for KRaft quorum. Some comments: 1. Follower transitions to: Prospective: After expiration of the election timeout -> Is this the fetch timeout, not election timeout? 2. I also agree we don't bump the epoch in prospective state. A

[jira] [Created] (KAFKA-15929) Flaky test - testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress - TopicCommandIntegrationTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15929: -- Summary: Flaky test - testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress - TopicCommandIntegrationTest Key: KAFKA-15929 URL:

[jira] [Created] (KAFKA-15928) Flaky test - testBalancePartitionLeaders - QuorumControllerTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15928: -- Summary: Flaky test - testBalancePartitionLeaders - QuorumControllerTest Key: KAFKA-15928 URL: https://issues.apache.org/jira/browse/KAFKA-15928 Project: Kafka

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

2023-11-29 Thread Apache Jenkins Server
See Changes: -- [...truncated 433105 lines...] Gradle Test Run :core:test > Gradle Test Executor 92 > ZkConfigMigrationClientTest > testWriteNewTopicConfigs() STARTED

[jira] [Created] (KAFKA-15927) Flaky test - testReplicateSourceDefault - MirrorConnectorsIntegrationExactlyOnceTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15927: -- Summary: Flaky test - testReplicateSourceDefault - MirrorConnectorsIntegrationExactlyOnceTest Key: KAFKA-15927 URL: https://issues.apache.org/jira/browse/KAFKA-15927

[jira] [Created] (KAFKA-15926) Flaky test - testReplicateSourceDefault - MirrorConnectorsIntegrationSSLTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15926: -- Summary: Flaky test - testReplicateSourceDefault - MirrorConnectorsIntegrationSSLTest Key: KAFKA-15926 URL: https://issues.apache.org/jira/browse/KAFKA-15926

[jira] [Created] (KAFKA-15925) Flaky test testReplicateSourceDefault - MirrorConnectorsIntegrationTransactionsTest

2023-11-29 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-15925: -- Summary: Flaky test testReplicateSourceDefault - MirrorConnectorsIntegrationTransactionsTest Key: KAFKA-15925 URL: https://issues.apache.org/jira/browse/KAFKA-15925

[jira] [Created] (KAFKA-15924) Flaky test - QuorumControllerTest.testFatalMetadataReplayErrorOnActive

2023-11-29 Thread Haruki Okada (Jira)
Haruki Okada created KAFKA-15924: Summary: Flaky test - QuorumControllerTest.testFatalMetadataReplayErrorOnActive Key: KAFKA-15924 URL: https://issues.apache.org/jira/browse/KAFKA-15924 Project:

[jira] [Created] (KAFKA-15923) Implement changes proposed in KIP-994

2023-11-29 Thread Raman Verma (Jira)
Raman Verma created KAFKA-15923: --- Summary: Implement changes proposed in KIP-994 Key: KAFKA-15923 URL: https://issues.apache.org/jira/browse/KAFKA-15923 Project: Kafka Issue Type: Task