Hello again, all,
Since it would be confusing to continue indefinitely with the old
and now PAPIs both not deprecated, I have decided to go
ahead with deprecating the old PAPI in AK 3.0.
Since KAFKA-10603 has not seen any progress, this means
that we actually do have to go ahead and
Hello all,
While reviewing the KIP document, I realized that I hadn't
submitted a PR to migrate the KStream.process() method to
the new API. Upon reflection, I think I'd better defer this
work for the same reason I deferred all the transform()
APIs. I believe that the new Processor interface will
Thanks John.
SGTM.
On 10/1/20 2:50 PM, John Roesler wrote:
> Hello again, all,
>
> I'm sorry to make another tweak to this KIP, but during the
> implementation of the design we've just agreed on, I
> realized that Processors would almost never need to
> reference the RecordMetadata. Therefore,
Hello again, all,
I'm sorry to make another tweak to this KIP, but during the
implementation of the design we've just agreed on, I
realized that Processors would almost never need to
reference the RecordMetadata. Therefore, I'm proposing to
streamline the API by moving the Optional to
the new
Thanks, Matthias!
I can certainly document it. I didn't bother because the old
Processor, Supplier, and Context will themselves be
deprecated, so any method that handles them won't be able to
avoid the deprecation warning. Nevertheless, it doesn't hurt
just to explicitly deprecated those methods.
Thanks John. I like the proposal.
Btw: I was just going over the KIP and realized that we add new methods
to `StreamBuilder`, `Topology`, and `KStream` that take the new
`ProcessorSupplier` class -- should we also deprecate the corresponding
existing ones that take the old `ProcessorSupplier`?
Thanks Paul and Sophie,
Your feedback certainly underscores the need to be explicit
in the javadoc about why that parameter is Optional. Getting
this kind of feedback before the release is exactly the kind
of outcome we hope to get from the KIP process!
Thanks,
-John
On Tue, 2020-09-29 at 22:32
John, I totally agree that adding a method to Processor is cumbersome and
not a good path. I was imagining maybe a separate interface that could be
used in the appropriate context, but I don't think that makes too much
sense - it's just too far away from what Kafka Streams is. I was
originally
FWIW, while I'm really not a fan of Optional in general, I agree that its
usage
here seems appropriate. Even for those rare software developers who
carefully
read all the docs several times over, I think it wouldn't be too hard to
miss a
note about the RecordMetadata possibly being null.
>
> Does my reply address your concerns?
Yes; also, I definitely misread part of the proposal earlier and thought
you had put
the timestamp field in RecordMetadata. Sorry for not giving things a closer
look
before responding! I'm not sure my original message made much sense given
the
Thanks for the reply, Sophie,
I think I may have summarized too much in my prior reply.
In the currently proposed KIP, any caller of forward() must
supply a Record, which consists of:
* key
* value
* timestamp
* headers (with a convenience constructor that sets empty
headers)
These aren't what
>
> However, the record metadata is only defined when the parent forwards
> while processing a
real record, not when it calls forward from the punctuator
Can we take a step back for a second...why wouldn't you be required to set
the RecordContext
yourself when calling forward from a Punctuator?
Oh, I guess one other thing I should have mentioned is that I’ve recently
discovered that in cases where the context is undefined, we currently just fill
in dummy values for the context. So there’s a good chance that real
applications in use are depending on undefined context without even
Thanks for the review, Paul!
I had read some of that debate before. There seems to be some subtext there,
because they advise against using Optional in cases like this, but there
doesn’t seem to be a specific reason why it’s inappropriate. I got the
impression they were just afraid that people
Looks pretty good to me, though the Processor#process(Record,
Optional) signature caught my eye. There's some debate (
https://stackoverflow.com/questions/31922866/why-should-java-8s-optional-not-be-used-in-arguments)
about whether to use Optionals in arguments, and while that's a bit of a
Hello again, all,
Thanks for the latest round of discussion. I've taken the
recent feedback and come up with an updated KIP that seems
actually quite a bit nicer than the prior proposal.
The specific diff on the KIP is here:
Interesting proposal. However, I am not totally convinced, because I see
a fundamental difference between "data" and "metadata".
Topic/partition/offset are "metadata" in the strong sense and they are
immutable.
On the other hand there is "primary" data like key and value, as well as
"secondary"
Thanks for this thought, Matthias!
To be honest, it's bugged me quite a bit that _all_ the
record information hasn't been an argument to `process`. I
suppose I was trying to be conservative in this proposal,
but then again, if we're adding new Processor and
ProcessorContext interfaces, then this
I think separating the different contexts make sense.
In fact, we could even go one step further and remove the record context
from the processor context completely and we add a third parameter to
`process(key, value, recordContext)`. This would make it clear that the
context is for the input
Thanks for the reply, Paul!
I certainly intend to make sure that the changelogging layer
continues to work the way it does now, by hook or by crook.
I think the easiest path for me is to just "cheat" and get
the real ProcessorContext into the ChangeLoggingStore
implementation somehow. I'll tag
>
> when you use a HashMap or RocksDB or other "state stores", you don't
> expect them to automatically know extra stuff about the record you're
> storing.
So, I don't think there is any reason we *can't* retain the record context
> in the StateStoreContext, and if any users came along with a
Thanks for the conversation, Sophie! Sorry for the ambiguity
I introduced into it, though.
Thanks,
-John
On Thu, 2020-09-10 at 16:10 -0700, Sophie Blee-Goldman
wrote:
> Aha, I did misinterpret the example in your previous response regarding the
> range query after all. I thought you just meant a
Aha, I did misinterpret the example in your previous response regarding the
range query after all. I thought you just meant a time-range query inside a
punctuator. It genuinely did not occur to me that users might be looking up
and/or updating records of other keys from within a Processor. Sorry
Ah, thanks Sophie,
I'm sorry for misinterpreting your resonse. Yes, we
absolutely can and should clear the context before
punctuating.
My secondary concern is maybe more far-fetched. I was
thinking that inside process(key,value), a Processor might
do a get/put of a _different_ key. Consider, for
>
> Regarding your first sentence, "...the processor would null
> out the record context...", this is not possible, since the
> processor doesn't have write access to the context. We could
> add it,
>
Sorry, this was poorly phrased, I definitely did not mean to imply that we
should make the
Thanks, Sophie,
Yes, now that you point it out, I can see that the record
context itself should be nulled out by Streams before
invoking punctuators. From that perspective, we don't need
to think about the second-order problem of what's in the
context for the state store when called from a
I was just thinking that the processor would null out the record context
after it
finished processing the record, so I'm not sure I follow why this would not
be
possible? AFAIK we never call a punctuator in the middle of processing a
record through the topology, and even if we did, we still know
Thanks for the thoughts, Sophie.
I agree that the extra information could be useful. My only concern is that it
doesn’t seem like we can actually supply that extra information correctly. So,
then we have a situation where the system offers useful API calls that are only
correct in a narrow
>
> If you were to call "put" from a punctuator, or do a
> `range()` query and then update one of those records with
> `put()`, you'd have a very subtle bug on your hands.
Can you elaborate on this a bit? I agree that the punctuator case is an
obvious exemption to the assumption that store
Hi Paul,
It's good to hear from you!
I'm glad you're in favor of the direction. Especially when
it comes to public API and usability concens, I tend to
think that "the folks who matter" are actually the folks who
have to use the APIs to accomplish real tasks. It can be
hard for me to be sure I'm
John,
It's exciting to see this KIP head in this direction! In the last year or
so I've tried to sketch out some usability improvements for custom state
stores, and I also ended up splitting out the StateStoreContext from the
ProcessorContext in an attempt to facilitate what I was doing. I sort
Hello all,
I've been slowly pushing KIP-478 forward over the last year,
and I'm happy to say that we're making good progress now.
However, several issues with the original design have come
to light.
The major changes:
We discovered that the original plan of just adding generic
parameters to
Thanks, everyone, for the really good discussion.
The vote has been open for 6 days, and has three binding votes (Guozhang,
Bill, Matthias), in addition to my own non-binding +1, so the KIP vote
passes!
Next, I'll close my POC PR and put together an actual change set for review.
Thanks again,
+1 (binding)
On 7/29/19 11:59 AM, Bill Bejeck wrote:
> Thanks for the KIP.
>
> +1 (binding)
>
> -Bill
>
> On Wed, Jul 24, 2019 at 12:12 PM Guozhang Wang wrote:
>
>> Yeah I think I agree with you.
>>
>> +1 (binding) from me.
>>
>>
>> Guozhang
>>
>>
>> On Wed, Jul 24, 2019 at 7:43 AM John
Thanks for the KIP.
+1 (binding)
-Bill
On Wed, Jul 24, 2019 at 12:12 PM Guozhang Wang wrote:
> Yeah I think I agree with you.
>
> +1 (binding) from me.
>
>
> Guozhang
>
>
> On Wed, Jul 24, 2019 at 7:43 AM John Roesler wrote:
>
> > Hi Guozhang,
> >
> > Thanks! I just replied in the discuss
Yeah I think I agree with you.
+1 (binding) from me.
Guozhang
On Wed, Jul 24, 2019 at 7:43 AM John Roesler wrote:
> Hi Guozhang,
>
> Thanks! I just replied in the discuss thread. I agree with what you're
> proposing, but would like to consider it outside the scope of this KIP, if
> that's
Hi Guozhang,
Thanks! I just replied in the discuss thread. I agree with what you're
proposing, but would like to consider it outside the scope of this KIP, if
that's ok with you.
-John
On Tue, Jul 23, 2019 at 5:38 PM Guozhang Wang wrote:
> Hi John,
>
> I left another question regarding
Hi John,
I left another question regarding Transformer in the DISCUSS thread. Other
than that I think this KIP is ready. Thanks!
Guozhang
On Tue, Jul 16, 2019 at 9:01 AM John Roesler wrote:
> Hi Dev,
>
> After a good discussion, I'd like to start the vote for KIP-478
>
38 matches
Mail list logo