Re: [DISCUSS] KIP-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-09 Thread Bruno Cadonna
Hi Yishun, Thank you for the KIP. I have a couple of comments: 1. Could you please add an example to the KIP that demonstrates how the mocks should be used in a test? 2. I am wondering, whether the MockKeyValueStore needs to be backed by an actual KeyValueStore (in your KIP

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-04-09 Thread Tom Bentley
Hi Rajini, I'd be happy to do that. I'll try to get it done in the next few days. Although there's been quite a lot of interest this, the vote thread never got any binding +1, so it's been stuck in limbo for a long time. It would be great to get this moving again. Kind regards, Tom On Tue,

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-04-09 Thread Jason Gustafson
Hi Colin, On a related note, what do you think about the idea of storing the > reassigning replicas in > /brokers/topics/[topic]/partitions/[partitionId]/state, rather than in the > reassignment znode? I don't think this requires a major change to the > proposal-- when the controller becomes

Re: [VOTE] KIP-360: Improve handling of unknown producer when using EOS

2019-04-09 Thread Guozhang Wang
+1 (binding). Thanks for the written KIP! The approach lgtm. One minor thing: the name of "last epoch" maybe a bit misleading (although it is for internal usage only and will not be exposed to users) for future developers, how about rename it to "required_epoch" and if it is set to "-1" it means

Re: [VOTE] KIP-360: Improve handling of unknown producer when using EOS

2019-04-09 Thread Jason Gustafson
Bump (for Guozhang) On Mon, Apr 8, 2019 at 8:55 AM Jason Gustafson wrote: > Hi All, > > I'd like to start a vote on KIP-360: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-360%3A+Improve+handling+of+unknown+producer > . > > +1 from me (duh) > > Thanks, > Jason >

Re: [DISCUSS] KIP-450: Sliding Window Aggregations in the DSL

2019-04-09 Thread Guozhang Wang
Hi Sophie, Thanks for the proposed KIP. I've made a pass over it and here are some thoughts: 1. "The window size is effectively the grace and retention period". The grace time is defined as "the time to admit late-arriving events after the end of the window." hence it is the additional time

Re: [DISCUSS] KIP-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-09 Thread Sophie Blee-Goldman
Hi Yishun, thanks for the KIP! I have a few initial questions/comments: 1) It may be useful to capture the iterator results as well (eg with a MockIterator that wraps the underlying iterator and records the same way the MockStore wraps/records the underlying store) 2) a. Where is the

[jira] [Created] (KAFKA-8206) A consumer can't discover new group coordinator when the cluster was partly restarted

2019-04-09 Thread alex gabriel (JIRA)
alex gabriel created KAFKA-8206: --- Summary: A consumer can't discover new group coordinator when the cluster was partly restarted Key: KAFKA-8206 URL: https://issues.apache.org/jira/browse/KAFKA-8206

[jira] [Created] (KAFKA-8205) Kafka SSL encryption of dataat rest

2019-04-09 Thread Niten Aggarwal (JIRA)
Niten Aggarwal created KAFKA-8205: - Summary: Kafka SSL encryption of dataat rest Key: KAFKA-8205 URL: https://issues.apache.org/jira/browse/KAFKA-8205 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-8204) Streams may flush state stores in the incorrect order

2019-04-09 Thread John Roesler (JIRA)
John Roesler created KAFKA-8204: --- Summary: Streams may flush state stores in the incorrect order Key: KAFKA-8204 URL: https://issues.apache.org/jira/browse/KAFKA-8204 Project: Kafka Issue

Re: Dynamic window size for aggregations

2019-04-09 Thread Rajesh Kalyanasundaram
Thanks Boyang & Matthias for your replies. Boyang you are right.. More generically, I want to create a calendar based window. If my window is in months then the window shall tumble or hop end of every month. If my window is in days, then the window shall tumble/advanceby end of every day.

Re: Need Doc of Kafka with GO Language

2019-04-09 Thread Gwen Shapira
Confluent's Go client is here: https://github.com/confluentinc/confluent-kafka-go The repository has examples and the API documentation is here: https://docs.confluent.io/current/clients/confluent-kafka-go/index.html Sarama is another popular Go client: https://github.com/Shopify/sarama And the

Re: Dynamic window size for aggregations

2019-04-09 Thread Boyang Chen
Hey Rajesh, my understanding is that you want to create a month-based time window, is that correct? Boyang From: Matthias J. Sax Sent: Tuesday, April 9, 2019 11:42 PM To: dev@kafka.apache.org Subject: Re: Dynamic window size for aggregations Feel free to

Re: Need Doc of Kafka with GO Language

2019-04-09 Thread vinodh kumar gulumuru
Hi, Right now I working on GO language. I need to integrate Kafka into my project for data streaming. I have searched for GO language with kafka. But I didn't get proper documentation. can you provide any tutorial or documentation? hope you are accept my request. Thank you. Regards, vinodh

Re: [DISCUSS] KIP-448: Add State Stores Unit Test Support to Kafka Streams Test Utils

2019-04-09 Thread Yishun Guan
Bumping this up again, thanks! On Fri, Apr 5, 2019, 14:36 Yishun Guan wrote: > Hi, bumping this up again. Thanks! > > On Tue, Apr 2, 2019, 13:07 Yishun Guan wrote: > >> Hi All, >> >> I like to start a discussion on KIP-448 >> (https://cwiki.apache.org/confluence/x/SAeZBg). It is about adding

Re: [DISCUSS] KIP-360: Improve handling of unknown producer

2019-04-09 Thread Matthias J. Sax
@Adam: As workaround, you can increase the repartition topic config `segment.bytes` and set a larger segment size. This should mitigate the issue. -Matthias On 4/4/19 3:47 PM, Jason Gustafson wrote: > Hi Everyone, > > Sorry for the long delay on this KIP. I have updated it to include the >

Re: Dynamic window size for aggregations

2019-04-09 Thread Matthias J. Sax
Feel free to create a feature request JIRA. For now, you could use a custom processor (via `.transform()`) that uses an attached window store to implement the logic you need. -Matthias On 4/9/19 4:09 AM, Rajesh Kalyanasundaram wrote: > Hi all, > We have a requirement to implement aggregations

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-04-09 Thread Rajini Sivaram
Hi Tom, Are you planning to extend this KIP to also include dynamic broker config update (currently covered under AlterConfigPolicy)? May be worth sending another note to make progress on this KIP since it has been around a while and reading through the threads, it looks like there has been a

Re: [VOTE] KIP-445: In-memory session store

2019-04-09 Thread Bill Bejeck
Thanks for the KIP Sophie. +1(binding) -Bill On Tue, Apr 9, 2019 at 12:14 AM Matthias J. Sax wrote: > Thanks for the KIP Sophie! > > +1 (binding) > > > -Matthias > > On 4/8/19 5:26 PM, Sophie Blee-Goldman wrote: > > Hello all, > > > > There has been a positive reception so I'd like to call

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

2019-04-09 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Remove redundant access specifiers from metrics interfaces -- [...truncated 2.36 MB...] org.apache.kafka.connect.source.SourceRecordTest >

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

2019-04-09 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-8203) plaintext connections to SSL secured broker can be handled more elegantly

2019-04-09 Thread Jorg Heymans (JIRA)
Jorg Heymans created KAFKA-8203: --- Summary: plaintext connections to SSL secured broker can be handled more elegantly Key: KAFKA-8203 URL: https://issues.apache.org/jira/browse/KAFKA-8203 Project: Kafka

Dynamic window size for aggregations

2019-04-09 Thread Rajesh Kalyanasundaram
Hi all, We have a requirement to implement aggregations with TimedWindows which may have varying window sizes. For example, I may want a tumbling window that tumbles on 31st of Jan and 28th of Feb, 31st of March and so on. We did initial analysis of TimedWindows. Found that the windoSize is

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

2019-04-09 Thread Apache Jenkins Server
See Changes: [mjsax] MINOR: Correct KStream documentation (#6552) -- [...truncated 2.36 MB...] org.apache.kafka.streams.kstream.internals.FullChangeSerdeTest >

[jira] [Created] (KAFKA-8202) StackOverflowError on producer when splitting batches

2019-04-09 Thread Daniel Krawczyk (JIRA)
Daniel Krawczyk created KAFKA-8202: -- Summary: StackOverflowError on producer when splitting batches Key: KAFKA-8202 URL: https://issues.apache.org/jira/browse/KAFKA-8202 Project: Kafka

[jira] [Created] (KAFKA-8201) Kafka streams repartitioning topic settings crashing multiple nodes

2019-04-09 Thread Anders Aagaard (JIRA)
Anders Aagaard created KAFKA-8201: - Summary: Kafka streams repartitioning topic settings crashing multiple nodes Key: KAFKA-8201 URL: https://issues.apache.org/jira/browse/KAFKA-8201 Project: Kafka

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

2019-04-09 Thread Apache Jenkins Server
See -- [...truncated 545 B...] > git config remote.origin.url https://github.com/apache/kafka.git # timeout=10 Fetching upstream changes from https://github.com/apache/kafka.git > git

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

2019-04-09 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-435: Incremental Partition Reassignment

2019-04-09 Thread George Li
Hi Colin / Jason,  Reassignment should really be doing a batches.  I am not too worried about reassignment znode getting larger.  In a real production environment,  too many concurrent reassignment and too frequent submission of reassignments seemed to cause latency spikes of kafka cluster.