[jira] [Resolved] (KAFKA-15823) NodeToControllerChannelManager: authentication error prevents controller update

2024-03-31 Thread Luke Chen (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-15823?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Luke Chen resolved KAFKA-15823.
---
Resolution: Fixed

> NodeToControllerChannelManager: authentication error prevents controller 
> update
> ---
>
> Key: KAFKA-15823
> URL: https://issues.apache.org/jira/browse/KAFKA-15823
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 3.6.0, 3.5.1
>Reporter: Gaurav Narula
>Priority: Major
> Fix For: 3.8.0
>
>
> NodeToControllerChannelManager caches the activeController address in an 
> AtomicReference which is updated when:
>  # activeController [has not been 
> set|https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala#L422]
>  # networkClient [disconnnects from the 
> controller|https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala#L395C7-L395C7]
>  # A node replies with 
> `[Errors.NOT_CONTROLLER|https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala#L408]`,
>  and
>  # When a controller changes from [Zk mode to Kraft 
> mode|https://github.com/apache/kafka/blob/832627fc78484fdc7c8d6da8a2d20e7691dbf882/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala#L325]
>  
> When running multiple Kafka clusters in a dynamic environment, there is a 
> chance that a controller's IP may get reassigned to another cluster's broker 
> when the controller is bounced. In this scenario, the requests from Node to 
> the Controller may fail with an AuthenticationException and are then retried 
> indefinitely. This causes the node to get stuck as the new controller's 
> information is never set.
>  
> A potential fix would be disconnect the network client and invoke 
> `updateControllerAddress(null)` as we do in the `Errors.NOT_CONTROLLER` case.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-16323) Failing test: fix testRemoteFetchExpiresPerSecMetric

2024-03-31 Thread Luke Chen (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-16323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Luke Chen resolved KAFKA-16323.
---
Fix Version/s: 3.8.0
   3.7.1
   Resolution: Fixed

> Failing test: fix testRemoteFetchExpiresPerSecMetric 
> -
>
> Key: KAFKA-16323
> URL: https://issues.apache.org/jira/browse/KAFKA-16323
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Reporter: Johnny Hsu
>Assignee: Johnny Hsu
>Priority: Major
>  Labels: test-failure
> Fix For: 3.8.0, 3.7.1
>
>
> Refer to 
> [https://ci-builds.apache.org/job/Kafka/job/kafka/job/trunk/2685/testReport/junit/kafka.server/ReplicaManagerTest/Build___JDK_21_and_Scala_2_13___testRemoteFetchExpiresPerSecMetric__/]
> This test is failing, and this ticket aims to address this 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-1007: Introduce Remote Storage Not Ready Exception

2024-03-31 Thread Kamal Chandraprakash
Hi all,

While testing the patch [1], realised that introducing a
new REMOTE_STORAGE_NOT_READY error-code
is not compatible with the consumer. Consumer does not retry the FETCH
request for all the retriable
exceptions [2] instead it retries only for specific error codes. Dropping
the KIP-1007

as
it is not compatible
with the older clients. Thanks!

[1]: https://github.com/apache/kafka/pull/14822
[2]:
https://sourcegraph.com/github.com/apache/kafka@trunk/-/blob/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java?L325

--
Kamal

On Fri, Jan 5, 2024 at 5:03 PM Divij Vaidya  wrote:

> Thank you for addressing my concerns Kamal. Though, instead of the KIP, I
> actually was suggesting to add it in JavaDoc so that someone looking at the
> exception is able to understand what it means. We can discuss that during
> the PR review though.
>
> The KIP looks good to me.
>
> --
> Divij Vaidya
>
>
>
> On Fri, Jan 5, 2024 at 10:44 AM Satish Duggana 
> wrote:
>
> > Thanks for the KIP Kamal, LGTM.
> >
> > On Tue, 26 Dec 2023 at 10:23, Kamal Chandraprakash
> >  wrote:
> > >
> > > Hi Divij,
> > >
> > > Thanks for reviewing the KIP! I've updated the KIP with the below
> > > documentation. Let me know if it needs to be changed:
> > >
> > > The consumer can read the local data as long as it knows the offset
> from
> > > where to fetch the data from.
> > > When there is no initial offset, the consumer decides the offset based
> on
> > > the below config:
> > >
> > > ```
> > > auto.offset.reset = earliest / latest / none
> > > ```
> > >
> > >- For `earliest` offset policy and any offset that lies in the
> remote
> > >storage, the consumer (FETCH request)
> > >cannot be able to make progress until the remote log metadata gets
> > >synced.
> > >- In a FETCH request, when there are multiple partitions where a
> > subset
> > >of them are consuming from local
> > >and others from remote, then only the partitions which are consuming
> > >from the remote cannot make
> > >progress and the partitions that fetch data from local storage
> should
> > be
> > >able to make progress.
> > >- In a FETCH request, when the fetch-offset for a partition is
> within
> > >the local-storage, then it should be able
> > >to consume the messages.
> > >- All the calls to LIST_OFFETS will fail until the remote log
> metadata
> > >gets synced.
> > >
> > >
> > > The code link that is mentioned is referring to the `LIST_OFFSETS`
> > handler.
> > > Usually, consumers don't use this API
> > > unless it's explicitly called by the user.
> > >
> > >
> > > On Fri, Dec 22, 2023 at 4:10 PM Divij Vaidya 
> > > wrote:
> > >
> > > > Thanks for the KIP, Kamal.
> > > >
> > > > The change looks good to me, though, I think we can do a better job
> at
> > > > documenting what the error means for the clients and users.
> > > >
> > > > Correct me if I'm wrong, when remote metadata is being synced on a
> new
> > > > leader, we cannot fetch even the local data (as per [1]), hence,
> > partition
> > > > is considered "unreadable" but writes (and all other operations such
> as
> > > > admin operations) can continue to work on that partition. If my
> > > > understanding is correct, perhaps, please clarify this in the error
> > > > description. In absence of it, it is difficult to determine what this
> > error
> > > > means for operations that can be performed on a partition.
> > > >
> > > > [1]
> > > >
> > > >
> >
> https://github.com/apache/kafka/blob/82808873cbf6a95611243c2e7984c4aa6ff2cfff/core/src/main/scala/kafka/log/UnifiedLog.scala#L1336
> > > >
> > > >
> > > > --
> > > > Divij Vaidya
> > > >
> > > >
> > > >
> > > > On Tue, Dec 12, 2023 at 9:58 AM Kamal Chandraprakash <
> > > > kamal.chandraprak...@gmail.com> wrote:
> > > >
> > > > > Thanks Luke for reviewing this KIP!
> > > > >
> > > > > If there are no more comments from others, I'll start the VOTE
> since
> > this
> > > > > is a minor KIP.
> > > > >
> > > > > On Mon, Dec 11, 2023 at 1:01 PM Luke Chen 
> wrote:
> > > > >
> > > > > > Hi Kamal,
> > > > > >
> > > > > > Thanks for the KIP!
> > > > > > LGTM.
> > > > > >
> > > > > > Thanks.
> > > > > > Luke
> > > > > >
> > > > > > On Wed, Nov 22, 2023 at 7:28 PM Kamal Chandraprakash <
> > > > > > kamal.chandraprak...@gmail.com> wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > >
> > > > > > > I would like to start a discussion to introduce a new error
> code
> > for
> > > > > > > retriable remote storage errors. Please take a look at the
> > proposal:
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1007%3A+Introduce+Remote+Storage+Not+Ready+Exception
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
>


Re: [VOTE] KIP-1007: Introduce Remote Storage Not Ready Exception

2024-03-31 Thread Kamal Chandraprakash
Hi all,

While testing the patch [1], realised that introducing a
new REMOTE_STORAGE_NOT_READY error-code
is not compatible with the consumer. Consumer does not retry the FETCH
request for all the retriable
exceptions [2] instead it retries only for specific error codes. Dropping
the KIP-1007

as it is not compatible
with the older clients. Thanks!

[1]: https://github.com/apache/kafka/pull/14822
[2]:
https://sourcegraph.com/github.com/apache/kafka@trunk/-/blob/clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchCollector.java?L325


On Sun, Jan 7, 2024 at 5:36 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Thank you for all the votes! I'm closing the vote thread as it is open for
> more than 72 hours.
> The KIP has been passed with 3 binding and 1 non-binding votes.
>
> --
> Kamal
>
>
>
> On Fri, Jan 5, 2024 at 5:51 PM Satish Duggana 
> wrote:
>
>> Thanks Kamal for the KIP.
>>
>> +1 (binding)
>>
>> On Fri, 5 Jan 2024 at 17:04, Divij Vaidya 
>> wrote:
>> >
>> > +1 (binding)
>> >
>> > --
>> > Divij Vaidya
>> >
>> >
>> >
>> > On Thu, Dec 21, 2023 at 10:30 AM Luke Chen  wrote:
>> >
>> > > Hi Kamal,
>> > >
>> > > Thanks for the KIP.
>> > > +1 (binding) from me.
>> > >
>> > > Luke
>> > >
>> > > On Thu, Dec 21, 2023 at 4:51 PM Christo Lolov > >
>> > > wrote:
>> > >
>> > > > Heya Kamal,
>> > > >
>> > > > The proposed change makes sense to me as it will be a more explicit
>> > > > behaviour than what Kafka does today - I am happy with it!
>> > > >
>> > > > +1 (non-binding) from me
>> > > >
>> > > > Best,
>> > > > Christo
>> > > >
>> > > > On Tue, 12 Dec 2023 at 09:01, Kamal Chandraprakash <
>> > > > kamal.chandraprak...@gmail.com> wrote:
>> > > >
>> > > > > Hi,
>> > > > >
>> > > > > I would like to call a vote for KIP-1007
>> > > > > <
>> > > > >
>> > > >
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1007%3A+Introduce+Remote+Storage+Not+Ready+Exception
>> > > > > >.
>> > > > > This KIP aims to introduce a new error code for retriable remote
>> > > storage
>> > > > > errors. Thanks to everyone who reviewed the KIP!
>> > > > >
>> > > > > --
>> > > > > Kamal
>> > > > >
>> > > >
>> > >
>>
>


Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-31 Thread Nick Telford
Hi Matthias,

> For the oldest iterator metric, I would propose something simple like
> `iterator-opened-ms` and it would just be the actual timestamp when the
> iterator was opened. I don't think we need to compute the actual age,
> but user can to this computation themselves?

That works for me; it's easier to implement like that :-D I'm a little
concerned that the name "iterator-opened-ms" may not be obvious enough
without reading the docs.

> If we think reporting the age instead of just the timestamp is better, I
> would propose `iterator-max-age-ms`. I should be sufficient to call out
> (as it's kinda "obvious" anyway) that the metric applies to open
> iterator only.

While I think it's preferable to record the timestamp, rather than the age,
this does have the benefit of a more obvious metric name.

> Nit: the 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:

> The time window thing was just an idea. Happy to drop it.
>
> For the oldest iterator metric, I would propose something simple like
> `iterator-opened-ms` and it would just be the actual timestamp when the
> iterator was opened. I don't think we need to compute the actual age,
> but user can to this computation themselves?
>
> If we think reporting the age instead of just the timestamp is better, I
> would propose `iterator-max-age-ms`. I should be sufficient to call out
> (as it's kinda "obvious" anyway) that the metric applies to open
> iterator only.
>
> And yes, I was hoping that the code inside MetereXxxStore might already
> be setup in a way that custom stores would inherit the iterator metrics
> automatically -- I am just not sure, and left it as an exercise for
> somebody to confirm :)
>
>
> Nit: the 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?
>
>
>
> -Matthias
>
>
> On 3/28/24 2:52 PM, Nick Telford wrote:
> > 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 
> wrote:
> >
> >> Hi everyone,
> >>
> >> Sorry for leaving this for so long. So much for "3 weeks until KIP
> freeze"!
> >>
> >> On Sophie's comments:
> >> 1. Would Matthias's suggestion of a separate metric tracking the age of
> >> the oldest open iterator (within the tag set) satisfy this? That way we
> can
> >> keep iterator-duration-(avg|max) for closed iterators, which can be
> useful
> >> for performance debugging for iterators that don't leak. I'm not sure
> what
> >> we'd call this metric, maybe: "oldest-open-iterator-age-seconds"? Seems
> >> like a mouthful.
> >>
> >> 2. You're right, it makes more sense to provide
> >> iterator-duration-(avg|max). Honestly, I can't remember why I had
> "total"
> >> before, or why I was computing a rate-of-change over it.
> >>
> >> 3, 4, 5, 6. Agreed, I'll make all those changes as suggested.
> >>
> >> 7. Combined with Matthias's point about RocksDB, I'm convinced that this
> >> is the wrong KIP for these. I'll introduce the additional Rocks metrics
> in
> >> another KIP.
> >>
> >> On Matthias's comments:
> >> A. Not sure about the time window. I'm pretty sure all existing avg/max
> >> metrics are since the application was started? Any other suggestions
> here
> >> would be appreciated.
> >>
> >> B. Agreed. See point 1 above.
> >>
> >> C. Good point. My focus was very much on Rocks memory leaks when I wrote
> >> the first draft. I can generalise it. My only concern is that it might
> make
> >> it more difficult to detect Rocks iterator leaks caused *within* our
> >> high-level iterator, e.g. RocksJNI, RocksDB, RocksDBStore, etc. But we
> >> could always provide a RocksDB-specific metric for this, as you
> suggested.
> >>
> >> D. Hmm, we do already have MeteredKeyValueIterator, which automatically
> >> wraps the iterator from inner-stores of MeteredKeyValueStore. If we
> >> implemented these metrics there, then custom stores would automatically
> >> gain the functionality, right? This seems like a pretty logical place to
> >> implement these metrics, since MeteredKeyValueStore is all about adding
> >> metrics to state stores.
> >>
> >>> I imagine the best way to implement this would be to do so at the
> >>> high-level iterator rather than implementing 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 <
> sop...@responsive.dev>
> >> wrote:
> >>

[jira] [Created] (KAFKA-16452) Bound highwatermark offset to range b/w local-log-start-offset and log-end-offset

2024-03-31 Thread Kamal Chandraprakash (Jira)
Kamal Chandraprakash created KAFKA-16452:


 Summary: Bound highwatermark offset to range b/w 
local-log-start-offset and log-end-offset
 Key: KAFKA-16452
 URL: https://issues.apache.org/jira/browse/KAFKA-16452
 Project: Kafka
  Issue Type: Task
Reporter: Kamal Chandraprakash
Assignee: Kamal Chandraprakash


The high watermark should not go below the local-log-start offset. If the high 
watermark is less than the local-log-start-offset, then the 
[UnifiedLog#fetchHighWatermarkMetadata|https://sourcegraph.com/github.com/apache/kafka@d4caa1c10ec81b9c87eaaf52b73c83d5579b68d3/-/blob/core/src/main/scala/kafka/log/UnifiedLog.scala?L358]
 method will throw the OFFSET_OUT_OF_RANGE error when it converts the offset to 
metadata. Once this error happens, the followers will receive out-of-range 
exceptions and the producers won't be able to produce messages since the leader 
cannot move the high watermark.

This issue can happen when the partition undergoes recovery due to corruption 
in the checkpoint file and it gets elected as leader before it gets a chance to 
update the HW from the previous leader.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-989: RocksDB Iterator Metrics

2024-03-31 Thread Matthias J. Sax

The time window thing was just an idea. Happy to drop it.

For the oldest iterator metric, I would propose something simple like 
`iterator-opened-ms` and it would just be the actual timestamp when the 
iterator was opened. I don't think we need to compute the actual age, 
but user can to this computation themselves?


If we think reporting the age instead of just the timestamp is better, I 
would propose `iterator-max-age-ms`. I should be sufficient to call out 
(as it's kinda "obvious" anyway) that the metric applies to open 
iterator only.


And yes, I was hoping that the code inside MetereXxxStore might already 
be setup in a way that custom stores would inherit the iterator metrics 
automatically -- I am just not sure, and left it as an exercise for 
somebody to confirm :)



Nit: the 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?




-Matthias


On 3/28/24 2:52 PM, Nick Telford wrote:

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  wrote:


Hi everyone,

Sorry for leaving this for so long. So much for "3 weeks until KIP freeze"!

On Sophie's comments:
1. Would Matthias's suggestion of a separate metric tracking the age of
the oldest open iterator (within the tag set) satisfy this? That way we can
keep iterator-duration-(avg|max) for closed iterators, which can be useful
for performance debugging for iterators that don't leak. I'm not sure what
we'd call this metric, maybe: "oldest-open-iterator-age-seconds"? Seems
like a mouthful.

2. You're right, it makes more sense to provide
iterator-duration-(avg|max). Honestly, I can't remember why I had "total"
before, or why I was computing a rate-of-change over it.

3, 4, 5, 6. Agreed, I'll make all those changes as suggested.

7. Combined with Matthias's point about RocksDB, I'm convinced that this
is the wrong KIP for these. I'll introduce the additional Rocks metrics in
another KIP.

On Matthias's comments:
A. Not sure about the time window. I'm pretty sure all existing avg/max
metrics are since the application was started? Any other suggestions here
would be appreciated.

B. Agreed. See point 1 above.

C. Good point. My focus was very much on Rocks memory leaks when I wrote
the first draft. I can generalise it. My only concern is that it might make
it more difficult to detect Rocks iterator leaks caused *within* our
high-level iterator, e.g. RocksJNI, RocksDB, RocksDBStore, etc. But we
could always provide a RocksDB-specific metric for this, as you suggested.

D. Hmm, we do already have MeteredKeyValueIterator, which automatically
wraps the iterator from inner-stores of MeteredKeyValueStore. If we
implemented these metrics there, then custom stores would automatically
gain the functionality, right? This seems like a pretty logical place to
implement these metrics, since MeteredKeyValueStore is all about adding
metrics to state stores.


I imagine the best way to implement this would be to do so at the
high-level iterator rather than implementing 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 last two points: I completely agree that we should try to
make this independent of RocksDB, and should probably adopt a
general philosophy of being store-implementation agnostic unless
there is good reason to focus on a particular store type: eg if it was
only possible to implement for certain stores, or only made sense in
the context of a certain store type but not necessarily stores in general.

While leaking memory due to unclosed iterators on RocksDB stores is
certainly the most common issue, I think Matthias sufficiently
demonstrated that the problem of leaking iterators is not actually
unique to RocksDB, and we should consider including in-memory
stores at the very least. I also think that at this point, we may as well
just implement the metrics for *all* store types, whether rocksdb or
in-memory or custom. Not just because it probably applies to all
store types (leaking iterators are rarely a good thing!) but because
I imagine the best way to implement this would be to do so at the
high-level iterator rather than implementing it separately for each
specific iterator implementation for every store type.

That said, I haven't thought all that carefully about the implementation
yet -- it just strikes me as easiest to do at the top level of the store
hierarchy rather than at the bottom. My gut instinct may very well be
wrong, but that's what it's saying

On Thu, Mar 7, 2024 at 10:43 AM Matthias J. Sax  wrote:


Seems I am late to 

Re: [DISCUSS] KIP-1024: Make the restore behavior of GlobalKTables with custom processors configureable

2024-03-31 Thread Matthias J. Sax

Two more follow up thoughts:

(1) I am still not a big fan of the boolean parameter we introduce. Did 
you consider to use different method names, like 
`addReadOnlyGlobalStore()` (for the optimized method, that would not 
reprocess data on restore), and maybe add `addModifiableGlobalStore()` 
(not a good name, but we cannot re-use existing `addGlobalStore()` -- 
maybe somebody else has a good idea about a better `addXxxGlobalStore` 
that would describe it well).


(2) I was thinking about Bruno's comment to limit the scope the store 
builder for the optimized case. I think we should actually do something 
about it, because in the end, the runtime (ie, the `Processor` we hard 
wire) would need to pick a store it supports and cast to the 
corresponding store? If the cast fails, we hit a runtime exception, but 
by putting the store we cast to into the signature we can actually 
convert it into a compile time error what seems better. -- If we want, 
we could make it somewhat flexible and support both `KeyValueStore` and 
`TimestampedKeyValueStore` -- ie, the signature would be `KeyValueStore` 
but we explicitly check if the builder gives us a 
`TimestampedKeyValueStore` instance and use it properly.


If putting the signature does not work for some reason, we should at 
least clearly call it out in the JavaDocs what store type is expected.




-Matthias



On 3/28/24 5:05 PM, Walker Carlson wrote:

Hey all,

Thanks for the feedback Bruno, Almog and Matthias!

Almog: I like the idea, but I agree with Matthais. I actually looked at
that ticket a bit when doing this and found that while similar they are
actually pretty unrelated codewise. I would love to see it get taken care
of.

Bruno and Matthias: The Named parameter doesn't really make sense to me to
put it here. The store in the Store builder is already named through what
Matthais described and the processor doesn't actually have a name. That
would be the processor node that gets named via the Named parameter  (in
the DSL) and the internal streams builder uses the consumed object to make
a source name. I think we should just keep the Consumed object and used
that for the processor node name.

As for the limitation of the store builder interface I don't think it is
necessary. It could be something we add later if we really want to.

Anyways I think we are getting close enough to consensus that I'm going to
open a vote and hopefully we can get it voted on soon!

best,
Walker

On Thu, Mar 28, 2024 at 5:55 AM Matthias J. Sax  wrote:


Hey,

looking into the API, I am wondering why we would need to add an
overload talking a `Named` parameter?

StreamsBuilder.addGlobalStore() (and .addGlobalTable()) already takes a
`Consumed` parameter that allows to set a name.



2.
I do not understand what you mean with "maximum flexibility". The

built-in processor needs to assume a given state store interface. That
means, users have to provide a state store that offers that interface. If
they do not they will get a runtime exception. If we require a store
builder for a given interface, we can catch the mistake at compile time.
Let me know whether I misunderstood something.

Yes, we could catch it at runtime. But I guess what I was trying to say
is different: I was trying to say, we should not limit the API to always
require a specific store, such that global stores can only be of a
certain type. Global Stores should be allowed to be of any type. Hence,
if we add a built-in processor, it can only be one option, and we always
need to support custom processor, and might also want to try to allow
the restore optimization for custom processor (and thus other store
types), not just for our built-in processor (and our built-in stores).
Coupling the optimization to built-in stores would prevent us to apply
the optimization to custom stores.



@Almog: interesting idea. I tend to think that both issues are
orthogonal. If users pick to apply the optimization "added" by this KIP,
the bug you mentioned would still apply to global stores, and thus this
KIP is not addressing the issue you mentioned.

Personally, I also think that we don't need a KIP to fix the ticket you
mentioned? In the end, we need to skip records during restore, and it
seems it does not make sense to make this configurable?



-Matthias


On 3/26/24 4:24 PM, Almog Gavra wrote:

Thanks for the thoughts Bruno!


Do you mean a API to configure restoration instead of boolean flag

reprocessOnRestore?

Yes, this is exactly the type of thing I was musing (but I don't have any
concrete suggestions). It feels like that would give the flexibility to

do

things like the motivation section of the KIP (allow bulk loading of
records without reprocessing) while also solving other limitations.

I'm supportive of the KIP as-is but was hoping somebody with more
experience would have a sudden inspiration for how to solve both issues
with one API! Anyway, I'll slide back into the lurking shadows for now

and

let the discussion