Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-17 Thread Jan Filipiak
This is not a performance optimisation. Its a fundamental design choice. I never really took a look how streams does exactly once. (its a trap anyways and you usually can deal with at least once donwstream pretty easy). But I am very certain its not gonna get somewhere if offset commit and rec

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

2018-10-17 Thread Per Steffensen
Lets use X for the the point in time where commit() is called. Lets use Rs(X) for the recorders returned by poll()s at time X. At time X, it is not necessarily true that all records in Rs(X) have been sent to Kafka (and acknowledged) and had their offsets flushed to offset-store. Example * Tim

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Zahari Dichev
Hi there, Are there any opinions on the matter described in my previous email? I think this is quite important when it comes to implementing any non trivial functionality that relies on pause/resume. Of course if I am mistaken, feel free to elaborate. Thanks, Zahari On Tue, Oct 16, 2018 at 10:29

[jira] [Created] (KAFKA-7516) Client (Producer and/or Consumer) crashes during initialization on Android

2018-10-17 Thread alex kamenetsky (JIRA)
alex kamenetsky created KAFKA-7516: -- Summary: Client (Producer and/or Consumer) crashes during initialization on Android Key: KAFKA-7516 URL: https://issues.apache.org/jira/browse/KAFKA-7516 Project:

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

2018-10-17 Thread Per Steffensen
The fix will definitely "facilitate" the source-connectors I have written. It will make them work 100% correctly. Today they dont. Fine for me to change "Acknowledged" to "Acked" in the method-naming. Not sure I would like to give a Collection instead of a List as the argument to offstesFlushe

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

2018-10-17 Thread Ryanne Dolan
> There is no guarantee that the data in R has been sent/acknowledged > to/by Kafka, nor that the offsets in R has been flushed to offset-store (it > is likely, though). I see, thanks. On the other hand, the commitRecord() callback provides the functionality you require in this case. In commitRec

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Ryanne Dolan
Zahari, 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. Ryanne On Wed, Oct 17, 2018 at 7:35 AM Zahari Dichev wrote: > Hi there, > > Are there any opinions

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

2018-10-17 Thread Per Steffensen
On 17/10/2018 16.43, Ryanne Dolan wrote: I see, thanks. On the other hand, the commitRecord() callback provides the functionality you require in this case. In commitRecord() your SourceTask can track the offsets of records that have been ack'd by the producer client, and then in commit() you can

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-17 Thread Ryanne Dolan
Jan, these are two separate issues. 1) consumer coordination should not, ideally, involve unreliable or slow connections. Naively, a KafkaSourceConnector would coordinate via the source cluster. We can do better than this, but I'm deferring this optimization for now. 2) exactly-once between two c

[jira] [Created] (KAFKA-7517) Add a minimum retention.bytes config value

2018-10-17 Thread Stanislav Kozlovski (JIRA)
Stanislav Kozlovski created KAFKA-7517: -- Summary: Add a minimum retention.bytes config value Key: KAFKA-7517 URL: https://issues.apache.org/jira/browse/KAFKA-7517 Project: Kafka Issue Ty

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

2018-10-17 Thread Ryanne Dolan
> this does not guarantee that the > offsets of R have been written/flushed at the next commit() call True, but does it matter? So long as you can guarantee the records are delivered to the downstream Kafka cluster, it shouldn't matter if they have been committed or not. The worst that can happen

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

2018-10-17 Thread Konstantine Karantasis
Indeed, implying that the flushing and acknowledgement of records happens in order reveals an implementation detail that is not required by the interface. Strictly speaking if that was required then you'd only need a single record as an argument to offstesFlushedAndAcked to indicate up to which rec

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-17 Thread Harsha
Hi Ryanne, Thanks for the KIP. I am also curious about why not use the uReplicator design as the foundation given it alreadys resolves some of the fundamental issues in current MIrrorMaker, updating the confifgs on the fly and running the mirror maker agents in a worker model whic

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-17 Thread Harsha
Congrats Mani!! Very well deserved. --Harsha On Tue, Oct 16, 2018, at 5:20 PM, Attila Sasvari wrote: > Congratulations Manikumar! Keep up the good work. > > On Tue, Oct 16, 2018 at 12:30 AM Jungtaek Lim wrote: > > > Congrats Mani! > > On Tue, 16 Oct 2018 at 1:45 PM Abhimanyu Nagrath < > > abhim

Re: [DISCUSS] KIP-382: MirrorMaker 2.0

2018-10-17 Thread Ryanne Dolan
Harsha, yes I can do that. I'll update the KIP accordingly, thanks. Ryanne On Wed, Oct 17, 2018 at 12:18 PM Harsha wrote: > Hi Ryanne, >Thanks for the KIP. I am also curious about why not use the > uReplicator design as the foundation given it alreadys resolves some of the > fun

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-17 Thread Ray Chiang
Congrats Mani. -Ray On 10/17/18 10:19 AM, Harsha wrote: Congrats Mani!! Very well deserved. --Harsha On Tue, Oct 16, 2018, at 5:20 PM, Attila Sasvari wrote: Congratulations Manikumar! Keep up the good work. On Tue, Oct 16, 2018 at 12:30 AM Jungtaek Lim wrote: Congrats Mani! On Tue, 16 Oct

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Zahari Dichev
Hi there Ryanne, Thanks for the response ! There is most likely quite a lot that I am missing here, but after I read the docs, it seems to me that the pause/resume API has been provided with the very purpose of implementing bespoke flow control. That being said, I see it as quite natural to be abl

[DISCUSS] KIP-383 Pluggable interface for SSL Factory

2018-10-17 Thread Pellerin, Clement
I would like feedback on this proposal to make it possible to replace SslFactory with a custom implementation. https://cwiki.apache.org/confluence/display/KAFKA/KIP-383%3A++Pluggable+interface+for+SSL+Factory

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

2018-10-17 Thread Apache Jenkins Server
See

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

2018-10-17 Thread Apache Jenkins Server
See Changes: [me] KAFKA-7080 and KAFKA-7222: Cleanup overlapping KIP changes Part 2 -- [...truncated 2.83 MB...] org.apache.kafka.streams.test.OutputVerifierTest > should

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Ryanne Dolan
Zahira, Kafka consumers use a pull model. I'm not sure what backpressure means in this context. If a consumer isn't ready for more records, it just doesn't poll() for more. The documentation talks about "flow control" but doesn't mention "backpressure". I think these are related but different con

Build failed in Jenkins: kafka-2.1-jdk8 #30

2018-10-17 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-7080 and KAFKA-7222: Cleanup overlapping KIP changes Part 2 -- [...truncated 232.08 KB...] kafka.admin.ConfigCommandTest > testDynamicBrokerConf

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2018-10-17 Thread vito jeng
Just open a PR for further discussion: https://github.com/apache/kafka/pull/5814 Any suggestion is welcome. Thanks! --- Vito On Thu, Oct 11, 2018 at 12:14 AM vito jeng wrote: > Hi John, > > Thanks for reviewing the KIP. > > > I didn't follow the addition of a new method to the QueryableStoreT

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Jan Filipiak
Hi Zahari, would you be willing to scan through the KIP-349 discussion a little? I think it has suggestions that could be interesting for you Best Jan On 16.10.2018 09:29, Zahari Dichev wrote: Hi there Kafka developers, I am currently trying to find a solution to an issue that has been manife

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Jan Filipiak
especially my suggestions ;) On 18.10.2018 08:30, Jan Filipiak wrote: Hi Zahari, would you be willing to scan through the KIP-349 discussion a little? I think it has suggestions that could be interesting for you Best Jan On 16.10.2018 09:29, Zahari Dichev wrote: Hi there Kafka developers, I

Re: Throwing away prefetched records optimisation.

2018-10-17 Thread Zahari Dichev
Thanks a lot Jan, I will read it. Zahari On Thu, Oct 18, 2018 at 9:31 AM Jan Filipiak wrote: > especially my suggestions ;) > > On 18.10.2018 08:30, Jan Filipiak wrote: > > Hi Zahari, > > > > would you be willing to scan through the KIP-349 discussion a little? > > I think it has suggestions t