On 5/3/24 6:29 PM, Matthias J. Sax wrote:
101: Yes, but what I am saying is, that we don't need to flush the
.position file to disk periodically, but only maintain it in main
memory, and only write it to disk on close() to preserve it across
restarts. This way, it would never be ahead
-3910-4c25-bfad-ea2b98953db3-StreamThread-9
Message: [Consumer
clientId=kafka-streams-exec-0-test-store-6d676cf0-3910-4c25-bfad-ea2b98953db3-StreamThread-9-consumer,
groupId=kafka-streams-exec-0-test-store ] Node 102 disconnected.
On Mon, Apr 22, 2024 at 7:16 AM Matthias J. Sax wrote
, May 1, 2024 at 11:00 PM Matthias J. Sax wrote:
Any updates on this KIP?
On 3/28/24 4:11 AM, Matthias J. Sax wrote:
It seems that `MockRecordMetadata` is a private class, and thus not part
of the public API. If there are any changes required, we don't need to
discuss on the KIP
:43 AM, Matthias J. Sax wrote:
Thanks Sophie. My bad. You are of course right about `TaskAssignment`
and the StreamsPartitionAssignor's responsibitliy to map tasks of a
instance to consumers. When I wrote my reply, I forgot about this detail.
Seems you did not add `UNKNOWN_TASK_ID` error yet as pr
eams/state/internals/RocksDBStore.java#L397
On 5/1/24 3:00 AM, Matthias J. Sax wrote:
Thanks Bruno.
101: I think I understand this better now. But just want to make sure
I do. What do you mean by "they can diverge" and "Recovering after a
failure might load inconsistent offse
+1 (binding)
On 5/3/24 8:52 AM, Federico Valeri wrote:
Hi Fred, this is a useful addition.
+1 non binding
Thanks
On Fri, May 3, 2024 at 4:11 PM Andrew Schofield
wrote:
Hi Fred,
Thanks for the KIP. It’s turned out nice and elegant I think. Definitely a
worthwhile improvement.
+1
What about (106) to unify both exiting callback methods of
`ProductionExceptionHandler` into a single one, instead of adding two
new ones?
Damien's last reply about it was:
I will think about unifying, I do agree it would be cleaner.
There was not follow up on this question, and the KIP
I left one more nit on the discuss thread. But overall LGTM.
+1 (binding)
Thanks Rohan and Sophie for driving this KIP.
-Matthias
On 4/29/24 2:07 PM, Sophie Blee-Goldman wrote:
+1 (binding)
thanks for driving this KIP!
On Tue, Apr 16, 2024 at 1:46 PM Rohan Desai wrote:
have a lower level indention? It caught me for a sec until I
realized there are just two categories.
* In TaskAssignmentUtils , why not let those util functions return
`TaskAssignment` (to me it feels more consistent with the user APIs),
but instead return a Map?
Guozhang
On Tue, Apr 30, 2024
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Component/s: clients
> Deprecate window.size.ms and window.inner.serde.cl
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Description:
{{window.size.ms}} and {{window.inner.serde.class}} are not a true
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Description:
{{indwindow.size.ms}} and `is not a true KafkaStreams config, and results
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Description:
{{window.size.ms}} and `is not a true KafkaStreams config, and results
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Summary: Deprecate window.size.ms and window.inner.serde.class in
StreamsConfig
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Summary: Deprecate window.size.ms and inner.serde.class in StreamsConfig
(was: Deprecate
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Description:
{{window.size.ms}} is not a true KafkaStreams config, and results
[
https://issues.apache.org/jira/browse/KAFKA-16260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16260:
Labels: KIP (was: needs-kip)
> Deprecate window.size.ms in StreamsCon
Any updates on this KIP?
On 3/28/24 4:11 AM, Matthias J. Sax wrote:
It seems that `MockRecordMetadata` is a private class, and thus not part
of the public API. If there are any changes required, we don't need to
discuss on the KIP.
For `CapturedPunctuator` and `CapturedForward` it's
oes process one sided joins after the
skipped record. Do you have any docs on the "dropper records" metric? I did
a Google search and didn't find many good results for that.
Thanks,
Chad
On Tue, Apr 30, 2024 at 2:49 PM Matthias J. Sax wrote:
Thanks for the information. I ran the cod
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842805#comment-17842805
]
Matthias J. Sax commented on KAFKA-16514:
-
Thanks for the background! Makes sense.
> Ka
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16644.
-
Resolution: Duplicate
> FK join emits duplicate tombstone on left-side del
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16644.
-
Resolution: Duplicate
> FK join emits duplicate tombstone on left-side del
Thanks for the update.
I am wondering if we should use `ReadOnlyHeaders` instead of
`ImmutableHeaders` as interface name?
Also, the returned `Header` interface is technically not immutable
either, because `Header#key()` returns a mutable byte-array... Would we
need a `ReadOnlyHeader`
that we should try to support downgrades without wipes. At least
Nick should state in the KIP why we do not support it.
Best,
Bruno
On 4/23/24 8:13 AM, Matthias J. Sax wrote:
Thanks for splitting out this KIP. The discussion shows, that it is a
complex beast by itself, so worth to discuss by
I like the idea of error codes. Not sure if the name are ideal?
UNKNOWN_PROCESS_ID makes sense, but the other two seems a little bit
difficult to understand?
Should we be very descriptive (and also try to avoid coupling it to the
threading model -- important for the first error code):
-
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842514#comment-17842514
]
Matthias J. Sax commented on KAFKA-16644:
-
Sorry. Wrong link. Fixed -> https://issues.apache.
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16644:
Description:
We introduced a regression bug in 3.7.0 release via KAFKA-14748. When
n on to tell me what is going on?
Basically, I'm looking for some pointers on where I can start looking.
Thanks,
Chad
On Tue, Apr 30, 2024 at 10:26 AM Matthias J. Sax wrote:
I expect the join to
execute after the 25 with one side of the join containing a record and
the
other being null
Give
I expect the join to
execute after the 25 with one side of the join containing a record and the
other being null
Given that you also have a grace period of 5 minutes, the result will
only be emitted after the grace-period passed and the window is closed
(not when window end time is reached).
[
https://issues.apache.org/jira/browse/KAFKA-16382?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842423#comment-17842423
]
Matthias J. Sax commented on KAFKA-16382:
-
Not yet from our side... Working on other things atm
[
https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842419#comment-17842419
]
Matthias J. Sax commented on KAFKA-16645:
-
I believe fixing these CVEs should be a blocker
[
https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16645:
Priority: Blocker (was: Major)
> CVEs in 3.7.0 docker im
[
https://issues.apache.org/jira/browse/KAFKA-16645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16645:
Fix Version/s: 3.8.0
3.7.1
> CVEs in 3.7.0 docker im
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842417#comment-17842417
]
Matthias J. Sax commented on KAFKA-16644:
-
Thanks for the pointer. I did not have time yet
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842229#comment-17842229
]
Matthias J. Sax commented on KAFKA-16514:
-
Cool. You can find details on the wiki:
[https
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16644:
Description:
We introduced a regression bug in 3.7.0 release via KAFKA-14778. When
[
https://issues.apache.org/jira/browse/KAFKA-16644?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16644:
Summary: FK join emits duplicate tombstone on left-side delete (was: FK
join emit
Matthias J. Sax created KAFKA-16644:
---
Summary: FK join emit duplicate tombstone on left-side delete
Key: KAFKA-16644
URL: https://issues.apache.org/jira/browse/KAFKA-16644
Project: Kafka
Matthias J. Sax created KAFKA-16644:
---
Summary: FK join emit duplicate tombstone on left-side delete
Key: KAFKA-16644
URL: https://issues.apache.org/jira/browse/KAFKA-16644
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17842183#comment-17842183
]
Matthias J. Sax commented on KAFKA-16514:
-
I would not call it a bug, but more like a feature
Hi,
as many of you know, there is a issue with creating new wiki accounts
right now: https://issues.apache.org/jira/browse/INFRA-25451
A fix might take some more time. In the meantime, please reply to the
INFRA ticket with your email address, and accounts will be created
manually in the
It's a known issue and INFRA is working on a solution:
https://issues.apache.org/jira/browse/INFRA-25451
In the mean time, users can be added manually (cf the ticket, and reply
there to get added).
-Matthias
On 3/28/24 5:10 AM, Prashant Jagtap wrote:
Hi,
I hope this email finds you well.
Did you look into .windowedBy(...).emitStrategy(...) ?
Using emit-final you would get an downstream even only after the window
closed.
-Matthias
On 4/29/24 1:43 AM, Santhoshi Mekala wrote:
Hi Team,
We have the below requirement:
We are processing batch logs in kstreams. Currently, we are
[
https://issues.apache.org/jira/browse/KAFKA-16584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17841332#comment-17841332
]
Matthias J. Sax commented on KAFKA-16584:
-
There is an issue with creating new account:
https
Thanks. You should be all set.
-Matthias
On 4/25/24 10:49 PM, Claude Warren wrote:
My Confluence ID is "claude"
On Thu, Apr 25, 2024 at 8:40 PM Matthias J. Sax wrote:
What's your wiki ID? We can grant write access on our side if you have
already an account.
-Matthias
On 4/25/
believe that we
shouldn't call certain kinds of assignments "invalid" (such as a task not
being assigned to anyone). For the clearly-invalid assignment cases, I'd
err on the side of not holding users hands too much for now, but again:
would be happy to defer if anyone has another suggestion and/or s
What's your wiki ID? We can grant write access on our side if you have
already an account.
-Matthias
On 4/25/24 4:06 AM, Claude Warren wrote:
I would like to get edit access to the Kafka confluence so that I can work
on KIP-936. Can someone here do that or do I need to go through Infra?
[
https://issues.apache.org/jira/browse/KAFKA-16585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17840907#comment-17840907
]
Matthias J. Sax commented on KAFKA-16585:
-
{quote}I can use the regular Processor, but as I
aying -- but I'm now realizing that
I completely misinterpreted what your concern was. Sorry for the
long-winded and ultimately irrelevant answer. I'm completely fine with
having the return type be a simple Set with additional info such as TaskId
in the AssignedTask class (and I see Rohan already made this
[
https://issues.apache.org/jira/browse/KAFKA-16584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17840639#comment-17840639
]
Matthias J. Sax commented on KAFKA-16584:
-
Yes, it does required to write code. – It does also
Congrats!
On 4/24/24 2:29 PM, Bill Bejeck wrote:
Congrats Igor!
-Bill
On Wed, Apr 24, 2024 at 2:37 PM Tom Bentley wrote:
Congratulations Igor!
On Thu, 25 Apr 2024 at 6:27 AM, Chia-Ping Tsai wrote:
Congratulations, Igor! you are one of the best Kafka developers!!!
Mickael Maison 於
Congrats!
On 4/24/24 2:29 PM, Bill Bejeck wrote:
Congrats Igor!
-Bill
On Wed, Apr 24, 2024 at 2:37 PM Tom Bentley wrote:
Congratulations Igor!
On Thu, 25 Apr 2024 at 6:27 AM, Chia-Ping Tsai wrote:
Congratulations, Igor! you are one of the best Kafka developers!!!
Mickael Maison 於
[
https://issues.apache.org/jira/browse/KAFKA-16584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17840232#comment-17840232
]
Matthias J. Sax commented on KAFKA-16584:
-
I would prefer to make if configurable personally
[
https://issues.apache.org/jira/browse/KAFKA-16584?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16584:
Labels: needs-kip newbie (was: )
> Make log processing summary configurable or de
et to null. Therefore, the KIP provides no
default implementation of the interface.
- We follow the interface solution as described in the
Rejected Alternetives section.
Cheers,
Alieh
On Thu, Apr 18, 2024 at 8:11 PM Matthias J. Sax wrote:
Thanks for the KIP Alieh! It addresses an important case
Thanks for splitting out this KIP. The discussion shows, that it is a
complex beast by itself, so worth to discuss by its own.
Couple of question / comment:
100 `StateStore#commit()`: The JavaDoc says "must not be called by
users" -- I would propose to put a guard in place for this, by
gaps in a separate
KIP.
Best,
Bruno
On 4/11/24 12:06 AM, Matthias J. Sax wrote:
Thanks for the KIP. Great discussion.
I am not sure if I understand the proposal from Bruno to hand in the
processor node id? Isn't this internal (could not even find it
quickly).
We do have a processor name, r
[
https://issues.apache.org/jira/browse/KAFKA-16585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839910#comment-17839910
]
Matthias J. Sax commented on KAFKA-16585:
-
Well, the use-case make sense, but the question
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839907#comment-17839907
]
Matthias J. Sax commented on KAFKA-16567:
-
Thanks Bruno – makes sense to me – should we move
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16567:
Priority: Major (was: Blocker)
> Add New Stream Metrics based on KIP-
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16567:
Fix Version/s: (was: 4.0.0)
> Add New Stream Metrics based on KIP-
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839905#comment-17839905
]
Matthias J. Sax commented on KAFKA-16514:
-
Well, in general we can, but the internal flag
Not sure either, but it sounds like a bug to me. Can you reproduce this
reliably? What version are you using?
It would be best if you could file a Jira ticket and we can take it from
there.
-Matthias
On 4/21/24 5:38 PM, Penumarthi Durga Prasad Chowdary wrote:
Hi ,
I have an issue in
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839441#comment-17839441
]
Matthias J. Sax commented on KAFKA-16514:
-
You are right that there is alway a member-id etc – I
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839440#comment-17839440
]
Matthias J. Sax commented on KAFKA-16567:
-
I see – this raises a few questions... Given that KIP
, Matthias J. Sax wrote:
Great KIP. I have some minor comments/questions:
100 The KIP says: "In the future, additional plugins can use the same
partition.assignor prefix". What does this mean?
101 (nit) The KIP says: "Note that the thread-level assignment will
remain an
Great KIP. I have some minor comments/questions:
100 The KIP says: "In the future, additional plugins can use the same
partition.assignor prefix". What does this mean?
101 (nit) The KIP says: "Note that the thread-level assignment will
remain an un-configurable internal implementation
[
https://issues.apache.org/jira/browse/KAFKA-16486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16486.
-
Fix Version/s: 3.8.0
Resolution: Done
> Integrate metric measurability chan
[
https://issues.apache.org/jira/browse/KAFKA-16486?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16486.
-
Fix Version/s: 3.8.0
Resolution: Done
> Integrate metric measurability chan
Matthias J. Sax created KAFKA-16586:
---
Summary: Test TaskAssignorConvergenceTest failing
Key: KAFKA-16586
URL: https://issues.apache.org/jira/browse/KAFKA-16586
Project: Kafka
Issue Type
Matthias J. Sax created KAFKA-16586:
---
Summary: Test TaskAssignorConvergenceTest failing
Key: KAFKA-16586
URL: https://issues.apache.org/jira/browse/KAFKA-16586
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838829#comment-17838829
]
Matthias J. Sax commented on KAFKA-16514:
-
Thanks for the input. I was not reviewing/voting
[
https://issues.apache.org/jira/browse/KAFKA-16573?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16573:
Priority: Minor (was: Major)
> Streams does not specify where a Serde is nee
[
https://issues.apache.org/jira/browse/KAFKA-16573?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838827#comment-17838827
]
Matthias J. Sax commented on KAFKA-16573:
-
Thanks for filing this ticket. I think your idea
[
https://issues.apache.org/jira/browse/KAFKA-16280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16280.
-
Resolution: Done
> Expose method to determine Metric Measurabil
[
https://issues.apache.org/jira/browse/KAFKA-16280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16280:
Issue Type: Improvement (was: Bug)
> Expose method to determine Metric Measurabil
[
https://issues.apache.org/jira/browse/KAFKA-16280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax resolved KAFKA-16280.
-
Resolution: Done
> Expose method to determine Metric Measurabil
Thanks for the KIP Alieh! It addresses an important case for error handling.
I agree that using this handler would be an expert API, as mentioned by
a few people. But I don't think it would be a reason to not add it. It's
always a tricky tradeoff what to expose to users and to avoid foot guns,
Andrew, thanks for the details about Consumer internals. That's super
useful for this discussion! -- And it confirms my understanding.
I don't think we want to use ConsumerRecord type thought,
because for a DLQ the handler wants to write the message into some DLQ
topic, and thus needs the key
The main difference is the internal implementation. Semantically, both
are equivalent.
suppress() uses an in-memory buffer, while `emitStrategy()` does not,
but modifies the upstream aggregation operator impl, and waits to send
results downstream, and thus, it's RocksDB based.
-Matthias
I don't think that there is any specific recommendation. However, there
is an overall max-message-size config that you need to keep in mind.
-Matthias
On 4/16/24 9:42 AM, Gabriel Giussi wrote:
I have logic in my service to capture exceptions being thrown during
message processing and produce
[
https://issues.apache.org/jira/browse/KAFKA-16585?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838730#comment-17838730
]
Matthias J. Sax commented on KAFKA-16585:
-
Thanks for raising this ticket. Wondering how we
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838728#comment-17838728
]
Matthias J. Sax commented on KAFKA-16567:
-
Why is this ticket marked as "blocker&quo
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16567:
Component/s: streams
> Add New Stream Metrics based on KIP-
[
https://issues.apache.org/jira/browse/KAFKA-16567?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16567:
Labels: kip (was: )
> Add New Stream Metrics based on KIP-
[
https://issues.apache.org/jira/browse/KAFKA-16263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838727#comment-17838727
]
Matthias J. Sax commented on KAFKA-16263:
-
Yes, these are the handlers this ticket refers do
[
https://issues.apache.org/jira/browse/KAFKA-16336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838724#comment-17838724
]
Matthias J. Sax commented on KAFKA-16336:
-
The next planned release is 3.8, but we can work
Hi,
I am actually not sure if using `Record` is really the right thing?
While `Record` is technically public API, it does not seem to be
intended to be exposed to end users?
But I guess my main question is really about what metadata we really
want to add to `RecordDeserializationException`?
Congrats Greg!
On 4/15/24 10:44 AM, Hector Geraldino (BLOOMBERG/ 919 3RD A) wrote:
Congrats! Well deserved
From: d...@kafka.apache.org At: 04/13/24 14:42:22 UTC-4:00To:
d...@kafka.apache.org
Subject: [ANNOUNCE] New Kafka PMC Member: Greg Harris
Hi all,
Greg Harris has been a Kafka
Congrats Greg!
On 4/15/24 10:44 AM, Hector Geraldino (BLOOMBERG/ 919 3RD A) wrote:
Congrats! Well deserved
From: dev@kafka.apache.org At: 04/13/24 14:42:22 UTC-4:00To:
dev@kafka.apache.org
Subject: [ANNOUNCE] New Kafka PMC Member: Greg Harris
Hi all,
Greg Harris has been a Kafka committer
Matthias J. Sax created KAFKA-16575:
---
Summary: Automatically remove KTable aggregation result when group
becomes empty
Key: KAFKA-16575
URL: https://issues.apache.org/jira/browse/KAFKA-16575
Matthias J. Sax created KAFKA-16575:
---
Summary: Automatically remove KTable aggregation result when group
becomes empty
Key: KAFKA-16575
URL: https://issues.apache.org/jira/browse/KAFKA-16575
[
https://issues.apache.org/jira/browse/KAFKA-16514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17836671#comment-17836671
]
Matthias J. Sax commented on KAFKA-16514:
-
CloseOption was introduced via
[https
Thanks for the KIP. Great discussion.
I am not sure if I understand the proposal from Bruno to hand in the
processor node id? Isn't this internal (could not even find it quickly).
We do have a processor name, right? Or do I mix up something?
Another question is about `ProcessingContext` --
Thanks for the KIP Fred.
Couple of nits: it's not clear from the "Public API" section what is new
and what is existing API w/o going back to the code. For existing
methods which are not changed, it's also best to actually omit them. --
It would also be best to only put the interface itself
Matthias J. Sax created KAFKA-16508:
---
Summary: Infinte loop if output topic does not exisit
Key: KAFKA-16508
URL: https://issues.apache.org/jira/browse/KAFKA-16508
Project: Kafka
Issue
Matthias J. Sax created KAFKA-16508:
---
Summary: Infinte loop if output topic does not exisit
Key: KAFKA-16508
URL: https://issues.apache.org/jira/browse/KAFKA-16508
Project: Kafka
Issue
[
https://issues.apache.org/jira/browse/KAFKA-16505?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16505:
Component/s: streams
> KIP-1034: Dead letter queue in Kafka Stre
[
https://issues.apache.org/jira/browse/KAFKA-16505?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16505:
Labels: KIP (was: )
> KIP-1034: Dead letter queue in Kafka Stre
[
https://issues.apache.org/jira/browse/KAFKA-16502?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Matthias J. Sax updated KAFKA-16502:
Component/s: streams
unit tests
> Fix fl
1 - 100 of 11111 matches
Mail list logo