Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-26 Thread Luke Chen
Hi Kamal,

Thanks for the comments.

For this:
> If we throw an exception from the server for invalid config, then there
will be inconsistency between the CLI tools and the actual state of the
topic in the cluster. This can cause some confusion to the users whether
tiered storage is disabled or not. I don't know how the Kraft topic config
propagation/validation works.

I've confirmed we can validate the topic configuration change on the
controller level, by comparing existing configuration and new changed
configuration.
In my local POC, we can fail the configuration change if it's invalid like
this:

# Disable with remote.log.delete.on.disable=false (default)
bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
   --alter --entity-type topics --entity-name {topic-name} \
   --add-config 'remote.storage.enable=false'

Error while executing config command with args '--bootstrap-server
{bootstrap-string} --entity-type topics --entity-name {topic-name} --alter
--add-config remote.storage.enable=false'
java.util.concurrent.ExecutionException:
org.apache.kafka.common.errors.InvalidConfigurationException: It is invalid
to disable remote storage without deleting remote data. If you want to keep
the remote data, but turn to read only, please set `remote.copy.disabled=
true`. If you want to disable remote storage and delete all remote data,
please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.

I've updated the KIP. Please take a look when available.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement

Thank you.
Luke


On Fri, Jul 26, 2024 at 2:05 AM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Correction:
>
> (2): Wait for all the remote segments to be deleted async due to breach by
> retention time (or) size,
>then set the `remote.storage.enable = false` and
> `remote.log.delete.on.disable = true`. This step is optional.
>
> On Thu, Jul 25, 2024 at 11:13 PM Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
>
> > Hi Chia-Ping,
> >
> > Thanks for the review!
> >
> > >  If so, what is the purpose of `remote.log.delete.on.disable=false`?
> >
> > IIUC, the purpose of `remote.log.delete.on.disable` is to get explicit
> > confirmation from the user
> > before deleting the remote log segments. The concern raised in the thread
> > is that if the user
> > accidentally changes the value of `remote.storage.enable` from true to
> > false, then remote segments
> > get lost.
> >
> > For ungraceful disablement, (ie) disabling the remote storage for the
> > topic and deleting all the
> > remote segments, the user should set both the configs at once:
> >
> > (1) remote.storage.enable = false and remote.log.delete.on.disable = true
> >
> > If the user accidentally sets only the remote.storage.enable = true and
> > leaves the `remote.log.delete.on.disable`
> > with default value of `false`, then we will throw ConfigException to
> > prevent the deletion of remote logs.
> >
> > For graceful disablement, the user should set:
> >
> > (1): remote.copy.disabled = true.
> > (2): Wait for all the remote segments to be deleted async due to breach
> by
> > retention time (or) size,
> >then set the `remote.storage.enable = false`. This step is
> > optional.
> >
> > Luke,
> >
> > In ZK mode, once the topic config value gets updated, then it gets saved
> > in the /configs/topics/ znode.
> > If we throw an exception from the server for invalid config, then there
> > will be inconsistency between the CLI tools
> > and the actual state of the topic in the cluster. This can cause some
> > confusion to the users whether tiered storage
> > is disabled or not. I don't know how the Kraft topic config
> > propagation/validation works.
> >
> > --
> > Kamal
> >
> > On Thu, Jul 25, 2024 at 7:10 PM Chia-Ping Tsai 
> wrote:
> >
> >> remote.storage.enable=false
> >> remote.log.delete.on.disable=false (default)
> >> If the topic config is set to this, or changed to this, we'll return
> >> ConfigException during validation.
> >>
> >> Pardon me, I'm a bit confused.
> >>
> >> when `remote.storage.enable=true`, `remote.log.delete.on.disable=false`
> is
> >> no-op
> >> when `remote.storage.enable=false`, `remote.log.delete.on.disable=false`
> >> is
> >> error
> >>
> >> If `remote.log.delete.on.disable` must be true when setting
> >> `remote.storage.enable`
> >> to false, does it mean changing `remote.storage.enable` to false is
> >> expected to delete remote storage topic data"?
> >>
> >>  If so, what is the purpose of `remote.log.delete.on.disable=false`?
> >>
> >> Best,
> >> Chia-Ping
> >>
> >> Luke Chen  於 2024年7月25日 週四 下午8:51寫道:
> >>
> >> > Hi Christo,
> >> >
> >> > Thanks for your reply.
> >> >
> >> > > keep the remote.log.disable.policy, but only allow it to take a
> value
> >> of
> >> > "delete".
> >> >
> >> > I agree, or maybe make it a boolean value, and rename it to
> >> > `remote.log.delete.on.disable`, which is clearer.
> >> > And because 

[jira] [Created] (KAFKA-17205) Allow topic config validation in controller level

2024-07-26 Thread Luke Chen (Jira)
Luke Chen created KAFKA-17205:
-

 Summary: Allow topic config validation in controller level 
 Key: KAFKA-17205
 URL: https://issues.apache.org/jira/browse/KAFKA-17205
 Project: Kafka
  Issue Type: Sub-task
Reporter: Luke Chen
Assignee: Luke Chen


Allow topic config validation in controller level. This is required because we 
need to fail the invalid config change before it is written into metadata log, 
especially for tiered storage feature.



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


Re: [VOTE] KIP-1068: New metrics for the new KafkaConsumer

2024-07-26 Thread Andrew Schofield
HI Brenden,
Thanks for the KIP.

+1 (non-binding)

Thanks,
Andrew

> On 25 Jul 2024, at 23:26, Apoorv Mittal  wrote:
>
> Hi Brenden,
> The metrics look good. Thanks for the KIP.
>
> +1 (Non-Binding)
>
> Regards,
> Apoorv Mittal
>
>
> On Thu, Jul 25, 2024 at 5:04 PM Philip Nee 
> wrote:
>
>> Hi Brenden,
>>
>> Thanks for the KIP. +1 (non-binding).
>>
>> P
>>
>> On Thu, Jul 25, 2024 at 6:25 AM Lianet M.  wrote:
>>
>>> Hi Brenden,
>>>
>>> +1 (non-binding) from me.
>>>
>>> Thanks for the KIP!
>>>
>>> Lianet
>>>
>>> On Thu, Jul 25, 2024 at 3:18 AM Bruno Cadonna 
>> wrote:
>>>
 Hi Brenden,

 Thanks for the KIP!

 +1 (binding)

 Best,
 Bruno

 On 7/24/24 11:55 PM, Brenden Deluna wrote:
> Hello everyone,
>
> I would like to start the vote on KIP-1068:
>

>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1068%3A+New+metrics+for+the+new+KafkaConsumer
>
> This KIP introduces new metrics for the new KafkaConsumer
>>> implementation
> for broader metrics coverage.
>
> Thanks,
> Brenden
>

>>>
>>



Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-26 Thread Kamal Chandraprakash
Luke,

Thanks for confirming the topic config change validation on the controller
and updating the KIP.
The updated KIP LGTM.

1. Can we update the below sentence in the KIP to clarify that
remote.storage.enable should be true during graceful disablement?

> Users set the configuration
"remote.storage.enable=false,remote.log.delete.on.disable=true", or
"remote.copy.disabled=true" for the desired topic, indicating the
disablement of tiered storage.
to
> Users set the configuration
"remote.storage.enable=false,remote.log.delete.on.disable=true", or
"remote.storage.enable=true,remote.copy.disabled=true" for the desired
topic, indicating the disablement of tiered storage.

2. Can we clarify in the public interface that the StopReplica v5,
tiered_epoch, and tiered_state changes are required only for ZK mode and
won't be implemented?

Thanks,
Kamal

On Fri, Jul 26, 2024 at 1:40 PM Luke Chen  wrote:

> Hi Kamal,
>
> Thanks for the comments.
>
> For this:
> > If we throw an exception from the server for invalid config, then there
> will be inconsistency between the CLI tools and the actual state of the
> topic in the cluster. This can cause some confusion to the users whether
> tiered storage is disabled or not. I don't know how the Kraft topic config
> propagation/validation works.
>
> I've confirmed we can validate the topic configuration change on the
> controller level, by comparing existing configuration and new changed
> configuration.
> In my local POC, we can fail the configuration change if it's invalid like
> this:
>
> # Disable with remote.log.delete.on.disable=false (default)
> bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
>--alter --entity-type topics --entity-name {topic-name} \
>--add-config 'remote.storage.enable=false'
>
> Error while executing config command with args '--bootstrap-server
> {bootstrap-string} --entity-type topics --entity-name {topic-name} --alter
> --add-config remote.storage.enable=false'
> java.util.concurrent.ExecutionException:
> org.apache.kafka.common.errors.InvalidConfigurationException: It is invalid
> to disable remote storage without deleting remote data. If you want to keep
> the remote data, but turn to read only, please set `remote.copy.disabled=
> true`. If you want to disable remote storage and delete all remote data,
> please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.
>
> I've updated the KIP. Please take a look when available.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
>
> Thank you.
> Luke
>
>
> On Fri, Jul 26, 2024 at 2:05 AM Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
>
> > Correction:
> >
> > (2): Wait for all the remote segments to be deleted async due to breach
> by
> > retention time (or) size,
> >then set the `remote.storage.enable = false` and
> > `remote.log.delete.on.disable = true`. This step is optional.
> >
> > On Thu, Jul 25, 2024 at 11:13 PM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> > > Hi Chia-Ping,
> > >
> > > Thanks for the review!
> > >
> > > >  If so, what is the purpose of `remote.log.delete.on.disable=false`?
> > >
> > > IIUC, the purpose of `remote.log.delete.on.disable` is to get explicit
> > > confirmation from the user
> > > before deleting the remote log segments. The concern raised in the
> thread
> > > is that if the user
> > > accidentally changes the value of `remote.storage.enable` from true to
> > > false, then remote segments
> > > get lost.
> > >
> > > For ungraceful disablement, (ie) disabling the remote storage for the
> > > topic and deleting all the
> > > remote segments, the user should set both the configs at once:
> > >
> > > (1) remote.storage.enable = false and remote.log.delete.on.disable =
> true
> > >
> > > If the user accidentally sets only the remote.storage.enable = true and
> > > leaves the `remote.log.delete.on.disable`
> > > with default value of `false`, then we will throw ConfigException to
> > > prevent the deletion of remote logs.
> > >
> > > For graceful disablement, the user should set:
> > >
> > > (1): remote.copy.disabled = true.
> > > (2): Wait for all the remote segments to be deleted async due to breach
> > by
> > > retention time (or) size,
> > >then set the `remote.storage.enable = false`. This step is
> > > optional.
> > >
> > > Luke,
> > >
> > > In ZK mode, once the topic config value gets updated, then it gets
> saved
> > > in the /configs/topics/ znode.
> > > If we throw an exception from the server for invalid config, then there
> > > will be inconsistency between the CLI tools
> > > and the actual state of the topic in the cluster. This can cause some
> > > confusion to the users whether tiered storage
> > > is disabled or not. I don't know how the Kraft topic config
> > > propagation/validation works.
> > >
> > > --
> > > Kamal
> > >
> > > On Thu, Jul 25, 2024 at 7:10 PM Chia-Ping Tsai 
> > wrote:
> > >
> > >> remote.storage.ena

Re: [VOTE] KIP-950: Tiered Storage Disablement

2024-07-26 Thread Luke Chen
Thanks Kamal for the comments.
KIP updated.

Thanks.
Luke

On Fri, Jul 26, 2024 at 6:56 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> Luke,
>
> Thanks for confirming the topic config change validation on the controller
> and updating the KIP.
> The updated KIP LGTM.
>
> 1. Can we update the below sentence in the KIP to clarify that
> remote.storage.enable should be true during graceful disablement?
>
> > Users set the configuration
> "remote.storage.enable=false,remote.log.delete.on.disable=true", or
> "remote.copy.disabled=true" for the desired topic, indicating the
> disablement of tiered storage.
> to
> > Users set the configuration
> "remote.storage.enable=false,remote.log.delete.on.disable=true", or
> "remote.storage.enable=true,remote.copy.disabled=true" for the desired
> topic, indicating the disablement of tiered storage.
>
> 2. Can we clarify in the public interface that the StopReplica v5,
> tiered_epoch, and tiered_state changes are required only for ZK mode and
> won't be implemented?
>
> Thanks,
> Kamal
>
> On Fri, Jul 26, 2024 at 1:40 PM Luke Chen  wrote:
>
> > Hi Kamal,
> >
> > Thanks for the comments.
> >
> > For this:
> > > If we throw an exception from the server for invalid config, then there
> > will be inconsistency between the CLI tools and the actual state of the
> > topic in the cluster. This can cause some confusion to the users whether
> > tiered storage is disabled or not. I don't know how the Kraft topic
> config
> > propagation/validation works.
> >
> > I've confirmed we can validate the topic configuration change on the
> > controller level, by comparing existing configuration and new changed
> > configuration.
> > In my local POC, we can fail the configuration change if it's invalid
> like
> > this:
> >
> > # Disable with remote.log.delete.on.disable=false (default)
> > bin/kafka-configs.sh --bootstrap-server {bootstrap-string} \
> >--alter --entity-type topics --entity-name {topic-name} \
> >--add-config 'remote.storage.enable=false'
> >
> > Error while executing config command with args '--bootstrap-server
> > {bootstrap-string} --entity-type topics --entity-name {topic-name}
> --alter
> > --add-config remote.storage.enable=false'
> > java.util.concurrent.ExecutionException:
> > org.apache.kafka.common.errors.InvalidConfigurationException: It is
> invalid
> > to disable remote storage without deleting remote data. If you want to
> keep
> > the remote data, but turn to read only, please set `remote.copy.disabled=
> > true`. If you want to disable remote storage and delete all remote data,
> > please set
> `remote.storage.enable=false,remote.log.delete.on.disable=true`.
> >
> > I've updated the KIP. Please take a look when available.
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-950%3A++Tiered+Storage+Disablement
> >
> > Thank you.
> > Luke
> >
> >
> > On Fri, Jul 26, 2024 at 2:05 AM Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> >
> > > Correction:
> > >
> > > (2): Wait for all the remote segments to be deleted async due to breach
> > by
> > > retention time (or) size,
> > >then set the `remote.storage.enable = false` and
> > > `remote.log.delete.on.disable = true`. This step is optional.
> > >
> > > On Thu, Jul 25, 2024 at 11:13 PM Kamal Chandraprakash <
> > > kamal.chandraprak...@gmail.com> wrote:
> > >
> > > > Hi Chia-Ping,
> > > >
> > > > Thanks for the review!
> > > >
> > > > >  If so, what is the purpose of
> `remote.log.delete.on.disable=false`?
> > > >
> > > > IIUC, the purpose of `remote.log.delete.on.disable` is to get
> explicit
> > > > confirmation from the user
> > > > before deleting the remote log segments. The concern raised in the
> > thread
> > > > is that if the user
> > > > accidentally changes the value of `remote.storage.enable` from true
> to
> > > > false, then remote segments
> > > > get lost.
> > > >
> > > > For ungraceful disablement, (ie) disabling the remote storage for the
> > > > topic and deleting all the
> > > > remote segments, the user should set both the configs at once:
> > > >
> > > > (1) remote.storage.enable = false and remote.log.delete.on.disable =
> > true
> > > >
> > > > If the user accidentally sets only the remote.storage.enable = true
> and
> > > > leaves the `remote.log.delete.on.disable`
> > > > with default value of `false`, then we will throw ConfigException to
> > > > prevent the deletion of remote logs.
> > > >
> > > > For graceful disablement, the user should set:
> > > >
> > > > (1): remote.copy.disabled = true.
> > > > (2): Wait for all the remote segments to be deleted async due to
> breach
> > > by
> > > > retention time (or) size,
> > > >then set the `remote.storage.enable = false`. This step is
> > > > optional.
> > > >
> > > > Luke,
> > > >
> > > > In ZK mode, once the topic config value gets updated, then it gets
> > saved
> > > > in the /configs/topics/ znode.
> > > > If we throw an exception from the server for invalid config, t

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

2024-07-26 Thread Apache Jenkins Server
See 




Queries regarding kafka logs

2024-07-26 Thread Pasupuleti Bhanusri
Hi,

I am facing issue with kafka logs clearing. Please take this to development 
team i want to discuss with them.

Issue :kafka server is running,kafka consumer is running is running in a wpf 
application, a batch file to delete kafka logs is running for every one hour 
from task scheduler at that time getting error in application as below

2024-07-25 17:43:13,264 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:DESKTOP-NDMP5G8:9092/0]: DESKTOP-NDMP5G8:9092/0: Disconnected (after 
15950ms in state UP)
2024-07-25 17:43:13,269 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:GroupCoordinator]: GroupCoordinator: DESKTOP-NDMP5G8:9092: Disconnected 
(after 16019ms in state UP)
2024-07-25 17:43:13,271 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:127.0.0.1:9092/bootstrap]: 127.0.0.1:9092/bootstrap: Disconnected (after 
16025ms in state UP)
2024-07-25 17:43:13,272 ERROR TestBench.Diagnostics.Logger - Kafka error: 
GroupCoordinator: DESKTOP-NDMP5G8:9092: Disconnected (after 16019ms in state UP)
2024-07-25 17:43:13,272 ERROR TestBench.Diagnostics.Logger - Kafka error: 
DESKTOP-NDMP5G8:9092/0: Disconnected (after 15950ms in state UP)
2024-07-25 17:43:13,272 ERROR TestBench.Diagnostics.Logger - Kafka error: 
127.0.0.1:9092/bootstrap: Disconnected (after 16025ms in state UP)
2024-07-25 17:43:13,272 ERROR TestBench.Diagnostics.Logger - Kafka error: 3/3 
brokers are down
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:127.0.0.1:9092/bootstrap]: 127.0.0.1:9092/bootstrap: Connect to 
ipv4#127.0.0.1:9092 failed: Unknown error (after 2034ms in state CONNECT)
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:GroupCoordinator]: GroupCoordinator: DESKTOP-NDMP5G8:9092: Connect to 
ipv4#192.168.1.103:9092 failed: Unknown error (after 2034ms in state CONNECT)
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:DESKTOP-NDMP5G8:9092/0]: DESKTOP-NDMP5G8:9092/0: Connect to 
ipv4#192.168.1.103:9092 failed: Unknown error (after 2034ms in state CONNECT)
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka error: 
127.0.0.1:9092/bootstrap: Connect to ipv4#127.0.0.1:9092 failed: Unknown error 
(after 2034ms in state CONNECT)
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka error: 
DESKTOP-NDMP5G8:9092/0: Connect to ipv4#192.168.1.103:9092 failed: Unknown 
error (after 2034ms in state CONNECT)
2024-07-25 17:43:15,305 ERROR TestBench.Diagnostics.Logger - Kafka error: 
GroupCoordinator: DESKTOP-NDMP5G8:9092: Connect to ipv4#192.168.1.103:9092 
failed: Unknown error (after 2034ms in state CONNECT)
The thread 0x4510 has exited with code 0 (0x0).
2024-07-25 17:43:22,346 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:main]: Consumer group session timed out (in join-state steady) after 
10022 ms without a successful response from the group coordinator (broker 0, 
last error was Success): revoking assignment and rejoining group
2024-07-25 17:43:25,340 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:127.0.0.1:9092/bootstrap]: 127.0.0.1:9092/bootstrap: Disconnected while 
requesting ApiVersion: might be caused by incorrect security.protocol 
configuration (connecting to a SSL listener?) or broker version is < 0.10 (see 
api.version.request) (after 8723ms in state APIVERSION_QUERY)
2024-07-25 17:43:25,340 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:DESKTOP-NDMP5G8:9092/0]: DESKTOP-NDMP5G8:9092/0: Disconnected while 
requesting ApiVersion: might be caused by incorrect security.protocol 
configuration (connecting to a SSL listener?) or broker version is < 0.10 (see 
api.version.request) (after 8508ms in state APIVERSION_QUERY)
2024-07-25 17:43:25,340 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:GroupCoordinator]: GroupCoordinator: DESKTOP-NDMP5G8:9092: Disconnected 
(after 8508ms in state APIVERSION_QUERY)
2024-07-25 17:43:25,341 ERROR TestBench.Diagnostics.Logger - Kafka error: 
127.0.0.1:9092/bootstrap: Disconnected while requesting ApiVersion: might be 
caused by incorrect security.protocol configuration (connecting to a SSL 
listener?) or broker version is < 0.10 (see api.version.request) (after 8723ms 
in state APIVERSION_QUERY)
2024-07-25 17:43:25,341 ERROR TestBench.Diagnostics.Logger - Kafka error: 
DESKTOP-NDMP5G8:9092/0: Disconnected while requesting ApiVersion: might be 
caused by incorrect security.protocol configuration (connecting to a SSL 
listener?) or broker version is < 0.10 (see api.version.request) (after 8508ms 
in state APIVERSION_QUERY)
2024-07-25 17:43:25,341 ERROR TestBench.Diagnostics.Logger - Kafka error: 
GroupCoordinator: DESKTOP-NDMP5G8:9092: Disconnected (after 8508ms in state 
APIVERSION_QUERY)
2024-07-25 17:43:27,720 ERROR TestBench.Diagnostics.Logger - Kafka log: 
[thrd:GroupCoordinator]: GroupCoordinator: DESKTOP-NDMP5G8:9092: Connect to 
ipv4#192.168.1.103:9092 failed: Unknown error (after 2040ms in state CONNECT)
2024-07-25 17:43:27,721 ERRO

Re: [VOTE] 3.8.0 RC3

2024-07-26 Thread Josep Prat
Thank you all who reviewed this release candidate and the previous ones.

This vote passes with 7 +1 votes (3 bindings) and no 0 or -1 votes.

+1 votes
PMC Members:
* Chris Egerton
* Greg Harris
* Mickael Maison

Committers:
* Igor Soarez

Community:
* Murali Basani
* Jakub Scholz
* Krishna Agarwal

0 votes
* No votes

-1 votes
* No votes

I'll continue with the release process and the release announcement will
follow in the next few days.

Best,

On Thu, Jul 25, 2024 at 7:44 PM Krishna Agarwal <
krishna0608agar...@gmail.com> wrote:

> Hi Josep,
>
> Thanks for the release candidate.
>
> +1 (non-binding)
>
> I tested and verified the docker image artifact
> apache/kafka-native:3.8.0-rc3
> - verified create topic, produce messages and consume messages flow.
> - verified the html documentation for docker image.
>
> I also ran the System tests by bringing up Kafka in the native mode.
> Following tests failed in the run.
>
> -
> >
> tests/kafkatest/tests/tools/replica_verification_test.py::ReplicaVerificationToolTest.test_replica_lags@
> > {"metadata_quorum":"ISOLATED_KRAFT"}
> > -
> >
> tests/kafkatest/tests/tools/replica_verification_test.py::ReplicaVerificationToolTest.test_replica_lags@
> > {"metadata_quorum":"ZK"}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"client-id","old_client_throttling_behavior":true}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"(user,client-id)","override_quota":true}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"client-id","old_broker_throttling_behavior":true}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"client-id","consumer_num":2}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"(user,client-id)","override_quota":false}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"client-id","override_quota":false}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"user","override_quota":true}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"client-id","override_quota":true}
> > - tests/kafkatest/tests/client/quota_test.py::QuotaTest.test_quota@
> > {"quota_type":"user","override_quota":false}
> > -
> >
> tests/kafkatest/tests/connect/connect_distributed_test.py::ConnectDistributedTest.test_dynamic_logging
> > -
> >
> tests/kafkatest/tests/connect/connect_distributed_test.py::ConnectDistributedTest.test_pause_and_resume_sink@
> >
> {"connect_protocol":"sessioned","metadata_quorum":"ISOLATED_KRAFT","use_new_coordinator":true,"group_protocol":"consumer"}
> >
>
> On re-running the failed tests locally, *all tests passed except*
>
> `tests/kafkatest/tests/tools/replica_verification_test.py::ReplicaVerificationToolTest.test_replica_lags@
> {"metadata_quorum":"ZK"}`(timeout
> error).
>
> All looks good for this docker image artifact!
>
> Regards,
> Krishna
>
> On Wed, Jul 24, 2024 at 10:46 PM Mickael Maison 
> wrote:
>
> > Hi Josep,
> >
> > I have:
> > - checked signatures and checksums
> > - run ZK and KRaft quickstarts with the Scala 2.13 binaries and with
> > both the kafka and kafka-native images
> > - built and run tests from source with Java 17
> >
> > +1 (binding)
> >
> > Thanks,
> > Mickael
> >
> > On Wed, Jul 24, 2024 at 4:22 PM Igor Soarez  wrote:
> > >
> > > Hi Josep,
> > >
> > > That makes sense to me, thanks for clarifying.
> > >
> > > +1 non-binding from me then.
> > >
> > > --
> > > Igor
> >
>


-- 
[image: Aiven] 

*Josep Prat*
Open Source Engineering Director, *Aiven*
josep.p...@aiven.io   |   +491715557497
aiven.io    |   
     
*Aiven Deutschland GmbH*
Alexanderufer 3-7, 10117 Berlin
Geschäftsführer: Oskari Saarenmaa, Hannu Valtonen,
Anna Richardson, Kenneth Chen
Amtsgericht Charlottenburg, HRB 209739 B


[jira] [Created] (KAFKA-17206) Use v1 of LeaderChangeMessage when kraft.version is 1

2024-07-26 Thread Alyssa Huang (Jira)
Alyssa Huang created KAFKA-17206:


 Summary: Use v1 of LeaderChangeMessage when kraft.version is 1
 Key: KAFKA-17206
 URL: https://issues.apache.org/jira/browse/KAFKA-17206
 Project: Kafka
  Issue Type: Task
Reporter: Alyssa Huang


[https://github.com/apache/kafka/pull/16668] introduced v1 of LCM but still 
uses v0 of the schema.



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


[jira] [Resolved] (KAFKA-17204) KafkaStreamsCloseOptionsIntegrationTest.before leaks AdminClient

2024-07-26 Thread Matthias J. Sax (Jira)


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

Matthias J. Sax resolved KAFKA-17204.
-
Fix Version/s: 3.9.0
   Resolution: Fixed

> KafkaStreamsCloseOptionsIntegrationTest.before leaks AdminClient
> 
>
> Key: KAFKA-17204
> URL: https://issues.apache.org/jira/browse/KAFKA-17204
> Project: Kafka
>  Issue Type: Test
>  Components: streams
>Affects Versions: 3.9.0
>Reporter: Greg Harris
>Assignee: TengYao Chi
>Priority: Minor
>  Labels: newbie
> Fix For: 3.9.0
>
>
> The before method creates an AdminClient, but this client is never closed. It 
> should be closed either in `after` or `closeCluster`.



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


[PR] Minor update website 38 release [kafka-site]

2024-07-26 Thread via GitHub


jlprat opened a new pull request, #616:
URL: https://github.com/apache/kafka-site/pull/616

   Add the banner for older versions. Links the pages to the new version


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [VOTE] KIP-1070: Deprecate MockProcessorContext

2024-07-26 Thread Matthias J. Sax

I updated the KIP accordingly.

If there is no further follow usp, I'll close the vote shortly.


-Matthias

On 7/21/24 10:24 PM, Matthias J. Sax wrote:

I just realized that both

- ValueTransformerWithKey and
- ValueTransfromerWithKeySupplier

are still used by non-deprecated `KTable#transformValues()`. So we 
cannot deprecate them with this KIP.


I filed https://issues.apache.org/jira/browse/KAFKA-17178 for tracking.

As an alternative, we could extend this KIP and include K17178 that I 
just filed, but it would be a larger change. I frankly just wanted to 
tie up a few loose ends w/o the need to do a larger KIP.


Hope you are all ok with keeping the KIP as-is and simple, and we wait 
for somebody else to pickup K17178.


Thoughts?


-Matthias

On 7/15/24 8:43 PM, Sophie Blee-Goldman wrote:

Makes sense to me -- seems like an oversight since we did correctly
deprecate the old Processor, ProcessorSupplier, etc (not to mention the
#transform, #transformValues methods). Still a +1 (binding) from me

On Fri, Jul 12, 2024 at 4:41 PM Matthias J. Sax  wrote:


I just realized, that there is more interfaces with a similar situation:

- Transformer
- TransformerSupplier
- ValueTransformer
- ValueTransfomerSupplier
- ValueTransformerWithKey
- ValueTransfromerWithKeySupplier

Given that `KStream#transform` and `KStream#transformValues` are
deprecated, it seems we should deprecate all of them, too?



-Matthias


On 7/12/24 1:06 AM, Lucas Brutschy wrote:

Sounds good to me!

+1 (binding)

On Fri, Jul 12, 2024 at 12:55 AM Bill Bejeck  wrote:


+1 (binding)

On Thu, Jul 11, 2024 at 5:07 PM Sophie Blee-Goldman <

sop...@responsive.dev>

wrote:


Makes sense to me, +1 (binding)

On Thu, Jul 11, 2024 at 9:24 AM Matthias J. Sax 

wrote:



Hi,

I want to propose a very small KIP. Skipping the DISCUSS step, and
calling for a VOTE directly.






https://cwiki.apache.org/confluence/display/KAFKA/KIP-1070%3A+deprecate+MockProcessorContext



-Matthias









Re: [VOTE] KIP-1068: New metrics for the new KafkaConsumer

2024-07-26 Thread Matthias J. Sax

Thanks for the KIP Brenden.

+1 (binding)

On 7/26/24 1:16 AM, Andrew Schofield wrote:

HI Brenden,
Thanks for the KIP.

+1 (non-binding)

Thanks,
Andrew


On 25 Jul 2024, at 23:26, Apoorv Mittal  wrote:

Hi Brenden,
The metrics look good. Thanks for the KIP.

+1 (Non-Binding)

Regards,
Apoorv Mittal


On Thu, Jul 25, 2024 at 5:04 PM Philip Nee 
wrote:


Hi Brenden,

Thanks for the KIP. +1 (non-binding).

P

On Thu, Jul 25, 2024 at 6:25 AM Lianet M.  wrote:


Hi Brenden,

+1 (non-binding) from me.

Thanks for the KIP!

Lianet

On Thu, Jul 25, 2024 at 3:18 AM Bruno Cadonna 

wrote:



Hi Brenden,

Thanks for the KIP!

+1 (binding)

Best,
Bruno

On 7/24/24 11:55 PM, Brenden Deluna wrote:

Hello everyone,

I would like to start the vote on KIP-1068:






https://cwiki.apache.org/confluence/display/KAFKA/KIP-1068%3A+New+metrics+for+the+new+KafkaConsumer


This KIP introduces new metrics for the new KafkaConsumer

implementation

for broader metrics coverage.

Thanks,
Brenden











Re: [VOTE] KIP-1068: New metrics for the new KafkaConsumer

2024-07-26 Thread Bill Bejeck
Hi Brenden,

Thanks for the KIP!

+1(binding)

-Bill

On Fri, Jul 26, 2024 at 1:54 PM Matthias J. Sax  wrote:

> Thanks for the KIP Brenden.
>
> +1 (binding)
>
> On 7/26/24 1:16 AM, Andrew Schofield wrote:
> > HI Brenden,
> > Thanks for the KIP.
> >
> > +1 (non-binding)
> >
> > Thanks,
> > Andrew
> >
> >> On 25 Jul 2024, at 23:26, Apoorv Mittal 
> wrote:
> >>
> >> Hi Brenden,
> >> The metrics look good. Thanks for the KIP.
> >>
> >> +1 (Non-Binding)
> >>
> >> Regards,
> >> Apoorv Mittal
> >>
> >>
> >> On Thu, Jul 25, 2024 at 5:04 PM Philip Nee 
> >> wrote:
> >>
> >>> Hi Brenden,
> >>>
> >>> Thanks for the KIP. +1 (non-binding).
> >>>
> >>> P
> >>>
> >>> On Thu, Jul 25, 2024 at 6:25 AM Lianet M.  wrote:
> >>>
>  Hi Brenden,
> 
>  +1 (non-binding) from me.
> 
>  Thanks for the KIP!
> 
>  Lianet
> 
>  On Thu, Jul 25, 2024 at 3:18 AM Bruno Cadonna 
> >>> wrote:
> 
> > Hi Brenden,
> >
> > Thanks for the KIP!
> >
> > +1 (binding)
> >
> > Best,
> > Bruno
> >
> > On 7/24/24 11:55 PM, Brenden Deluna wrote:
> >> Hello everyone,
> >>
> >> I would like to start the vote on KIP-1068:
> >>
> >
> 
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1068%3A+New+metrics+for+the+new+KafkaConsumer
> >>
> >> This KIP introduces new metrics for the new KafkaConsumer
>  implementation
> >> for broader metrics coverage.
> >>
> >> Thanks,
> >> Brenden
> >>
> >
> 
> >>>
> >
>


[jira] [Created] (KAFKA-17207) ConnectWorkerIntegrationTest.testRequestTimeouts times out in stop(), leaking clients

2024-07-26 Thread Greg Harris (Jira)
Greg Harris created KAFKA-17207:
---

 Summary: ConnectWorkerIntegrationTest.testRequestTimeouts times 
out in stop(), leaking clients
 Key: KAFKA-17207
 URL: https://issues.apache.org/jira/browse/KAFKA-17207
 Project: Kafka
  Issue Type: Test
  Components: connect
Reporter: Greg Harris


The testRequestTimeouts deletes the internal config topic, putting the connect 
workers into a bad state. When the test goes to clean up, it calls 
DistributedHerder#stop, which waits for the herder executor to stop. This times 
out, because the herder executor is blocked closing the 
KafkaConfigBackingStore's producer. This log message gets printed:
{noformat}
[2024-07-26 11:52:50,817] ERROR Executor 
java.util.concurrent.ThreadPoolExecutor@7ae97a58[Shutting down, pool size = 1, 
active threads = 1, queued tasks = 0, completed tasks = 0] did not terminate in 
time (org.apache.kafka.common.utils.ThreadUtils:83){noformat}

This effectively leaks the Kafka clients for the workers' internal topics, and 
the herder executor thread. Instead, either the producer should not block 
indefinitely on a missing topic, or the cluster state should be healed enough 
for the producer to shutdown cleanly.



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


Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #3143

2024-07-26 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 406639 lines...]
[2024-07-26T18:55:21.064Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testUpdateExistingPartitions() STARTED
[2024-07-26T18:55:21.064Z] 
[2024-07-26T18:55:21.064Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testUpdateExistingPartitions() PASSED
[2024-07-26T18:55:21.064Z] 
[2024-07-26T18:55:21.064Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testEmptyWrite() STARTED
[2024-07-26T18:55:22.261Z] 
[2024-07-26T18:55:22.261Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testEmptyWrite() PASSED
[2024-07-26T18:55:22.261Z] 
[2024-07-26T18:55:22.261Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testReadMigrateAndWriteProducerId() STARTED
[2024-07-26T18:55:23.366Z] 
[2024-07-26T18:55:23.366Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testReadMigrateAndWriteProducerId() PASSED
[2024-07-26T18:55:23.366Z] 
[2024-07-26T18:55:23.366Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testExistingKRaftControllerClaim() STARTED
[2024-07-26T18:55:23.366Z] 
[2024-07-26T18:55:23.366Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testExistingKRaftControllerClaim() PASSED
[2024-07-26T18:55:23.366Z] 
[2024-07-26T18:55:23.366Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testMigrateTopicConfigs() STARTED
[2024-07-26T18:55:24.472Z] 
[2024-07-26T18:55:24.472Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testMigrateTopicConfigs() PASSED
[2024-07-26T18:55:24.472Z] 
[2024-07-26T18:55:24.472Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testNonIncreasingKRaftEpoch() STARTED
[2024-07-26T18:55:25.968Z] 
[2024-07-26T18:55:25.968Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testNonIncreasingKRaftEpoch() PASSED
[2024-07-26T18:55:25.968Z] 
[2024-07-26T18:55:25.968Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testMigrateEmptyZk() STARTED
[2024-07-26T18:55:25.968Z] 
[2024-07-26T18:55:25.968Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testMigrateEmptyZk() PASSED
[2024-07-26T18:55:25.968Z] 
[2024-07-26T18:55:25.968Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testTopicAndBrokerConfigsMigrationWithSnapshots() 
STARTED
[2024-07-26T18:55:27.087Z] 
[2024-07-26T18:55:27.087Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testTopicAndBrokerConfigsMigrationWithSnapshots() 
PASSED
[2024-07-26T18:55:27.087Z] 
[2024-07-26T18:55:27.087Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testClaimAndReleaseExistingController() STARTED
[2024-07-26T18:55:28.531Z] 
[2024-07-26T18:55:28.531Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testClaimAndReleaseExistingController() PASSED
[2024-07-26T18:55:28.531Z] 
[2024-07-26T18:55:28.531Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testClaimAbsentController() STARTED
[2024-07-26T18:55:28.531Z] 
[2024-07-26T18:55:28.531Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testClaimAbsentController() PASSED
[2024-07-26T18:55:28.531Z] 
[2024-07-26T18:55:28.531Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testIdempotentCreateTopics() STARTED
[2024-07-26T18:55:29.638Z] 
[2024-07-26T18:55:29.638Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testIdempotentCreateTopics() PASSED
[2024-07-26T18:55:29.638Z] 
[2024-07-26T18:55:29.638Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testCreateNewTopic() STARTED
[2024-07-26T18:55:31.174Z] 
[2024-07-26T18:55:31.174Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testCreateNewTopic() PASSED
[2024-07-26T18:55:31.174Z] 
[2024-07-26T18:55:31.174Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testUpdateExistingTopicWithNewAndChangedPartitions() 
STARTED
[2024-07-26T18:55:31.174Z] 
[2024-07-26T18:55:31.174Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZkMigrationClientTest > testUpdateExistingTopicWithNewAndChangedPartitions() 
PASSED
[2024-07-26T18:55:31.174Z] 
[2024-07-26T18:55:31.174Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZooKeeperClientTest > testZNodeChangeHandlerForDataChange() STARTED
[2024-07-26T18:55:32.278Z] 
[2024-07-26T18:55:32.278Z] Gradle Test Run :core:test > Gradle Test Executor 96 
> ZooKeeperClientTest 

[jira] [Created] (KAFKA-17208) replica_scale_test.py fails for new consumer

2024-07-26 Thread Kirk True (Jira)
Kirk True created KAFKA-17208:
-

 Summary: replica_scale_test.py fails for new consumer
 Key: KAFKA-17208
 URL: https://issues.apache.org/jira/browse/KAFKA-17208
 Project: Kafka
  Issue Type: Bug
  Components: clients, consumer, system tests
Affects Versions: 3.8.0
Reporter: Kirk True
Assignee: Kirk True
 Fix For: 4.0.0


{{replica_scale_test}}’s {{test_produce_consume}} fails when using a 
{{group_protocol}} of {{CONSUMER}}:
 
{noformat}
TimeoutError('replicas-consume-workload failed to finish in the expected amount 
of time.')
Traceback (most recent call last):
  File 
"/home/kafka/venv/lib/python3.8/site-packages/ducktape-0.11.1-py3.8.egg/ducktape/tests/runner_client.py",
 line 183, in _do_run
data = self.run_test()
  File 
"/home/kafka/venv/lib/python3.8/site-packages/ducktape-0.11.1-py3.8.egg/ducktape/tests/runner_client.py",
 line 243, in run_test
return self.test_context.function(self.test)
  File 
"/home/kafka/venv/lib/python3.8/site-packages/ducktape-0.11.1-py3.8.egg/ducktape/mark/_mark.py",
 line 433, in wrapper
return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
  File "/home/kafka/tests/kafkatest/tests/core/replica_scale_test.py", line 
116, in test_produce_consume
consume_workload.wait_for_done(timeout_sec=600)
  File "/home/kafka/tests/kafkatest/services/trogdor/trogdor.py", line 352, in 
wait_for_done
wait_until(lambda: self.done(),
  File 
"/home/kafka/venv/lib/python3.8/site-packages/ducktape-0.11.1-py3.8.egg/ducktape/utils/util.py",
 line 58, in wait_until
raise TimeoutError(err_msg() if callable(err_msg) else err_msg) from 
last_exception
ducktape.errors.TimeoutError: replicas-consume-workload failed to finish in the 
expected amount of time.
{noformat}



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


Re: [DISCUSS] KIP-512: make Record Headers available in onAcknowledgement

2024-07-26 Thread Andrew Schofield
Hi Rich,
Thanks for resurrecting this KIP. It seems like a useful idea to me and
I’d be interested in seeing the proposed public interfaces.

I note that you specifically called out the 
ProducerInterceptor.onAcknowledgement
method, as opposed to the producer Callback.onCompletion method.

Thanks,
Andrew

> On 26 Jul 2024, at 04:54, Rich C.  wrote:
>
> Hi Kevin,
>
> Thanks for your support.
>
> Hi Matthias,
>
> I apologize for the confusion. I've deleted the Public Interface sections
> for now. I think we should focus on discussing its necessity with the
> community. I'll let it sit for a few more days, and if there are no
> objections, I will propose changes over the weekend and share them here
> again.
>
> Regards,
> Rich
>
>
> On Thu, Jul 25, 2024 at 5:51 PM Matthias J. Sax  wrote:
>
>> Rich,
>>
>> thanks for resurrecting this KIP. I was not part of the original
>> discussion back in the day, but personally agree with your assessment
>> that making headers available in the callbacks would make developer's
>> life much simpler.
>>
>> For the KIP itself, starting with "Public Interface" section, everything
>> is formatted as "strike through". Can you fix this? It's confusing as
>> it's apparently not correctly formatted, but unclear which (if any)
>> parts should be formatted like this. In general, wiki pages have
>> history, so strike-through should be used rather rarely but the wiki
>> page should just contain the latest proposal. (If one want to see the
>> history, it's there anyway).
>>
>>
>> -Matthias
>>
>> On 7/23/24 6:36 AM, Kevin Lam wrote:
>>> Hi,
>>>
>>> Thanks for starting the discussion. Latency Measurement and Tracing
>>> Completeness are both good reasons to support this feature, and would be
>>> interested to see this move forward.
>>>
>>> On Mon, Jul 22, 2024 at 11:15 PM Rich C.  wrote:
>>>
 Hi Everyone,

 I hope this email finds you well.

 I would like to start a discussion on KIP-512. The initial version of
 KIP-512 was created in 2019, and I have resurrected it in 2024 with more
 details about the motivation behind it.

 You can view the current version of the KIP here: KIP-512: Make Record
 Headers Available in onAcknowledgement.
 <

>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-512%3A+make+Record+Headers+available+in+onAcknowledgement
>

 Let's focus on discussing the necessity of this feature first. If we
>> agree
 on its importance, we can then move on to discussing the proposed
>> changes.

 Looking forward to your feedback.

 Best regards,
 Rich

>>>
>>



Build failed in Jenkins: Kafka » kafka-2.7-jdk8 #193

2024-07-26 Thread Apache Jenkins Server
See 


Changes:


--
[...truncated 3.46 MB...]

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSourceSpecificDeserializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldReturnAllStores[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotCreateStateDirectoryForStatelessTopology[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnAllStoresNames[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPassRecordHeadersIntoSerializersAndDeserializers[Eos enabled = false] 
STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPassRecordHeadersIntoSerializersAndDeserializers[Eos enabled = false] 
PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessConsumerRecordList[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUseSinkSpecificSerializers[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldFlushStoreForFirstInput[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessFromSourceThatMatchPattern[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureSinkTopicNamesIfWrittenInto[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureSinkTopicNamesIfWrittenInto[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldUpdateStoreForNewKey[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldSendRecordViaCorrectSourceTopicDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTime[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldSetRecordMetadata[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldSetRecordMetadata[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotUpdateStoreForLargerValue[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldNotUpdateStoreForLargerValue[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectInMemoryStoreTypeOnly[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldReturnCorrectInMemoryStoreTypeOnly[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > shouldThrowForMissingTime[Eos 
enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > shouldThrowForMissingTime[Eos 
enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureInternalTopicNamesIfWrittenInto[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldCaptureInternalTopicNamesIfWrittenInto[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTimeDeprecated[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldPunctuateOnWallClockTimeDeprecated[Eos enabled = false] PASSED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessRecordForTopic[Eos enabled = false] STARTED

org.apache.kafka.streams.TopologyTestDriverTest > 
shouldProcessRecordForTopic[Eos enabled = false] PASSED

org.apache.kafka.streams.Top

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

2024-07-26 Thread Apache Jenkins Server
See