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

2018-03-08 Thread Jan Filipiak
Hi Dong, are you actually reading my emails, or are you just using the thread I started for general announcements regarding the KIP? I tried to argue really hard against linear hashing. Growing the topic by an integer factor does not require any state redistribution at all. I fail to see

Jenkins build is back to normal : kafka-trunk-jdk7 #3232

2018-03-08 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-03-08 Thread Rajini Sivaram
Hi Ron, Thanks for the KIP. Sorry for the delay in reviewing this. I have a few questions/comments. 1. Are all of the classes listed in the KIP intended to be public classes/interfaces? Since it requires more effort to maintain public classes, it will be good if we can make more of

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

2018-03-08 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-250 Add Support for Quorum-based Producer Acknowledgment

2018-03-08 Thread Jason Gustafson
Initially I thought this proposal was just about improving producer latency. So acks=quorum (or whatever) was sort of the midway point between acks=1 and acks=all, but offsets would only be exposed according to current high watermark semantics (meaning full ISR replication). However, it looks like

[jira] [Created] (KAFKA-6627) Console producer default config values override explicitly provided properties

2018-03-08 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6627: -- Summary: Console producer default config values override explicitly provided properties Key: KAFKA-6627 URL: https://issues.apache.org/jira/browse/KAFKA-6627

Re: [VOTE] 1.1.0 RC1

2018-03-08 Thread Jeff Chao
Hello, We at Heroku have run 1.1.0 RC1 through our normal performance and regression test suite and have found performance to be comparable to 1.0.0. That said, we're however -1 (non-binding) since this release includes Zookeeper 3.4.11 which is

Re: [VOTE] KIP-265: Make Windowed Serde to public APIs

2018-03-08 Thread Damian Guy
Thanks Guozhang +1 On Tue, 6 Mar 2018 at 00:26 Hu Xi wrote: > +1 (non-binding) > > > 发件人: Matthias J. Sax > 发送时间: 2018年3月6日 8:19 > 收件人: dev@kafka.apache.org > 主题: Re: [VOTE] KIP-265: Make Windowed Serde to public APIs

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

2018-03-08 Thread Dong Lin
Hey Jan, Sorry for the frustration. I haven't finished replying to all comments. For example in my last email it is mentioned that "I will reply after I finish reading the documentation and code". It takes time to think through comments thoroughly. I have been busy with my daily work and haven't

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread Guozhang Wang
Hello John, Thanks for the KIP. I made a pass over the wiki page and here are some comments: 1. Meta-comment: there is an internal class MockProcessorContext under the o.a.k.test package, which should be replaced as part of this KIP. 2. In @Override StreamsMetrics metrics(), will you return a

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-08 Thread Guozhang Wang
Hello Matthias, thanks for the KIP. I've read through the upgrade patch section and it looks good to me, if you already have a WIP PR for it could you also share it here so that people can take a look? I'm +1 on the KIP itself. But large KIPs like this there are always some devil hidden in the

[jira] [Created] (KAFKA-6628) RocksDBSegmentedBytesStoreTest does not cover time window serdes

2018-03-08 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6628: Summary: RocksDBSegmentedBytesStoreTest does not cover time window serdes Key: KAFKA-6628 URL: https://issues.apache.org/jira/browse/KAFKA-6628 Project: Kafka

[jira] [Created] (KAFKA-6630) Speed up the processing of StopReplicaResponse events on the controller

2018-03-08 Thread Lucas Wang (JIRA)
Lucas Wang created KAFKA-6630: - Summary: Speed up the processing of StopReplicaResponse events on the controller Key: KAFKA-6630 URL: https://issues.apache.org/jira/browse/KAFKA-6630 Project: Kafka

[jira] [Created] (KAFKA-6629) SegmentedCacheFunctionTest does not cover session window serdes

2018-03-08 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6629: Summary: SegmentedCacheFunctionTest does not cover session window serdes Key: KAFKA-6629 URL: https://issues.apache.org/jira/browse/KAFKA-6629 Project: Kafka

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-08 Thread John Roesler
Hey Matthias, The KIP looks good to me. I had several questions queued up, but they were all in the "rejected alternatives" section... oh, well. One very minor thought re changing the state directory from "//< application.id>//rocksdb/storeName/" to "//< application.id>//rocksdb-v2/storeName/":

[jira] [Created] (KAFKA-6631) Kafka Streams - Rebalancing exception in Kafka 1.0.0

2018-03-08 Thread Alexander Ivanichev (JIRA)
Alexander Ivanichev created KAFKA-6631: -- Summary: Kafka Streams - Rebalancing exception in Kafka 1.0.0 Key: KAFKA-6631 URL: https://issues.apache.org/jira/browse/KAFKA-6631 Project: Kafka

Re: [VOTE] KIP-265: Make Windowed Serde to public APIs

2018-03-08 Thread Guozhang Wang
+1 from myself as well. I'm closing the vote for KIP-265 with the following results: binding +1: Matthias, Damian, Guozhang non-binding +1: Xi, Ted, Bill Thanks everyone for the votes. Guozhang On Thu, Mar 8, 2018 at 11:18 AM, Damian Guy wrote: > Thanks Guozhang +1

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
Thanks, Matthias, 1. I can move it into the o.a.k.streams.processor package; that makes sense. 2. I'm expecting most users to use in-memory state stores, so they won't need a state directory. In the "real" code path, the stateDir is extracted from the config by

Re: [VOTE] KIP-251: Allow timestamp manipulation in Processor API

2018-03-08 Thread Matthias J. Sax
Guozhang, I updated the code slightly to avoid object creation and I did some perf investigations. 1) JMH Benchmark with the below topology using TopologyTestDriver to pipe data throw the topology: > StreamsBuilder builder = new StreamsBuilder(); > KStream stream =

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
I think what you're suggesting is to: 1. compile the main streams code, but not the tests 2. compile test-utils (and compile and run the test-utils tests) 3. compile and run the streams tests This works in theory, since the test-utils depends on the main streams code, but not the streams tests.

Re: Request to be added to the contributor list so that i can assign JIRAs to myself

2018-03-08 Thread Matthias J. Sax
What is your user ID? -Matthias On 3/8/18 8:11 PM, Sanket Band wrote: > Thanks > Sanket > signature.asc Description: OpenPGP digital signature

Re: Request to be added to the contributor list so that i can assign JIRAs to myself

2018-03-08 Thread Matthias J. Sax
I meant your JIRA ID -- sorry for the confusion. -Matthias On 3/8/18 10:48 PM, Sanket Band wrote: > you mean the github id ? it is sband > > Thanks > Sanket Band > > On Fri, Mar 9, 2018 at 11:22 AM, Matthias J. Sax > wrote: > >> What is your user ID? >> >> -Matthias

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

2018-03-08 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix incorrect references to the max transaction timeout config [wangguoz] KAFKA-6560: [FOLLOW-UP] don't deserialize null byte array in window

Request to be added to the contributor list so that i can assign JIRAs to myself

2018-03-08 Thread Sanket Band
Thanks Sanket

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

2018-03-08 Thread Matthias J. Sax
Jun, There is one more case: non-windowed aggregations. For windowed aggregation, the changelog topic will be compact+delete. However, for non-windowed aggregation the policy is compact only. Even if we assume that windowed aggregations are dominant and non-windowed aggregation are used rarely,

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

2018-03-08 Thread Matthias J. Sax
As I just mentioned joins: For Kafka Streams it might also be required to change the partition count for multiple topics in a coordinated way that allows to maintain the co-partitioning property that Kafka Streams uses to computed joins. Any thoughts how this could be handled? -Matthias On

Re: [VOTE] KIP-251: Allow timestamp manipulation in Processor API

2018-03-08 Thread Guozhang Wang
Thanks Matthias, that sounds good to me. I'm +1 on the KIP itself. Guozhang On Thu, Mar 8, 2018 at 5:46 PM, Matthias J. Sax wrote: > Guozhang, > > I updated the code slightly to avoid object creation and I did some perf > investigations. > > 1) JMH Benchmark with the

Re: Request to be added to the contributor list so that i can assign JIRAs to myself

2018-03-08 Thread Sanket Band
you mean the github id ? it is sband Thanks Sanket Band On Fri, Mar 9, 2018 at 11:22 AM, Matthias J. Sax wrote: > What is your user ID? > > -Matthias > > On 3/8/18 8:11 PM, Sanket Band wrote: > > Thanks > > Sanket > > > >

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

2018-03-08 Thread Matthias J. Sax
@Jan: You suggest to copy the data from one topic to a new topic, and provide an "offset mapping" from the old to the new topic for the consumers. I don't quite understand how this would work. Let's say there are 2 partitions in the original topic and 3 partitions in the new topic. If we assume

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-08 Thread Ted Yu
Matthias: Nicely written KIP. "in_place" : can this be "in-place" ? Underscore may sometimes be miss typed (as '-'). I think using '-' is more friendly to user. public interface ReadOnlyKeyValueTimestampStore { Is ReadOnlyKeyValueStoreWithTimestamp better name for the class ? Thanks On

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

2018-03-08 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
Actually, replacing the MockProcessorContext in o.a.k.test could be a bit tricky, since it would make the "streams" module depend on "streams:test-utils", but "streams:test-utils" already depends on "streams". At first glance, it seems like the options are: 1. leave the two separate

Jenkins build is back to normal : kafka-trunk-jdk7 #3234

2018-03-08 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-08 Thread Ted Yu
Matthias: For my point #1, I don't have preference as to which separator is chosen. Given the background you mentioned, current choice is good. For #2, I think my proposal is better since it is closer to English grammar. Would be good to listen to what other people think. On Thu, Mar 8, 2018 at

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

2018-03-08 Thread Jun Rao
Hi, Matthis, My understanding is that in KStream, the only case when a changelog topic needs to be compacted is when the corresponding input is a KTable. In all other cases, the changelog topics are of compacted + deletion. So, if most KTables are not high volume, there may not be a need to

Re: [DISCUSS] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-08 Thread John Roesler
Thanks for the review, Guozhang, In response: 1. I missed that! I'll look into it and update the KIP. 2. I was planning to use the real implementation, since folks might register some metrics in the processors and want to verify the values that get recorded. If the concern is about initializing

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

2018-03-08 Thread Jun Rao
Hi, Jan, Thanks for the feedback. Just some comments on the earlier points that you mentioned. 50. You brought up the question of whether existing data needs to be copied during partition expansion. My understand of your view is that avoid copying existing data will be more efficient, but it

Jenkins build is back to normal : kafka-trunk-jdk9 #455

2018-03-08 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-08 Thread Matthias J. Sax
Thanks for the comments! @Guozhang: So far, there is one PR for the rebalance metadata upgrade fix (addressing the mentioned https://issues.apache.org/jira/browse/KAFKA-6054) It give a first impression how the metadata upgrade works including a system test: