[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r497257420 ## File path: core/src/main/scala/kafka/server/FinalizedFeatureCache.scala ## @@ -20,26 +20,31 @@ package kafka.server import kafka.utils.Logging import o

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r497256475 ## File path: core/src/main/scala/kafka/server/BrokerFeatures.scala ## @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r497256162 ## File path: core/src/main/scala/kafka/server/BrokerFeatures.scala ## @@ -0,0 +1,187 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r497255894 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java ## @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software F

[jira] [Commented] (KAFKA-10513) Newly added topic or partitions are not assigned to running consumer groups using static membership

2020-09-29 Thread Marlon Ou (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10513?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17204432#comment-17204432 ] Marlon Ou commented on KAFKA-10513: --- Thank you both so much for the help! I tried sett

[GitHub] [kafka] chia7712 commented on a change in pull request #9347: KAFKA-10531: Check for negative values to Thread.sleep call

2020-09-29 Thread GitBox
chia7712 commented on a change in pull request #9347: URL: https://github.com/apache/kafka/pull/9347#discussion_r497229208 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -133,11 +133,14 @@ public void start() { List

[GitHub] [kafka] zhaohaidao commented on a change in pull request #9311: KAFKA-9910: Implement new transaction timed out error

2020-09-29 Thread GitBox
zhaohaidao commented on a change in pull request #9311: URL: https://github.com/apache/kafka/pull/9311#discussion_r497217290 ## File path: core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala ## @@ -381,24 +385,35 @@ class TransactionCoordinator(broke

[jira] [Assigned] (KAFKA-10471) TimeIndex handling may cause data loss in certain back to back failure

2020-09-29 Thread Raman Verma (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10471?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Raman Verma reassigned KAFKA-10471: --- Assignee: Raman Verma > TimeIndex handling may cause data loss in certain back to back fail

[GitHub] [kafka] guozhangwang commented on pull request #9354: KAFKA-10134 Follow-up: Set the re-join flag in heartbeat failure

2020-09-29 Thread GitBox
guozhangwang commented on pull request #9354: URL: https://github.com/apache/kafka/pull/9354#issuecomment-701049068 @ableegoldman @abbccdda to review This is an automated message from the Apache Git Service. To respond to the

[GitHub] [kafka] guozhangwang opened a new pull request #9354: KAFKA-10134 Follow-up: Set the re-join flag in heartbeat failure

2020-09-29 Thread GitBox
guozhangwang opened a new pull request #9354: URL: https://github.com/apache/kafka/pull/9354 ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including up

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9273: KAFKA-9331: changes for shutdownRequest

2020-09-29 Thread GitBox
wcarlson5 commented on a change in pull request #9273: URL: https://github.com/apache/kafka/pull/9273#discussion_r497080111 ## File path: streams/src/main/java/org/apache/kafka/streams/errors/StreamsUncaughtExceptionHandler.java ## @@ -0,0 +1,49 @@ +/* + * Licensed to the Apac

[jira] [Updated] (KAFKA-10545) Create topic IDs and propagate to brokers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10545: --- Summary: Create topic IDs and propagate to brokers (was: Create Topic IDs and Propagate to

[jira] [Updated] (KAFKA-10550) Update kafka-topics.sh to support topic IDs

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10550?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10550: --- Summary: Update kafka-topics.sh to support topic IDs (was: Update kafka-topics.sh to suppor

[GitHub] [kafka] hachikuji opened a new pull request #9353: KAFKA-10521; Skip partition watch registration when `AlterIsr` is expected

2020-09-29 Thread GitBox
hachikuji opened a new pull request #9353: URL: https://github.com/apache/kafka/pull/9353 Before `AlterIsr` with KIP-497, the controller would register watches in Zookeeper for each reassigning partition so that it could be notified immediately when the ISR was expanded and the reassignmen

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

2020-09-29 Thread GitBox
dongjinleekr commented on pull request #7898: URL: https://github.com/apache/kafka/pull/7898#issuecomment-701008405 Hi All, Here is the update. All compatibility breaks caused by the root logger name change between log4j and log4j2 (`"root"` → `""`) is now resolved. Plus, I also mig

[GitHub] [kafka] scanterog commented on a change in pull request #9313: [mm2] Fix consumer/producer properties override

2020-09-29 Thread GitBox
scanterog commented on a change in pull request #9313: URL: https://github.com/apache/kafka/pull/9313#discussion_r497079320 ## File path: connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java ## @@ -199,8 +199,8 @@ protected static fina

[jira] [Resolved] (KAFKA-9061) StreamStreamJoinIntegrationTest flaky test failures

2020-09-29 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-9061. -- Resolution: Cannot Reproduce > StreamStreamJoinIntegrationTest flaky test failures > --

[jira] [Commented] (KAFKA-9061) StreamStreamJoinIntegrationTest flaky test failures

2020-09-29 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17204301#comment-17204301 ] Guozhang Wang commented on KAFKA-9061: -- I have not seen this failure for a while, wi

[jira] [Commented] (KAFKA-10553) Track handling of topic deletion during reassignment

2020-09-29 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10553?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17204285#comment-17204285 ] Jason Gustafson commented on KAFKA-10553: - Thanks for filing the issue. I agree

[jira] [Updated] (KAFKA-10553) Track handling of topic deletion during reassignment

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10553: --- Summary: Track handling of topic deletion during reassignment (was: Track handling of topic

[jira] [Updated] (KAFKA-10553) Track handling of topic deletion during reassignment

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10553: --- Description: Currently deletion is blocked during partition reassignment, but KIP-516 and t

[GitHub] [kafka] C0urante commented on a change in pull request #8910: KAFKA-10188: Prevent SinkTask::preCommit from being called after SinkTask::stop

2020-09-29 Thread GitBox
C0urante commented on a change in pull request #8910: URL: https://github.com/apache/kafka/pull/8910#discussion_r497049737 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java ## @@ -138,6 +139,7 @@ public WorkerSinkTask(ConnectorTas

[jira] [Updated] (KAFKA-10553) Track handling of topic deletion with topic IDs during reassignment

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10553?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10553: --- Summary: Track handling of topic deletion with topic IDs during reassignment (was: Track ha

[jira] [Created] (KAFKA-10553) Track handling of topic deletion during reassignment

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10553: -- Summary: Track handling of topic deletion during reassignment Key: KAFKA-10553 URL: https://issues.apache.org/jira/browse/KAFKA-10553 Project: Kafka Issu

[GitHub] [kafka] soondenana commented on a change in pull request #9347: KAFKA-10531: Check for negative values to Thread.sleep call

2020-09-29 Thread GitBox
soondenana commented on a change in pull request #9347: URL: https://github.com/apache/kafka/pull/9347#discussion_r497034435 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -133,11 +133,14 @@ public void start() { Lis

[GitHub] [kafka] scanterog commented on a change in pull request #9313: [mm2] Fix consumer/producer properties override

2020-09-29 Thread GitBox
scanterog commented on a change in pull request #9313: URL: https://github.com/apache/kafka/pull/9313#discussion_r495871017 ## File path: connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java ## @@ -199,8 +199,8 @@ protected static fina

[jira] [Assigned] (KAFKA-10510) Reassigning partitions should not allow increasing RF of a partition unless configured with it

2020-09-29 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-10510?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Piotr Rżysko reassigned KAFKA-10510: Assignee: Piotr Rżysko > Reassigning partitions should not allow increasing RF of a parti

[GitHub] [kafka] chia7712 commented on a change in pull request #9347: KAFKA-10531: Check for negative values to Thread.sleep call

2020-09-29 Thread GitBox
chia7712 commented on a change in pull request #9347: URL: https://github.com/apache/kafka/pull/9347#discussion_r497017395 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -133,11 +133,14 @@ public void start() { List

[jira] [Created] (KAFKA-10552) Update directory structure to use topic IDs

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10552: -- Summary: Update directory structure to use topic IDs Key: KAFKA-10552 URL: https://issues.apache.org/jira/browse/KAFKA-10552 Project: Kafka Issue Type: S

[GitHub] [kafka] soondenana commented on a change in pull request #9347: KAFKA-10531: Check for negative values to Thread.sleep call

2020-09-29 Thread GitBox
soondenana commented on a change in pull request #9347: URL: https://github.com/apache/kafka/pull/9347#discussion_r496983721 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -133,11 +133,14 @@ public void start() { Lis

[jira] [Created] (KAFKA-10551) Support topic IDs in Produce request

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10551: -- Summary: Support topic IDs in Produce request Key: KAFKA-10551 URL: https://issues.apache.org/jira/browse/KAFKA-10551 Project: Kafka Issue Type: Sub-task

[jira] [Resolved] (KAFKA-10479) Throw exception if users try to update configs of existent listeners

2020-09-29 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-10479. - Fix Version/s: 2.7.0 Resolution: Fixed > Throw exception if users try to update c

[GitHub] [kafka] hachikuji merged pull request #9284: KAFKA-10479 Throw exception if users try to update configs of existen…

2020-09-29 Thread GitBox
hachikuji merged pull request #9284: URL: https://github.com/apache/kafka/pull/9284 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] nizhikov commented on pull request #9196: KAFKA-10402: Upgrade system tests to python3

2020-09-29 Thread GitBox
nizhikov commented on pull request #9196: URL: https://github.com/apache/kafka/pull/9196#issuecomment-700910936 @edenhill Thanks for the help! Appreciate it This is an automated message from the Apache Git Service. To respond

[GitHub] [kafka] hachikuji opened a new pull request #9352: KAFKA-10533; KafkaRaftClient should flush log after appends

2020-09-29 Thread GitBox
hachikuji opened a new pull request #9352: URL: https://github.com/apache/kafka/pull/9352 This patch adds missing flush logic to `KafkaRaftClient`. The initial flushing behavior is simplistic. We guarantee that the leader will not replicate above the last flushed offset and we guarantee th

[GitHub] [kafka] chia7712 commented on pull request #9284: KAFKA-10479 Throw exception if users try to update configs of existen…

2020-09-29 Thread GitBox
chia7712 commented on pull request #9284: URL: https://github.com/apache/kafka/pull/9284#issuecomment-700904592 @hachikuji Thanks for reviews and update! This is an automated message from the Apache Git Service. To respond to

[jira] [Created] (KAFKA-10550) Update kafka-topics.sh to support Topic IDs

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10550: -- Summary: Update kafka-topics.sh to support Topic IDs Key: KAFKA-10550 URL: https://issues.apache.org/jira/browse/KAFKA-10550 Project: Kafka Issue Type: S

[jira] [Created] (KAFKA-10549) Add topic ID support to DeleteTopics,ListOffsets, OffsetForLeaders, StopReplica

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10549: -- Summary: Add topic ID support to DeleteTopics,ListOffsets, OffsetForLeaders, StopReplica Key: KAFKA-10549 URL: https://issues.apache.org/jira/browse/KAFKA-10549 P

[jira] [Created] (KAFKA-10548) Implement Type field and logic for LeaderAndIsrRequests

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10548: -- Summary: Implement Type field and logic for LeaderAndIsrRequests Key: KAFKA-10548 URL: https://issues.apache.org/jira/browse/KAFKA-10548 Project: Kafka I

[jira] [Created] (KAFKA-10547) Add topic IDs to MetadataResponse, UpdateMetadata, and Fetch

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10547: -- Summary: Add topic IDs to MetadataResponse, UpdateMetadata, and Fetch Key: KAFKA-10547 URL: https://issues.apache.org/jira/browse/KAFKA-10547 Project: Kafka

[jira] [Updated] (KAFKA-8872) Improvements to controller "deleting" state / topic Identifiers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-8872: -- Description: Kafka currently uniquely identifies a topic by its name. This is generally suffic

[jira] [Updated] (KAFKA-10545) Create Topic IDs and Propagate to Brokers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-10545: --- Parent: KAFKA-8872 Issue Type: Sub-task (was: Improvement) > Create Topic IDs and P

[jira] [Updated] (KAFKA-8872) Improvements to controller "deleting" state / topic Identifiers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-8872: -- Description: Kafka currently uniquely identifies a topic by its name. This is generally suffic

[jira] [Updated] (KAFKA-8872) Improvements to controller "deleting" state / topic Identifiers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-8872: -- Description: Kafka currently uniquely identifies a topic by its name. This is generally suffic

[jira] [Updated] (KAFKA-8872) Improvements to controller "deleting" state / topic Identifiers

2020-09-29 Thread Justine Olshan (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Justine Olshan updated KAFKA-8872: -- Description: Kafka currently uniquely identifies a topic by its name. This is generally suffic

[jira] [Updated] (KAFKA-10437) KIP-478: Implement test-utils changes

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler updated KAFKA-10437: - Summary: KIP-478: Implement test-utils changes (was: Convert test-utils (and StateStore) for KI

[jira] [Assigned] (KAFKA-10543) Convert KTable joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler reassigned KAFKA-10543: Assignee: John Roesler > Convert KTable joins to new PAPI > -

[jira] [Created] (KAFKA-10545) Create Topic IDs and Propagate to Brokers

2020-09-29 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-10545: -- Summary: Create Topic IDs and Propagate to Brokers Key: KAFKA-10545 URL: https://issues.apache.org/jira/browse/KAFKA-10545 Project: Kafka Issue Type: Imp

[jira] [Created] (KAFKA-10546) KIP-478: Deprecate old PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10546: Summary: KIP-478: Deprecate old PAPI Key: KAFKA-10546 URL: https://issues.apache.org/jira/browse/KAFKA-10546 Project: Kafka Issue Type: Sub-task

[jira] [Assigned] (KAFKA-10540) Convert KStream aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler reassigned KAFKA-10540: Assignee: John Roesler > Convert KStream aggregations to new PAPI > -

[jira] [Assigned] (KAFKA-10544) Convert KTable aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler reassigned KAFKA-10544: Assignee: John Roesler > Convert KTable aggregations to new PAPI > --

[jira] [Assigned] (KAFKA-10542) Convert KTable maps to new PAPI

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10542?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler reassigned KAFKA-10542: Assignee: John Roesler > Convert KTable maps to new PAPI > --

[jira] [Created] (KAFKA-10544) Convert KTable aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10544: Summary: Convert KTable aggregations to new PAPI Key: KAFKA-10544 URL: https://issues.apache.org/jira/browse/KAFKA-10544 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10542) Convert KTable maps to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10542: Summary: Convert KTable maps to new PAPI Key: KAFKA-10542 URL: https://issues.apache.org/jira/browse/KAFKA-10542 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10543) Convert KTable joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10543: Summary: Convert KTable joins to new PAPI Key: KAFKA-10543 URL: https://issues.apache.org/jira/browse/KAFKA-10543 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10541) Convert KTable filters to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10541: Summary: Convert KTable filters to new PAPI Key: KAFKA-10541 URL: https://issues.apache.org/jira/browse/KAFKA-10541 Project: Kafka Issue Type: Sub-task

[GitHub] [kafka] ahuang98 removed a comment on pull request #9340: Improving Fetch Session Caching for KAFKA-9401

2020-09-29 Thread GitBox
ahuang98 removed a comment on pull request #9340: URL: https://github.com/apache/kafka/pull/9340#issuecomment-700886220 retest this please This is an automated message from the Apache Git Service. To respond to the message, p

[jira] [Created] (KAFKA-10540) Convert KStream aggregations to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10540: Summary: Convert KStream aggregations to new PAPI Key: KAFKA-10540 URL: https://issues.apache.org/jira/browse/KAFKA-10540 Project: Kafka Issue Type: Sub-task

[jira] [Assigned] (KAFKA-10539) Convert KStreamImpl joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler reassigned KAFKA-10539: Assignee: John Roesler > Convert KStreamImpl joins to new PAPI >

[GitHub] [kafka] ahuang98 commented on pull request #9340: Improving Fetch Session Caching for KAFKA-9401

2020-09-29 Thread GitBox
ahuang98 commented on pull request #9340: URL: https://github.com/apache/kafka/pull/9340#issuecomment-700886220 retest this please This is an automated message from the Apache Git Service. To respond to the message, please lo

[jira] [Created] (KAFKA-10539) Convert KStreamImpl joins to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10539: Summary: Convert KStreamImpl joins to new PAPI Key: KAFKA-10539 URL: https://issues.apache.org/jira/browse/KAFKA-10539 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10538) Convert KStreamImpl maps to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10538: Summary: Convert KStreamImpl maps to new PAPI Key: KAFKA-10538 URL: https://issues.apache.org/jira/browse/KAFKA-10538 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10537) Convert KStreamImpl filters to new PAPI

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10537: Summary: Convert KStreamImpl filters to new PAPI Key: KAFKA-10537 URL: https://issues.apache.org/jira/browse/KAFKA-10537 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10536) KIP-478: Implement KStream changes

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10536: Summary: KIP-478: Implement KStream changes Key: KAFKA-10536 URL: https://issues.apache.org/jira/browse/KAFKA-10536 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-10535) KIP-478: Implement StateStoreContext and Record

2020-09-29 Thread John Roesler (Jira)
John Roesler created KAFKA-10535: Summary: KIP-478: Implement StateStoreContext and Record Key: KAFKA-10535 URL: https://issues.apache.org/jira/browse/KAFKA-10535 Project: Kafka Issue Type: S

[GitHub] [kafka] leosilvadev opened a new pull request #9351: remove unecessary int-long widening

2020-09-29 Thread GitBox
leosilvadev opened a new pull request #9351: URL: https://github.com/apache/kafka/pull/9351 There is a JIRA ticket requesting this small change (https://issues.apache.org/jira/browse/KAFKA-10047) ### Committer Checklist (excluded from commit message) - [ ] Verify design and implem

[GitHub] [kafka] abbccdda commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
abbccdda commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496901588 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java ## @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] kobebryantlin0 opened a new pull request #9350: KAFKA-10534: change params type to avoid avoid redundant judgments.

2020-09-29 Thread GitBox
kobebryantlin0 opened a new pull request #9350: URL: https://github.com/apache/kafka/pull/9350 Modify the AbstractConfig class, convert params `originals` type from Map to Map to avoid redundant judgments in the code. ### Committer Checklist (excluded from commit message) - [ ] Ver

[GitHub] [kafka] chia7712 commented on a change in pull request #9347: KAFKA-10531: Check for negative values to Thread.sleep call

2020-09-29 Thread GitBox
chia7712 commented on a change in pull request #9347: URL: https://github.com/apache/kafka/pull/9347#discussion_r496904812 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -133,11 +133,14 @@ public void start() { List

[jira] [Updated] (KAFKA-10534) Modify the AbstractConfig class, convert parameter `originals` type from Map to Map to avoid redundant judgments in the code

2020-09-29 Thread linenwei (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] linenwei updated KAFKA-10534: - Summary: Modify the AbstractConfig class, convert parameter `originals` type from Map to Map to avoid

[jira] [Updated] (KAFKA-10534) Modify the originals parameter type of the AbstractConfig class to avoid redundant judgments in the code

2020-09-29 Thread linenwei (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10534?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] linenwei updated KAFKA-10534: - Flags: Patch Description: Source Code: {code:java} @SuppressWarnings("unchecked") publi

[jira] [Created] (KAFKA-10534) Modify the originals parameter type of the AbstractConfig class to avoid redundant judgments in the code

2020-09-29 Thread linenwei (Jira)
linenwei created KAFKA-10534: Summary: Modify the originals parameter type of the AbstractConfig class to avoid redundant judgments in the code Key: KAFKA-10534 URL: https://issues.apache.org/jira/browse/KAFKA-10534

[jira] [Updated] (KAFKA-10533) Add log flush semantics to simulation test

2020-09-29 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10533?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-10533: Description: In order to do KAFKA-10526, it is useful to add support for flush semantics t

[jira] [Created] (KAFKA-10533) Add log flush semantics to simulation test

2020-09-29 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-10533: --- Summary: Add log flush semantics to simulation test Key: KAFKA-10533 URL: https://issues.apache.org/jira/browse/KAFKA-10533 Project: Kafka Issue Type:

[GitHub] [kafka] edenhill commented on pull request #9196: KAFKA-10402: Upgrade system tests to python3

2020-09-29 Thread GitBox
edenhill commented on pull request #9196: URL: https://github.com/apache/kafka/pull/9196#issuecomment-700817177 We'll want to use a released version of ducktape to get stable tests. I'm working on getting a ducktape release out.

[GitHub] [kafka] kowshik edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik edited a comment on pull request #9001: URL: https://github.com/apache/kafka/pull/9001#issuecomment-700585584 @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2. I've provided a response to all of your com

[GitHub] [kafka] rhauch commented on pull request #8134: KAFKA-9546 Allow custom tasks through configuration

2020-09-29 Thread GitBox
rhauch commented on pull request #8134: URL: https://github.com/apache/kafka/pull/8134#issuecomment-700796780 Thanks for the PR, @gcsaba2. However, I'm going to close this per my comments on https://issues.apache.org/jira/browse/KAFKA-9546. TLDR; the `FileStreamSourceConnector` clas

[GitHub] [kafka] rhauch closed pull request #8134: KAFKA-9546 Allow custom tasks through configuration

2020-09-29 Thread GitBox
rhauch closed pull request #8134: URL: https://github.com/apache/kafka/pull/8134 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

[jira] [Resolved] (KAFKA-9546) Make FileStreamSourceTask extendable with generic streams

2020-09-29 Thread Randall Hauch (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9546?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-9546. -- Resolution: Won't Fix I'm going to close this as WONTFIX, per my previous comment. > Make File

[jira] [Resolved] (KAFKA-9514) The protocol generator generated useless condition when a field is made nullable and flexible version is used

2020-09-29 Thread David Jacot (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Jacot resolved KAFKA-9514. Fix Version/s: 2.7.0 Resolution: Fixed > The protocol generator generated useless condition

[GitHub] [kafka] vvcephei commented on a change in pull request #9020: KAFKA-10271 Performance regression while fetching a key from a single partition

2020-09-29 Thread GitBox
vvcephei commented on a change in pull request #9020: URL: https://github.com/apache/kafka/pull/9020#discussion_r496796429 ## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java ## @@ -46,11 +46,22 @@ public void setStoreQueryPa

[GitHub] [kafka] nizhikov commented on pull request #9196: KAFKA-10402: Upgrade system tests to python3

2020-09-29 Thread GitBox
nizhikov commented on pull request #9196: URL: https://github.com/apache/kafka/pull/9196#issuecomment-700766947 Hello, @guozhangwang , @mimaison The only thing we need to have perfect merge is ducktape release that contains python3 fixes. Can we go with the current PR, which poin

[jira] [Updated] (KAFKA-9584) Removing headers causes ConcurrentModificationException

2020-09-29 Thread John Roesler (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] John Roesler updated KAFKA-9584: Fix Version/s: 2.4.2 > Removing headers causes ConcurrentModificationException > --

[GitHub] [kafka] vvcephei commented on pull request #8181: KAFKA-9584 Headers ConcurrentModificationException

2020-09-29 Thread GitBox
vvcephei commented on pull request #8181: URL: https://github.com/apache/kafka/pull/8181#issuecomment-700766402 Cherry-picked to 2.4 as well. This is an automated message from the Apache Git Service. To respond to the message

[GitHub] [kafka] rajinisivaram merged pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599)

2020-09-29 Thread GitBox
rajinisivaram merged pull request #9344: URL: https://github.com/apache/kafka/pull/9344 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

[GitHub] [kafka] rajinisivaram commented on pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599)

2020-09-29 Thread GitBox
rajinisivaram commented on pull request #9344: URL: https://github.com/apache/kafka/pull/9344#issuecomment-700735225 Builds are good, merging to trunk. This is an automated message from the Apache Git Service. To respond to t

[GitHub] [kafka] nizhikov commented on a change in pull request #9196: KAFKA-10402: Upgrade system tests to python3

2020-09-29 Thread GitBox
nizhikov commented on a change in pull request #9196: URL: https://github.com/apache/kafka/pull/9196#discussion_r496670159 ## File path: tests/docker/Dockerfile ## @@ -32,9 +32,11 @@ ARG ducker_creator=default LABEL ducker.creator=$ducker_creator # Update Linux and install

[GitHub] [kafka] rajinisivaram commented on pull request #9345: KAFKA-10338; Support PEM format for SSL key and trust stores (KIP-651)

2020-09-29 Thread GitBox
rajinisivaram commented on pull request #9345: URL: https://github.com/apache/kafka/pull/9345#issuecomment-700664402 retest this please This is an automated message from the Apache Git Service. To respond to the message, plea

[GitHub] [kafka] dajac commented on pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599)

2020-09-29 Thread GitBox
dajac commented on pull request #9344: URL: https://github.com/apache/kafka/pull/9344#issuecomment-700644563 @rajinisivaram Thanks for your suggestion. I have updated the PR. This is an automated message from the Apache Git S

[GitHub] [kafka] dajac commented on a change in pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599)

2020-09-29 Thread GitBox
dajac commented on a change in pull request #9344: URL: https://github.com/apache/kafka/pull/9344#discussion_r496645598 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java ## @@ -1554,6 +1575,13 @@ private ConfigEntry configEntry(CreatableT

[GitHub] [kafka] kowshik edited a comment on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik edited a comment on pull request #9001: URL: https://github.com/apache/kafka/pull/9001#issuecomment-700585584 @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2. I've provided a response to all of your com

[GitHub] [kafka] rajinisivaram commented on a change in pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599

2020-09-29 Thread GitBox
rajinisivaram commented on a change in pull request #9344: URL: https://github.com/apache/kafka/pull/9344#discussion_r496579003 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java ## @@ -1554,6 +1575,13 @@ private ConfigEntry configEntry(Cr

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496511604 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/FinalizedVersionRange.java ## @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software F

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496509282 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/DescribeFeaturesOptions.java ## @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496509097 ## File path: clients/src/main/resources/common/message/UpdateFeaturesRequest.json ## @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF)

[GitHub] [kafka] kowshik commented on pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on pull request #9001: URL: https://github.com/apache/kafka/pull/9001#issuecomment-700585584 @junrao Thanks a lot for the review! I've addressed the comments in the recent commit: 06d8b47131f168db88e4f7d5bda3dd025ba9a2a2. -

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496550557 ## File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java ## @@ -40,14 +40,16 @@ public static FinalizedVersionRange f

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496550557 ## File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java ## @@ -40,14 +40,16 @@ public static FinalizedVersionRange f

[GitHub] [kafka] kowshik commented on a change in pull request #9001: KAFKA-10028: Implement write path for feature versioning system (KIP-584)

2020-09-29 Thread GitBox
kowshik commented on a change in pull request #9001: URL: https://github.com/apache/kafka/pull/9001#discussion_r496550557 ## File path: clients/src/main/java/org/apache/kafka/common/feature/FinalizedVersionRange.java ## @@ -40,14 +40,16 @@ public static FinalizedVersionRange f

[GitHub] [kafka] dajac commented on pull request #9344: MINOR; Preserve ThrottlingQuotaExceededException when request timeouts after being retried due to a quota violation (KIP-599)

2020-09-29 Thread GitBox
dajac commented on pull request #9344: URL: https://github.com/apache/kafka/pull/9344#issuecomment-700557270 @rajinisivaram Thanks for your comments. I have updated the PR. This is an automated message from the Apache Git Ser

  1   2   >