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
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.
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
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
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
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,
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
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
) -> 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 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.
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:
>
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
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
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
/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
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
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
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
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:
>
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
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
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 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
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
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
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
, 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
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
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,
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
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
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
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
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
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
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
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
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,
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 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
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
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
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
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
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
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
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
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
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
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
s when the FAIL
DeserializationHandlerResponse is used.
Let me know what you think!
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
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
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
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
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
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.
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
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
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
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
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
already grown quite
large!
On Fri, 21 Jul 2023 at 21:33, Nick Telford wrote:
> Hi everyone,
>
> I've updated the KIP (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores)
> with the latest changes; mostly bringing back &quo
e either of these, so my primary test environment
doesn't test these areas.
I'm going on Parental Leave starting next week for a few weeks, so will not
have time to move this forward until late August. That said, your feedback
is welcome and appreciated, I just won't be able to respo
xity is drastically
reduced, I may try bringing back Atomic Checkpoints into this KIP.
Regards,
Nick
On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna wrote:
> Hi Nick,
>
> Thanks for the insights! Very interesting!
>
> As far as I understand, you want to atomically update the state store
>
ommit
I'm also open to suggestions.
Regards,
Nick
On Thu, 22 Jun 2023 at 14:06, Nick Telford wrote:
> Hi Bruno!
>
> 3.
> By "less predictable for users", I meant in terms of understanding the
> performance profile under various circumstances. The more complex the
&
ising, but it will
need to undergo some performance testing. I haven't (yet) worked on
removing the `newTransaction()` stuff, but I would expect that,
behaviourally, it should make no difference. The branch is available at
https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is interes
I was
overly alarmist in my phrasing.
My concern is only that, while the concept seems quite reasonable, there
are no doubt hidden issues lurking.
On Wed, 21 Jun 2023 at 18:25, John Roesler wrote:
> Thanks Nick,
>
> That sounds good to me.
>
> I can't let (2) slide, though..
anagement
outlined by John, then we can do so with little/no API changes.
Regards,
Nick
llow-up KIP, but that we keep it in mind during the design of
this one.
What do you think?
Regards,
Nick
On Tue, 20 Jun 2023, 22:26 John Roesler, wrote:
> Oh, that's a good point.
>
> On the topic of a behavioral switch for disabled caches, the typical use
> case for disabling th
probably not be
> performant, but benchmarking may prove different. Or maybe we can have
> some reserved cache space on top of the user-configured cache, which we
> would have reclaimed from the memtable space. Or some other, more
> creative solution.
>
> Thanks,
> -John
>
&g
ds be read by joins?
On Tue, 20 Jun 2023, 20:51 John Roesler, wrote:
> Ah, sorry Nick,
>
> I just meant the regular heap based cache that we maintain in Streams. I
> see that it's not called "RecordCache" (my mistake).
>
> The actual cache is ThreadCache:
>
> ht
,
due to an error writing to the changelog (e.g. a changelog transaction
rollback or a timeout). This is something that KIP-892 aims to improve on,
as the current design would ensure that records are only observed by IQ
*after* they have been committed to the Kafka changelog.
That said, it definitely sounds *feasible*.
Regards,
Nick
Hi John,
By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find any
class called "RecordCache"...
Cheers,
Nick
On Tue, 20 Jun 2023 at 19:42, John Roesler wrote:
> Hi Nick,
>
> Thanks for picking this up again!
>
> I did have
ou mean failing over an Active Task to
an instance already hosting a Standby Task?
Thanks again and sorry for the essay of a response!
Regards,
Nick
On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna wrote:
> Hi Nick,
>
> Thanks for the updates!
>
> I really appreciate that you simplified
Hi everyone,
Quick update: I've added a new section to the KIP: "Offsets for Consumer
Rebalances", that outlines my solution to the problem that
StreamsPartitionAssignor needs to read StateStore offsets even if they're
not currently open.
Regards,
Nick
On Wed, 3 May 2023 at
ing
support for transactionality to VersionedStateStores. I will probably need
to revise my implementation for transactional "segmented" stores, both to
accommodate VersionedStateStore, and to clean up some other stuff.
Regards,
Nick
On Tue, 2 May 2023 at 13:45, Bruno Cadonna wrote:
> Hi Nick,
&g
. The downside
is that we wouldn't be able to remove the explicit RocksDB flush on-commit,
which likely hurts performance.
If anyone has any thoughts or ideas on this subject, I would appreciate it!
Regards,
Nick
On Wed, 19 Apr 2023 at 15:05, Nick Telford wrote:
> Hi Colt,
>
&g
fsets, restoring the small amount of data that hadn't been flushed when
the app exited/crashed.
Regards,
Nick
On Wed, 19 Apr 2023 at 14:35, Colt McNealy wrote:
> Nick,
>
> Thanks for your reply. Ack to A) and B).
>
> For item C), I see what you're referring to. Your pro
ta they checkpoint allows us
to write the checkpoint offsets *on every commit*, not just on clean
shutdown.
Regards,
Nick
On Tue, 18 Apr 2023 at 15:39, Colt McNealy wrote:
> Nick,
>
> Thank you for continuing this work. I have a few minor clarifying
> questions.
>
>
a bunch of interface changes relating to
Atomic Checkpointing, which is the final piece of the puzzle to making
everything robust.
Let me know what you think!
Regards,
Nick
On Tue, 3 Jan 2023 at 11:33, Nick Telford wrote:
> Hi Lucas,
>
> Thanks for looking over my KIP.
>
> A)
batched writes in RocksDB performing
better than non-batched writes.[1]
Regards,
Nick
1: https://github.com/adamretter/rocksjava-write-methods-benchmark#results
On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy
wrote:
> Hi Nick,
>
> I'm just starting to read up on the whole di
ade in the
thread.
Please let me know if there are areas that are unclear or need more
clarification.
I have a (nearly) working implementation. I'm confident that the remaining
work (making Segments behave) will not impact the documented design.
Regards,
Nick
On Tue, 6 Dec 2022 at 19:24, Colt
ore. I think I'm getting
close, although sadly I won't be able to progress much until next week due
to some work commitments.
Regards,
Nick
On Thu, 1 Dec 2022 at 00:01, Colt McNealy wrote:
> Nick,
>
> Thank you for the explanation, and also for the updated KIP. I am quite
>
rns around
concurrency, especially in the presence of Iterators. I'm thinking of
wrapping WriteBatchWithIndex with a reference-counting copy-on-write
implementation (that only makes a copy if there's an active iterator), but
I'm open to suggestions.
Regards,
Nick
On Mon, 28 Nov 20
re is the
major factor, with the 2 extra memory copies per-Record contributing a
significant amount too.
Regards,
Nick
On Mon, 28 Nov 2022 at 16:12, Colt McNealy wrote:
> Hi all,
>
> Out of curiosity, why does the performance of the store degrade so
> significantly with the 844
dy
be explicitly configuring this for their purposes.
I think a further optimization for ALOS to only commit the StateStores when
exceeding this limit is reasonable, to preserve the user's desired
commit.interval.ms as much as possible.
On Mon, 28 Nov 2022 at 15:55, Nick Telford wrote:
>
end on the processing.mode; under ALOS it would allow more
frequently committing stores, whereas under EOS it couldn't.
Any better ideas?
On Wed, 23 Nov 2022 at 16:25, Nick Telford wrote:
> Hi Alex,
>
> Thanks for the feedback.
>
> I've updated the discussion of OOM issues
-benchmark#results
* Just kidding, no prizes, sadly.
On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
wrote:
> Hey Nick,
>
> Thank you for the KIP! With such a significant performance degradation in
> the secondary store approach, we should definitely consider
> WriteBatchWithIn
consistent with
the records that are read from the database.
Regards,
Nick
On Tue, 22 Nov 2022 at 16:25, John Roesler wrote:
> Thanks for publishing this alternative, Nick!
>
> The benchmark you mentioned in the KIP-844 discussion seems like a
> compelling reason to revisit the b
let me know if you'd like me to push them for review in advance
of a vote.
Regards,
Nick
.
Regards,
Nick
On Mon, 21 Nov 2022 at 11:25, Alexander Sorokoumov
wrote:
> Hey Nick,
>
> Thank you for the prototype testing and benchmarking, and sorry for the
> late reply!
>
> I agree that it is worth revisiting the WriteBatchWithIndex approach. I
> will implement a for
.
Regards,
Nick
org.apache.kafka.streams.processor.internals package) in a KIP, or are they
considered implementation details?
Cheers,
Nick
On Sat, 12 Nov 2022 at 03:59, John Roesler wrote:
> Hi all,
>
> Just to clarify: there actually is a position file. It was a small detail
> of the IQv2 implementati
s the Task when this size exceeds a
configured threshold.
Thanks for your time, and let me know what you think!
--
Nick
1: https://rocksdb.org/blog/2015/02/27/write-batch-with-index.html
On Thu, 6 Oct 2022 at 19:31, Alexander Sorokoumov
wrote:
> Hey Nick,
>
> It is going to be
re a store, and b) which offsets to
restore from?
If I've understood the above correctly, is there any scope to potentially
replace checkpoint files with StateStore#position()?
Regards,
Nick
Can't wait!
+1 (non-binding)
On Wed, 12 Oct 2022, 18:02 Guozhang Wang,
wrote:
> Hello all,
>
> I'd like to start a vote for the following KIP, aiming to improve Kafka
> Stream's restoration visibility via new metrics and callback methods:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/K
Hi Guozhang,
What you propose sounds good to me. Having the more detailed Task-level
metrics at DEBUG makes sense.
Regards,
Nick
read-level metrics, which correspond to a
StreamThread.
Perhaps the record counting metrics would be more useful as
task/store-level metrics? That way, they can be aggregated by users to
determine things like total records remaining to restore by store, across
the entire app etc.
Regards,
Nick
On T
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 197 matches
Mail list logo