Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.2 #48

2022-05-10 Thread Apache Jenkins Server
See 




Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.1 #120

2022-05-10 Thread Apache Jenkins Server
See 




Re: [DISCUSS] Apache Kafka 3.3.0 Release

2022-05-10 Thread John Roesler
+1 from me! Thanks for volunteering, José. 
-John

On Tue, May 10, 2022, at 17:53, José Armando García Sancio wrote:
> Hi all,
>
> I would like to volunteer for the release of Apache Kafka 3.3.0. If
> people agree, I'll start working on the release plan and update this
> thread.
>
> Thanks,
> -José


Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-10 Thread Chris Egerton
Hi Francois,

Thanks for your thoughts. I think it's worth noting that in regards to item
2, it's possible to explicitly declare the type parameters for a builder
without capturing it in a variable; for example:

KafkaProducer p = new Builder(...)
.withKeySerializer(new StringSerializer())
.withValueSerializer(new IntegerSerializer())
.build();

That aside, given the three binding votes already cast on the vote thread,
it's probably too late to be worth changing direction at this point. Thanks
for entertaining the proposal, and congratulations on your KIP!

Cheers,

Chris

On Tue, May 10, 2022 at 5:33 PM François Rosière 
wrote:

> Hello Chris,
>
> Thanks for the feedback. Builders is definitely the pattern to apply when
> an object needs to be built using different arguments/combinations.
>
> Here are my thoughts about the proposal:
>
> 1. The builder should only expose meaningful methods for the users such as
> the interceptors, the serializer/deserializer, partitioner, etc. A method
> like the configured instances is internal and should not be exposed if we
> don't want to expose the config itself. Using this internal method is the
> only way to solve the issue if the config is exposed.
>
> 2. As the key and value types are not given, a variable will need to be
> created for the builder before being used. Otherwise, there is no way to
> infer the type correctly. Breaks a bit the inline usage with DSL style.
>
> 3. What about existing constructors, they would need to stay to keep
> compatibility with existing o could they be removed in benefit of the
> builder?
>
> 4. Having an access to the config also gives a way to also fine tune other
> aspects such as the logging related to the config. Log unused, skip some
> properties, etc.
>
> 5. What about streams? Shouldn't it be aligned?
>
> So, to summarise, the KIP was a best effort solution to support already
> configured instances related to both the producer and the consumer.
> The builder will work, it's just a matter of deciding the best approach...
> for me, both solutions are fine, I just need a way to inject already
> configured dependencies into the producers and consumers.
>
> If we conclude, I will drop a PR on Github.
>
> Kr,
>
> F.
>
> Le mar. 10 mai 2022 à 15:01, Chris Egerton  a
> écrit :
>
> > Hi Francois,
> >
> > Thanks for the KIP! I sympathize with the issue you're facing and with
> > John's reluctance to let perfect be the enemy of good, and if KIP freeze
> > were tomorrow, I think this would be good enough. Given that we still
> have
> > some time to work with, I'd like to propose an alternative approach and
> see
> > what your thoughts are.
> >
> > There are a few issues with the current client APIs that are closely
> > related to the KIP:
> > 1. Too many constructors (there are currently four each for KafkaProducer
> > and KafkaConsumer, yet they all do basically the same thing)
> > 2. Lack of type safety with interceptors (you have no way to enforce at
> > compile time that your ProducerInterceptor is used with
> a
> > Serializer and Serializer, for example)
> > 3. Inflexibility and inconsistency with instantiation of pluggable
> > interfaces (you can bring your own (de)serializers, but everything else
> > gets instantiated and configured for you at producer/consumer
> construction
> > time)
> >
> > The KIP as it exists now will only address item 3, and will exacerbate
> item
> > 1.
> >
> > In addition, there are a few new issues introduced by the KIP as it
> exists
> > now:
> > 1. Tighter coupling between the ProducerConfig/ConsumerConfig classes and
> > the KafkaProducer/KafkaConsumer classes. Any change we make in the future
> > that breaks either of these config classes in unexpected ways (but which
> > does not break the KafkaProducer/KafkaConsumer constructors that do not
> > accept these classes as parameters) will now have a much higher chance to
> > also break a user's entire producer/consumer application.
> > 2. Complexity for users like yourself who would like to override behavior
> > in a ProducerConfig/ConsumerConfig in order to inject pre-instantiated
> > dependencies. The example in the KIP overrides
> > AbstractConfig::getConfiguredInstances [1] in order to achieve this. But
> > there are two other overloaded variants of getConfiguredInstances, and
> two
> > AbstractConfig::getConfiguredInstance methods that also exist. We'd
> either
> > need to establish a dependency graph between these methods (e.g., some
> > methods are guaranteed to invoke another overloaded variant) as part of
> the
> > public API for the AbstractConfig, or users would need to override every
> > single one of these methods in order to ensure that their code won't
> break
> > at runtime after bumping their Kafka version.
> >
> > I think introducing the builder pattern for KafkaProducer and
> KafkaConsumer
> > would alleviate all of these issues. As a rough draft of what the API
> might
> > look like for KafkaProducer:
> >
> > public 

Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Jim Hughes
Hi Matthias,

I like it.  I've updated the KIP to reflect that detail; I put the details
in the docs for pause.

Cheers,

Jim

On Tue, May 10, 2022 at 7:51 PM Matthias J. Sax  wrote:

> Thanks for the KIP. Overall LGTM.
>
> Can we clarify one question: would it be allowed to call `pause()`
> before calling `start()`? I don't see any reason why we would need to
> disallow it?
>
> It could be helpful to start a KafkaStreams client in paused state --
> otherwise there is a race between calling `start()` and calling `pause()`.
>
> If we allow it, we should clearly document it.
>
>
> -Matthias
>
> On 5/10/22 12:04 PM, Jim Hughes wrote:
> > Hi Bill, all,
> >
> > Thank you.  I've updated the KIP to reflect pausing standby tasks as
> well.
> > I think all the outstanding points have been addressed and I'm going to
> > start the vote thread!
> >
> > Cheers,
> >
> > Jim
> >
> >
> >
> > On Tue, May 10, 2022 at 2:43 PM Bill Bejeck  wrote:
> >
> >> Hi Jim,
> >>
> >> After reading the comments on the KIP, I agree that it makes sense to
> pause
> >> all activities and any changes can be made later on.
> >>
> >> Thanks,
> >> Bill
> >>
> >> On Tue, May 10, 2022 at 4:03 AM Bruno Cadonna 
> wrote:
> >>
> >>> Hi Jim,
> >>>
> >>> Thanks for the KIP!
> >>>
> >>> I am fine with the KIP in general.
> >>>
> >>> However, I am with Sophie and John to also pause the standbys for the
> >>> reasons they brought up. Is there a specific reason you want to keep
> >>> standbys going? It feels like premature optimization to me. We can
> still
> >>> add keeping standby running in a follow up if needed.
> >>>
> >>> Best,
> >>> Bruno
> >>>
> >>> On 10.05.22 05:15, Sophie Blee-Goldman wrote:
>  Thanks Jim, just one note/question on the standby tasks:
> 
>  At the minute, my moderately held position is that standby tasks ought
> >> to
> > continue reading and remain caught up.  If standby tasks would run
> out
> >>> of
> > space, there are probably bigger problems.
> 
> 
>  For a single node application, or when the #pause API is invoked on
> all
>  instances,
>  then there won't be any further active processing and thus nothing to
> >>> keep
>  up with,
>  right? So for that case, it's just a matter of whether any standbys
> >> that
>  are lagging
>  will have the chance to catch up to the (paused) active task state
> >> before
>  they stop
>  as well, in which case having them continue feels fine to me. However
> >>> this
>  is a
>  relatively trivial benefit and I would only consider it as a deciding
>  factor when all
>  things are equal otherwise.
> 
>  My concern is the more interesting case: when this feature is used to
> >>> pause
>  only
>  one nodes, or some subset of the overall application. In this case,
> >> yes,
>  the standby
>  tasks will indeed fall out of sync. But the only reason I can imagine
>  someone using
>  the pause feature in such a way is because there is something going
> >>> wrong,
>  or about
>  to go wrong, on that particular node. For example as mentioned above,
> >> if
>  the user
>  wants to cut down on costs without stopping everything, or if the node
> >> is
>  about to
>  run out of disk or needs to be debugged or so on. And in this case,
>  continuing to
>  process the standby tasks while other instances continue to run would
>  pretty much
>  defeat the purpose of pausing it entirely, and might have unpleasant
>  consequences
>  for the unsuspecting developer.
> 
>  All that said, I don't want to block this KIP so if you have strong
>  feelings about the
>  standby behavior I'm happy to back down. I'm only pushing back now
> >>> because
>  it
>  felt like there wasn't any particular motivation for the standbys to
>  continue processing
>  or not, and I figured I'd try to fill in this gap with my thoughts on
> >> the
>  matter :)
>  Either way we should just make sure that this behavior is documented
>  clearly,
>  since it may be surprising if we decide to only pause active
> processing
>  (another option
>  is to rename the method something like #pauseProcessing or
>  #pauseActiveProcessing
>  so that it's hard to miss).
> 
>  Thanks! Sorry for the lengthy response, but hopefully we won't need to
>  debate this any
>  further. Beyond this I'm satisfied with the latest proposal
> 
>  On Mon, May 9, 2022 at 5:16 PM John Roesler 
> >> wrote:
> 
> > Thanks for the updates, Jim!
> >
> > After this discussion and your updates, this KIP looks good to me.
> >
> > Thanks,
> > John
> >
> > On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:
> >> Hi Sophie, all,
> >>
> >> I've updated the KIP with feedback from the discussion so far:
> >>
> >
> >>>
> >>
> 

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Guozhang Wang
Thanks Jim. +1 from me.

On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax  wrote:

> I had one minor question on the discuss thread. It's mainly about
> clarifying and document the user contract. I am fine either way.
>
> +1 (binding)
>
>
> -Matthias
>
> On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > Thanks for the KIP! +1 (binding)
> >
> > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna  wrote:
> >
> >> Thanks Jim,
> >>
> >> +1 (binding)
> >>
> >> Best,
> >> Bruno
> >>
> >> On 10.05.22 21:19, John Roesler wrote:
> >>> Thanks Jim,
> >>>
> >>> I’m +1 (binding)
> >>>
> >>> -John
> >>>
> >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
>  Hi all,
> 
>  I'm asking for a vote on KIP-834:
> 
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> 
>  Thanks in advance!
> 
>  Jim
> >>
> >
>


-- 
-- Guozhang


[jira] [Resolved] (KAFKA-7527) Enable Dependency Injection for Kafka Streams handlers (KIP-378)

2022-05-10 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax resolved KAFKA-7527.

Resolution: Fixed

> Enable Dependency Injection for Kafka Streams handlers (KIP-378)
> 
>
> Key: KAFKA-7527
> URL: https://issues.apache.org/jira/browse/KAFKA-7527
> Project: Kafka
>  Issue Type: Improvement
>  Components: streams
>Affects Versions: 2.0.0, 2.1.0
>Reporter: Wladimir Schmidt
>Priority: Minor
>  Labels: kip, usability
>
> Implement solution proposed in the KIP-378 (Enable Dependency Injection for 
> Kafka Streams handlers).
> Link to 
> [KIP-378|https://cwiki.apache.org/confluence/display/KAFKA/KIP-378:+Enable+Dependency+Injection+for+Kafka+Streams+handlers]



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Matthias J. Sax
I had one minor question on the discuss thread. It's mainly about 
clarifying and document the user contract. I am fine either way.


+1 (binding)


-Matthias

On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:

Thanks for the KIP! +1 (binding)

On Tue, May 10, 2022, 12:24 PM Bruno Cadonna  wrote:


Thanks Jim,

+1 (binding)

Best,
Bruno

On 10.05.22 21:19, John Roesler wrote:

Thanks Jim,

I’m +1 (binding)

-John

On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:

Hi all,

I'm asking for a vote on KIP-834:


https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832


Thanks in advance!

Jim






Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Matthias J. Sax

Thanks for the KIP. Overall LGTM.

Can we clarify one question: would it be allowed to call `pause()` 
before calling `start()`? I don't see any reason why we would need to 
disallow it?


It could be helpful to start a KafkaStreams client in paused state -- 
otherwise there is a race between calling `start()` and calling `pause()`.


If we allow it, we should clearly document it.


-Matthias

On 5/10/22 12:04 PM, Jim Hughes wrote:

Hi Bill, all,

Thank you.  I've updated the KIP to reflect pausing standby tasks as well.
I think all the outstanding points have been addressed and I'm going to
start the vote thread!

Cheers,

Jim



On Tue, May 10, 2022 at 2:43 PM Bill Bejeck  wrote:


Hi Jim,

After reading the comments on the KIP, I agree that it makes sense to pause
all activities and any changes can be made later on.

Thanks,
Bill

On Tue, May 10, 2022 at 4:03 AM Bruno Cadonna  wrote:


Hi Jim,

Thanks for the KIP!

I am fine with the KIP in general.

However, I am with Sophie and John to also pause the standbys for the
reasons they brought up. Is there a specific reason you want to keep
standbys going? It feels like premature optimization to me. We can still
add keeping standby running in a follow up if needed.

Best,
Bruno

On 10.05.22 05:15, Sophie Blee-Goldman wrote:

Thanks Jim, just one note/question on the standby tasks:

At the minute, my moderately held position is that standby tasks ought

to

continue reading and remain caught up.  If standby tasks would run out

of

space, there are probably bigger problems.



For a single node application, or when the #pause API is invoked on all
instances,
then there won't be any further active processing and thus nothing to

keep

up with,
right? So for that case, it's just a matter of whether any standbys

that

are lagging
will have the chance to catch up to the (paused) active task state

before

they stop
as well, in which case having them continue feels fine to me. However

this

is a
relatively trivial benefit and I would only consider it as a deciding
factor when all
things are equal otherwise.

My concern is the more interesting case: when this feature is used to

pause

only
one nodes, or some subset of the overall application. In this case,

yes,

the standby
tasks will indeed fall out of sync. But the only reason I can imagine
someone using
the pause feature in such a way is because there is something going

wrong,

or about
to go wrong, on that particular node. For example as mentioned above,

if

the user
wants to cut down on costs without stopping everything, or if the node

is

about to
run out of disk or needs to be debugged or so on. And in this case,
continuing to
process the standby tasks while other instances continue to run would
pretty much
defeat the purpose of pausing it entirely, and might have unpleasant
consequences
for the unsuspecting developer.

All that said, I don't want to block this KIP so if you have strong
feelings about the
standby behavior I'm happy to back down. I'm only pushing back now

because

it
felt like there wasn't any particular motivation for the standbys to
continue processing
or not, and I figured I'd try to fill in this gap with my thoughts on

the

matter :)
Either way we should just make sure that this behavior is documented
clearly,
since it may be surprising if we decide to only pause active processing
(another option
is to rename the method something like #pauseProcessing or
#pauseActiveProcessing
so that it's hard to miss).

Thanks! Sorry for the lengthy response, but hopefully we won't need to
debate this any
further. Beyond this I'm satisfied with the latest proposal

On Mon, May 9, 2022 at 5:16 PM John Roesler 

wrote:



Thanks for the updates, Jim!

After this discussion and your updates, this KIP looks good to me.

Thanks,
John

On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:

Hi Sophie, all,

I've updated the KIP with feedback from the discussion so far:






https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832


As a terse summary of my current position:
Pausing will only stop processing and punctuation (respecting modular
topologies).
Paused topologies will still a) consume from input topics, b) call

the

usual commit pathways (commits will happen basically as they would

have),

and c) standBy tasks will still be processed.

Shout if the KIP or those details still need some TLC.  Responding to
Sophie inline below.


On Mon, May 9, 2022 at 6:06 PM Sophie Blee-Goldman
 wrote:


Don't worry, I'm going to be adding the APIs for topology-level

pausing

as

part of the modular topologies KIP,
so we don't need to worry about that for now. That said, I don't

think

we

should brush it off entirely and design
this feature in a way that's going to be incompatible or hugely

raise

the

LOE on bringing the (mostly already
implemented) modular topologies feature into the public API, just
because it "won the race to write a KIP" :)



Yes, I'm hoping that this is all 

[DISCUSS] Apache Kafka 3.3.0 Release

2022-05-10 Thread José Armando García Sancio
Hi all,

I would like to volunteer for the release of Apache Kafka 3.3.0. If
people agree, I'll start working on the release plan and update this
thread.

Thanks,
-José


[jira] [Resolved] (KAFKA-13892) Dedupe RemoveAccessControlEntryRecord in deleteAcls of AclControlManager

2022-05-10 Thread Andrew Grant (Jira)


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

Andrew Grant resolved KAFKA-13892.
--
Resolution: Fixed

> Dedupe RemoveAccessControlEntryRecord in deleteAcls of AclControlManager
> 
>
> Key: KAFKA-13892
> URL: https://issues.apache.org/jira/browse/KAFKA-13892
> Project: Kafka
>  Issue Type: Bug
>Reporter: Andrew Grant
>Assignee: Andrew Grant
>Priority: Major
>
> In 
> [https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java#L143]
>  we loop through the ACL filters and and add RemoveAccessControlEntryRecord 
> records to the response list for each matching ACL. I think there's a bug 
> where if two filters match the same ACL, we create two 
> RemoveAccessControlEntryRecord records for that same ACL. This is an issue 
> because upon replay we throw an exception 
> (https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java#L195)
>  if the ACL is not in the in-memory data structures which will happen to the 
> second RemoveAccessControlEntryRecord.
> Maybe we can just de-dupe both List and 
> List? I think something like (just showing code for 
> ApiMessageAndVersion):
> {code:java}
> private List 
> deDupeApiMessageAndVersion(List messages) {
> return new HashSet<>(messages).stream().collect(Collectors.toList());
> }{code}
> should suffice as I don't think the ordering matters within the list of 
> response objects.  



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Resolved] (KAFKA-13879) Exponential backoff for reconnect does not work

2022-05-10 Thread Chern Yih Cheah (Jira)


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

Chern Yih Cheah resolved KAFKA-13879.
-
Resolution: Fixed

> Exponential backoff for reconnect does not work
> ---
>
> Key: KAFKA-13879
> URL: https://issues.apache.org/jira/browse/KAFKA-13879
> Project: Kafka
>  Issue Type: Bug
>  Components: network
>Affects Versions: 2.7.0
>Reporter: Chern Yih Cheah
>Assignee: Chern Yih Cheah
>Priority: Minor
>
> When a client connects to a SSL listener using PLAINTEXT security protocol, 
> after the TCP connection is setup, the client considers the channel setup is 
> complete (in reality the channel setup is not complete yet). The client 
> issues API version request after that. When issuing API version request, 
> reconnection exponential backoff is reset. Since the broker expects SSL 
> handshake, client's API version request will cause the connection to 
> disconnect. Reconnect will happen without exponential backoff since it has 
> been reset.
> [https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java#L249.]
>   



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [DISCUSS] KIP-835: Monitor KRaft Controller Quorum Health

2022-05-10 Thread José Armando García Sancio
Thanks for your feedback Jason, much appreciated.

Here are the changes to the KIP:
https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=211883219=5=4

On Tue, May 10, 2022 at 1:34 PM Jason Gustafson
 wrote:
> The approach sounds reasonable. By the way, I think one of the gaps we have
> today is when the leader gets partitioned from the remaining voters. I
> believe it continues acting as a leader indefinitely. I was considering
> whether this periodic write can address the issue. Basically it can be used
> to force a leader to prove it is still the leader by committing some data.
> Say, for example, that the leader fails to commit the record after the
> fetch timeout expires, then perhaps it could start a new election. What do
> you think?

We have an issue for this at
https://issues.apache.org/jira/browse/KAFKA-13621. I updated the issue
with your feedback and included some of my thoughts. Do you mind if we
move this conversation to that issue?

> A couple additional questions:
>
> - What is the default value for `metadata.monitor.write.interval.ms`? Also,
> I'm wondering if `controller` would be a more suitable prefix?

Yeah. I am not sure. Looking at the current configuration we have both
prefixes. For example, with the `controller` prefix we have
`controller.quorum.voters`, `controller.listener.names`,
`controller.quota.window.num`, etc. For the `metadata` prefix we have
`metadata.log.dir`, `metadata.log.*` and `metadat.max.retention.ms`,
etc.
I get the impression that we use `metadata` for things that are kinda
log/disk related and `controller` for things that are not. I am
thinking that the `metadata` prefix is more consistent with the
current situation. What do you think Jason?

> - Could we avoid letting BrokerMetadataPublisher escape into the metric
> name? Letting the classnames leak into the metrics tends to cause
> compatibility issues over time.

Good point. For Raft we use `kafka.server:type=raft-metrics,name=...`.
I'll change it to
`kafka.server:type=broker-metadata-metrics,name=...`.

Thanks,
-José


Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-10 Thread François Rosière
Hello Chris,

Thanks for the feedback. Builders is definitely the pattern to apply when
an object needs to be built using different arguments/combinations.

Here are my thoughts about the proposal:

1. The builder should only expose meaningful methods for the users such as
the interceptors, the serializer/deserializer, partitioner, etc. A method
like the configured instances is internal and should not be exposed if we
don't want to expose the config itself. Using this internal method is the
only way to solve the issue if the config is exposed.

2. As the key and value types are not given, a variable will need to be
created for the builder before being used. Otherwise, there is no way to
infer the type correctly. Breaks a bit the inline usage with DSL style.

3. What about existing constructors, they would need to stay to keep
compatibility with existing o could they be removed in benefit of the
builder?

4. Having an access to the config also gives a way to also fine tune other
aspects such as the logging related to the config. Log unused, skip some
properties, etc.

5. What about streams? Shouldn't it be aligned?

So, to summarise, the KIP was a best effort solution to support already
configured instances related to both the producer and the consumer.
The builder will work, it's just a matter of deciding the best approach...
for me, both solutions are fine, I just need a way to inject already
configured dependencies into the producers and consumers.

If we conclude, I will drop a PR on Github.

Kr,

F.

Le mar. 10 mai 2022 à 15:01, Chris Egerton  a
écrit :

> Hi Francois,
>
> Thanks for the KIP! I sympathize with the issue you're facing and with
> John's reluctance to let perfect be the enemy of good, and if KIP freeze
> were tomorrow, I think this would be good enough. Given that we still have
> some time to work with, I'd like to propose an alternative approach and see
> what your thoughts are.
>
> There are a few issues with the current client APIs that are closely
> related to the KIP:
> 1. Too many constructors (there are currently four each for KafkaProducer
> and KafkaConsumer, yet they all do basically the same thing)
> 2. Lack of type safety with interceptors (you have no way to enforce at
> compile time that your ProducerInterceptor is used with a
> Serializer and Serializer, for example)
> 3. Inflexibility and inconsistency with instantiation of pluggable
> interfaces (you can bring your own (de)serializers, but everything else
> gets instantiated and configured for you at producer/consumer construction
> time)
>
> The KIP as it exists now will only address item 3, and will exacerbate item
> 1.
>
> In addition, there are a few new issues introduced by the KIP as it exists
> now:
> 1. Tighter coupling between the ProducerConfig/ConsumerConfig classes and
> the KafkaProducer/KafkaConsumer classes. Any change we make in the future
> that breaks either of these config classes in unexpected ways (but which
> does not break the KafkaProducer/KafkaConsumer constructors that do not
> accept these classes as parameters) will now have a much higher chance to
> also break a user's entire producer/consumer application.
> 2. Complexity for users like yourself who would like to override behavior
> in a ProducerConfig/ConsumerConfig in order to inject pre-instantiated
> dependencies. The example in the KIP overrides
> AbstractConfig::getConfiguredInstances [1] in order to achieve this. But
> there are two other overloaded variants of getConfiguredInstances, and two
> AbstractConfig::getConfiguredInstance methods that also exist. We'd either
> need to establish a dependency graph between these methods (e.g., some
> methods are guaranteed to invoke another overloaded variant) as part of the
> public API for the AbstractConfig, or users would need to override every
> single one of these methods in order to ensure that their code won't break
> at runtime after bumping their Kafka version.
>
> I think introducing the builder pattern for KafkaProducer and KafkaConsumer
> would alleviate all of these issues. As a rough draft of what the API might
> look like for KafkaProducer:
>
> public class Builder {
> private final Map props;
> private Serializer keySerializer;
> private Serializer valueSerializer;
> private List> interceptors;
> private Map configuredInstances;
> private Map> configuredInstanceLists;
>
> public Builder(Map props) {
> this.props = props;
> this.interceptors = new ArrayList<>();
> this.configuredInstances = new HashMap<>();
> this.configuredInstanceLists = new HashMap<>();
> }
>
> // Use this serializer, if non-null
> // Will take precedence over any serializer class specified in the
> properties for this producer
> public Builder withKeySerializer(Serializer serializer) {
> this.keySerializer = serializer;
> return this;
> }
>
> public Builder withValueSerializer(Serializer serializer) {
>

Re: [DISCUSS] KIP-835: Monitor KRaft Controller Quorum Health

2022-05-10 Thread Jason Gustafson
Hi Jose,

Thanks for the KIP.

The approach sounds reasonable. By the way, I think one of the gaps we have
today is when the leader gets partitioned from the remaining voters. I
believe it continues acting as a leader indefinitely. I was considering
whether this periodic write can address the issue. Basically it can be used
to force a leader to prove it is still the leader by committing some data.
Say, for example, that the leader fails to commit the record after the
fetch timeout expires, then perhaps it could start a new election. What do
you think?

A couple additional questions:

- What is the default value for `metadata.monitor.write.interval.ms`? Also,
I'm wondering if `controller` would be a more suitable prefix?
- Could we avoid letting BrokerMetadataPublisher escape into the metric
name? Letting the classnames leak into the metrics tends to cause
compatibility issues over time.

Best,
Jason




On Fri, May 6, 2022 at 12:02 PM José Armando García Sancio
 wrote:

> Hi all,
>
> I created a KIP for adding a mechanism to monitor the health of the
> KRaft Controller quorum through metrics. See KIP-835:
> https://cwiki.apache.org/confluence/x/0xShD
>
> Thanks for your feedback,
> -José
>


Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-10 Thread Niket Goel
> @Niket does it make sense to add the Admin API to this KIP?

Thanks Deng for pointing this out. I agree with Jason's suggestion. I will
go ahead and add the admin API to this KIP.

- Niket

On Tue, May 10, 2022 at 11:44 AM Jason Gustafson 
wrote:

> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> it’s unnecessary to submit a KIP to change it, you can just submit a PR to
> accomplish this.
>
> Hey Ziming, I think it is public. It was documented in KIP-595 and we have
> implemented the API on the server. However, it looks like I never added
> the Admin API (even though it is assumed by the `kafka-metadata-quorum.sh`
> tool). @Niket does it make sense to add the Admin API to this KIP?
>
> Best,
> Jason
>
> On Mon, May 9, 2022 at 8:09 PM deng ziming 
> wrote:
>
> > Hello Niket, currently DescribeQuorumResponse is not a public API, we
> > don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> > it’s unnecessary to submit a KIP to change it, you can just submit a PR
> to
> > accomplish this.
> >
> > --
> > Thanks
> > Ziming
> >
> > > On May 10, 2022, at 1:33 AM, Niket Goel 
> > wrote:
> > >
> > > Hi all,
> > >
> > > I created a KIP to add some more information to
> > `DesscribeQuorumResponse` to enable ascertaining voter lag in the quorum
> a
> > little better.
> > > Please see KIP --
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> > >
> > > Thanks for your feedback,
> > > Niket Goel
> >
> >
>


-- 
- Niket


Re: [VOTE] KIP-832 Allow creating a producer/consumer using a producer/consumer

2022-05-10 Thread Bill Bejeck
Thanks for the KIP, François.

+1 (binding)

-Bill

On Tue, May 10, 2022 at 10:45 AM John Roesler  wrote:

> +1 (binding)
>
> Thanks, François!
> -John
>
> On Tue, May 10, 2022, at 03:09, Bruno Cadonna wrote:
> > Hi Francois,
> >
> > Thanks for the KIP whose link is:
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578
> >
> > +1 (binding)
> >
> > Best,
> > Bruno
> >
> > On 09.05.22 23:23, François Rosière wrote:
> >>
> >>
>


[jira] [Resolved] (KAFKA-13862) Add And Subtract multiple config values is not supported in KRaft mode

2022-05-10 Thread Jason Gustafson (Jira)


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

Jason Gustafson resolved KAFKA-13862.
-
Fix Version/s: 3.3.0
   Resolution: Fixed

> Add And Subtract multiple config values is not supported in KRaft mode
> --
>
> Key: KAFKA-13862
> URL: https://issues.apache.org/jira/browse/KAFKA-13862
> Project: Kafka
>  Issue Type: Bug
>Reporter: dengziming
>Assignee: dengziming
>Priority: Major
> Fix For: 3.3.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Sophie Blee-Goldman
Thanks for the KIP! +1 (binding)

On Tue, May 10, 2022, 12:24 PM Bruno Cadonna  wrote:

> Thanks Jim,
>
> +1 (binding)
>
> Best,
> Bruno
>
> On 10.05.22 21:19, John Roesler wrote:
> > Thanks Jim,
> >
> > I’m +1 (binding)
> >
> > -John
> >
> > On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> >> Hi all,
> >>
> >> I'm asking for a vote on KIP-834:
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> >>
> >> Thanks in advance!
> >>
> >> Jim
>


Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Bruno Cadonna

Thanks Jim,

+1 (binding)

Best,
Bruno

On 10.05.22 21:19, John Roesler wrote:

Thanks Jim,

I’m +1 (binding)

-John

On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:

Hi all,

I'm asking for a vote on KIP-834:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832

Thanks in advance!

Jim


Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread John Roesler
Thanks Jim,

I’m +1 (binding)

-John

On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> Hi all,
>
> I'm asking for a vote on KIP-834:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
>
> Thanks in advance!
>
> Jim


[VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Jim Hughes
Hi all,

I'm asking for a vote on KIP-834:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832

Thanks in advance!

Jim


Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Jim Hughes
Hi Bill, all,

Thank you.  I've updated the KIP to reflect pausing standby tasks as well.
I think all the outstanding points have been addressed and I'm going to
start the vote thread!

Cheers,

Jim



On Tue, May 10, 2022 at 2:43 PM Bill Bejeck  wrote:

> Hi Jim,
>
> After reading the comments on the KIP, I agree that it makes sense to pause
> all activities and any changes can be made later on.
>
> Thanks,
> Bill
>
> On Tue, May 10, 2022 at 4:03 AM Bruno Cadonna  wrote:
>
> > Hi Jim,
> >
> > Thanks for the KIP!
> >
> > I am fine with the KIP in general.
> >
> > However, I am with Sophie and John to also pause the standbys for the
> > reasons they brought up. Is there a specific reason you want to keep
> > standbys going? It feels like premature optimization to me. We can still
> > add keeping standby running in a follow up if needed.
> >
> > Best,
> > Bruno
> >
> > On 10.05.22 05:15, Sophie Blee-Goldman wrote:
> > > Thanks Jim, just one note/question on the standby tasks:
> > >
> > > At the minute, my moderately held position is that standby tasks ought
> to
> > >> continue reading and remain caught up.  If standby tasks would run out
> > of
> > >> space, there are probably bigger problems.
> > >
> > >
> > > For a single node application, or when the #pause API is invoked on all
> > > instances,
> > > then there won't be any further active processing and thus nothing to
> > keep
> > > up with,
> > > right? So for that case, it's just a matter of whether any standbys
> that
> > > are lagging
> > > will have the chance to catch up to the (paused) active task state
> before
> > > they stop
> > > as well, in which case having them continue feels fine to me. However
> > this
> > > is a
> > > relatively trivial benefit and I would only consider it as a deciding
> > > factor when all
> > > things are equal otherwise.
> > >
> > > My concern is the more interesting case: when this feature is used to
> > pause
> > > only
> > > one nodes, or some subset of the overall application. In this case,
> yes,
> > > the standby
> > > tasks will indeed fall out of sync. But the only reason I can imagine
> > > someone using
> > > the pause feature in such a way is because there is something going
> > wrong,
> > > or about
> > > to go wrong, on that particular node. For example as mentioned above,
> if
> > > the user
> > > wants to cut down on costs without stopping everything, or if the node
> is
> > > about to
> > > run out of disk or needs to be debugged or so on. And in this case,
> > > continuing to
> > > process the standby tasks while other instances continue to run would
> > > pretty much
> > > defeat the purpose of pausing it entirely, and might have unpleasant
> > > consequences
> > > for the unsuspecting developer.
> > >
> > > All that said, I don't want to block this KIP so if you have strong
> > > feelings about the
> > > standby behavior I'm happy to back down. I'm only pushing back now
> > because
> > > it
> > > felt like there wasn't any particular motivation for the standbys to
> > > continue processing
> > > or not, and I figured I'd try to fill in this gap with my thoughts on
> the
> > > matter :)
> > > Either way we should just make sure that this behavior is documented
> > > clearly,
> > > since it may be surprising if we decide to only pause active processing
> > > (another option
> > > is to rename the method something like #pauseProcessing or
> > > #pauseActiveProcessing
> > > so that it's hard to miss).
> > >
> > > Thanks! Sorry for the lengthy response, but hopefully we won't need to
> > > debate this any
> > > further. Beyond this I'm satisfied with the latest proposal
> > >
> > > On Mon, May 9, 2022 at 5:16 PM John Roesler 
> wrote:
> > >
> > >> Thanks for the updates, Jim!
> > >>
> > >> After this discussion and your updates, this KIP looks good to me.
> > >>
> > >> Thanks,
> > >> John
> > >>
> > >> On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:
> > >>> Hi Sophie, all,
> > >>>
> > >>> I've updated the KIP with feedback from the discussion so far:
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > >>>
> > >>> As a terse summary of my current position:
> > >>> Pausing will only stop processing and punctuation (respecting modular
> > >>> topologies).
> > >>> Paused topologies will still a) consume from input topics, b) call
> the
> > >>> usual commit pathways (commits will happen basically as they would
> > have),
> > >>> and c) standBy tasks will still be processed.
> > >>>
> > >>> Shout if the KIP or those details still need some TLC.  Responding to
> > >>> Sophie inline below.
> > >>>
> > >>>
> > >>> On Mon, May 9, 2022 at 6:06 PM Sophie Blee-Goldman
> > >>>  wrote:
> > >>>
> >  Don't worry, I'm going to be adding the APIs for topology-level
> > pausing
> > >> as
> >  part of the modular topologies KIP,
> >  so we don't need to worry about that for now. That said, I don't
> think
> > >> we
> >  should brush it off entirely and 

Re: [DISCUSS] KIP-836: Addition of Information in DescribeQuorumResponse about Voter Lag

2022-05-10 Thread Jason Gustafson
> Hello Niket, currently DescribeQuorumResponse is not a public API, we
don’t have a Admin api or shell script to get DescribeQuorumResponse, so
it’s unnecessary to submit a KIP to change it, you can just submit a PR to
accomplish this.

Hey Ziming, I think it is public. It was documented in KIP-595 and we have
implemented the API on the server. However, it looks like I never added
the Admin API (even though it is assumed by the `kafka-metadata-quorum.sh`
tool). @Niket does it make sense to add the Admin API to this KIP?

Best,
Jason

On Mon, May 9, 2022 at 8:09 PM deng ziming  wrote:

> Hello Niket, currently DescribeQuorumResponse is not a public API, we
> don’t have a Admin api or shell script to get DescribeQuorumResponse, so
> it’s unnecessary to submit a KIP to change it, you can just submit a PR to
> accomplish this.
>
> --
> Thanks
> Ziming
>
> > On May 10, 2022, at 1:33 AM, Niket Goel 
> wrote:
> >
> > Hi all,
> >
> > I created a KIP to add some more information to
> `DesscribeQuorumResponse` to enable ascertaining voter lag in the quorum a
> little better.
> > Please see KIP --
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Additional+Information+in+DescribeQuorumResponse+about+Voter+Lag
> >
> > Thanks for your feedback,
> > Niket Goel
>
>


Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Bill Bejeck
Hi Jim,

After reading the comments on the KIP, I agree that it makes sense to pause
all activities and any changes can be made later on.

Thanks,
Bill

On Tue, May 10, 2022 at 4:03 AM Bruno Cadonna  wrote:

> Hi Jim,
>
> Thanks for the KIP!
>
> I am fine with the KIP in general.
>
> However, I am with Sophie and John to also pause the standbys for the
> reasons they brought up. Is there a specific reason you want to keep
> standbys going? It feels like premature optimization to me. We can still
> add keeping standby running in a follow up if needed.
>
> Best,
> Bruno
>
> On 10.05.22 05:15, Sophie Blee-Goldman wrote:
> > Thanks Jim, just one note/question on the standby tasks:
> >
> > At the minute, my moderately held position is that standby tasks ought to
> >> continue reading and remain caught up.  If standby tasks would run out
> of
> >> space, there are probably bigger problems.
> >
> >
> > For a single node application, or when the #pause API is invoked on all
> > instances,
> > then there won't be any further active processing and thus nothing to
> keep
> > up with,
> > right? So for that case, it's just a matter of whether any standbys that
> > are lagging
> > will have the chance to catch up to the (paused) active task state before
> > they stop
> > as well, in which case having them continue feels fine to me. However
> this
> > is a
> > relatively trivial benefit and I would only consider it as a deciding
> > factor when all
> > things are equal otherwise.
> >
> > My concern is the more interesting case: when this feature is used to
> pause
> > only
> > one nodes, or some subset of the overall application. In this case, yes,
> > the standby
> > tasks will indeed fall out of sync. But the only reason I can imagine
> > someone using
> > the pause feature in such a way is because there is something going
> wrong,
> > or about
> > to go wrong, on that particular node. For example as mentioned above, if
> > the user
> > wants to cut down on costs without stopping everything, or if the node is
> > about to
> > run out of disk or needs to be debugged or so on. And in this case,
> > continuing to
> > process the standby tasks while other instances continue to run would
> > pretty much
> > defeat the purpose of pausing it entirely, and might have unpleasant
> > consequences
> > for the unsuspecting developer.
> >
> > All that said, I don't want to block this KIP so if you have strong
> > feelings about the
> > standby behavior I'm happy to back down. I'm only pushing back now
> because
> > it
> > felt like there wasn't any particular motivation for the standbys to
> > continue processing
> > or not, and I figured I'd try to fill in this gap with my thoughts on the
> > matter :)
> > Either way we should just make sure that this behavior is documented
> > clearly,
> > since it may be surprising if we decide to only pause active processing
> > (another option
> > is to rename the method something like #pauseProcessing or
> > #pauseActiveProcessing
> > so that it's hard to miss).
> >
> > Thanks! Sorry for the lengthy response, but hopefully we won't need to
> > debate this any
> > further. Beyond this I'm satisfied with the latest proposal
> >
> > On Mon, May 9, 2022 at 5:16 PM John Roesler  wrote:
> >
> >> Thanks for the updates, Jim!
> >>
> >> After this discussion and your updates, this KIP looks good to me.
> >>
> >> Thanks,
> >> John
> >>
> >> On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:
> >>> Hi Sophie, all,
> >>>
> >>> I've updated the KIP with feedback from the discussion so far:
> >>>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> >>>
> >>> As a terse summary of my current position:
> >>> Pausing will only stop processing and punctuation (respecting modular
> >>> topologies).
> >>> Paused topologies will still a) consume from input topics, b) call the
> >>> usual commit pathways (commits will happen basically as they would
> have),
> >>> and c) standBy tasks will still be processed.
> >>>
> >>> Shout if the KIP or those details still need some TLC.  Responding to
> >>> Sophie inline below.
> >>>
> >>>
> >>> On Mon, May 9, 2022 at 6:06 PM Sophie Blee-Goldman
> >>>  wrote:
> >>>
>  Don't worry, I'm going to be adding the APIs for topology-level
> pausing
> >> as
>  part of the modular topologies KIP,
>  so we don't need to worry about that for now. That said, I don't think
> >> we
>  should brush it off entirely and design
>  this feature in a way that's going to be incompatible or hugely raise
> >> the
>  LOE on bringing the (mostly already
>  implemented) modular topologies feature into the public API, just
>  because it "won the race to write a KIP" :)
> 
> >>>
> >>> Yes, I'm hoping that this is all compatible with modular topologies.  I
> >>> haven't seen anything so far which seems to be a problem; this KIP is
> >> just
> >>> in a weird state to discuss details of acting on modular topologies.:)
> >>>
> >>>
>  I may 

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #920

2022-05-10 Thread Apache Jenkins Server
See 




[jira] [Created] (KAFKA-13892) Dedupe response objects in deleteAcls of AclControlManager

2022-05-10 Thread Andrew Grant (Jira)
Andrew Grant created KAFKA-13892:


 Summary: Dedupe response objects in deleteAcls of 
AclControlManager 
 Key: KAFKA-13892
 URL: https://issues.apache.org/jira/browse/KAFKA-13892
 Project: Kafka
  Issue Type: Bug
Reporter: Andrew Grant


In 
[https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java#L143]
 we loop through the ACL filters and and add RemoveAccessControlEntryRecord 
records to the response list for each matching ACL. I think there's a bug where 
if two filters match the same ACL, we create two RemoveAccessControlEntryRecord 
records for that same ACL. This is an issue because upon replay we throw an 
exception 
(https://github.com/apache/kafka/blob/trunk/metadata/src/main/java/org/apache/kafka/controller/AclControlManager.java#L195)
 if the ACL is not in the in-memory data structures which will happen to the 
second RemoveAccessControlEntryRecord.

Maybe we can just de-dupe both List and 
List? I think something like (just showing code for 
ApiMessageAndVersion):
{code:java}
private List 
deDupeApiMessageAndVersion(List messages) {
return new HashSet<>(messages).stream().collect(Collectors.toList());
}{code}
should suffice as I don't think the ordering matters within the list of 
response objects.  



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [HEADS-UP] Modification to KIP Template

2022-05-10 Thread Mickael Maison
Hi,

I did not see any comments nor concerns so I went ahead and added the
Test Plan section to the KIP template.

Thanks,
Mickael

On Fri, Apr 1, 2022 at 5:53 PM Mickael Maison  wrote:
>
> Hi,
>
> Unearthing this old thread as today I stumbled on the issue that
> Ismael reported. It looks like this was never fixed!
>
> The "Test Plan" section was only added in the KIP-template page [0]
> and not in the actual KIP-template template [1] that is used when
> doing `Create -> KIP-Template` or by clicking on `Create KIP` on
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>
> I think this new section makes sense and it's very easy to add it to
> the actual template. Before doing it, I just want to ping the dev list
> to see if anybody has suggestions or concerns since this was discussed
> many years ago now.
>
> 0: https://cwiki.apache.org/confluence/display/KAFKA/KIP-Template
> 1: 
> https://cwiki.apache.org/confluence/pages/templates2/viewpagetemplate.action?entityId=54329345=KAFKA
>
> Thanks,
> Mickael
>
> On Fri, May 27, 2016 at 10:55 AM Ismael Juma  wrote:
> >
> > Hi Gwen,
> >
> > Thanks for adding the "Test Plans" section. I think it may be worth adding
> > a note about performance testing plans too (whenever relevant). By the way,
> > even though the following page has the new section, if I use `Create ->
> > KIP-Template`, the new section doesn't appear. Do you know why?
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-Template
> >
> > Ismael
> >
> > On Fri, May 27, 2016 at 3:24 AM, Gwen Shapira  wrote:
> >
> > > Hi Kafka Developers,
> > >
> > > Just a quick heads-up that I added a new section to the KIP template: 
> > > "Test
> > > Plans".
> > > I think its a good habit to think about how a feature will be tested while
> > > planning it. I'm talking about high-level notes on system tests, not 
> > > gritty
> > > details.
> > >
> > > This will apply to new KIPs, not ones in discussion/implementation phases
> > > (although if your KIP is under discussion and you want to add test plans,
> > > it will be very nice of you).
> > >
> > > I figured we all agree that thinking a bit about tests is a good idea, so 
> > > I
> > > added it first and started a discussion later. If you strongly object,
> > > please respond with strong objections. Wikis are easy to edit :)
> > >
> > > Gwen
> > >


Re: [DISCUSS] KIP-827: Expose logdirs total and usable space via Kafka API

2022-05-10 Thread Mickael Maison
Hi Colin,

Thanks for the suggestion.

I guess there are pros and cons with both methods. In my mind I'm
expecting these values to always be there in the long run (once people
have upgraded to brokers that support this feature). So I thought
having a primitive directly may be nicer to use in the future. On the
other hand OptionalLong avoids exposing magic values to users and
makes error cases (brokers that support the feature but hit an error)
a bit nicer.

We seem to already be using Optional fields in a few *Result classes
so I'll follow your suggestion. As a bonus that also resolves the
naming questions from Divij!

I've updated the KIP accordingly.

Thanks,
Mickael

On Wed, May 4, 2022 at 2:52 PM Igor Soarez  wrote:
>
> Hi Mickael,
>
> Thanks for writing this KIP. This would be a very useful improvement!
>
> --
> Igor
>
> On Thu, Apr 7, 2022, at 10:16 AM, Mickael Maison wrote:
> > Hi,
> >
> > I wrote a small KIP to expose the total and usable space of logdirs
> > via the DescribeLogDirs API:
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-827%3A+Expose+logdirs+total+and+usable+space+via+Kafka+API
> >
> > Please take a look and let me know if you have any feedback.
> >
> > Thanks,
> > Mickael


Re: [VOTE] KIP-832 Allow creating a producer/consumer using a producer/consumer

2022-05-10 Thread John Roesler
+1 (binding)

Thanks, François!
-John

On Tue, May 10, 2022, at 03:09, Bruno Cadonna wrote:
> Hi Francois,
>
> Thanks for the KIP whose link is:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578
>
> +1 (binding)
>
> Best,
> Bruno
>
> On 09.05.22 23:23, François Rosière wrote:
>> 
>>


Re: [DISCUSS] KIP-831: Add metric for log recovery progress

2022-05-10 Thread Luke Chen
Hi James and all,

I checked again and I can see when creating UnifiedLog, we expected the
logs/indexes/snapshots are in good state.
So, I don't think we should break the current design to expose the
`RemainingBytesToRecovery`
metric.

If there is no other comments, I'll start a vote within this week.

Thank you.
Luke

On Fri, May 6, 2022 at 6:00 PM Luke Chen  wrote:

> Hi James,
>
> Thanks for your input.
>
> For the `RemainingBytesToRecovery` metric proposal, I think there's one
> thing I didn't make it clear.
> Currently, when log manager start up, we'll try to load all logs
> (segments), and during the log loading, we'll try to recover logs if
> necessary.
> And the logs loading is using "thread pool" as you thought.
>
> So, here's the problem:
> All segments in each log folder (partition) will be loaded in each log
> recovery thread, and until it's loaded, we can know how many segments (or
> how many Bytes) needed to recover.
> That means, if we have 10 partition logs in one broker, and we have 2 log
> recovery threads (num.recovery.threads.per.data.dir=2), before the
> threads load the segments in each log, we only know how many logs
> (partitions) we have in the broker (i.e. RemainingLogsToRecover metric).
> We cannot know how many segments/Bytes needed to recover until each thread
> starts to load the segments under one log (partition).
>
> So, the example in the KIP, it shows:
> Currently, there are still 5 logs (partitions) needed to recover under
> /tmp/log1 dir. And there are 2 threads doing the jobs, where one thread has
> 1 segments needed to recover, and the other one has 3 segments needed
> to recover.
>
>- kafka.log
>   - LogManager
>  - RemainingLogsToRecover
> - /tmp/log1 => 5← there are 5 logs under
> /tmp/log1 needed to be recovered
> - /tmp/log2 => 0
>  - RemainingSegmentsToRecover
> - /tmp/log1 ← 2 threads are doing log
> recovery for /tmp/log1
> - 0 => 1 ← there are 1 segments needed to be
>recovered for thread 0
>- 1 => 3
>- /tmp/log2
>- 0 => 0
>- 1 => 0
>
>
> So, after a while, the metrics might look like this:
> It said, now, there are only 4 logs needed to recover in /tmp/log1, and
> the thread 0 has 9000 segments left, and thread 1 has 5 segments left
> (which should imply the thread already completed 2 logs recovery in the
> period)
>
>- kafka.log
>   - LogManager
>  - RemainingLogsToRecover
> - /tmp/log1 => 3← there are 3 logs under
> /tmp/log1 needed to be recovered
> - /tmp/log2 => 0
>  - RemainingSegmentsToRecover
> - /tmp/log1 ← 2 threads are doing log
> recovery for /tmp/log1
> - 0 => 9000 ← there are 9000 segments needed to be
>recovered for thread 0
>- 1 => 5
>- /tmp/log2
>- 0 => 0
>- 1 => 0
>
>
> That said, the `RemainingBytesToRecovery` metric is difficult to achieve
> as you expected. I think the current proposal with `RemainingLogsToRecover`
> and `RemainingSegmentsToRecover` should already provide enough info for
> the log recovery progress.
>
> I've also updated the KIP example to make it clear.
>
>
> Thank you.
> Luke
>
>
> On Thu, May 5, 2022 at 3:31 AM James Cheng  wrote:
>
>> Hi Luke,
>>
>> Thanks for adding RemainingSegmentsToRecovery.
>>
>> Another thought: different topics can have different segment sizes. I
>> don't know how common it is, but it is possible. Some topics might want
>> small segment sizes to more granular expiration of data.
>>
>> The downside of RemainingLogsToRecovery and RemainingSegmentsToRecovery
>> is that the rate that they will decrement depends on the configuration and
>> patterns of the topics and partitions and segment sizes. If someone is
>> monitoring those metrics, they might see times where the metric decrements
>> slowly, followed by a burst where it decrements quickly.
>>
>> What about RemainingBytesToRecovery? This would not depend on the
>> configuration of the topic or of the data. It would actually be a pretty
>> good metric, because I think that this metric would change at a constant
>> rate (based on the disk I/O speed that the broker allocates to recovery).
>> Because it changes at a constant rate, you would be able to use the
>> rate-of-change to predict when it hits zero, which will let you know when
>> the broker is going to start up. Like, I would imagine if we graphed
>> RemainingBytesToRecovery that we'd see a fairly straight line that is
>> decrementing at a steady rate towards zero.
>>
>> What do you think about adding RemainingBytesToRecovery?
>>
>> Or, what would you think about making the primary metric be
>> RemainingBytesToRecovery, and getting rid of the others?
>>

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #919

2022-05-10 Thread Apache Jenkins Server
See 




Re: [DISCUSS] KIP-832 Allow creating a producer/consumer using a producer/consumer config

2022-05-10 Thread Chris Egerton
Hi Francois,

Thanks for the KIP! I sympathize with the issue you're facing and with
John's reluctance to let perfect be the enemy of good, and if KIP freeze
were tomorrow, I think this would be good enough. Given that we still have
some time to work with, I'd like to propose an alternative approach and see
what your thoughts are.

There are a few issues with the current client APIs that are closely
related to the KIP:
1. Too many constructors (there are currently four each for KafkaProducer
and KafkaConsumer, yet they all do basically the same thing)
2. Lack of type safety with interceptors (you have no way to enforce at
compile time that your ProducerInterceptor is used with a
Serializer and Serializer, for example)
3. Inflexibility and inconsistency with instantiation of pluggable
interfaces (you can bring your own (de)serializers, but everything else
gets instantiated and configured for you at producer/consumer construction
time)

The KIP as it exists now will only address item 3, and will exacerbate item
1.

In addition, there are a few new issues introduced by the KIP as it exists
now:
1. Tighter coupling between the ProducerConfig/ConsumerConfig classes and
the KafkaProducer/KafkaConsumer classes. Any change we make in the future
that breaks either of these config classes in unexpected ways (but which
does not break the KafkaProducer/KafkaConsumer constructors that do not
accept these classes as parameters) will now have a much higher chance to
also break a user's entire producer/consumer application.
2. Complexity for users like yourself who would like to override behavior
in a ProducerConfig/ConsumerConfig in order to inject pre-instantiated
dependencies. The example in the KIP overrides
AbstractConfig::getConfiguredInstances [1] in order to achieve this. But
there are two other overloaded variants of getConfiguredInstances, and two
AbstractConfig::getConfiguredInstance methods that also exist. We'd either
need to establish a dependency graph between these methods (e.g., some
methods are guaranteed to invoke another overloaded variant) as part of the
public API for the AbstractConfig, or users would need to override every
single one of these methods in order to ensure that their code won't break
at runtime after bumping their Kafka version.

I think introducing the builder pattern for KafkaProducer and KafkaConsumer
would alleviate all of these issues. As a rough draft of what the API might
look like for KafkaProducer:

public class Builder {
private final Map props;
private Serializer keySerializer;
private Serializer valueSerializer;
private List> interceptors;
private Map configuredInstances;
private Map> configuredInstanceLists;

public Builder(Map props) {
this.props = props;
this.interceptors = new ArrayList<>();
this.configuredInstances = new HashMap<>();
this.configuredInstanceLists = new HashMap<>();
}

// Use this serializer, if non-null
// Will take precedence over any serializer class specified in the
properties for this producer
public Builder withKeySerializer(Serializer serializer) {
this.keySerializer = serializer;
return this;
}

public Builder withValueSerializer(Serializer serializer) {
this.valueSerializer = serializer;
return this;
}

// Use these interceptors (has no effect if null)
// Each must already be configured
// Will be combined with any interceptor classes also specified in the
properties for this producer
public Builder withInterceptors(List>
interceptors) {
if (interceptors != null) {
this.interceptors.addAll(interceptors);
}
return this;
}

// Use this plugin instance, if non-null
// Must already be configured
// Will take precedence over any plugin class specified for the same
property in the properties for this producer (wording here needs work but
you get the idea)
public Builder withConfiguredInstance(String property, Object instance)
{
this.configuredInstances.put(property, instance);
return this;
}

// Use these plugin instances (has no effect if null)
// Each must already be configured
// Will be combined with any plugin classes also specified for the same
property in the properties for this consumer
public Builder withConfiguredInstances(String property, List
instances) {
this.configuredInstanceLists.put(property, instances);
return this;
}

public KafkaProducer build() { ... }
}

Thoughts?

[1] -
https://kafka.apache.org/31/javadoc/org/apache/kafka/common/config/AbstractConfig.html#getConfiguredInstances(java.lang.String,java.lang.Class,java.util.Map)

Cheers,

Chris

On Mon, May 9, 2022 at 4:55 PM Bruno Cadonna  wrote:

> Hi Francois,
>
> I think you can go ahead and call for votes.
>
> Could you please also clean up a little bit the KIP since it has still
> parts that refer to its first version? For 

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.1 #119

2022-05-10 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 505500 lines...]
[2022-05-10T12:44:18.193Z] 
[2022-05-10T12:44:18.193Z] KafkaZkClientTest > testGetTopicsAndPartitions() 
STARTED
[2022-05-10T12:44:18.193Z] 
[2022-05-10T12:44:18.193Z] KafkaZkClientTest > testGetTopicsAndPartitions() 
PASSED
[2022-05-10T12:44:18.193Z] 
[2022-05-10T12:44:18.193Z] KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[1] STARTED
[2022-05-10T12:44:19.215Z] 
[2022-05-10T12:44:19.215Z] KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[1] PASSED
[2022-05-10T12:44:19.215Z] 
[2022-05-10T12:44:19.215Z] KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[2] STARTED
[2022-05-10T12:44:19.215Z] 
[2022-05-10T12:44:19.215Z] KafkaZkClientTest > testChroot(boolean) > 
kafka.zk.KafkaZkClientTest.testChroot(boolean)[2] PASSED
[2022-05-10T12:44:19.215Z] 
[2022-05-10T12:44:19.215Z] KafkaZkClientTest > testRegisterBrokerInfo() STARTED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > testRegisterBrokerInfo() PASSED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > testRetryRegisterBrokerInfo() 
STARTED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > testRetryRegisterBrokerInfo() 
PASSED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > testConsumerOffsetPath() STARTED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > testConsumerOffsetPath() PASSED
[2022-05-10T12:44:20.407Z] 
[2022-05-10T12:44:20.407Z] KafkaZkClientTest > 
testDeleteRecursiveWithControllerEpochVersionCheck() STARTED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > 
testDeleteRecursiveWithControllerEpochVersionCheck() PASSED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > testTopicAssignments() STARTED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > testTopicAssignments() PASSED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > 
testControllerManagementMethods() STARTED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > 
testControllerManagementMethods() PASSED
[2022-05-10T12:44:21.458Z] 
[2022-05-10T12:44:21.458Z] KafkaZkClientTest > testTopicAssignmentMethods() 
STARTED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testTopicAssignmentMethods() 
PASSED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testConnectionViaNettyClient() 
STARTED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testConnectionViaNettyClient() 
PASSED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testPropagateIsrChanges() STARTED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testPropagateIsrChanges() PASSED
[2022-05-10T12:44:22.481Z] 
[2022-05-10T12:44:22.481Z] KafkaZkClientTest > testControllerEpochMethods() 
STARTED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > testControllerEpochMethods() 
PASSED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > testDeleteRecursive() STARTED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > testDeleteRecursive() PASSED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > testGetTopicPartitionStates() 
STARTED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > testGetTopicPartitionStates() 
PASSED
[2022-05-10T12:44:23.505Z] 
[2022-05-10T12:44:23.505Z] KafkaZkClientTest > 
testCreateConfigChangeNotification() STARTED
[2022-05-10T12:44:24.527Z] 
[2022-05-10T12:44:24.527Z] KafkaZkClientTest > 
testCreateConfigChangeNotification() PASSED
[2022-05-10T12:44:24.527Z] 
[2022-05-10T12:44:24.527Z] KafkaZkClientTest > testDelegationTokenMethods() 
STARTED
[2022-05-10T12:44:24.527Z] 
[2022-05-10T12:44:24.527Z] KafkaZkClientTest > testDelegationTokenMethods() 
PASSED
[2022-05-10T12:44:24.527Z] 
[2022-05-10T12:44:24.527Z] LogCleanerLagIntegrationTest > 
cleanerTest(CompressionType) > 
kafka.log.LogCleanerLagIntegrationTest.cleanerTest(CompressionType)[1] STARTED
[2022-05-10T12:44:47.232Z] 
[2022-05-10T12:44:47.232Z] LogCleanerLagIntegrationTest > 
cleanerTest(CompressionType) > 
kafka.log.LogCleanerLagIntegrationTest.cleanerTest(CompressionType)[1] PASSED
[2022-05-10T12:44:47.232Z] 
[2022-05-10T12:44:47.232Z] LogCleanerLagIntegrationTest > 
cleanerTest(CompressionType) > 
kafka.log.LogCleanerLagIntegrationTest.cleanerTest(CompressionType)[2] STARTED
[2022-05-10T12:45:22.298Z] 
[2022-05-10T12:45:22.298Z] LogCleanerLagIntegrationTest > 
cleanerTest(CompressionType) > 
kafka.log.LogCleanerLagIntegrationTest.cleanerTest(CompressionType)[2] PASSED

[jira] [Created] (KAFKA-13891) sync group failed with rebalanceInProgress error cause rebalance many rounds in coopeartive

2022-05-10 Thread Shawn Wang (Jira)
Shawn Wang created KAFKA-13891:
--

 Summary: sync group failed with rebalanceInProgress error cause 
rebalance many rounds in coopeartive
 Key: KAFKA-13891
 URL: https://issues.apache.org/jira/browse/KAFKA-13891
 Project: Kafka
  Issue Type: Bug
  Components: clients
Affects Versions: 3.0.0
Reporter: Shawn Wang


This issue was first found in 
[KAFKA-13419|https://issues.apache.org/jira/browse/KAFKA-13419]

But the previous PR forgot to reset generation when sync group failed with 
rebalanceInProgress error. So the previous bug still exists and it may cause 
consumer to rebalance many rounds before final stable.

Here's the example ({*}bold is added{*}):
 # consumer A joined and synced group successfully with generation 1 *( with 
ownedPartition P1/P2 )*
 # New rebalance started with generation 2, consumer A joined successfully, but 
somehow, consumer A doesn't send out sync group immediately
 # other consumer completed sync group successfully in generation 2, except 
consumer A.
 # After consumer A send out sync group, the new rebalance start, with 
generation 3. So consumer A got REBALANCE_IN_PROGRESS error with sync group 
response
 # When receiving REBALANCE_IN_PROGRESS, we re-join the group, with generation 
3, with the assignment (ownedPartition) in generation 1.
 # So, now, we have out-of-date ownedPartition sent, with unexpected results 
happened
 # *After the generation-3 rebalance, consumer A got P3/P4 partition. the 
ownedPartition is ignored because of old generation.*
 # *consumer A revoke P1/P2 and re-join to start a new round of rebalance*
 # *if some other consumer C failed to syncGroup before consumer A's joinGroup. 
the same issue will happens again and result in many rounds of rebalance before 
stable*

 



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: [DISCUSS] KIP-830: Allow disabling JMX Reporter

2022-05-10 Thread Mickael Maison
Hi Ismael,

The last option you suggested seems like a good tradeoff, I've updated
the KIP to follow this idea.

Thanks,
Mickael

On Mon, May 2, 2022 at 10:32 PM Ismael Juma  wrote:
>
> The other option would be to add a deprecated config that then goes away in
> AK 4.0. Something like "auto.include.jmx.reporter" and set it to `true` by
> default. In 4.0, the default would effectively switch to `false` and the
> config would be removed. Anyone with a config that would _change_ after
> this config goes away would get a warning. For people who want to remove
> the jmx reporter, they would set this to `false`.
>
> Ismael
>
> On Mon, May 2, 2022 at 9:35 AM Mickael Maison 
> wrote:
>
> > Hi Ismael,
> >
> > Thanks for the feedback.
> > Kafka 4.0 still feels a bit far away. This is not really urgent so I
> > guess we could wait and save us adding a new configuration setting.
> >
> > Thanks,
> > Mickael
> >
> > On Mon, May 2, 2022 at 6:12 PM Ismael Juma  wrote:
> > >
> > > Hi Mickael,
> > >
> > > I think Federico is suggesting that we would do this in AK 4.0 and hence
> > > the breaking change (with the appropriate deprecation period) would be
> > > possible. I personally prefer that option too. Is there an urgent need to
> > > do this before 4.0?
> > >
> > > Ismael
> > >
> > > On Mon, May 2, 2022 at 8:58 AM Mickael Maison 
> > > wrote:
> > >
> > > > HI Federico,
> > > >
> > > > Thanks for taking a look.
> > > >
> > > > I think the method you suggest is what I included in the rejected
> > > > alternatives section. The reason I rejected it is because it would
> > > > force people to change their configuration in order to preserve the
> > > > current behavior, hence a breaking change.
> > > >
> > > > Thanks,
> > > > Mickael
> > > >
> > > > On Fri, Apr 22, 2022 at 6:26 PM Federico Valeri 
> > > > wrote:
> > > > >
> > > > > Hi Mickael, what about setting the default value to JmxReporter while
> > > > > also maintaining the old behavior? When a user is setting
> > > > > metric.reporters without explicitly including the JmxReporter, we
> > > > > could simply print a warning message which says "The automatic
> > > > > addition of the JmxReporter is deprecated. Add it explicitly to the
> > > > > metric.reporters list if needed.", or something like that. That way
> > we
> > > > > can avoid adding a new property, and drop the implicit setting in the
> > > > > next major release.
> > > > >
> > > > > On Wed, Apr 13, 2022 at 11:49 AM Mickael Maison
> > > > >  wrote:
> > > > > >
> > > > > > Hi,
> > > > > >
> > > > > > I've created a small KIP to allow disabling JMXreporter:
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-830%3A+Allow+disabling+JMX+Reporter
> > > > > >
> > > > > > Let me know if you have any feedback.
> > > > > >
> > > > > > Thanks,
> > > > > > Mickael
> > > >
> >


Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #918

2022-05-10 Thread Apache Jenkins Server
See 




Jenkins build is still unstable: Kafka » Kafka Branch Builder » 3.1 #118

2022-05-10 Thread Apache Jenkins Server
See 




Re: [VOTE] KIP-832 Allow creating a producer/consumer using a producer/consumer

2022-05-10 Thread Bruno Cadonna

Hi Francois,

Thanks for the KIP whose link is:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882578

+1 (binding)

Best,
Bruno

On 09.05.22 23:23, François Rosière wrote:





Re: [DISCUSS] KIP-834: Pause / Resume KafkaStreams Topologies

2022-05-10 Thread Bruno Cadonna

Hi Jim,

Thanks for the KIP!

I am fine with the KIP in general.

However, I am with Sophie and John to also pause the standbys for the 
reasons they brought up. Is there a specific reason you want to keep 
standbys going? It feels like premature optimization to me. We can still 
add keeping standby running in a follow up if needed.


Best,
Bruno

On 10.05.22 05:15, Sophie Blee-Goldman wrote:

Thanks Jim, just one note/question on the standby tasks:

At the minute, my moderately held position is that standby tasks ought to

continue reading and remain caught up.  If standby tasks would run out of
space, there are probably bigger problems.



For a single node application, or when the #pause API is invoked on all
instances,
then there won't be any further active processing and thus nothing to keep
up with,
right? So for that case, it's just a matter of whether any standbys that
are lagging
will have the chance to catch up to the (paused) active task state before
they stop
as well, in which case having them continue feels fine to me. However this
is a
relatively trivial benefit and I would only consider it as a deciding
factor when all
things are equal otherwise.

My concern is the more interesting case: when this feature is used to pause
only
one nodes, or some subset of the overall application. In this case, yes,
the standby
tasks will indeed fall out of sync. But the only reason I can imagine
someone using
the pause feature in such a way is because there is something going wrong,
or about
to go wrong, on that particular node. For example as mentioned above, if
the user
wants to cut down on costs without stopping everything, or if the node is
about to
run out of disk or needs to be debugged or so on. And in this case,
continuing to
process the standby tasks while other instances continue to run would
pretty much
defeat the purpose of pausing it entirely, and might have unpleasant
consequences
for the unsuspecting developer.

All that said, I don't want to block this KIP so if you have strong
feelings about the
standby behavior I'm happy to back down. I'm only pushing back now because
it
felt like there wasn't any particular motivation for the standbys to
continue processing
or not, and I figured I'd try to fill in this gap with my thoughts on the
matter :)
Either way we should just make sure that this behavior is documented
clearly,
since it may be surprising if we decide to only pause active processing
(another option
is to rename the method something like #pauseProcessing or
#pauseActiveProcessing
so that it's hard to miss).

Thanks! Sorry for the lengthy response, but hopefully we won't need to
debate this any
further. Beyond this I'm satisfied with the latest proposal

On Mon, May 9, 2022 at 5:16 PM John Roesler  wrote:


Thanks for the updates, Jim!

After this discussion and your updates, this KIP looks good to me.

Thanks,
John

On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:

Hi Sophie, all,

I've updated the KIP with feedback from the discussion so far:


https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832


As a terse summary of my current position:
Pausing will only stop processing and punctuation (respecting modular
topologies).
Paused topologies will still a) consume from input topics, b) call the
usual commit pathways (commits will happen basically as they would have),
and c) standBy tasks will still be processed.

Shout if the KIP or those details still need some TLC.  Responding to
Sophie inline below.


On Mon, May 9, 2022 at 6:06 PM Sophie Blee-Goldman
 wrote:


Don't worry, I'm going to be adding the APIs for topology-level pausing

as

part of the modular topologies KIP,
so we don't need to worry about that for now. That said, I don't think

we

should brush it off entirely and design
this feature in a way that's going to be incompatible or hugely raise

the

LOE on bringing the (mostly already
implemented) modular topologies feature into the public API, just
because it "won the race to write a KIP" :)



Yes, I'm hoping that this is all compatible with modular topologies.  I
haven't seen anything so far which seems to be a problem; this KIP is

just

in a weird state to discuss details of acting on modular topologies.:)



I may be biased (ok, I definitely am), but I'm not in favor of adding

this

as a state regardless of the modular topologies.
First of all any change to the KafkaStreams state machine is a breaking
change, no? So we would have to wait until
the next major release which seems like an unnecessary thing to block

on.

(Whether to add this as a state to the
StreamThread's FSM is an implementation detail).



+1.  I am sold on skipping out on new states.  I had that as a rejected
alternative in the KIP and have added a few more words to that bit.



Also, the semantics of using an `isPaused` method to distinguish a

paused

instance (or topology) make more sense
to me -- this is a user-specified status, whereas the KafkaStreams

state is


Re: Contributor access

2022-05-10 Thread Bruno Cadonna

Hi Jakub,

I added you to the contributors in Jira. You are now able to assign 
tickets to yourself.


Thank you for your interest in Apache Kafka!

Best,
Bruno

On 10.05.22 00:22, Jakub Miroś wrote:

Hello,

I want to offer a help as a new contributor. Please add me to the Jira, my
ID is SpacRocket

Regards,

Jakub Miroś