[GitHub] [kafka] ableegoldman commented on a change in pull request #11686: KAFKA-12648: invoke exception handler for MissingSourceTopicException with named topologies

2022-01-19 Thread GitBox
ableegoldman commented on a change in pull request #11686: URL: https://github.com/apache/kafka/pull/11686#discussion_r788430597 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java ## @@ -515,11 +518,23 @@ private

[GitHub] [kafka] guozhangwang commented on a change in pull request #11609: KAFKA-12648: fixes for query APIs and blocking calls

2022-01-19 Thread GitBox
guozhangwang commented on a change in pull request #11609: URL: https://github.com/apache/kafka/pull/11609#discussion_r788356598 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java ## @@ -428,7 +469,7 @@ private void

[jira] [Created] (KAFKA-13604) Add pluggable logging framework support

2022-01-19 Thread Dongjin Lee (Jira)
Dongjin Lee created KAFKA-13604: --- Summary: Add pluggable logging framework support Key: KAFKA-13604 URL: https://issues.apache.org/jira/browse/KAFKA-13604 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-13603) empty active segment can trigger recovery after clean shutdown and restart

2022-01-19 Thread Cong Ding (Jira)
Cong Ding created KAFKA-13603: - Summary: empty active segment can trigger recovery after clean shutdown and restart Key: KAFKA-13603 URL: https://issues.apache.org/jira/browse/KAFKA-13603 Project: Kafka

[jira] [Assigned] (KAFKA-13603) empty active segment can trigger recovery after clean shutdown and restart

2022-01-19 Thread Cong Ding (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13603?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cong Ding reassigned KAFKA-13603: - Assignee: Cong Ding > empty active segment can trigger recovery after clean shutdown and

[jira] [Created] (KAFKA-13602) Allow to broadcast a result record

2022-01-19 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-13602: --- Summary: Allow to broadcast a result record Key: KAFKA-13602 URL: https://issues.apache.org/jira/browse/KAFKA-13602 Project: Kafka Issue Type: New

[jira] [Updated] (KAFKA-13602) Allow to broadcast a result record

2022-01-19 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13602?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-13602: Labels: needs-kip newbie++ (was: needs-kip) > Allow to broadcast a result record >

[GitHub] [kafka] showuon commented on pull request #11692: MINOR: Upgrade jetty-server to 9.4.44.v20210927

2022-01-19 Thread GitBox
showuon commented on pull request #11692: URL: https://github.com/apache/kafka/pull/11692#issuecomment-1017027090 @dajac , I saw you're working on the same issue as me. could you check this PR: https://github.com/apache/kafka/pull/11656 ? That also upgrade some other libraries with

[GitHub] [kafka] hachikuji commented on a change in pull request #11665: KAFKA-13585; Fix flaky test `ReplicaManagerTest.testReplicaAlterLogDirsWithAndWithoutIds`

2022-01-19 Thread GitBox
hachikuji commented on a change in pull request #11665: URL: https://github.com/apache/kafka/pull/11665#discussion_r788276315 ## File path: core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala ## @@ -3541,7 +3541,14 @@ class ReplicaManagerTest { val version = if

[GitHub] [kafka] Kvicii edited a comment on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
Kvicii edited a comment on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1017010507 @dajac I used `computeUntilTrue`, but it is also very difficult to reproduce using a script.I tried a few other ways, but none seem to work -- This is an automated

[GitHub] [kafka] Kvicii commented on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
Kvicii commented on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1017010507 @dajac I used `computeUntilTrue`, but it is also very difficult to reproduce using a script☹️ -- This is an automated message from the Apache Git Service. To respond to the

[jira] [Comment Edited] (KAFKA-13497) Debug Log RegexRouter transform

2022-01-19 Thread Spencer Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479034#comment-17479034 ] Spencer Carlson edited comment on KAFKA-13497 at 1/20/22, 12:12 AM:

[jira] [Comment Edited] (KAFKA-13497) Debug Log RegexRouter transform

2022-01-19 Thread Spencer Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479034#comment-17479034 ] Spencer Carlson edited comment on KAFKA-13497 at 1/20/22, 12:09 AM:

[jira] [Comment Edited] (KAFKA-13497) Debug Log RegexRouter transform

2022-01-19 Thread Spencer Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479034#comment-17479034 ] Spencer Carlson edited comment on KAFKA-13497 at 1/20/22, 12:09 AM:

[jira] [Commented] (KAFKA-13497) Debug Log RegexRouter transform

2022-01-19 Thread Spencer Carlson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479034#comment-17479034 ] Spencer Carlson commented on KAFKA-13497: - My request is very similar to what is already being

[GitHub] [kafka] mattwong949 opened a new pull request #11694: MINOR: deleteHorizonMs update to documentation and DumpLogSegments tool

2022-01-19 Thread GitBox
mattwong949 opened a new pull request #11694: URL: https://github.com/apache/kafka/pull/11694 This PR updates the documentation and tooling to match the changes made in https://github.com/apache/kafka/pull/10914 In the documentation, changes include the adding the new attribute and

[jira] [Commented] (KAFKA-13486) Kafka Connect: Failed to start task due to NPE

2022-01-19 Thread Jordan Moore (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479033#comment-17479033 ] Jordan Moore commented on KAFKA-13486: -- You seem to be missing part of the stacktrace. The workers

[jira] [Commented] (KAFKA-13497) Debug Log RegexRouter transform

2022-01-19 Thread Jordan Moore (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17479029#comment-17479029 ] Jordan Moore commented on KAFKA-13497: -- While extra logging would be useful, creating a Matcher and

[GitHub] [kafka] vvcephei commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vvcephei commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r788231897 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java ## @@ -145,4 +145,9 @@ public void

[GitHub] [kafka] dongjinleekr commented on a change in pull request #7898: KAFKA-9366: Change log4j dependency into log4j2

2022-01-19 Thread GitBox
dongjinleekr commented on a change in pull request #7898: URL: https://github.com/apache/kafka/pull/7898#discussion_r788207067 ## File path: bin/connect-distributed.sh ## @@ -23,13 +23,16 @@ fi base_dir=$(dirname $0) if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then +echo

[GitHub] [kafka] junrao commented on pull request #11345: Allow empty last segment to have missing offset index during recovery

2022-01-19 Thread GitBox
junrao commented on pull request #11345: URL: https://github.com/apache/kafka/pull/11345#issuecomment-1016906026 @ccding : The PR is kind of large now. Could you associate the PR with a jira? Thanks. -- This is an automated message from the Apache Git Service. To respond to the message,

[GitHub] [kafka] florin-akermann edited a comment on pull request #11456: KAFKA-13351: Add possibility to write kafka headers in Kafka Console Producer

2022-01-19 Thread GitBox
florin-akermann edited a comment on pull request #11456: URL: https://github.com/apache/kafka/pull/11456#issuecomment-1016901381 @dajac Thanks a lot for the PR review and the guidance from a-Z. Just like with Mickael, the whole process was a nice experience. -- This is an automated

[GitHub] [kafka] florin-akermann commented on pull request #11456: KAFKA-13351: Add possibility to write kafka headers in Kafka Console Producer

2022-01-19 Thread GitBox
florin-akermann commented on pull request #11456: URL: https://github.com/apache/kafka/pull/11456#issuecomment-1016901381 @dajac Thanks a lot for the PR review and the guidance from A-Z. Just like with Mickael, the whole process was a nice experience. -- This is an automated message

[GitHub] [kafka] florin-akermann commented on a change in pull request #11456: KAFKA-13351: Add possibility to write kafka headers in Kafka Console Producer

2022-01-19 Thread GitBox
florin-akermann commented on a change in pull request #11456: URL: https://github.com/apache/kafka/pull/11456#discussion_r786289278 ## File path: core/src/main/scala/kafka/tools/ConsoleProducer.scala ## @@ -264,39 +278,92 @@ object ConsoleProducer { var reader:

[GitHub] [kafka] hachikuji merged pull request #11452: KAFKA-13412; Ensure initTransactions() safe for retry after timeout

2022-01-19 Thread GitBox
hachikuji merged pull request #11452: URL: https://github.com/apache/kafka/pull/11452 -- 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:

[GitHub] [kafka] hachikuji opened a new pull request #11693: MINOR: Convert LogLoader into a class

2022-01-19 Thread GitBox
hachikuji opened a new pull request #11693: URL: https://github.com/apache/kafka/pull/11693 The logic for log loading is encapsulated in `LogLoader`. Currently all the methods are static and we pass the parameters through a separate object `LogLoaderParams`. It seems simpler to turn

[GitHub] [kafka] guozhangwang commented on a change in pull request #11686: KAFKA-12648: invoke exception handler for MissingSourceTopicException with named topologies

2022-01-19 Thread GitBox
guozhangwang commented on a change in pull request #11686: URL: https://github.com/apache/kafka/pull/11686#discussion_r788072458 ## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ## @@ -523,16 +528,21 @@ private boolean wrappedExceptionIsIn(final

[GitHub] [kafka] mjsax commented on pull request #11613: MINOR: Update streamResetter option description

2022-01-19 Thread GitBox
mjsax commented on pull request #11613: URL: https://github.com/apache/kafka/pull/11613#issuecomment-1016804211 Thanks for the PR! Merged to `trunk`. -- 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

[GitHub] [kafka] mjsax merged pull request #11613: MINOR: Update streamResetter option description

2022-01-19 Thread GitBox
mjsax merged pull request #11613: URL: https://github.com/apache/kafka/pull/11613 -- 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:

[GitHub] [kafka] mjsax commented on a change in pull request #11592: KAFKA-13501: Avoid state restore via rebalance if standbys are enabled

2022-01-19 Thread GitBox
mjsax commented on a change in pull request #11592: URL: https://github.com/apache/kafka/pull/11592#discussion_r788073751 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java ## @@ -228,8 +228,9 @@ public void initializeIfNeeded() {

[GitHub] [kafka] mjsax commented on pull request #11211: KAFKA-12960: Enforcing strict retention time for WindowStore and Sess…

2022-01-19 Thread GitBox
mjsax commented on pull request #11211: URL: https://github.com/apache/kafka/pull/11211#issuecomment-1016797562 Just catching up on this one... I am fine with pushing it into the most inner store as proposed by Guozhang. Two requests: (1) can we update the JavaDocs of the store

[GitHub] [kafka] guozhangwang commented on pull request #11686: KAFKA-12648: invoke exception handler for MissingSourceTopicException with named topologies

2022-01-19 Thread GitBox
guozhangwang commented on pull request #11686: URL: https://github.com/apache/kafka/pull/11686#issuecomment-1016796802 @cadonna I was referring to https://issues.apache.org/jira/browse/KAFKA-13543. I chatted with @ableegoldman a bit offline and now I get it that we want to keep up

[jira] [Updated] (KAFKA-13543) Consumer may pass stale cluster metadata to the assignor following a subscription update

2022-01-19 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-13543: -- Labels: new-rebalance-should-fix (was: ) > Consumer may pass stale cluster metadata to the

[jira] [Updated] (KAFKA-13601) Add option to support sync offset commit in Kafka Connect Sink

2022-01-19 Thread Anil Dasari (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anil Dasari updated KAFKA-13601: Description: Exactly once in s3 connector with scheduled rotation and field partitioner can be

[jira] [Updated] (KAFKA-13601) Add option to support sync offset commit in Kafka Connect Sink

2022-01-19 Thread Anil Dasari (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anil Dasari updated KAFKA-13601: Description: Exactly once in s3 connector with scheduled rotation and field partitioner can be

[jira] [Commented] (KAFKA-12879) Compatibility break in Admin.listOffsets()

2022-01-19 Thread Kirk True (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478899#comment-17478899 ] Kirk True commented on KAFKA-12879: --- [~tombentley] - is there a specific way you're invoking

[GitHub] [kafka] mjsax commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
mjsax commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788055037 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java ## @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788053269 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java ## @@ -2221,14 +2231,94 @@ public void

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788053117 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java ## @@ -2221,14 +2231,94 @@ public void

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788052839 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java ## @@ -2221,14 +2231,94 @@ public void

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788051969 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java ## @@ -2221,14 +2231,94 @@ public void

[jira] [Updated] (KAFKA-13601) Add option to support sync offset commit in Kafka Connect Sink

2022-01-19 Thread Anil Dasari (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anil Dasari updated KAFKA-13601: Description: Exactly once in s3 connector with scheduled rotation and field partitioner can be

[jira] [Created] (KAFKA-13601) Add option to support sync offset commit in Kafka Connect Sink

2022-01-19 Thread Anil Dasari (Jira)
Anil Dasari created KAFKA-13601: --- Summary: Add option to support sync offset commit in Kafka Connect Sink Key: KAFKA-13601 URL: https://issues.apache.org/jira/browse/KAFKA-13601 Project: Kafka

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788050048 ## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java ## @@ -2221,14 +2231,94 @@ public void

[GitHub] [kafka] Gerrrr commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
Ge commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788034547 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java ## @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache

[GitHub] [kafka] rgoers commented on a change in pull request #7898: KAFKA-9366: Change log4j dependency into log4j2

2022-01-19 Thread GitBox
rgoers commented on a change in pull request #7898: URL: https://github.com/apache/kafka/pull/7898#discussion_r788034050 ## File path: bin/connect-distributed.sh ## @@ -23,13 +23,16 @@ fi base_dir=$(dirname $0) if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then +echo "DEPRECATED:

[GitHub] [kafka] mjsax commented on a change in pull request #11683: KAFKA-6502: Update consumed offsets on corrupted records.

2022-01-19 Thread GitBox
mjsax commented on a change in pull request #11683: URL: https://github.com/apache/kafka/pull/11683#discussion_r788014733 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java ## @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache

[GitHub] [kafka] viktorsomogyi commented on a change in pull request #7898: KAFKA-9366: Change log4j dependency into log4j2

2022-01-19 Thread GitBox
viktorsomogyi commented on a change in pull request #7898: URL: https://github.com/apache/kafka/pull/7898#discussion_r788006560 ## File path: bin/connect-distributed.sh ## @@ -23,13 +23,16 @@ fi base_dir=$(dirname $0) if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then +echo

[GitHub] [kafka] dineshudayakumar edited a comment on pull request #11476: KAFKA-13407: Always start controller when broker wins election

2022-01-19 Thread GitBox
dineshudayakumar edited a comment on pull request #11476: URL: https://github.com/apache/kafka/pull/11476#issuecomment-1016688000 I was seeing this issue in 3.0 in our on-prem 3 nodes K8s cluster Applied this patch and after that I wasn't able to reproduce it. Is there a reason why

[GitHub] [kafka] dineshudayakumar commented on pull request #11476: KAFKA-13407: Always start controller when broker wins election

2022-01-19 Thread GitBox
dineshudayakumar commented on pull request #11476: URL: https://github.com/apache/kafka/pull/11476#issuecomment-1016688000 I was seeing this issue in 3.0. Applied this patch and after that I wasn't able to reproduce it. Is there a reason why this is not merged to a release yet?

[GitHub] [kafka] dajac commented on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
dajac commented on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1016614978 @Kvicii `ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions` does not fail all the time but only occasionally. It seems that there is a race condition in

[GitHub] [kafka] vpapavas commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vpapavas commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r787893711 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java ## @@ -148,8 +150,12 @@ default void init(final

[GitHub] [kafka] Kvicii commented on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
Kvicii commented on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1016606448 @dajac The first build failed didn't seem to be because of this test, so I rebuilt it.Can you tell me what the expectations should be, thanks a lot -- This is an automated

[GitHub] [kafka] dajac opened a new pull request #11692: MINOR: Upgrade jetty-server to 9.4.44.v20210927

2022-01-19 Thread GitBox
dajac opened a new pull request #11692: URL: https://github.com/apache/kafka/pull/11692 Release notes: https://github.com/eclipse/jetty.project/releases/tag/jetty-9.4.44.v20210927 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation -

[GitHub] [kafka] Kvicii removed a comment on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
Kvicii removed a comment on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1016586326 > The first time it failed due to other tests, so I rebuilt it. During the build I see that testTopicIdCreatedOnUpgrade has passed. -- This is an automated

[GitHub] [kafka] Kvicii commented on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
Kvicii commented on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1016586326 > The first time it failed due to other tests, so I rebuilt it. During the build I see that testTopicIdCreatedOnUpgrade has passed. -- This is an automated message

[jira] [Commented] (KAFKA-13435) Group won't consume partitions added after static member restart

2022-01-19 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13435?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478765#comment-17478765 ] David Jacot commented on KAFKA-13435: - [~rleslie] I have published a small KIP to address this

[GitHub] [kafka] dajac commented on pull request #11687: KAFKA-13592:Fix flaky test ControllerIntegrationTest.testTopicIdUpgradeAfterReassigningPartitions

2022-01-19 Thread GitBox
dajac commented on pull request #11687: URL: https://github.com/apache/kafka/pull/11687#issuecomment-1016569873 @Kvicii Thanks for the PR. I am a bit confused because the PR refactors the code but it does not seem to fix any issues. Did I miss something? Have you been able to make the

[jira] [Commented] (KAFKA-13584) Fix `kafka.network.SocketServerTest.testUnmuteChannelWithBufferedReceives` flaky test

2022-01-19 Thread Kvicii.Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478742#comment-17478742 ] Kvicii.Yu commented on KAFKA-13584: --- [~dajac]  hello, I will try fix this test, maybe need you help.

[GitHub] [kafka] vpapavas commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vpapavas commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r787752628 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java ## @@ -148,8 +150,12 @@ default void init(final

[jira] [Commented] (KAFKA-13590) Rename InternalTopologyBuilder's #topicGroups method to be more descriptive

2022-01-19 Thread Kvicii.Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13590?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478702#comment-17478702 ] Kvicii.Yu commented on KAFKA-13590: --- [~ableegoldman] i think `topicGroupsInfo` may be more appropriate

[jira] (KAFKA-13590) Rename InternalTopologyBuilder's #topicGroups method to be more descriptive

2022-01-19 Thread Kvicii.Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13590 ] Kvicii.Yu deleted comment on KAFKA-13590: --- was (Author: JIRAUSER283467): rename is a problem.But what is more appropriate? > Rename InternalTopologyBuilder's #topicGroups method to be more

[GitHub] [kafka] jonathan-albrecht-ibm commented on pull request #11690: KAFKA-13599: Upgrade RocksDB to 6.27.3

2022-01-19 Thread GitBox
jonathan-albrecht-ibm commented on pull request #11690: URL: https://github.com/apache/kafka/pull/11690#issuecomment-1016498932 Thanks @cadonna! -- 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

[GitHub] [kafka] vpapavas commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vpapavas commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r787752628 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java ## @@ -148,8 +150,12 @@ default void init(final

[GitHub] [kafka] vpapavas commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vpapavas commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r787752628 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java ## @@ -148,8 +150,12 @@ default void init(final

[GitHub] [kafka] vpapavas commented on a change in pull request #11682: KAFKA-13524: Add IQv2 query handling to the caching layer

2022-01-19 Thread GitBox
vpapavas commented on a change in pull request #11682: URL: https://github.com/apache/kafka/pull/11682#discussion_r787725141 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java ## @@ -83,10 +90,67 @@ public void init(final

[jira] [Comment Edited] (KAFKA-7575) 'Error while writing to checkpoint file' Issue

2022-01-19 Thread Ivan Sager (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7575?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478646#comment-17478646 ] Ivan Sager edited comment on KAFKA-7575 at 1/19/22, 12:40 PM: -- Much thanks

[jira] [Commented] (KAFKA-7575) 'Error while writing to checkpoint file' Issue

2022-01-19 Thread Ivan Sager (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7575?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478646#comment-17478646 ] Ivan Sager commented on KAFKA-7575: --- Much thanks [~Jacol84] Jacek. I'll move to 2.8 and give it a spin

[GitHub] [kafka] venkatteki commented on a change in pull request #11615: KAFKA-13546: Do not fail connector if default topic creation group is explicitly specified

2022-01-19 Thread GitBox
venkatteki commented on a change in pull request #11615: URL: https://github.com/apache/kafka/pull/11615#discussion_r787676776 ## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceConnectorConfigTest.java ## @@ -64,6 +67,18 @@ return

[GitHub] [kafka] cadonna commented on a change in pull request #11686: KAFKA-12648: invoke exception handler for MissingSourceTopicException with named topologies

2022-01-19 Thread GitBox
cadonna commented on a change in pull request #11686: URL: https://github.com/apache/kafka/pull/11686#discussion_r787544943 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java ## @@ -515,11 +518,23 @@ private

[jira] [Updated] (KAFKA-13598) idempotence producer is not enabled by default if not set explicitly

2022-01-19 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen updated KAFKA-13598: -- Summary: idempotence producer is not enabled by default if not set explicitly (was: producer config

[GitHub] [kafka] patrickstuedi commented on a change in pull request #11676: feat: checkpoint position in state stores

2022-01-19 Thread GitBox
patrickstuedi commented on a change in pull request #11676: URL: https://github.com/apache/kafka/pull/11676#discussion_r787515967 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java ## @@ -94,7 +94,8 @@ * @throws

[jira] [Commented] (KAFKA-13598) producer config didn't throw exception when retries and acks config changed

2022-01-19 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17478455#comment-17478455 ] David Jacot commented on KAFKA-13598: - Sounds good. Thanks [~showuon]. > producer config didn't

[jira] [Updated] (KAFKA-13598) producer config didn't throw exception when retries and acks config changed

2022-01-19 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot updated KAFKA-13598: Fix Version/s: 3.1.1 3.2.0 > producer config didn't throw exception when

[GitHub] [kafka] dajac merged pull request #11456: KAFKA-13351: Add possibility to write kafka headers in Kafka Console Producer

2022-01-19 Thread GitBox
dajac merged pull request #11456: URL: https://github.com/apache/kafka/pull/11456 -- 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:

[GitHub] [kafka] ableegoldman commented on a change in pull request #11675: KAFKA-12648: POC for committing tasks on error

2022-01-19 Thread GitBox
ableegoldman commented on a change in pull request #11675: URL: https://github.com/apache/kafka/pull/11675#discussion_r787472259 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java ## @@ -84,6 +84,7 @@ private final

[GitHub] [kafka] cadonna commented on pull request #11690: KAFKA-13599: Upgrade RocksDB to 6.27.3

2022-01-19 Thread GitBox
cadonna commented on pull request #11690: URL: https://github.com/apache/kafka/pull/11690#issuecomment-1016214618 @jonathan-albrecht-ibm Thank you for the PR! I will review it this or next week. -- This is an automated message from the Apache Git Service. To respond to the message,