Re: Throwing away prefetched records optimisation.

2018-10-21 Thread Ismael Juma
Hi, I think a KIP to discuss a concrete proposal makes sense. One suggestion is to explore the possibility of fixing the issue without a new config. Would that break existing users? Generally, we should strive for avoiding configs if at all possible. Ismael On 16 Oct 2018 12:30 am, "Zahari Diche

Re: Throwing away prefetched records optimisation.

2018-10-21 Thread Colin McCabe
Ryanne Dolan wrote: > It sounds to me like this problem is due to Akka attempting to implement > additional backpressure on top of the Consumer API. I'd suggest they not do > that, and then this problem goes away. Imagine a very simple case where you want to consume from three partitions at abou

Re: [VOTE] KIP-349 Priorities for Source Topics

2018-10-21 Thread Colin McCabe
On Thu, Oct 18, 2018, at 09:23, n...@afshartous.com wrote: > > > > On Oct 12, 2018, at 5:06 PM, Colin McCabe wrote: > > > > Maybe there's some really cool use-case that I haven't thought of. But so > > far I can't really think of any time I would need topic priorities if I was > > muting top

Re: Throwing away prefetched records optimisation.

2018-10-21 Thread Matthias J. Sax
It's spread out multiple classes... Good starting point is here: https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java#L806 It implements the main-loop that polls, addRecordsToTasks() (ie, put the into buffers), and process

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

2018-10-21 Thread Mickael Maison
+1( non-binding) Thanks for the KIP! On Sun, Oct 21, 2018, 03:31 Harsha Chintalapani wrote: > +1(binding). LGTM. > -Harsha > On Oct 20, 2018, 4:49 PM -0700, Dong Lin , wrote: > > Thanks much for the KIP Patrick. Looks pretty good. > > > > +1 (binding) > > > > On Fri, Oct 19, 2018 at 10:17 AM Pat

Re: [DISCUSS] KIP-378: Enable Dependency Injection for Kafka Streams handlers

2018-10-21 Thread Wladimir Schmidt
Hi Damian, The first approach was added only because it had been initially proposed in my pull request, which started a discussion and thus, the KIP-378 was born. Yes, I would like to have something "injectable". In this regard, a `ConfiguredStreamsFactory` (name is a subject to discussion)

[jira] [Created] (KAFKA-7527) Enable Dependency Injection for Kafka Streams handlers (KIP-378)

2018-10-21 Thread Wladimir Schmidt (JIRA)
Wladimir Schmidt created KAFKA-7527: --- Summary: Enable Dependency Injection for Kafka Streams handlers (KIP-378) Key: KAFKA-7527 URL: https://issues.apache.org/jira/browse/KAFKA-7527 Project: Kafka

Re: Throwing away prefetched records optimisation.

2018-10-21 Thread Zahari Dichev
Thanks for your feedback Matthias, Do you think you can point me to the part where Kafka streams deals with all of that so I can take a look. Will try and see whether your suggested approach works for us before trying to argue my point further. Just a mall thing to mention though, blocking on the p

Re: Throwing away prefetched records optimisation.

2018-10-21 Thread Matthias J. Sax
> You have one consumer that is quite >> slow so lets say it call poll every 5 seconds, while you need to call poll >> every 1 second to issue a heartbeat (these are made up numbers of course). Heartbeats are send by a backgroud thread since 0.10.0.0. And there is a second configuration `max.poll.

[DISCUSS] KIP-385: Provide configuration allowing consumer to no throw away prefetched data

2018-10-21 Thread Zahari Dichev
Hi there, although it has been discussed briefly already in this thread , I decided to follow the process and initiate a DISCUSS thread. Comments and suggestions are more than w

[jira] [Created] (KAFKA-7526) Allow for not throwing away prefetched data of paused partitions

2018-10-21 Thread Zahari Dichev (JIRA)
Zahari Dichev created KAFKA-7526: Summary: Allow for not throwing away prefetched data of paused partitions Key: KAFKA-7526 URL: https://issues.apache.org/jira/browse/KAFKA-7526 Project: Kafka

Re: [DISCUSS] KIP-381 Connect: Tell about records that had their offsets flushed in callback

2018-10-21 Thread Per Steffensen
On 18/10/2018 16.46, Ryanne Dolan wrote: Per Steffenson, getting sequence numbers correct is definitely difficult, but this is not Connect's fault. I'd like to see Connect implement exactly-once from end-to-end, but that requires coordination between sources and sinks along the lines that you a

[jira] [Created] (KAFKA-7525) Handling corrupt records

2018-10-21 Thread Katarzyna Solnica (JIRA)
Katarzyna Solnica created KAFKA-7525: Summary: Handling corrupt records Key: KAFKA-7525 URL: https://issues.apache.org/jira/browse/KAFKA-7525 Project: Kafka Issue Type: Improvement

Jenkins build is back to normal : kafka-2.0-jdk8 #170

2018-10-21 Thread Apache Jenkins Server
See

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

2018-10-21 Thread Apache Jenkins Server
See Changes: [me] MINOR: Fix some typos [me] KAFKA-7131: Update release script to generate announcement email text -- [...truncated 3.52 MB...] org.apache.kafka.streams.t