[jira] [Commented] (KAFKA-12261) Splitting partition causes message loss for consumers with auto.offset.reset=latest

2021-02-18 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12261?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286901#comment-17286901 ] Haruki Okada commented on KAFKA-12261: -- Yeah, actually I've reconsidered about that and changed the

[jira] [Commented] (KAFKA-12261) Splitting partition causes message loss for consumers with auto.offset.reset=latest

2021-02-18 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12261?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286892#comment-17286892 ] Luke Chen commented on KAFKA-12261: --- I think we should document it, but not change the default setting

[jira] [Created] (KAFKA-12346) punctuate is called at twice the duration passed as the first argument to Processor.Schedule (with PunctuationType.WALL_CLOCK_TIME)

2021-02-18 Thread Arindam Ray (Jira)
Arindam Ray created KAFKA-12346: --- Summary: punctuate is called at twice the duration passed as the first argument to Processor.Schedule (with PunctuationType.WALL_CLOCK_TIME) Key: KAFKA-12346 URL:

[jira] [Updated] (KAFKA-12261) Splitting partition causes message loss for consumers with auto.offset.reset=latest

2021-02-18 Thread Haruki Okada (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12261?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Haruki Okada updated KAFKA-12261: - Description: As of now, auto.offset.reset of ConsumerConfig is "latest" by default.   This

[GitHub] [kafka] chia7712 commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
chia7712 commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578965629 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -366,6 +346,42 @@ private void readToLogEnd() {

[jira] [Assigned] (KAFKA-12261) Splitting partition causes message loss for consumers with auto.offset.reset=latest

2021-02-18 Thread Luke Chen (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12261?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luke Chen reassigned KAFKA-12261: - Assignee: Luke Chen > Splitting partition causes message loss for consumers with >

[jira] [Updated] (KAFKA-12273) InterBrokerSendThread#pollOnce throws FatalExitError even though it is shutdown correctly

2021-02-18 Thread Chia-Ping Tsai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai updated KAFKA-12273: --- Fix Version/s: 3.0.0 > InterBrokerSendThread#pollOnce throws FatalExitError even though it

[jira] [Assigned] (KAFKA-12336) custom stream naming does not work while calling stream[K, V](topicPattern: Pattern) API with named Consumed parameter

2021-02-18 Thread GeordieMai (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] GeordieMai reassigned KAFKA-12336: -- Assignee: GeordieMai > custom stream naming does not work while calling stream[K,

[GitHub] [kafka] tinawenqiao commented on pull request #9235: KAFKA-10449: Add some important parameter desc in connect-distributed.properties

2021-02-18 Thread GitBox
tinawenqiao commented on pull request #9235: URL: https://github.com/apache/kafka/pull/9235#issuecomment-781848357 > Thanks for the PR @tinawenqiao. > > It would be nice to also mention `rest.port` and `rest.host.name` are deprecated in their descriptions. Could you add something

[GitHub] [kafka] mjsax commented on pull request #10072: KAFKA-9274: Throw TaskCorruptedException instead of TimeoutException when TX commit times out

2021-02-18 Thread GitBox
mjsax commented on pull request #10072: URL: https://github.com/apache/kafka/pull/10072#issuecomment-781840391 Updated. This is an automated message from the Apache Git Service. To respond to the message, please log on to

[GitHub] [kafka] mjsax commented on a change in pull request #10072: KAFKA-9274: Throw TaskCorruptedException instead of TimeoutException when TX commit times out

2021-02-18 Thread GitBox
mjsax commented on a change in pull request #10072: URL: https://github.com/apache/kafka/pull/10072#discussion_r578937359 ## File path: streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java ## @@ -32,6 +33,7 @@ public final int topicGroupId; /** The ID

[GitHub] [kafka] hachikuji commented on pull request #10157: MINOR: Raft request thread should discover api versions

2021-02-18 Thread GitBox
hachikuji commented on pull request #10157: URL: https://github.com/apache/kafka/pull/10157#issuecomment-781825379 > Okay. I guess we need another KIP that explains how to upgrade a cluster and to determine when it is safe to enable the RaftClient on all of the brokers of a cluster.

[GitHub] [kafka] rhauch commented on pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on pull request #10158: URL: https://github.com/apache/kafka/pull/10158#issuecomment-781813200 @chia7712, for what it's worth, I still think your #10152 PR is still an important fix. This is an automated

[GitHub] [kafka] rhauch commented on pull request #10152: KAFKA-12339 Starting new connector cluster with new internal topics e…

2021-02-18 Thread GitBox
rhauch commented on pull request #10152: URL: https://github.com/apache/kafka/pull/10152#issuecomment-781812825 Thanks, @chia7712. I've reviewed much of this, and it seems straightforward, though I plan to review more thoroughly tomorrow (~12 hours). I've asked a few others that might be

[GitHub] [kafka] rhauch commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578916881 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -390,7 +410,11 @@ public void run() {

[GitHub] [kafka] rhauch commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578919850 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -366,6 +348,44 @@ private void readToLogEnd() {

[GitHub] [kafka] rhauch commented on pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on pull request #10158: URL: https://github.com/apache/kafka/pull/10158#issuecomment-781810306 Thanks for the quick review, @chia7712. I've incorporated several of your suggestions, and added comments/responses on the others. BTW, a run of the relevant Connect

[GitHub] [kafka] rhauch commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578917177 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -366,6 +348,44 @@ private void readToLogEnd() {

[GitHub] [kafka] rhauch commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578916881 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -390,7 +410,11 @@ public void run() {

[GitHub] [kafka] chia7712 merged pull request #10154: MINOR: Added missing import to kafka.py

2021-02-18 Thread GitBox
chia7712 merged pull request #10154: URL: https://github.com/apache/kafka/pull/10154 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] chia7712 commented on pull request #10154: MINOR: Added missing import to kafka.py

2021-02-18 Thread GitBox
chia7712 commented on pull request #10154: URL: https://github.com/apache/kafka/pull/10154#issuecomment-781801914 run `downgrade_test` with this patch on my local. pass will merge this patch to trunk and 2.8 This is

[GitHub] [kafka] hachikuji commented on pull request #10157: MINOR: Raft request thread should discover api versions

2021-02-18 Thread GitBox
hachikuji commented on pull request #10157: URL: https://github.com/apache/kafka/pull/10157#issuecomment-781799540 @jsancio I may be misunderstanding your question, but api versions are negotiated internally in `NetworkClient`. If there are no compatible versions, then the client will

[GitHub] [kafka] chia7712 commented on a change in pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
chia7712 commented on a change in pull request #10158: URL: https://github.com/apache/kafka/pull/10158#discussion_r578907382 ## File path: connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java ## @@ -366,6 +348,44 @@ private void readToLogEnd() {

[GitHub] [kafka] dengziming commented on pull request #10147: MINOR: Add raft resigned state metric name

2021-02-18 Thread GitBox
dengziming commented on pull request #10147: URL: https://github.com/apache/kafka/pull/10147#issuecomment-781799200 @guozhangwang @hachikuji , Hello, PTAL. This is an automated message from the Apache Git Service. To respond

[jira] [Resolved] (KAFKA-12258) Change the BatchAccumulator to split records into batches

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12258. - Resolution: Fixed > Change the BatchAccumulator to split records into batches >

[GitHub] [kafka] hachikuji merged pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
hachikuji merged pull request #10063: URL: https://github.com/apache/kafka/pull/10063 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] jsancio commented on pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
jsancio commented on pull request #10063: URL: https://github.com/apache/kafka/pull/10063#issuecomment-781790693 The following tests failed: ``` core: ConfigCommandTest. shouldFailIfUnresolvableHost() ConfigCommandTest. shouldFailIfUnresolvableHost()

[GitHub] [kafka] jsancio commented on pull request #10157: MINOR: Raft request thread should discover api versions

2021-02-18 Thread GitBox
jsancio commented on pull request #10157: URL: https://github.com/apache/kafka/pull/10157#issuecomment-781784110 > We do not plan to rely on the IBP in order to determine API versions for raft requests. Instead, we want to discover them through the ApiVersions API. This patch enables the

[GitHub] [kafka] dengziming opened a new pull request #10159: KAFKA-12338: Consolidate MetadataRecordSerde and MetadataParser serial/deserial code

2021-02-18 Thread GitBox
dengziming opened a new pull request #10159: URL: https://github.com/apache/kafka/pull/10159 *More detailed description of your change* The logics are duplicated except that `MetadataRecordSerde` has an extra `DEFAULT_FRAME_VERSION`, if we want to change the serial/deserial format of

[GitHub] [kafka] jsancio commented on pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
jsancio commented on pull request #10063: URL: https://github.com/apache/kafka/pull/10063#issuecomment-781778478 Running the following commands ``` ./gradlew -version Gradle 6.8.1

[GitHub] [kafka] mjsax commented on pull request #10091: KAFKA-9524: increase retention time for window and grace periods longer than one day

2021-02-18 Thread GitBox
mjsax commented on pull request #10091: URL: https://github.com/apache/kafka/pull/10091#issuecomment-781777265 Thanks for the PR @MarcoLotz! And congrats to your first code contribution! Thanks for reviewing @vcrfxia! Merged to `trunk` and cherry-picked to `2.8` branch.

[jira] [Resolved] (KAFKA-9524) Default window retention does not consider grace period

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9524. Fix Version/s: 2.8.0 Resolution: Fixed > Default window retention does not consider

[jira] [Updated] (KAFKA-9524) Default window retention does not consider grace period

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-9524: --- Issue Type: Improvement (was: Bug) > Default window retention does not consider grace

[GitHub] [kafka] rhauch opened a new pull request #10158: KAFKA-12343: Handle exceptions better in TopicAdmin, including UnsupportedVersionException

2021-02-18 Thread GitBox
rhauch opened a new pull request #10158: URL: https://github.com/apache/kafka/pull/10158 Refactored the KafkaBasedLog logic to read end offsets into a separate method to make it easier to test. Also changed the TopicAdmin.endOffsets method to throw the original

[GitHub] [kafka] jsancio commented on a change in pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
jsancio commented on a change in pull request #10063: URL: https://github.com/apache/kafka/pull/10063#discussion_r578886362 ## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java ## @@ -307,4 +309,38 @@ public int writeRecord(

[jira] [Updated] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alok Nikhil updated KAFKA-12345: Affects Version/s: 2.8.0 > KIP-500: AlterIsrManager crashes on broker idle-state >

[jira] [Commented] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286821#comment-17286821 ] Alok Nikhil commented on KAFKA-12345: - Hi [~dengziming]. I have updated the Kafka version and the

[jira] [Updated] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alok Nikhil updated KAFKA-12345: Priority: Minor (was: Major) > KIP-500: AlterIsrManager crashes on broker idle-state >

[GitHub] [kafka] showuon edited a comment on pull request #10118: KAFKA-10192: increase starting up waiting time

2021-02-18 Thread GitBox
showuon edited a comment on pull request #10118: URL: https://github.com/apache/kafka/pull/10118#issuecomment-781768574 > Fine by me, flaky tests suck. Indeed! Thanks, @C0urante @kkonstantine , could you check this PR? Thanks.

[GitHub] [kafka] hachikuji commented on a change in pull request #10129: KAFKA-10817; Add clusterId validation to Fetch handling

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10129: URL: https://github.com/apache/kafka/pull/10129#discussion_r578873790 ## File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java ## @@ -354,7 +355,8 @@ "Requested position is not greater

[GitHub] [kafka] showuon commented on pull request #10118: KAFKA-10192: increase starting up waiting time

2021-02-18 Thread GitBox
showuon commented on pull request #10118: URL: https://github.com/apache/kafka/pull/10118#issuecomment-781768574 > Fine by me, flaky tests suck. Indeed! Thanks, @C0urante @kkonstantine , could you check this PR? Thanks.

[GitHub] [kafka] hachikuji opened a new pull request #10157: MINOR: Raft request thread should discover api versions

2021-02-18 Thread GitBox
hachikuji opened a new pull request #10157: URL: https://github.com/apache/kafka/pull/10157 We do not plan to rely on the IBP in order to determine API versions for raft requests. Instead, we want to discover them through the ApiVersions API. This patch enables the flag to do so.

[GitHub] [kafka] junrao commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
junrao commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578876029 ## File path: metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java ## @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] mjsax merged pull request #10091: KAFKA-9524: increase retention time for window and grace periods longer than one day

2021-02-18 Thread GitBox
mjsax merged pull request #10091: URL: https://github.com/apache/kafka/pull/10091 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] hachikuji commented on a change in pull request #10129: KAFKA-10817; Add clusterId validation to Fetch handling

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10129: URL: https://github.com/apache/kafka/pull/10129#discussion_r578873790 ## File path: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java ## @@ -354,7 +355,8 @@ "Requested position is not greater

[GitHub] [kafka] hachikuji commented on a change in pull request #10129: KAFKA-10817; Add clusterId validation to Fetch handling

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10129: URL: https://github.com/apache/kafka/pull/10129#discussion_r578876834 ## File path: raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java ## @@ -1751,6 +1768,7 @@ private FetchRequestData buildFetchRequest() {

[GitHub] [kafka] dengziming commented on pull request #10021: KAFKA-12205: Delete snapshots less than the snapshot at the log start

2021-02-18 Thread GitBox
dengziming commented on pull request #10021: URL: https://github.com/apache/kafka/pull/10021#issuecomment-781762837 ping @hachikuji . retest this, please. This is an automated message from the Apache Git Service. To

[jira] [Comment Edited] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread dengziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286815#comment-17286815 ] dengziming edited comment on KAFKA-12345 at 2/19/21, 2:02 AM: -- hello,

[jira] [Commented] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread dengziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286815#comment-17286815 ] dengziming commented on KAFKA-12345: hello, please provide your specified Kafka version, or the

[GitHub] [kafka] hachikuji commented on a change in pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10063: URL: https://github.com/apache/kafka/pull/10063#discussion_r578870759 ## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java ## @@ -298,28 +342,30 @@ public void close() { public

[GitHub] [kafka] hachikuji commented on a change in pull request #10063: KAFKA-12258: Add support for splitting appending records

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10063: URL: https://github.com/apache/kafka/pull/10063#discussion_r578870321 ## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java ## @@ -307,4 +309,38 @@ public int writeRecord(

[jira] [Commented] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286810#comment-17286810 ] Alok Nikhil commented on KAFKA-12345: - Adding a bit more context. Seems like there was a controller

[jira] [Updated] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alok Nikhil updated KAFKA-12345: Description: Occasionally, a scheduler thread on a broker crashes with this stack   {code:java}

[jira] [Commented] (KAFKA-10847) Avoid spurious left/outer join results in stream-stream join

2021-02-18 Thread Guozhang Wang (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286808#comment-17286808 ] Guozhang Wang commented on KAFKA-10847: --- [~spena][~vcrfxia][~mjsax] I'd like to dump some more

[jira] [Updated] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alok Nikhil updated KAFKA-12345: Description: Occasionally, a scheduler thread on a broker crashes with this stack ```

[jira] [Created] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
Alok Nikhil created KAFKA-12345: --- Summary: KIP-500: AlterIsrManager crashes on broker idle-state Key: KAFKA-12345 URL: https://issues.apache.org/jira/browse/KAFKA-12345 Project: Kafka Issue

[GitHub] [kafka] cmccabe commented on pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on pull request #10070: URL: https://github.com/apache/kafka/pull/10070#issuecomment-781745956 > I think we need to handle preferred leader election in a special way. For example, if the assigned replicas are 1,2,3, isr is 2,3 and the current leader is 3, when doing

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578857828 ## File path: metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java ## @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software

[jira] [Resolved] (KAFKA-12232) Distinguish API scope by broker/controller

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12232?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12232. - Resolution: Duplicate > Distinguish API scope by broker/controller >

[jira] [Commented] (KAFKA-12278) Keep api versions consistent with api scope

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286803#comment-17286803 ] Jason Gustafson commented on KAFKA-12278: - [~tombentley] Haha, I was about to write something

[jira] [Resolved] (KAFKA-12278) Keep api versions consistent with api scope

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12278. - Resolution: Fixed > Keep api versions consistent with api scope >

[GitHub] [kafka] junrao commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
junrao commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578851481 ## File path: metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java ## @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578850804 ## File path: metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java ## @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578850598 ## File path: metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java ## @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] jsancio commented on pull request #10138: KAFKA-12331: Use LEO for the base offset of LeaderChangeMessage batch

2021-02-18 Thread GitBox
jsancio commented on pull request #10138: URL: https://github.com/apache/kafka/pull/10138#issuecomment-781734293 We got the following failures: ``` core: ConfigCommandTest. shouldFailIfUnresolvableHost() ConfigCommandTest. shouldFailIfUnresolvableHostUsingZookeeper()

[jira] [Resolved] (KAFKA-12331) KafkaRaftClient should use the LEO when appending LeaderChangeMessage

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12331?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12331. - Resolution: Fixed > KafkaRaftClient should use the LEO when appending

[GitHub] [kafka] hachikuji merged pull request #10138: KAFKA-12331: Use LEO for the base offset of LeaderChangeMessage batch

2021-02-18 Thread GitBox
hachikuji merged pull request #10138: URL: https://github.com/apache/kafka/pull/10138 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] jolshan edited a comment on pull request #10143: KAFKA-12332: Error partitions from topics with invalid IDs in LISR requests

2021-02-18 Thread GitBox
jolshan edited a comment on pull request #10143: URL: https://github.com/apache/kafka/pull/10143#issuecomment-781732199 Ran tests locally with newest code from trunk. Here are the tests that failed: ``` ConsumerBounceTest.testClose()

[GitHub] [kafka] jolshan edited a comment on pull request #10143: KAFKA-12332: Error partitions from topics with invalid IDs in LISR requests

2021-02-18 Thread GitBox
jolshan edited a comment on pull request #10143: URL: https://github.com/apache/kafka/pull/10143#issuecomment-781732199 Ran tests locally with newest code from trunk. Here are the tests that failed: ``` ConsumerBounceTest.testClose()

[GitHub] [kafka] jolshan commented on pull request #10143: KAFKA-12332: Error partitions from topics with invalid IDs in LISR requests

2021-02-18 Thread GitBox
jolshan commented on pull request #10143: URL: https://github.com/apache/kafka/pull/10143#issuecomment-781732199 Ran tests locally with newest code from trunk. Here are the tests that failed: ConsumerBounceTest. testClose() ConnectionQuotasTest.

[GitHub] [kafka] jsancio commented on pull request #10138: KAFKA-12331: Use LEO for the base offset of LeaderChangeMessage batch

2021-02-18 Thread GitBox
jsancio commented on pull request #10138: URL: https://github.com/apache/kafka/pull/10138#issuecomment-781730638 Ran the following commands locally: ``` $ ./gradlew -version Gradle 6.8.1

[GitHub] [kafka] cmccabe closed pull request #10066: KAFKA-12278; Ensure exposed api versions are consistent within listener scopes

2021-02-18 Thread GitBox
cmccabe closed pull request #10066: URL: https://github.com/apache/kafka/pull/10066 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] cmccabe commented on pull request #10066: KAFKA-12278; Ensure exposed api versions are consistent within listener scopes

2021-02-18 Thread GitBox
cmccabe commented on pull request #10066: URL: https://github.com/apache/kafka/pull/10066#issuecomment-781727318 pushed This is an automated message from the Apache Git Service. To respond to the message, please log on to

[GitHub] [kafka] mjsax commented on pull request #10134: TRIVIAL: fix JavaDocs formatting

2021-02-18 Thread GitBox
mjsax commented on pull request #10134: URL: https://github.com/apache/kafka/pull/10134#issuecomment-781715946 Merged to `trunk` and cherry-picked to `2.8` branch. This is an automated message from the Apache Git Service. To

[GitHub] [kafka] mjsax merged pull request #10134: TRIVIAL: fix JavaDocs formatting

2021-02-18 Thread GitBox
mjsax merged pull request #10134: URL: https://github.com/apache/kafka/pull/10134 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

[jira] [Commented] (KAFKA-10192) Flaky test BlockingConnectorTest#testBlockInConnectorStop

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10192?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286776#comment-17286776 ] Matthias J. Sax commented on KAFKA-10192: - Failed in setup: 

[jira] [Commented] (KAFKA-10579) Flaky test connect.integration.InternalTopicsIntegrationTest.testStartWhenInternalTopicsCreatedManuallyWithCompactForBrokersDefaultCleanupPolicy

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10579?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286774#comment-17286774 ] Matthias J. Sax commented on KAFKA-10579: - Different test method for same test:

[jira] [Updated] (KAFKA-12336) custom stream naming does not work while calling stream[K, V](topicPattern: Pattern) API with named Consumed parameter

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12336: Labels: easy-fix newbie (was: ) > custom stream naming does not work while calling

[jira] [Updated] (KAFKA-12336) custom stream naming does not work while calling stream[K, V](topicPattern: Pattern) API with named Consumed parameter

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12336: Priority: Minor (was: Major) > custom stream naming does not work while calling

[jira] [Updated] (KAFKA-12328) Expose TaskId partition number

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12328?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12328: Summary: Expose TaskId partition number (was: Find out partition of a store iterator) >

[jira] [Updated] (KAFKA-12328) Find out partition of a store iterator

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12328?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12328: Labels: needs-kip (was: ) > Find out partition of a store iterator >

[jira] [Commented] (KAFKA-12328) Find out partition of a store iterator

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12328?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17286771#comment-17286771 ] Matthias J. Sax commented on KAFKA-12328: - Personally, I am open to expose the partition number

[GitHub] [kafka] cmccabe commented on a change in pull request #10066: KAFKA-12278; Ensure exposed api versions are consistent within listener scopes

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10066: URL: https://github.com/apache/kafka/pull/10066#discussion_r578826004 ## File path: checkstyle/import-control.xml ## @@ -99,6 +100,7 @@ + Review comment: Hmm. Let's revisit this after

[GitHub] [kafka] cmccabe commented on a change in pull request #10066: KAFKA-12278; Ensure exposed api versions are consistent within listener scopes

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10066: URL: https://github.com/apache/kafka/pull/10066#discussion_r578825895 ## File path: core/src/main/scala/kafka/server/ApiVersionManager.scala ## @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF)

[GitHub] [kafka] hachikuji commented on a change in pull request #10094: KAFKA-12334: Add the KIP-631 metadata shell

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10094: URL: https://github.com/apache/kafka/pull/10094#discussion_r578823423 ## File path: bin/kafka-metadata-shell.sh ## @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +#

[GitHub] [kafka] hachikuji commented on a change in pull request #10094: KAFKA-12334: Add the KIP-631 metadata shell

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10094: URL: https://github.com/apache/kafka/pull/10094#discussion_r578818603 ## File path: bin/kafka-metadata-shell.sh ## @@ -0,0 +1,17 @@ +#!/bin/bash +# Licensed to the Apache Software Foundation (ASF) under one or more +#

[jira] [Updated] (KAFKA-12344) Support SlidingWindows in the Scala API

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12344: Affects Version/s: 2.7.0 > Support SlidingWindows in the Scala API >

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578817527 ## File path: metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java ## @@ -0,0 +1,367 @@ +/* + * Licensed to the Apache

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox
cmccabe commented on a change in pull request #10070: URL: https://github.com/apache/kafka/pull/10070#discussion_r578817611 ## File path: metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java ## @@ -0,0 +1,597 @@ +/* + * Licensed to the Apache

[jira] [Updated] (KAFKA-12344) Support SlidingWindows in the Scala API

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12344: Labels: (was: streams) > Support SlidingWindows in the Scala API >

[jira] [Updated] (KAFKA-12344) Support SlidingWindows in the Scala API

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12344?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-12344: Component/s: streams > Support SlidingWindows in the Scala API >

[GitHub] [kafka] hachikuji commented on a change in pull request #10094: KAFKA-12334: Add the KIP-631 metadata shell

2021-02-18 Thread GitBox
hachikuji commented on a change in pull request #10094: URL: https://github.com/apache/kafka/pull/10094#discussion_r578813351 ## File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java ## @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software

[GitHub] [kafka] rondagostino commented on pull request #10155: Fix Raft broker restart issue when offset partitions are deferred

2021-02-18 Thread GitBox
rondagostino commented on pull request #10155: URL: https://github.com/apache/kafka/pull/10155#issuecomment-781694442 Successfully ran `./gradlew build` after commenting out these two flaky tests, **both of which passed locally when run individually afterwards**:

[jira] [Created] (KAFKA-12344) Support SlidingWindows in the Scala API

2021-02-18 Thread Leah Thomas (Jira)
Leah Thomas created KAFKA-12344: --- Summary: Support SlidingWindows in the Scala API Key: KAFKA-12344 URL: https://issues.apache.org/jira/browse/KAFKA-12344 Project: Kafka Issue Type:

[GitHub] [kafka] abbccdda opened a new pull request #10156: KAFKA-10345 (WIP): File watch store reloading

2021-02-18 Thread GitBox
abbccdda opened a new pull request #10156: URL: https://github.com/apache/kafka/pull/10156 Add file-based store reloading mechanism, which does both file watch triggering and time based reloading in a separate thread. ### Committer Checklist (excluded from commit message) - [ ]

[GitHub] [kafka] ijuma merged pull request #10146: HOTFIX: fix Scala 2.12 build error caused by ControllerApisTest.scala

2021-02-18 Thread GitBox
ijuma merged pull request #10146: URL: https://github.com/apache/kafka/pull/10146 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] rondagostino commented on pull request #10146: HOTFIX: fix Scala 2.12 build error caused by ControllerApisTest.scala

2021-02-18 Thread GitBox
rondagostino commented on pull request #10146: URL: https://github.com/apache/kafka/pull/10146#issuecomment-781651604 Successfully confirmed this fix locally as per comment in #10155. This is an automated message from the

[GitHub] [kafka] rondagostino commented on pull request #10155: Fix Raft broker restart issue when offset partitions are deferred

2021-02-18 Thread GitBox
rondagostino commented on pull request #10155: URL: https://github.com/apache/kafka/pull/10155#issuecomment-781650728 I successfully compiled locally. Compiling with `-PscalaVersion=2.12` failed locally but then succeeded after applying the one-liner fix from

[GitHub] [kafka] ijuma commented on pull request #10113: MINOR: Add KIP-500 BrokerServer and ControllerServer

2021-02-18 Thread GitBox
ijuma commented on pull request #10113: URL: https://github.com/apache/kafka/pull/10113#issuecomment-781640866 @cmccabe Looks like the commit message is not as helpful as the PR description. We typically copy the PR description to the commit message.

[GitHub] [kafka] ijuma commented on pull request #10127: MINOR: Remove unused LeaderAndIsrResponse.partitions() since it has been replaced with partitionErrors()

2021-02-18 Thread GitBox
ijuma commented on pull request #10127: URL: https://github.com/apache/kafka/pull/10127#issuecomment-781639872 The implementation that was removed was more efficient than the new implementation. Is there a reason for that?

  1   2   >