In general, the official discussion thread for a KIP starts after the KIP is
written and posted. So you would typically start a new email thread with a
[DISCUSS] string in the title somewhere. You can certainly link back to this
email thread if you want, though, since it has some useful contex
Colin, I agree
I will take a closer looks at the Fetcher itself and see whether that is
feasible and update my KIP accordingly. I guess we can label this one, the
official discussion thread for it or should I start another one ?
Zahari
On Wed, Oct 24, 2018 at 6:43 AM Colin McCabe wrote:
> On T
On Tue, Oct 23, 2018, at 12:38, Zahari Dichev wrote:
> Hi there Matthias, I looked through the code of Kafka Streams. Quite
> impressive work ! If I have to put the logic of buffering within the
> context of what we are doing in Akka though, I might end up with the
> following situation.
>
> 1. Po
Thanks for the detailed explanation.
-Matthias
On 10/23/18 12:38 PM, Zahari Dichev wrote:
> Hi there Matthias, I looked through the code of Kafka Streams. Quite
> impressive work ! If I have to put the logic of buffering within the
> context of what we are doing in Akka though, I might end up wit
Hi there Matthias, I looked through the code of Kafka Streams. Quite
impressive work ! If I have to put the logic of buffering within the
context of what we are doing in Akka though, I might end up with the
following situation.
1. Poll is called with two partition being active *TP1, TP2*
2. We get
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
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
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
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
> 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.
Hi there Matthias,
Very useful thoughts indeed. I have considered the exact same approach but
what worries me a bit is that I do not think that will certainly solve the
issue. Imagine the following situation. You have one consumer that is quite
slow so lets say it call poll every 5 seconds, while
Just my 2 cents.
I am not 100% sure if we would need to change the consumer for this.
While I still think, that KIP-349 might be valuable, it seems to be
complementary/orthogonal to the issue discussed here.
For Kafka Streams, we have a related scenario and what Kafka Streams
does is, to add its
The idea for you would be that Messagechooser could hang on to the
prefetched messages.
ccing cmcc...@apache.org
@Collin
just for you to see that MessageChooser is a powerfull abstraction.
:)
Best jan
On 18.10.2018 13:59, Zahari Dichev wrote:
> Jan,
>
> Quite insightful indeed. I think your p
Thanks Ryanne,
I am glad it makes sense. Should I put a KIP together and call for
discussion on it ? Its my first KIP, so have not quite locked in the
process yet.
Zahari
On Thu, Oct 18, 2018 at 3:26 PM Ryanne Dolan wrote:
> Zahari, that makes sense, thanks for reframing your question. I suspec
Zahari, that makes sense, thanks for reframing your question. I suspect
that pause/resume was not intended to be called at high frequency like
that, but I agree with you that the current behavior is needlessly
inefficient. I like your idea of making it configurable.
Ryanne
On Thu, Oct 18, 2018, 6
Jan,
Quite insightful indeed. I think your propositions are valid.
Ryanne,
I understand that consumers are using a pull model... And yes, indeed if a
consumer is not ready for more records it surely should not call poll.
Except that it needs to do so periodically in order to indicate that its
li
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
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
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
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
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
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
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
Hi there Kafka developers,
I am currently trying to find a solution to an issue that has been
manifesting itself in the Akka streams implementation of the Kafka
connector. When it comes to consuming messages, the implementation relies
heavily on the fact that we can pause and resume partitions. In
24 matches
Mail list logo