Hi all,
With the acceptance of KIP-820 which will enable easier access to the
Record's metadata and headers and the potential design of a new version of
the DSL, I will set this KIP as inactive/dormant for the time being.
Thanks, everyone for the great discussions!
Jorge.
On Wed, 16 Feb 2022 at
Sorry for playing devil's advocate, but do we really think it's a good
way to design the API? To me, if feels like a cumbersome workaround.
Personally, I believe that we are hitting a point for the DSL that
requires a redesign from scratch. When the DSL was designed 5 years ago,
record timesta
Thanks for the update, Jorge,
I've just looked over the KIP again. Just one more small
concern:
5) We can't just change the type of Record#headers() to a
new fully qualified type. That would be a source-
incompatible breaking change for users.
Out options are:
* Deprecate the existing method and
John and team,
The following changes have been applied to the KIP following your feedback:
- Leverage `Record` instead of introducing a new type
(`RecordValue`).
- `RecordSerde` for stateful operations using `Record` as value.
- Extend `Record` to:
- Implement `RecordMetadata` to expose `topic`
> What do you think about instead adding topic and
partition to Record?
This is a very interesting idea. Forgot to consider this addition from
KIP-478.
`Record` would also require `offset`. Maybe implementing `RecordMetadata`
and adding these fields as part of the constructor to keep them immutab
Hello Jorge,
Thanks for bringing this up again!
I've just read over the current version of the KIP.
1) I wonder if we really need RecordValue, since we now have
Record, and they are almost the same, both in API and in
purpose. What do you think about instead adding topic and
partition to Record?
Hi Dev team,
I'd like to revamp the KIP again:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-634%3A+Complementary+support+for+headers+and+record+metadata+in+Kafka+Streams+DSL
- Reference implementation is now using the latest `Processor` API from
KIP-478: https://github.com/apache/kafka/p
Hi everyone!
I'd like to revamp this KIP. I have made some significant changes on the
scope:
- Added `mapRecordValue` to map not only headers, but other record
metadata: topic name, partition, offset, and timestamp into a new type
`RecordValue`.
- Added a serde for `RecordValue` to support statefu
Thanks Sophie! Haven't followed KIP-478 but sounds great.
I'll be happy to help on that migration to the new PAPI if it's still an
open issue. We can bump this KIP after that.
Cheers,
Jorge.
On Wed, Oct 28, 2020 at 7:00 PM Sophie Blee-Goldman
wrote:
> I *think* that the `To` Matthias was referr
I *think* that the `To` Matthias was referring to was not KStream#to but
the To class
which is accepted as a possible parameter of ProcessorContext#forward
(correct
me if wrong).
This was on the old ProcessorContext interface, which has now been
replaced with
the new api.ProcessorContext in KIP-47
Hi Matthias,
Sorry for the late reply.
I like the proposal. Just to check if I got it right:
We can extend the `kstream.to()` function to support setting headers. e.g.:
```
void to(final String topic,
final Produced produced,
final HeadersExtractor headersExtractor);
Jorge,
thanks a lot for this KIP. Being able to modify headers is a very
valuable feature.
However, before we actually expose them in the DSL, I am wondering if we
should improve how headers can be modified in the PAPI? Currently, it is
possible but very clumsy to work with headers in the Process
Hi everyone,
Bumping this thread to check if there's any feedback.
Cheers,
Jorge.
On Tue, Jun 30, 2020 at 12:46 AM Jorge Esteban Quilcate Otoya <
quilcate.jo...@gmail.com> wrote:
> Hi everyone,
>
> I would like to start the discussion for
> KIP-634:https://cwiki.apache.org/confluence/display/K
Hi everyone,
I would like to start the discussion for
KIP-634:https://cwiki.apache.org/confluence/display/KAFKA/KIP-634%3A+Complementary+support+for+headers+in+Kafka+Streams+DSL
Looking forward to your feedback.
Thanks!
Jorge.
14 matches
Mail list logo