Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-26 Thread Dong Lin
Hey John, Thanks much for the detailed comment. Here is my thought: Regarding thought 1: I agree. The current proposal seems to have addressed core problem. It has also provided the extra interface to support the stream use-case. I have yet to come up with the solution to do the 2-phase partition

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-26 Thread Jun Rao
Hi, John, Thanks for the comments. For your thought 1, I agree that it's better to decouple the design of core and Streams with respect to re-partitioning. From the state management perspective, a streaming application can be (1) stateless, (2) stateful and maintaining the states in a global data

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-26 Thread Jun Rao
Hi, Jan, Thanks for the reply. A few more comments inlined below. On Fri, Mar 23, 2018 at 7:15 AM, Jan Filipiak wrote: > >> I agree that decoupling the number of tasks in a consumer group from the >> number of partitions in the input topic is a good idea. This allows each >> consumer group to c

Build failed in Jenkins: kafka-0.10.1-jdk7 #133

2018-03-26 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6054: Fix upgrade path from Kafka Streams v0.10.0 (#4746) -- Started by an SCM change [EnvInject] - Loading node environment variables. Buildin

Re: [DISCUSSION] KIP-266: Add TimeoutException to KafkaConsumer#position()

2018-03-26 Thread Guozhang Wang
@Richard: TimeoutException inherits from RetriableException which inherits from ApiException. So users should explicitly try to capture RetriableException in their code and handle the exception. @Isamel, Ewen: I'm trying to push progress forward on this one, are we now on the same page for using f

Build failed in Jenkins: kafka-trunk-jdk8 #2503

2018-03-26 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix flaky standby task test (#4767) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H20 (

Build failed in Jenkins: kafka-0.10.2-jdk7 #206

2018-03-26 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6054: Fix upgrade path from Kafka Streams v0.10.0 (#4758) -- Started by an SCM change [EnvInject] - Loading node environment variables. Buildin

Build failed in Jenkins: kafka-trunk-jdk7 #3288

2018-03-26 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix flaky standby task test (#4767) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H27 (

Re: [DISCUSS] KIP-211: Revise Expiration Semantics of Consumer Group Offsets

2018-03-26 Thread Vahid S Hashemian
Hi all, Thanks for the feedback on this KIP so far. If there is no additional feedback, I'll start a vote on Wed. Thanks. --Vahid

Re: [VOTE] KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade

2018-03-26 Thread Matthias J. Sax
Thanks Ted for pointing out. Sorry for this mistake Zhenya. You should not have voted twice though ;) (Even if it was the same email address, so I could have figured this out by myself -- did not pay close enough attention.) No bid deal. The KIP is accepted with +6/+3 votes than. -Matthias On

Re: [VOTE] KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade

2018-03-26 Thread Matthias J. Sax
Thanks Ted for pointing out. Sorry for this mistake Zhenya. You should not have voted twice though ;) (Even if it was the same email address, so I could have figured this out by myself -- did not pay close enough attention.) No bid deal. The KIP is accepted with +6/+3 votes than. -Matthias On

Forcing log compaction

2018-03-26 Thread Cindy McMullen
How can I force the deletion of a record within a very small amount of time? That seems possible only by adding a bunch of records to the topic to force the segment containing the Tombstone record to become inactive. I need better control of my data so I can guarantee record deletes and update

Re: [VOTE] KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade

2018-03-26 Thread Ted Yu
Congratulations. BTW I believe 孙振亚 and Zhenya is the same person - Zhenya is the PinYin of 振亚 Cheers On Mon, Mar 26, 2018 at 12:51 PM, Matthias J. Sax wrote: > +1 (binding) > > > I am also cloeing this vote. The KIP is accepted with > > +7 non-binding (Richard, Ted, 孙振亚, Bill, James, John, Zhe

Re: [VOTE] KIP-268: Simplify Kafka Streams Rebalance Metadata Upgrade

2018-03-26 Thread Matthias J. Sax
+1 (binding) I am also cloeing this vote. The KIP is accepted with +7 non-binding (Richard, Ted, 孙振亚, Bill, James, John, Zhenya) +3 binding (Damian, Guozhang, Matthias) votes. Thanks a lot! -Matthias On 3/22/18 4:13 PM, zhenya Sun wrote: > +1 > > > | | > zhenya Sun > 邮箱:toke...@126.com

Re: [VOTE] KIP-257 - Configurable Quota Management

2018-03-26 Thread Viktor Somogyi
+1 (non-binding) Thanks for the KIP, Rajini On Fri, Mar 23, 2018 at 5:00 PM, Mickael Maison wrote: > +1 (non binding), thanks Rajini > > On Fri, Mar 23, 2018 at 9:38 AM, Edoardo Comar wrote: > > Thanks for the KIP, Rajini > > > > +1 (non binding) > > ---

[jira] [Created] (KAFKA-6716) discardChannel should be released in MockSelector#completeSend

2018-03-26 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-6716: - Summary: discardChannel should be released in MockSelector#completeSend Key: KAFKA-6716 URL: https://issues.apache.org/jira/browse/KAFKA-6716 Project: Kafka Issue Type: T

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-26 Thread John Roesler
Hi all, I hope you don't mind if I throw some thoughts in... For some reason, Jan's proposal wasn't really clear to me until this last message, and now that (I think) I understand it, I actually like it quite a lot. Thought 1: As much as I like Streams, I don't think it should become a requireme

[jira] [Resolved] (KAFKA-6712) Throw a specific exception with wrong topic name for interactive queries

2018-03-26 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6712?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-6712. Resolution: Duplicate > Throw a specific exception with wrong topic name for interactive que

Re: [DISCUSS] KIP-272: Add API version tag to broker's RequestsPerSec metric

2018-03-26 Thread Allen Wang
Hello, I would like to bring this to vote in the next day or two. Let me know if you have further comments. Thanks, Allen On Thu, Mar 22, 2018 at 11:37 AM, Xavier Léauté wrote: > > > > This kind of change will be problematic to us as the total RequestsPerSec > > will be double counted in our

[jira] [Created] (KAFKA-6715) Leader transition for all partitions lead by two brokers without visible reason

2018-03-26 Thread Uwe Eisele (JIRA)
Uwe Eisele created KAFKA-6715: - Summary: Leader transition for all partitions lead by two brokers without visible reason Key: KAFKA-6715 URL: https://issues.apache.org/jira/browse/KAFKA-6715 Project: Kafk

[jira] [Created] (KAFKA-6714) KafkaController marks all Brokers as "Shutting down", though only one broker has been shut down

2018-03-26 Thread Uwe Eisele (JIRA)
Uwe Eisele created KAFKA-6714: - Summary: KafkaController marks all Brokers as "Shutting down", though only one broker has been shut down Key: KAFKA-6714 URL: https://issues.apache.org/jira/browse/KAFKA-6714