Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

2018-10-29 Thread Boyang Chen
Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look: https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances KIP-345: Introduce static membership protocol to reduce

Re: [DISCUSS] KIP-370: Remove Orphan Partitions

2018-10-29 Thread xiongqi wu
Thanks Dong. I have updated the KIP. Instead of using a configure to specify the timeout, I switch it to use internal timer. User doesn't need a new configuration to use this feature. Xiongqi (Wesley) Wu On Mon, Oct 29, 2018 at 4:40 PM xiongqi wu wrote: > Dong, > > Thanks for the comments.

Re: [VOTE] KIP-380: Detect outdated control requests and bounced brokers using broker generation

2018-10-29 Thread Jun Rao
Hi, Patrick, Thanks for the updated KIP. +1 Jun On Wed, Oct 24, 2018 at 4:52 PM, Patrick Huang wrote: > Hi Jun, > > Sure. I already updated the KIP. Thanks! > > Best, > Zhanxiang (Patrick) Huang > > -- > *From:* Jun Rao > *Sent:* Wednesday, October 24, 2018 14:17

Re: [DISCUSS] KIP-370: Remove Orphan Partitions

2018-10-29 Thread xiongqi wu
Dong, Thanks for the comments. 1) With KIP-380, in theory we don't need the timeout phase. However, once orphan partitions are removed, they cannot be recovered. The question is should we rely on the fact that the first leaderandISR always contains correct information. For retention enabled

[jira] [Created] (KAFKA-7568) Return leader epoch in ListOffsets responses

2018-10-29 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7568: -- Summary: Return leader epoch in ListOffsets responses Key: KAFKA-7568 URL: https://issues.apache.org/jira/browse/KAFKA-7568 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-7567) Clean up internal metadata usage for consistency and extensibility

2018-10-29 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-7567: -- Summary: Clean up internal metadata usage for consistency and extensibility Key: KAFKA-7567 URL: https://issues.apache.org/jira/browse/KAFKA-7567 Project: Kafka

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-10-29 Thread xiongqi wu
Hi Dong, I have updated the KIP to address your comments. One correction to previous Email: after offline discussion with Dong, we decide to use MAX_LONG as default value for max.compaction.lag.ms. Xiongqi (Wesley) Wu On Mon, Oct 29, 2018 at 12:15 PM xiongqi wu wrote: > Hi Dong, > > Thank

Build failed in Jenkins: kafka-trunk-jdk11 #70

2018-10-29 Thread Apache Jenkins Server
See Changes: [colin] KAFKA-7515: Trogdor - Add Consumer Group Benchmark Specification (#5810) -- [...truncated 2.37 MB...] org.apache.kafka.streams.test.OutputVerifierTest

[jira] [Created] (KAFKA-7566) Add sidecar job to leader (or a random single follower) only

2018-10-29 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7566: -- Summary: Add sidecar job to leader (or a random single follower) only Key: KAFKA-7566 URL: https://issues.apache.org/jira/browse/KAFKA-7566 Project: Kafka

Re: [DISCUSS] KIP-354 Time-based log compaction policy

2018-10-29 Thread xiongqi wu
Hi Dong, Thank you for your comment. See my inline comments. I will update the KIP shortly. Xiongqi (Wesley) Wu On Sun, Oct 28, 2018 at 9:17 PM Dong Lin wrote: > Hey Xiongqi, > > Sorry for late reply. I have some comments below: > > 1) As discussed earlier in the email list, if the topic is

[jira] [Created] (KAFKA-7565) NPE in KafkaConsumer

2018-10-29 Thread Alexey Vakhrenev (JIRA)
Alexey Vakhrenev created KAFKA-7565: --- Summary: NPE in KafkaConsumer Key: KAFKA-7565 URL: https://issues.apache.org/jira/browse/KAFKA-7565 Project: Kafka Issue Type: Bug

Re: [VOTE] 2.0.1 RC0

2018-10-29 Thread Manikumar
Hi Eno, This looks like an existing issue occuring only on source artifacts. We are able to generate aggregate docs on cloned repo. I am getting similar error on previous release and 2.1.0 RC0 src artifacts. maybe related to gradle task ordering. I will look into it and try to fix it on trunk.

Re: [VOTE] 2.0.1 RC0

2018-10-29 Thread Eno Thereska
Thanks. Tested basic building and running of unit and integration tests. They work. Tested docs. The following fails. Is it a known issue? " ./gradlew aggregatedJavadoc with info: > Configure project : Building project 'core' with Scala version 2.11.12 Building project 'streams-scala' with Scala

[jira] [Created] (KAFKA-7564) Trogdor - Expose single task details from Trogdor Coordinator

2018-10-29 Thread Stanislav Kozlovski (JIRA)
Stanislav Kozlovski created KAFKA-7564: -- Summary: Trogdor - Expose single task details from Trogdor Coordinator Key: KAFKA-7564 URL: https://issues.apache.org/jira/browse/KAFKA-7564 Project:

Jenkins build is back to normal : kafka-trunk-jdk8 #3171

2018-10-29 Thread Apache Jenkins Server
See

Re: [VOTE] 2.1.0 RC0

2018-10-29 Thread Magnus Edenhill
+1 (non-binding) passes librdkafka integration test suite Den fre 26 okt. 2018 kl 15:58 skrev Manikumar : > minor observation: config sections are empty in the documentation page. > http://kafka.apache.org/21/documentation.html#producerconfigs > > On Wed, Oct 24, 2018 at 10:49 PM Ted Yu wrote:

Jenkins build is back to normal : kafka-trunk-jdk11 #68

2018-10-29 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-7563) Single broker sends incorrect metadata for topic partitions

2018-10-29 Thread Martin Kamp Jensen (JIRA)
Martin Kamp Jensen created KAFKA-7563: - Summary: Single broker sends incorrect metadata for topic partitions Key: KAFKA-7563 URL: https://issues.apache.org/jira/browse/KAFKA-7563 Project: Kafka

[jira] [Created] (KAFKA-7562) Onemirrormaker synchronization problem in 0.11.0.2

2018-10-29 Thread salasming (JIRA)
salasming created KAFKA-7562: Summary: Onemirrormaker synchronization problem in 0.11.0.2 Key: KAFKA-7562 URL: https://issues.apache.org/jira/browse/KAFKA-7562 Project: Kafka Issue Type: Bug