ter of the associated Jira ticket
https://issues.apache.org/jira/browse/KAFKA-6690
<https://issues.apache.org/jira/browse/KAFKA-6690>
has identified as a use-case he frequently encounters. I’ve asked him to
elaborate on the dev list though he has not responded yet.
Best,
--
Nick
event rate is high from web. A
solution to handle the starvation with a timeout after which the events are
consumed normally for a specified period of time would be great and help us use
our resources effectively.
--
Nick
> On Oct 18, 2018, at 12:23 PM, n...@afshartous.com wr
tization to make the API more expressive
and to make it easier for developers to do this.
Thanks Colin for your vote on the KIP and for all you input. I look forward to
hearing from others.
Cheers,
--
Nick
Hi All,
Bumping this thread for more votes
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics>
Cheers,
--
Nick
Bumping again for more votes.
--
Nick
> On Dec 26, 2018, at 12:36 PM, n...@afshartous.com wrote:
>
> Bumping this thread for more votes
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics
>
> <https://cwiki.ap
that it makes sense not to try and address starvation to Mathias’
point that this is intended by design. The KIP has been updated to reflect
this by removing the second method.
Regarding incremental fetch, Colin do you have any suggestion on which option
to adopt or how to proceed ?
--
Nick
Hi Jan,
As discussed, I’ve adopted the position that MessageChooser is orthogonal to
topic prioritization and hence outside the scope of KIP-349.
--
Nick
> On Jan 14, 2019, at 12:47 AM, Jan Filipiak wrote:
>
> On 14.01.2019 02:48, n...@afshartous.com <mailto:n...@afshartou
cases and
very positive feedback. Please review.
https://lists.apache.org/list.html?us...@kafka.apache.org
<https://lists.apache.org/list.html?us...@kafka.apache.org>
At this point I feel like we have enough info and would like to try and work
towards a vote or set the status of the K
Fetcher.fetchablePartitions.
Thanks for any input.
Regards,
--
Nick
ent from streams to
consumer.
--
Nick
Hi All,
Calling for a vote on KIP-349
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics>
Cheers,
--
Nick
it immediately return the high priority records
> without pulling low priority records?
My own interpretation would be to read from all the topics in priority order as
the consumer is subscribed to multiple topics.
--
Nick
>
>
> On Wed, Aug 8, 2018 at 8:39 PM <mailt
Does this clarify ?
--
Nick
> On Aug 9, 2018, at 7:44 PM, n...@afshartous.com wrote:
>
> Since there are questions I changed the heading from VOTE to DISCUSS
>
>> On Aug 8, 2018, at 9:09 PM, Matt Farmer mailto:m...@frmr.me>>
>> wrote:
>>
>> s
Hi All,
Calling for a vote on KIP-349
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics
--
Nick
I only saw one vote on KIP-349, just checking to see if anyone else would like
to vote before closing this out.
--
Nick
> On Aug 13, 2018, at 9:19 PM, n...@afshartous.com wrote:
>
>
> Hi All,
>
> Calling for a vote on KIP-349
>
> https://cwiki.apache.org/
s Jan for your input.
Here’s an attempt at incorporating a Samza MessageChooser type interface.
--
Nick
New interface TopicPrioritizer allows one to create a method implementation
that prioritizes topics. The topic priorities that were assigned with method
KafkaConsumer.subscribe may or may not
Just clarifying that the API below would be in addition to the API specified in
KIP-349
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics
--
Nick
> On Aug 30, 2018, at 9:57 AM, n...@afshartous.com wrote:
>
> Here’s an a
@Jan - can you comment on whether or not this is what you had in mind ?
--
Nick
> On Aug 30, 2018, at 10:18 AM, n...@afshartous.com wrote:
>
>
> Just clarifying that the API below would be in addition to the API specified
> in KIP-349
>
>
> https://cwi
rface has method
/* Notify the chooser that a new envelope is available for a processing. */
void update(IncomingMessageEnvelope envelope)
and I’m wondering how this method would be translated to Kafka API. In
particular what corresponds to IncomingMessageEnvelope.
Best,
--
Nick
.
Cheers,
--
Nick
hooser.
Cheers,
--
Nick
;https://lists.apache.org/list.html?dev@kafka.apache.org:lte=1M:kip-349>
to the KIP-349 page
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics>
Best,
--
Nick
s and performance would be
unaffected.
Best,
--
Nick
one by the caller. Its certainly doable, though it
seems like those who are requesting topic prioritization are asking for a more
expressive consumer API. My $0.02.
Best,
--
Nick
...@kafka.apache.org
> <https://lists.apache.org/list.html?us...@kafka.apache.org>
>
> At this point I feel like we have enough info and would like to try and work
> towards a vote or set the status of the KIP to dormant.
>
Hi Colin,
Just bumping this thread to see if you’ve had a chance to review the use-cases
on the thread on the user’s list.
Cheers,
--
Nick
of this KIP to dormant until if
and when someone else picks up it up.
Does anybody else have input on either having a vote or setting the KIP dormant
?
Cheers,
--
Nick
Hi Sönke,
Thanks for taking the time to review. I’ve put KIP-349 into hibernation.
Thanks also to everyone who participated in the discussion.
Best regards,
--
Nick
> On Jan 25, 2019, at 5:51 AM, Sönke Liebau
> wrote:
>
> a bit late to the party, sorry. I recently spe
Hi Adam,
This change is only intended for the basic consumer API.
Cheers,
--
Nick
From: Adam Bellemare
Sent: Sunday, January 6, 2019 11:45 AM
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-349 Priorities for Source Topics
Hi Nick
Is this change
Hi all,
Requesting permission to create a KIP in regards to
KAFKA-6690 Priorities for Source Topics
My Wiki ID is nafshartous.
Cheers,
--
Nick
Hi All,
This message is to initiate discussion on a feature to add optional priorities
to source topics. Please review
https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics
Cheers,
--
Nick
GitHub user nick-zh opened a pull request:
https://github.com/apache/kafka/pull/4249
fix typo in ProducerConfig doc
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/nick-zh/kafka trunk
Alternatively you can review and apply
I didn't get any hits on this on users@, so I'm forwarding this to dev@.
Any explanation for this would be greatly appreciated!
-- Forwarded message ------
From: Nick Travers
Date: Sat, Feb 18, 2017 at 5:04 PM
Subject: Producer acks=1, clean broker shutdown and data l
ror about the offsets column family not being opened, which
triggers a wipe and rebuild of the Task. Given that downgrades should be
uncommon, I think this is acceptable behaviour, as the end-state is
consistent, even if it results in an undesirable state restore.
Should I document the upgrade/dow
to
the serialization in writeOffset, that should also enhance performance of
state commit in the normal write-path.
Please let me know if this addresses the issue!
Regards,
Nick
On Mon, 11 Sept 2023 at 05:38, Colt McNealy wrote:
> Howdy folks,
>
> First I wanted to say fantastic work and th
r most users.
If we're happy to do so, we can make ALOS also use transactions.
Regards,
Nick
Link 1:
https://github.com/adamretter/rocksjava-write-methods-benchmark#results
On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna wrote:
> Hi Nick,
>
> Thanks for the updates and sorry for the delay on m
tionLevels, even
if the StateStore has many layers of wrappers (as is the case at the point
IQv1 deals with the store). Would this be acceptable, or do you have
another approach in mind?
Regards,
Nick
On Wed, 13 Sept 2023 at 10:57, Nick Telford wrote:
> Hi Bruno,
>
> Thanks for gettin
active Queries since the last commit.
I realise this is getting into the weeds of the implementation, and you'd
rather we focus on the API for now, but I think it's important to consider
how to implement the desired API, in case we come up with an API that
cannot be implemented efficient
IQ
Iterators in the same way that RocksDB WriteBatches do.
--
Nick
On Wed, 13 Sept 2023 at 16:58, Nick Telford wrote:
> Hi Bruno,
>
> I've updated the KIP based on our conversation. The only things I've not
> yet done are:
>
> 1. Using transactions under ALOS and EOS.
his best, but I don't mind
changing it if the nomenclature is unclear.
Sorry for the massive emails/essays!
--
Nick
1: https://github.com/facebook/rocksdb/wiki/Transactions
2: https://github.com/facebook/rocksdb/wiki/Snapshot
3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
Oh! One other concern I haven't mentioned: if we make IsolationLevel a
query-time constraint, then we need to add support for READ_COMMITTED to
InMemoryKeyValueStore too, which will require some changes to the
implementation.
On Mon, 18 Sept 2023 at 17:24, Nick Telford wrote:
> Hi
switch
to READ_COMMITTED when processing.mode is anything other than
at-least-once. Do you think this would be acceptable?
In a later KIP, we can add support for query-time isolation levels and
solve this particular problem there, which would relax this restriction.
Regards,
Nick
On Tue, 19 Sep
ce/display/KAFKA/KIP-989%3A+RocksDB+Iterator+Metrics
P.S. I'm not too sure about the formatting of the "New Metrics" table, any
advice there would be appreciated.
Regards,
Nick
new code. This pattern is repeated
throughout RocksDBStore, wherever a new RocksDbIterator is created.
Regards,
Nick
On Thu, 5 Oct 2023 at 12:32, Colt McNealy wrote:
> Thank you for the KIP, Nick!
>
> This would be highly useful for many reasons. Much more sane than checking
&g
s when the FAIL
DeserializationHandlerResponse is used.
Let me know what you think!
Regards,
Nick
this KIP over the line ASAP, I settled on
adding the restriction that READ_UNCOMMITTED would be unavailable under
EOS, with the goal of relaxing this in a future KIP.
If it turns out that this restriction is a blocker, then I'll try to find
the time to explore the possibility of adding a flag.
Regar
D should not be supported?
Regards,
Nick
On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna wrote:
> Hi Nick,
>
> I think the KIP is converging!
>
>
> 1.
> I am wondering whether it makes sense to write the position file during
> close as we do for the checkpoint file, so tha
Hi Bruno,
4.
I'll hold off on making that change until we have a consensus as to what
configuration to use to control all of this, as it'll be affected by the
decision on EOS isolation levels.
5.
Done. I've chosen "committedOffsets".
Regards,
Nick
On Fri, 13 Oct 202
further delay
this KIP, which was the reason I placed the restriction in the KIP in the
first place.
Regards,
Nick
On Sat, 14 Oct 2023 at 03:30, Guozhang Wang
wrote:
> Hello Nick,
>
> First of all, thanks a lot for the great effort you've put in driving
> this KIP! I really like it
ky Iterator, it should manifest as a
persistently climbing "open-iterators" metric, even on a busy node, because
each time that Iterator is used, it will leak another one. So even in the
presence of many non-leaky Iterators on a busy instance, the metric should
still consistently climb.
Regar
re what we should do here: retrying the commitTransaction seems
logical, but what if it times out again? Where do we draw the line and
shutdown the instance?
Regards,
Nick
On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy
wrote:
> Hi all,
>
> I think I liked your suggestion of allowing EOS
Hi Lucas,
TaskCorruptedException is how Streams signals that the Task state needs to
be wiped, so we can't retain that exception without also wiping state on
timeouts.
Regards,
Nick
On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy
wrote:
> Hi Nick,
>
> I think indeed the better beh
dex blocks)
- block-cache-filter-usage (number of bytes occupied by filter blocks)
Regards,
Nick
On Tue, 24 Oct 2023 at 07:09, Sophie Blee-Goldman
wrote:
> I actually think we could implement Lucas' suggestion pretty easily and
> without too much additional effort. We have full contr
know if they're safe to
not pause.
4c.
Regardless, I like this idea, but I have very little knowledge about making
changes to the rebalance/network protocol. It looks like this could be
added via StreamsPartitionAssignor#subscriptionUserData? I might need some
help designing this aspect of this KI
e constructor, but it looks like configs of
type Class cannot accept an already instantiated instance, and there's no
other way to inject information like that.
Perhaps we could add something to ProcessorContext that contains details on
the sub-topology being executed?
Regards,
Nick
On T
Hi everyone,
I'd like to call a vote for KIP-892: Transactional StateStores[1], which
makes Kafka Streams StateStores transactional under EOS.
Regards,
Nick
1:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
Hi everyone,
With +3 binding votes (and +1 non-binding), the vote passes.
KIP-892 Transactional StateStores is Adopted!
Regards,
Nick
On Tue, 14 Nov 2023 at 09:56, Bruno Cadonna wrote:
> Hi Nick!
>
> Thanks a lot for the KIP!
>
> Looking forward to the implementation!
Hi Stan,
I'd like to propose including KIP-892 in the 3.7 release. The KIP has been
accepted and I'm just working on rebasing the implementation against trunk
before I open a PR.
Regards,
Nick
On Tue, 21 Nov 2023 at 11:27, Mayank Shekhar Narula <
mayanks.nar...@gmail.com> w
Hi everyone,
I'd like to call a vote on the Kafka Streams KIP-989: RocksDB Iterator
Metrics:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-989%3A+RocksDB+Iterator+Metrics
All of the points in the discussion thread have now been addressed.
Regards,
Nick
guarantee that committed
offsets are available when the store is open.
I'll investigate this approach this week for feasibility and report back.
I think that covers all the outstanding feedback, unless I missed anything?
Regards,
Nick
On Mon, 6 May 2024 at 14:06, Bruno Cadonna wrote:
>
Woops! Thanks for the catch Lucas. Given this was just a typo, I don't
think this affects the voting.
Cheers,
Nick
On Tue, 14 May 2024 at 18:06, Lucas Brutschy
wrote:
> Hi Nick,
>
> you are still referring to oldest-open-iterator-age-ms in the
> `Proposed Changes` section.
&g
90k6s
Regards,
Nick
On Wed, 15 May 2024 at 17:47, Josep Prat
wrote:
> And my maths are wrong! I added 24 hours more to all the numbers in there.
> If after 72 hours no vetoes appear, I have no objections on adding this
> specific KIP as it shouldn't have a big blast radius of af
Good point! I've updated it to "Improved StateStore Iterator metrics for
detecting leaks" - let me know if you have a better suggestion.
This should affect the voting imo, as nothing of substance has changed.
Regards,
Nick
On Thu, 16 May 2024 at 01:39, Sophie Blee-Goldman
wrot
/current/streams/monitoring.html#state-store-metrics
On Thu, 16 May 2024 at 12:15, Nick Telford wrote:
> Good point! I've updated it to "Improved StateStore Iterator metrics for
> detecting leaks" - let me know if you have a better suggestion.
>
> This should affect th
Hi everyone,
With 3 binding votes and no objections, the vote passes.
KIP-989 is adopted.
Cheers,
Nick
On Wed, 15 May 2024 at 03:41, Sophie Blee-Goldman
wrote:
> +1 (binding)
>
> Thanks!
>
> On Tue, May 14, 2024 at 6:58 PM Matthias J. Sax wrote:
>
> > +1 (binding)
&g
Oh shoot, you're right. I miscounted.
The vote remains open.
On Thu, 16 May 2024, 20:11 Josep Prat, wrote:
> Hi Nick,
> I think you need one more day to reach the 72 hours. You opened the vote on
> the 14th, right?
>
> Best,
>
>
>
> Jose
IP later today to reflect this design, but I will try to
keep it high-level, so that the exact implementation can vary.
Regards,
Nick
On Thu, 16 May 2024 at 03:12, Sophie Blee-Goldman
wrote:
> 103: I like the idea of immediately deprecating #managesOffsets and aiming
> to make offset man
Hi everyone,
Sorry I haven't got around to updating the KIP yet. Now that I've wrapped
up KIP-989, I'm going to be working on 1035 starting today.
I'll update the KIP first, and then call a vote.
Regards,
Nick
On Wed, 29 May 2024 at 07:25, Bruno Cadonna wrote:
>
e this covers all the outstanding changes that were requested.
Please let me know if I've missed anything or you think further changes are
needed.
Regards,
Nick
On Wed, 29 May 2024 at 09:28, Nick Telford wrote:
> Hi everyone,
>
> Sorry I haven't got around to updating the KIP yet.
) -> numCommittedEntryStored()
flushedEntryRemoved(K) -> committedEntryRemoved(K)
flushedEntryStored(K) -> committedEntryStored(K)
The old methods will obviously be marked as @Deprecated.
Any objections before I add this to the KIP?
Regards,
Nick
On Wed, 29 May 2024 at 11:20, Nick Telford wr
e KIP has been accepted *and* I'm close enough to
completion that we can guarantee getting it all done by the next release.
--
Cheers,
Nick
On Tue, 4 Jun 2024 at 20:34, Matthias J. Sax wrote:
> Nick,
>
> Thanks a lot for updating the KIP. I made a pass over it. Overall LGT
Hi everyone,
I'd like to call a vote on KIP-1035[1].
Regards,
Nick
1:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1035%3A+StateStore+managed+changelog+offsets
xception without the
configured default.deserialization.exception.handler being invoked.
I think this would better align these with JVM norms, e.g.
Thread#setUncaughtExceptionHandler, which behaves the same (albeit
configured through code).
Regards,
Nick
On Thu, 13 Jun 2024 at 10:22, Muralidhar Basa
Hi everyone,
By my count, it passes with 4 binding +1s and no -1s.
Thanks for your votes, review and discussion. I'll update the KIP and start
opening PRs!
Regards,
Nick
On Wed, 19 Jun 2024 at 09:16, Lucas Brutschy
wrote:
> Thanks, Nick!
>
> +1 (binding)
>
> On Wed, Jun
brokers?
NT4.
It's implied that clients send the changelog offsets for *all* dormant
stateful Tasks, but the current behaviour is that clients will only send
the changelog offsets for the stateful Tasks that they are able to lock
on-disk. Since this is a change in behaviour, perhaps this should
s is not correct, and that the new assignor will make
the same assumptions as the old one?
Regards,
Nick
On Fri, 16 Aug 2024 at 10:17, Lucas Brutschy
wrote:
> Hi Nick!
>
> Thanks for getting involved in the discussion.
>
> NT1. We are always referring to offsets in the changelog to
ients to continue to supply the lags
for only a subset of the dormant Tasks on-disk? Wouldn't it be simpler to
just have them supply everything, since the assignor has to handle
overlapping sets anyway?
Cheers,
Nick
On Fri, 16 Aug 2024 at 13:51, Lucas Brutschy
wrote:
> Hi Nick,
>
> NT4.
butors to benefit from it locally.
Regards,
Nick
On Tue, 2 Jan 2024 at 13:00, Lucas Brutschy
wrote:
> Thanks for all the work that has already been done on this in the past
> days!
>
> Have we considered running our test suite with
> -XX:+HeapDumpOnOutOfMemoryError and upl
Addendum: I've opened a PR with what I believe are the changes necessary to
enable Remote Build Caching, if you choose to go that route:
https://github.com/apache/kafka/pull/15109
On Tue, 2 Jan 2024 at 14:31, Nick Telford wrote:
> Hi everyone,
>
> Regarding building a &qu
practice I think few users
have monolithic apps like ours, and most instead de-couple unrelated topics
via different apps, which reduces the impact of incompatible upstream
schema changes.
Thanks for your reviews and feedback, I've learned a lot, as always; this
time, mostly about how, when a
menting it separately for each
> specific iterator implementation for every store type.
Sophie, does MeteredKeyValueIterator fit with your recommendation?
Thanks for your thoughts everyone, I'll update the KIP now.
Nick
On Thu, 14 Mar 2024 at 03:37, Sophie Blee-Goldman
wrote:
> About your l
Quick addendum:
My suggested metric "oldest-open-iterator-age-seconds" should be
"oldest-open-iterator-age-ms". Milliseconds is obviously a better
granularity for such a metric.
Still accepting suggestions for a better name.
On Thu, 28 Mar 2024 at 13:41, Nick Telford w
treams.
Let me know what you think,
Nick
On Tue, 15 Feb 2022 at 16:23, Nick Telford wrote:
> In the KIP, for Option A I suggested a new path of:
>
> /state/dir/stores//
>
> I made the mistake of thinking that the rocksdb/ segment goes *after* the
> store name in the current schem
he KIP says it's a store-level metric, but I think it would be
> good to say explicitly that it's recorded with DEBUG level only?
Yes, I've already updated the KIP with this information in the table.
Regards,
Nick
On Sun, 31 Mar 2024 at 10:53, Matthias J. Sax wrote:
> Th
hard dependency on KIP-1035 has been added to KIP-892 in their place.
I'm hopeful that with some more focus on this set of changes, we can
deliver something that we're all happy with.
Regards,
Nick
eManager (aka.
ProcessorStateManager/GlobalStateManager) proves too expensive to hold open
for closed stores, we could always have a "StubStateManager" in its place,
that enables the querying of offsets, but nothing else?
IDK, what do you think?
Regards,
Nick
On Tue, 9 Apr 2024 at 15:00,
devil will be in
the detail.
I'll try to find some time to explore the idea to see if it's possible and
report back, because we'll need to determine this before we can vote on the
KIP.
Regards,
Nick
On Wed, 10 Apr 2024 at 11:36, Bruno Cadonna wrote:
> Hi Nick,
>
> Thanks f
, but it looks like we'll have to
live with it.
Unless you have any better ideas?
Regards,
Nick
On Wed, 10 Apr 2024 at 14:12, Nick Telford wrote:
> Hi Bruno,
>
> Immediately after I sent my response, I looked at the codebase and came to
> the same conclusion. If it's possibl
nds before 1033?
Regards,
Nick
On Fri, 12 Apr 2024 at 11:38, Damien Gasparina
wrote:
> In a general way, if the user does not configure the right ACL, that
> would be a security issue, but that's true for any topic.
>
> This KIP allows users to configure a Dead Letter Queue w
essed. I'd like to see
a section that considers these consequences, and perhaps make those risks
clear to users. For the record, this is exactly what sunk KIP-990, which
was an alternative approach to error handling that introduced the same
issues.
Cheers,
Nick
On Fri, 12 Apr 2024 at 1
can be included? In particular, I'm
concerned with "schema pointer" headers (like those set by Schema
Registry), that may need to be propagated, especially if the records are
fed back into the source topics for re-processing by the user.
Regards,
Nick
On Fri, 12 Apr 2024 at 13:20, Dami
Hi Sophie,
Interesting idea! Although what would that mean for the StateStore
interface? Obviously we can't require that the constructor take the TaskId.
Is it enough to add the parameter to the StoreSupplier?
Would doing this be in-scope for this KIP, or are we over-complicating it?
Nic
gt; }
>
> And of course add the TaskId parameter to each of the actual
> state store constructors returned here.
>
> Does that make sense? It's entirely possible I'm missing something
> important here, but I think this would be a pretty small addition that
Hi Walker,
Feel free to ask away, either on the mailing list of the Confluent
Community Slack, where I hang out :-)
The implementation is *mostly* complete, although it needs some polishing.
It's worth noting that KIP-1035 is a hard prerequisite for this.
Regards,
Nick
ts to revert to a previous number, and taking the max
would incorrectly assume the older offsets are correct.
Regards,
Nick
On Mon, 19 Aug 2024 at 15:00, Lucas Brutschy
wrote:
> Hi Nick,
>
> NT4: As discussed, we will still require locking in the new protocol
> to avoid concurrent read/wr
ether, it should be trivial to ensure that the provided argument is an
ancestor of the current node, by recursively traversing up the process
graph.
I hope this clarifies your questions. It's clear that the KIP needs more
work to better elaborate on these points. I haven't had a cha
itioning" in my head, as I
don't think it's quite right. It may turn out that the additional overload
(with the Produced argument) is not necessary.
Thanks for all your feedback so far. Let me know what you think!
Regards,
Nick
On Thu, 25 Aug 2022 at 17:46, Nick Telford wrote:
- The "recursively" method requires significantly less code than
recursion via an explicit topic, and is significantly easier to understand.
Ultimately, I don't think repartitioning inside the unary operator adds
much complexity to the implementation. Certainly no more than o
her before,
after or inside the "recursively" method. I can't see a scenario where the
recursion would cause problems with it.
Nick
On Tue, 6 Sept 2022 at 18:08, Nick Telford wrote:
> Hi Guozhang,
>
> I mentioned this in the "Rejected Alternatives" section. Reparti
by the restoration rate).
Regards,
Nick
On Mon, 19 Sept 2022 at 19:57, Guozhang Wang wrote:
> Hello Bruno,
>
> Thanks for your comments!
>
> 1. Regarding the metrics group name: originally I put
> "stream-state-metrics" as it's related to state store restorations
gards,
Nick
On Thu, 1 Sept 2022 at 12:16, Alexander Sorokoumov
wrote:
> Hey Guozhang,
>
> Sounds good. I annotated all added StateStore methods (commit, recover,
> transactional) with @Evolving.
>
> Best,
> Alex
>
>
>
> On Wed, Aug 31, 2022 at 7:32 PM
1 - 100 of 201 matches
Mail list logo