Re: Kafka Streams reaching ERROR state during rolling upgrade / restart of brokers

2023-10-02 Thread Matthias J. Sax
I did mean client side...  If KS goes into ERROR state, it should log 
the reason.


If the logs are indeed empty, try to register an 
uncaught-exception-handler via


KafkaStreamssetUncaughtExceptionHandler(...)


-Matthias

On 10/2/23 12:11 PM, Debraj Manna wrote:

Are you suggesting to check the Kafka broker logs? I do not see any other
errors logs on the client / application side.

On Fri, 29 Sep, 2023, 22:01 Matthias J. Sax,  wrote:


In general, Kafka Streams should keep running.

Can you inspect the logs to figure out why it's going into ERROR state
to begin with? Maybe you need to increase/change some timeouts/retries
configs.

The stack trace you shared, is a symptom, but not the root cause.

-Matthias

On 9/21/23 12:56 AM, Debraj Manna wrote:

I am using Kafka broker 2.8.1 (from AWS MSK) with Kafka clients and Kafka
stream 3.5.1.

I am observing that whenever some rolling upgrade is done on AWS MSK our
stream application reaches an error state. I get the below exception on
trying to query the state store

caused by: java.lang.IllegalStateException: KafkaStreams is not running.
State is ERROR.
  at


org.apache.kafka.streams.KafkaStreams.validateIsRunningOrRebalancing(KafkaStreams.java:381)

  at


org.apache.kafka.streams.KafkaStreams.queryMetadataForKey(KafkaStreams.java:1663)

  at


org.springframework.cloud.stream.binder.kafka.streams.InteractiveQueryService.lambda$getHostInfo$2(InteractiveQueryService.java:227)


Can someone let me know what the recommended way we can keep the stream
application running whenever some rolling upgrade/restart of brokers is
done in the background?







Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Proven Provenzano
Hi,

To verify the release of release 3.6.0 RC2 I did the following:

   - Downloaded the source, built and ran the tests.
   - Validated SCRAM with KRaft including creating credentials with
   kafka-storage.
   - Validated Delegation Tokens with KRaft

+1 (non-binding)

--Proven



On Mon, Oct 2, 2023 at 8:37 AM Divij Vaidya  wrote:

> + 1 (non-binding)
>
> Verifications:
> 1. I ran a produce-consume workload with plaintext auth, JDK17, zstd
> compression using an open messaging benchmark and found 3.6 to be better
> than or equal to 3.5.1 across all dimensions. Notably, 3.6 had consistently
> 6-7% lower CPU utilization, lesser spikes on P99 produce latencies and
> overall lower P99.8 latencies.
>
> 2. I have verified that detached signature is correct using
> https://www.apache.org/info/verification.html and the release manager
> public keys are available at
> https://keys.openpgp.org/search?q=F65DC3423D4CD7B9
>
> 3. I have verified that all metrics emitted in 3.5.1 (with Zk) are also
> being emitted in 3.6.0 (with Zk).
>
> Problems (but not blockers):
> 1. Metrics added in
>
> https://github.com/apache/kafka/commit/2f71708955b293658cec3b27e9a5588d39c38d7e
> aren't available in the documentation (cc: Justine). I don't consider this
> as a release blocker but we should add it as a fast follow-up.
>
> 2. Metric added in
>
> https://github.com/apache/kafka/commit/a900794ace4dcf1f9dadee27fbd8b63979532a18
> isn't available in documentation (cc: David). I don't consider this as a
> release blocker but we should add it as a fast follow-up.
>
> --
> Divij Vaidya
>
>
>
> On Mon, Oct 2, 2023 at 9:50 AM Federico Valeri 
> wrote:
>
> > Hi Satish, I did the following to verify the release:
> >
> > - Built from source with Java 17 and Scala 2.13
> > - Ran all unit and integration tests
> > - Spot checked documentation
> > - Ran custom client applications using staging artifacts on a 3-nodes
> > cluster
> > - Tested tiered storage with one of the available RSM implementations
> >
> > +1 (non binding)
> >
> > Thanks
> > Fede
> >
> > On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
> > >
> > > Hi Satish,
> > >
> > > I verified with:
> > > 1. Ran quick start in KRaft for scala 2.12 artifact
> > > 2. Making sure the checksum are correct
> > > 3. Browsing release notes, documents, javadocs, protocols.
> > > 4. Verified the tiered storage feature works well.
> > >
> > > +1 (binding).
> > >
> > > Thanks.
> > > Luke
> > >
> > >
> > >
> > > On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
> > >
> > > > +1 (non-binding). I used the Scala 2.13 binaries and the staged Maven
> > > > artifacts and run my tests. Everything seems to work fine for me.
> > > >
> > > > Thanks
> > > > Jakub
> > > >
> > > > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana <
> > satish.dugg...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Kafka users, developers and client-developers,
> > > > >
> > > > > This is the third candidate for the release of Apache Kafka 3.6.0.
> > > > > Some of the major features include:
> > > > >
> > > > > * KIP-405 : Kafka Tiered Storage
> > > > > * KIP-868 : KRaft Metadata Transactions
> > > > > * KIP-875: First-class offsets support in Kafka Connect
> > > > > * KIP-898: Modernize Connect plugin discovery
> > > > > * KIP-938: Add more metrics for measuring KRaft performance
> > > > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > > > * KIP-917: Additional custom metadata for remote log segment
> > > > >
> > > > > Release notes for the 3.6.0 release:
> > > > >
> https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > > > >
> > > > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > > > >
> > > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > > https://kafka.apache.org/KEYS
> > > > >
> > > > > * Release artifacts to be voted upon (source and binary):
> > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > > > >
> > > > > * Maven artifacts to be voted upon:
> > > > >
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > >
> > > > > * Javadoc:
> > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > > > >
> > > > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > > > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> > > > >
> > > > > * Documentation:
> > > > > https://kafka.apache.org/36/documentation.html
> > > > >
> > > > > * Protocol:
> > > > > https://kafka.apache.org/36/protocol.html
> > > > >
> > > > > * Successful Jenkins builds for the 3.6 branch:
> > > > > There are a few runs of unit/integration tests. You can see the
> > latest
> > > > > at https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/. We
> > will
> > > > > continue running a few more iterations.
> > > > > System tests:
> > > > > We will send an update once we have the results.
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > >
> >
>


Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Bill Bejeck
Hi Satish,

Thanks for running the release.
I performed the following steps:

   - Validated all the checksums, signatures, and keys
   - Built the release from source
   - Ran all unit tests
   - Quick start validations
  - ZK and Kraft
  - Connect
  - Kafka Streams
   - Spot checked java docs and documentation

+1 (binding)

- Bill

On Mon, Oct 2, 2023 at 10:23 AM Proven Provenzano
 wrote:

> Hi,
>
> To verify the release of release 3.6.0 RC2 I did the following:
>
>- Downloaded the source, built and ran the tests.
>- Validated SCRAM with KRaft including creating credentials with
>kafka-storage.
>- Validated Delegation Tokens with KRaft
>
> +1 (non-binding)
>
> --Proven
>
>
>
> On Mon, Oct 2, 2023 at 8:37 AM Divij Vaidya 
> wrote:
>
> > + 1 (non-binding)
> >
> > Verifications:
> > 1. I ran a produce-consume workload with plaintext auth, JDK17, zstd
> > compression using an open messaging benchmark and found 3.6 to be better
> > than or equal to 3.5.1 across all dimensions. Notably, 3.6 had
> consistently
> > 6-7% lower CPU utilization, lesser spikes on P99 produce latencies and
> > overall lower P99.8 latencies.
> >
> > 2. I have verified that detached signature is correct using
> > https://www.apache.org/info/verification.html and the release manager
> > public keys are available at
> > https://keys.openpgp.org/search?q=F65DC3423D4CD7B9
> >
> > 3. I have verified that all metrics emitted in 3.5.1 (with Zk) are also
> > being emitted in 3.6.0 (with Zk).
> >
> > Problems (but not blockers):
> > 1. Metrics added in
> >
> >
> https://github.com/apache/kafka/commit/2f71708955b293658cec3b27e9a5588d39c38d7e
> > aren't available in the documentation (cc: Justine). I don't consider
> this
> > as a release blocker but we should add it as a fast follow-up.
> >
> > 2. Metric added in
> >
> >
> https://github.com/apache/kafka/commit/a900794ace4dcf1f9dadee27fbd8b63979532a18
> > isn't available in documentation (cc: David). I don't consider this as a
> > release blocker but we should add it as a fast follow-up.
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Mon, Oct 2, 2023 at 9:50 AM Federico Valeri 
> > wrote:
> >
> > > Hi Satish, I did the following to verify the release:
> > >
> > > - Built from source with Java 17 and Scala 2.13
> > > - Ran all unit and integration tests
> > > - Spot checked documentation
> > > - Ran custom client applications using staging artifacts on a 3-nodes
> > > cluster
> > > - Tested tiered storage with one of the available RSM implementations
> > >
> > > +1 (non binding)
> > >
> > > Thanks
> > > Fede
> > >
> > > On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
> > > >
> > > > Hi Satish,
> > > >
> > > > I verified with:
> > > > 1. Ran quick start in KRaft for scala 2.12 artifact
> > > > 2. Making sure the checksum are correct
> > > > 3. Browsing release notes, documents, javadocs, protocols.
> > > > 4. Verified the tiered storage feature works well.
> > > >
> > > > +1 (binding).
> > > >
> > > > Thanks.
> > > > Luke
> > > >
> > > >
> > > >
> > > > On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
> > > >
> > > > > +1 (non-binding). I used the Scala 2.13 binaries and the staged
> Maven
> > > > > artifacts and run my tests. Everything seems to work fine for me.
> > > > >
> > > > > Thanks
> > > > > Jakub
> > > > >
> > > > > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana <
> > > satish.dugg...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Kafka users, developers and client-developers,
> > > > > >
> > > > > > This is the third candidate for the release of Apache Kafka
> 3.6.0.
> > > > > > Some of the major features include:
> > > > > >
> > > > > > * KIP-405 : Kafka Tiered Storage
> > > > > > * KIP-868 : KRaft Metadata Transactions
> > > > > > * KIP-875: First-class offsets support in Kafka Connect
> > > > > > * KIP-898: Modernize Connect plugin discovery
> > > > > > * KIP-938: Add more metrics for measuring KRaft performance
> > > > > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > > > > * KIP-917: Additional custom metadata for remote log segment
> > > > > >
> > > > > > Release notes for the 3.6.0 release:
> > > > > >
> > https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > > > > >
> > > > > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > > > > >
> > > > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > > > https://kafka.apache.org/KEYS
> > > > > >
> > > > > > * Release artifacts to be voted upon (source and binary):
> > > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > > > > >
> > > > > > * Maven artifacts to be voted upon:
> > > > > >
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > > >
> > > > > > * Javadoc:
> > > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > > > > >
> > > > > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > > > > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> > > > 

Re: Kafka Streams reaching ERROR state during rolling upgrade / restart of brokers

2023-10-02 Thread Debraj Manna
Are you suggesting to check the Kafka broker logs? I do not see any other
errors logs on the client / application side.

On Fri, 29 Sep, 2023, 22:01 Matthias J. Sax,  wrote:

> In general, Kafka Streams should keep running.
>
> Can you inspect the logs to figure out why it's going into ERROR state
> to begin with? Maybe you need to increase/change some timeouts/retries
> configs.
>
> The stack trace you shared, is a symptom, but not the root cause.
>
> -Matthias
>
> On 9/21/23 12:56 AM, Debraj Manna wrote:
> > I am using Kafka broker 2.8.1 (from AWS MSK) with Kafka clients and Kafka
> > stream 3.5.1.
> >
> > I am observing that whenever some rolling upgrade is done on AWS MSK our
> > stream application reaches an error state. I get the below exception on
> > trying to query the state store
> >
> > caused by: java.lang.IllegalStateException: KafkaStreams is not running.
> > State is ERROR.
> >  at
> >
> org.apache.kafka.streams.KafkaStreams.validateIsRunningOrRebalancing(KafkaStreams.java:381)
> >  at
> >
> org.apache.kafka.streams.KafkaStreams.queryMetadataForKey(KafkaStreams.java:1663)
> >  at
> >
> org.springframework.cloud.stream.binder.kafka.streams.InteractiveQueryService.lambda$getHostInfo$2(InteractiveQueryService.java:227)
> >
> > Can someone let me know what the recommended way we can keep the stream
> > application running whenever some rolling upgrade/restart of brokers is
> > done in the background?
> >
>


Re: mirror-maker

2023-10-02 Thread Richard Bosch
Hi Denny,

If you use mirrormaker to replicate data between clusters then it
needs access to all brokers.

There is also a clear reason for this, Mirrormaker acts as a normal
Kafka client when connecting to Kafka clusters.
Mirrormaker uses the normal produce and consume protocol, which means
that a producer needs to be able to contact the broker that is the
leader of the partition.
The consume functionality needs to connect to the leader, or an in
sync replica of the partition to retrieve the records for replication.

The basic rule for network access is that a Kafka client needs network
access to all advertised ports belonging to the same Kafka broker
listener (SASL, SSL, EXTERNAL-XX).

Kind regards,

Richard Bosch
Developer Advocate
Axual BV

E : richard.bo...@axual.com
M : +31 6 11 850 846
W : www.axual.com


On Mon, Oct 2, 2023 at 2:25 PM Denny Fuchs
 wrote:
>
> hi,
>
> I try to use mirror maker to replicate the data, and I have one trivial
> question:
>
> Do I need access to / add  **all** Kafka Brokers ?
>
> I have a 3.1 Cluster with 7 brokers and a new 3.5 with 5 brokers and I
> want to migrate from the 3.1 (Zoo) to 3.5 (KRaft). Both clusters are in
> different networks with a lot of firewalls and we allowed only access to
> one of the old clusters:
>
> My mirror-mager is on the end.
>
> The problem is .. that if I start the sync .. it seems to be stuck
> somewhere:
>
>
> ==
>
>   (org.apache.kafka.connect.mirror.MirrorHeartbeatConfig:369)
> [2023-10-02 14:07:50,091] INFO [MirrorHeartbeatConnector|task-0]
> WorkerSourceTask{id=MirrorHeartbeatConnector-0} Source task finished
> initialization and start
> (org.apache.kafka.connect.runtime.AbstractWorkerSource
> Task:275)
> [2023-10-02 14:07:50,113] INFO [MirrorHeartbeatConnector|task-0]
> [Producer clientId=connector-producer-MirrorHeartbeatConnector-0]
> Cluster ID: Rnpnd4EcRBeWo8vUrWlOIQ (org.apache.kafka.clients.Metadata:287)
> [2023-10-02 14:07:50,238] INFO [MirrorSourceConnector|worker]
> [AdminClient
> clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin]
> Disconnecting from node 7 due to socket connection setup timeout.
> The timeout value is 11336 ms. (org.apache.kafka.clients.NetworkClient:858)
> [2023-10-02 14:07:50,828] INFO [MirrorCheckpointConnector|worker]
> [AdminClient
> clientId=FC-R02->FC-R01|MirrorCheckpointConnector|checkpoint-source-admin]
> Disconnecting from node 7 due to socket connection setup ti
> meout. The timeout value is 11943 ms.
> (org.apache.kafka.clients.NetworkClient:858)
> [2023-10-02 14:07:58,045] INFO [MirrorCheckpointConnector|worker]
> [AdminClient
> clientId=FC-R02->FC-R01|MirrorCheckpointConnector|checkpoint-source-admin]
> Disconnecting from node 6 due to socket connection setup ti
> meout. The timeout value is 8916 ms.
> (org.apache.kafka.clients.NetworkClient:858)
> [2023-10-02 14:08:00,271] INFO [MirrorSourceConnector|worker]
> [AdminClient
> clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin]
> Disconnecting from node 6 due to socket connection setup timeout.
> The timeout value is 10018 ms. (org.apache.kafka.clients.NetworkClient:858)
> [2023-10-02 14:08:00,359] INFO [MirrorSourceConnector|worker]
> [AdminClient
> clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin]
> Disconnecting from node 7 due to socket connection setup timeout.
> The timeout value is 11266 ms. (org.apache.kafka.clients.NetworkClient:858)
> [2023-10-02 14:08:00,440] INFO [MirrorSourceConnector|worker] loading
> initial set of topic-partitions took 11359 ms
> (org.apache.kafka.connect.mirror.Scheduler:95)
> [2023-10-02 14:08:00,448] INFO [MirrorSourceConnector|worker] creating
> downstream topic-partitions took 7 ms
> (org.apache.kafka.connect.mirror.Scheduler:95)
> [2023-10-02 14:08:00,454] INFO [MirrorSourceConnector|worker] refreshing
> known target topics took 5 ms
> (org.apache.kafka.connect.mirror.Scheduler:95)
> [2023-10-02 14:08:00,457] INFO [MirrorSourceConnector|worker] Started
> MirrorSourceConnector with 0 topic-partitions.
> (org.apache.kafka.connect.mirror.MirrorSourceConnector:172)
> [2023-10-02 14:08:00,458] INFO [MirrorSourceConnector|worker] Starting
> MirrorSourceConnector took 11510 ms.
> (org.apache.kafka.connect.mirror.MirrorSourceConnector:173)
> [2023-10-02 14:08:00,461] INFO SourceConnectorConfig values:
>  config.action.reload = restart
>  connector.class =
> org.apache.kafka.connect.mirror.MirrorSourceConnector
>
> ==
>
> So, If I see it correct, it tries to connect to the other nodes .. which
> is not allowed yet .. right ?
>
> cu denny
>
>
> ###
>
> clusters = FC-R02, FC-R01
>
> FC-R02.bootstrap.servers = fc-r02-kafka-01.example.com:9093
> FC-R01.bootstrap.servers = fc-r01-kafka-01.example.com:9092
>
> FC-R02->FC-R01.enabled = true
> FC-R01->FC-R02.enabled = false
>
> FC-R02->FC-R01.topics =
> 

Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Josep Prat
Hi Satish,

+1 (non-binding) from me as well! Thanks for running the release

I run the following validations steps:
- Built from source with Java 11 and Scala 2.13
- Verified Signatures and hashes of the artifacts generated
- Navigated through Javadoc including links to JDK classes
- Run the unit tests
- Run integration tests
- Run the quickstart in KRaft and Zookeeper mode

On Mon, Oct 2, 2023 at 3:13 PM Kamal Chandraprakash <
kamal.chandraprak...@gmail.com> wrote:

> +1 (non-binding)
>
> 1. Built the source from 3.6 branch in scala 2.12 and 2.13
> 2. Ran all the unit and integration tests.
> 3. Ran quickstart and verified the produce-consume on a 3 node cluster.
> 4. Verified the tiered storage functionality with local-tiered storage.
>
> Thanks,
> Kamal
>
> On Mon, Oct 2, 2023 at 6:07 PM Divij Vaidya 
> wrote:
>
> > + 1 (non-binding)
> >
> > Verifications:
> > 1. I ran a produce-consume workload with plaintext auth, JDK17, zstd
> > compression using an open messaging benchmark and found 3.6 to be better
> > than or equal to 3.5.1 across all dimensions. Notably, 3.6 had
> consistently
> > 6-7% lower CPU utilization, lesser spikes on P99 produce latencies and
> > overall lower P99.8 latencies.
> >
> > 2. I have verified that detached signature is correct using
> > https://www.apache.org/info/verification.html and the release manager
> > public keys are available at
> > https://keys.openpgp.org/search?q=F65DC3423D4CD7B9
> >
> > 3. I have verified that all metrics emitted in 3.5.1 (with Zk) are also
> > being emitted in 3.6.0 (with Zk).
> >
> > Problems (but not blockers):
> > 1. Metrics added in
> >
> >
> https://github.com/apache/kafka/commit/2f71708955b293658cec3b27e9a5588d39c38d7e
> > aren't available in the documentation (cc: Justine). I don't consider
> this
> > as a release blocker but we should add it as a fast follow-up.
> >
> > 2. Metric added in
> >
> >
> https://github.com/apache/kafka/commit/a900794ace4dcf1f9dadee27fbd8b63979532a18
> > isn't available in documentation (cc: David). I don't consider this as a
> > release blocker but we should add it as a fast follow-up.
> >
> > --
> > Divij Vaidya
> >
> >
> >
> > On Mon, Oct 2, 2023 at 9:50 AM Federico Valeri 
> > wrote:
> >
> > > Hi Satish, I did the following to verify the release:
> > >
> > > - Built from source with Java 17 and Scala 2.13
> > > - Ran all unit and integration tests
> > > - Spot checked documentation
> > > - Ran custom client applications using staging artifacts on a 3-nodes
> > > cluster
> > > - Tested tiered storage with one of the available RSM implementations
> > >
> > > +1 (non binding)
> > >
> > > Thanks
> > > Fede
> > >
> > > On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
> > > >
> > > > Hi Satish,
> > > >
> > > > I verified with:
> > > > 1. Ran quick start in KRaft for scala 2.12 artifact
> > > > 2. Making sure the checksum are correct
> > > > 3. Browsing release notes, documents, javadocs, protocols.
> > > > 4. Verified the tiered storage feature works well.
> > > >
> > > > +1 (binding).
> > > >
> > > > Thanks.
> > > > Luke
> > > >
> > > >
> > > >
> > > > On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
> > > >
> > > > > +1 (non-binding). I used the Scala 2.13 binaries and the staged
> Maven
> > > > > artifacts and run my tests. Everything seems to work fine for me.
> > > > >
> > > > > Thanks
> > > > > Jakub
> > > > >
> > > > > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana <
> > > satish.dugg...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Kafka users, developers and client-developers,
> > > > > >
> > > > > > This is the third candidate for the release of Apache Kafka
> 3.6.0.
> > > > > > Some of the major features include:
> > > > > >
> > > > > > * KIP-405 : Kafka Tiered Storage
> > > > > > * KIP-868 : KRaft Metadata Transactions
> > > > > > * KIP-875: First-class offsets support in Kafka Connect
> > > > > > * KIP-898: Modernize Connect plugin discovery
> > > > > > * KIP-938: Add more metrics for measuring KRaft performance
> > > > > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > > > > * KIP-917: Additional custom metadata for remote log segment
> > > > > >
> > > > > > Release notes for the 3.6.0 release:
> > > > > >
> > https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > > > > >
> > > > > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > > > > >
> > > > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > > > https://kafka.apache.org/KEYS
> > > > > >
> > > > > > * Release artifacts to be voted upon (source and binary):
> > > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > > > > >
> > > > > > * Maven artifacts to be voted upon:
> > > > > >
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > > >
> > > > > > * Javadoc:
> > > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > > > > >
> > > > > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > > > > 

Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Kamal Chandraprakash
+1 (non-binding)

1. Built the source from 3.6 branch in scala 2.12 and 2.13
2. Ran all the unit and integration tests.
3. Ran quickstart and verified the produce-consume on a 3 node cluster.
4. Verified the tiered storage functionality with local-tiered storage.

Thanks,
Kamal

On Mon, Oct 2, 2023 at 6:07 PM Divij Vaidya  wrote:

> + 1 (non-binding)
>
> Verifications:
> 1. I ran a produce-consume workload with plaintext auth, JDK17, zstd
> compression using an open messaging benchmark and found 3.6 to be better
> than or equal to 3.5.1 across all dimensions. Notably, 3.6 had consistently
> 6-7% lower CPU utilization, lesser spikes on P99 produce latencies and
> overall lower P99.8 latencies.
>
> 2. I have verified that detached signature is correct using
> https://www.apache.org/info/verification.html and the release manager
> public keys are available at
> https://keys.openpgp.org/search?q=F65DC3423D4CD7B9
>
> 3. I have verified that all metrics emitted in 3.5.1 (with Zk) are also
> being emitted in 3.6.0 (with Zk).
>
> Problems (but not blockers):
> 1. Metrics added in
>
> https://github.com/apache/kafka/commit/2f71708955b293658cec3b27e9a5588d39c38d7e
> aren't available in the documentation (cc: Justine). I don't consider this
> as a release blocker but we should add it as a fast follow-up.
>
> 2. Metric added in
>
> https://github.com/apache/kafka/commit/a900794ace4dcf1f9dadee27fbd8b63979532a18
> isn't available in documentation (cc: David). I don't consider this as a
> release blocker but we should add it as a fast follow-up.
>
> --
> Divij Vaidya
>
>
>
> On Mon, Oct 2, 2023 at 9:50 AM Federico Valeri 
> wrote:
>
> > Hi Satish, I did the following to verify the release:
> >
> > - Built from source with Java 17 and Scala 2.13
> > - Ran all unit and integration tests
> > - Spot checked documentation
> > - Ran custom client applications using staging artifacts on a 3-nodes
> > cluster
> > - Tested tiered storage with one of the available RSM implementations
> >
> > +1 (non binding)
> >
> > Thanks
> > Fede
> >
> > On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
> > >
> > > Hi Satish,
> > >
> > > I verified with:
> > > 1. Ran quick start in KRaft for scala 2.12 artifact
> > > 2. Making sure the checksum are correct
> > > 3. Browsing release notes, documents, javadocs, protocols.
> > > 4. Verified the tiered storage feature works well.
> > >
> > > +1 (binding).
> > >
> > > Thanks.
> > > Luke
> > >
> > >
> > >
> > > On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
> > >
> > > > +1 (non-binding). I used the Scala 2.13 binaries and the staged Maven
> > > > artifacts and run my tests. Everything seems to work fine for me.
> > > >
> > > > Thanks
> > > > Jakub
> > > >
> > > > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana <
> > satish.dugg...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Kafka users, developers and client-developers,
> > > > >
> > > > > This is the third candidate for the release of Apache Kafka 3.6.0.
> > > > > Some of the major features include:
> > > > >
> > > > > * KIP-405 : Kafka Tiered Storage
> > > > > * KIP-868 : KRaft Metadata Transactions
> > > > > * KIP-875: First-class offsets support in Kafka Connect
> > > > > * KIP-898: Modernize Connect plugin discovery
> > > > > * KIP-938: Add more metrics for measuring KRaft performance
> > > > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > > > * KIP-917: Additional custom metadata for remote log segment
> > > > >
> > > > > Release notes for the 3.6.0 release:
> > > > >
> https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > > > >
> > > > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > > > >
> > > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > > https://kafka.apache.org/KEYS
> > > > >
> > > > > * Release artifacts to be voted upon (source and binary):
> > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > > > >
> > > > > * Maven artifacts to be voted upon:
> > > > >
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > > >
> > > > > * Javadoc:
> > > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > > > >
> > > > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > > > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> > > > >
> > > > > * Documentation:
> > > > > https://kafka.apache.org/36/documentation.html
> > > > >
> > > > > * Protocol:
> > > > > https://kafka.apache.org/36/protocol.html
> > > > >
> > > > > * Successful Jenkins builds for the 3.6 branch:
> > > > > There are a few runs of unit/integration tests. You can see the
> > latest
> > > > > at https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/. We
> > will
> > > > > continue running a few more iterations.
> > > > > System tests:
> > > > > We will send an update once we have the results.
> > > > >
> > > > > Thanks,
> > > > > Satish.
> > > > >
> > > >
> >
>


Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Divij Vaidya
+ 1 (non-binding)

Verifications:
1. I ran a produce-consume workload with plaintext auth, JDK17, zstd
compression using an open messaging benchmark and found 3.6 to be better
than or equal to 3.5.1 across all dimensions. Notably, 3.6 had consistently
6-7% lower CPU utilization, lesser spikes on P99 produce latencies and
overall lower P99.8 latencies.

2. I have verified that detached signature is correct using
https://www.apache.org/info/verification.html and the release manager
public keys are available at
https://keys.openpgp.org/search?q=F65DC3423D4CD7B9

3. I have verified that all metrics emitted in 3.5.1 (with Zk) are also
being emitted in 3.6.0 (with Zk).

Problems (but not blockers):
1. Metrics added in
https://github.com/apache/kafka/commit/2f71708955b293658cec3b27e9a5588d39c38d7e
aren't available in the documentation (cc: Justine). I don't consider this
as a release blocker but we should add it as a fast follow-up.

2. Metric added in
https://github.com/apache/kafka/commit/a900794ace4dcf1f9dadee27fbd8b63979532a18
isn't available in documentation (cc: David). I don't consider this as a
release blocker but we should add it as a fast follow-up.

--
Divij Vaidya



On Mon, Oct 2, 2023 at 9:50 AM Federico Valeri  wrote:

> Hi Satish, I did the following to verify the release:
>
> - Built from source with Java 17 and Scala 2.13
> - Ran all unit and integration tests
> - Spot checked documentation
> - Ran custom client applications using staging artifacts on a 3-nodes
> cluster
> - Tested tiered storage with one of the available RSM implementations
>
> +1 (non binding)
>
> Thanks
> Fede
>
> On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
> >
> > Hi Satish,
> >
> > I verified with:
> > 1. Ran quick start in KRaft for scala 2.12 artifact
> > 2. Making sure the checksum are correct
> > 3. Browsing release notes, documents, javadocs, protocols.
> > 4. Verified the tiered storage feature works well.
> >
> > +1 (binding).
> >
> > Thanks.
> > Luke
> >
> >
> >
> > On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
> >
> > > +1 (non-binding). I used the Scala 2.13 binaries and the staged Maven
> > > artifacts and run my tests. Everything seems to work fine for me.
> > >
> > > Thanks
> > > Jakub
> > >
> > > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana <
> satish.dugg...@gmail.com>
> > > wrote:
> > >
> > > > Hello Kafka users, developers and client-developers,
> > > >
> > > > This is the third candidate for the release of Apache Kafka 3.6.0.
> > > > Some of the major features include:
> > > >
> > > > * KIP-405 : Kafka Tiered Storage
> > > > * KIP-868 : KRaft Metadata Transactions
> > > > * KIP-875: First-class offsets support in Kafka Connect
> > > > * KIP-898: Modernize Connect plugin discovery
> > > > * KIP-938: Add more metrics for measuring KRaft performance
> > > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > > * KIP-917: Additional custom metadata for remote log segment
> > > >
> > > > Release notes for the 3.6.0 release:
> > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > > >
> > > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > > >
> > > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > > https://kafka.apache.org/KEYS
> > > >
> > > > * Release artifacts to be voted upon (source and binary):
> > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > > >
> > > > * Maven artifacts to be voted upon:
> > > >
> https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > > >
> > > > * Javadoc:
> > > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > > >
> > > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> > > >
> > > > * Documentation:
> > > > https://kafka.apache.org/36/documentation.html
> > > >
> > > > * Protocol:
> > > > https://kafka.apache.org/36/protocol.html
> > > >
> > > > * Successful Jenkins builds for the 3.6 branch:
> > > > There are a few runs of unit/integration tests. You can see the
> latest
> > > > at https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/. We
> will
> > > > continue running a few more iterations.
> > > > System tests:
> > > > We will send an update once we have the results.
> > > >
> > > > Thanks,
> > > > Satish.
> > > >
> > >
>


Re: mirror-maker

2023-10-02 Thread Denny Fuchs

hi,

it seems to be pretty clear .. TCPDump says, it tries to connect to the 
other hosts too, so I need to allow it.



cu denny


mirror-maker

2023-10-02 Thread Denny Fuchs

hi,

I try to use mirror maker to replicate the data, and I have one trivial 
question:


Do I need access to / add  **all** Kafka Brokers ?

I have a 3.1 Cluster with 7 brokers and a new 3.5 with 5 brokers and I 
want to migrate from the 3.1 (Zoo) to 3.5 (KRaft). Both clusters are in 
different networks with a lot of firewalls and we allowed only access to 
one of the old clusters:


My mirror-mager is on the end.

The problem is .. that if I start the sync .. it seems to be stuck 
somewhere:



==

 (org.apache.kafka.connect.mirror.MirrorHeartbeatConfig:369)
[2023-10-02 14:07:50,091] INFO [MirrorHeartbeatConnector|task-0] 
WorkerSourceTask{id=MirrorHeartbeatConnector-0} Source task finished 
initialization and start 
(org.apache.kafka.connect.runtime.AbstractWorkerSource

Task:275)
[2023-10-02 14:07:50,113] INFO [MirrorHeartbeatConnector|task-0] 
[Producer clientId=connector-producer-MirrorHeartbeatConnector-0] 
Cluster ID: Rnpnd4EcRBeWo8vUrWlOIQ (org.apache.kafka.clients.Metadata:287)
[2023-10-02 14:07:50,238] INFO [MirrorSourceConnector|worker] 
[AdminClient 
clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin] 
Disconnecting from node 7 due to socket connection setup timeout.

The timeout value is 11336 ms. (org.apache.kafka.clients.NetworkClient:858)
[2023-10-02 14:07:50,828] INFO [MirrorCheckpointConnector|worker] 
[AdminClient 
clientId=FC-R02->FC-R01|MirrorCheckpointConnector|checkpoint-source-admin] 
Disconnecting from node 7 due to socket connection setup ti
meout. The timeout value is 11943 ms. 
(org.apache.kafka.clients.NetworkClient:858)
[2023-10-02 14:07:58,045] INFO [MirrorCheckpointConnector|worker] 
[AdminClient 
clientId=FC-R02->FC-R01|MirrorCheckpointConnector|checkpoint-source-admin] 
Disconnecting from node 6 due to socket connection setup ti
meout. The timeout value is 8916 ms. 
(org.apache.kafka.clients.NetworkClient:858)
[2023-10-02 14:08:00,271] INFO [MirrorSourceConnector|worker] 
[AdminClient 
clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin] 
Disconnecting from node 6 due to socket connection setup timeout.

The timeout value is 10018 ms. (org.apache.kafka.clients.NetworkClient:858)
[2023-10-02 14:08:00,359] INFO [MirrorSourceConnector|worker] 
[AdminClient 
clientId=FC-R02->FC-R01|MirrorSourceConnector|replication-source-admin] 
Disconnecting from node 7 due to socket connection setup timeout.

The timeout value is 11266 ms. (org.apache.kafka.clients.NetworkClient:858)
[2023-10-02 14:08:00,440] INFO [MirrorSourceConnector|worker] loading 
initial set of topic-partitions took 11359 ms 
(org.apache.kafka.connect.mirror.Scheduler:95)
[2023-10-02 14:08:00,448] INFO [MirrorSourceConnector|worker] creating 
downstream topic-partitions took 7 ms 
(org.apache.kafka.connect.mirror.Scheduler:95)
[2023-10-02 14:08:00,454] INFO [MirrorSourceConnector|worker] refreshing 
known target topics took 5 ms 
(org.apache.kafka.connect.mirror.Scheduler:95)
[2023-10-02 14:08:00,457] INFO [MirrorSourceConnector|worker] Started 
MirrorSourceConnector with 0 topic-partitions. 
(org.apache.kafka.connect.mirror.MirrorSourceConnector:172)
[2023-10-02 14:08:00,458] INFO [MirrorSourceConnector|worker] Starting 
MirrorSourceConnector took 11510 ms. 
(org.apache.kafka.connect.mirror.MirrorSourceConnector:173)

[2023-10-02 14:08:00,461] INFO SourceConnectorConfig values:
    config.action.reload = restart
    connector.class = 
org.apache.kafka.connect.mirror.MirrorSourceConnector


==

So, If I see it correct, it tries to connect to the other nodes .. which 
is not allowed yet .. right ?


cu denny


###

clusters = FC-R02, FC-R01

FC-R02.bootstrap.servers = fc-r02-kafka-01.example.com:9093
FC-R01.bootstrap.servers = fc-r01-kafka-01.example.com:9092

FC-R02->FC-R01.enabled = true
FC-R01->FC-R02.enabled = false

FC-R02->FC-R01.topics = 
"(blacklist_transaction.*|foo_transaction.*|chargebacks.*)"

replication.factor=4
sync.topic.acls.enabled=false

checkpoints.topic.replication.factor=3
heartbeats.topic.replication.factor=3
offset-syncs.topic.replication.factor=3

offset.storage.replication.factor=3
status.storage.replication.factor=3
config.storage.replication.factor=3

security.protocol: SASL_SSL
ssl.truststore.type = PEM
ssl.truststore.location=/etc/ssl/certs/ca-certificates.crt
ssl.enabled.protocols=TLSv1.2
security.protocol = SASL_SSL

FC-R01.sasl.mechanism=PLAIN
FC-R01.ssl.endpoint.identification.algorithm=https
FC-R01.sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule 
required \

    serviceName='kafka' \
    username='kafka' \
    password='secret';

FC-R02.security.protocol: SSL
FC-R02.sasl.mechanism=PLAIN
FC-R02.ssl.endpoint.identification.algorithm=https
FC-R02.sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule 
required \

    serviceName='kafka' \
    username='kafka' \
    

**SOLVED** Re: Offline migration from Zookeeper cluster (3.1) to Kraft enabled Cluster 3.5

2023-10-02 Thread Denny Fuchs

hi,


Am 01.10.23 um 22:49 schrieb Denny Fuchs:

What is the way, to do this ?


it seems, that mirror-maker is the right tool

cu denny


Re: Kafka Protocol : Compact Array or Array ?

2023-10-02 Thread Neeraj Vaidya
Thanks Luke. That helps.
Where can I find information about schemas of all requests ?

Regards,
Neeraj

Sent from my iPhone

> On 2 Oct 2023, at 6:03 pm, Luke Chen  wrote:
> 
> Hi Neeraj,
> 
> Yes, for MetadataRequest, version 0 ~ 8, the topic is ARRAY type. After
> version 9, it'll be COMPACT_ARRAY.
> It's because of this definition: "flexibleVersions": "9+".
> You can check KIP-482 for more information:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-482
> 
> Thanks.
> Luke
> 
>> On Mon, Oct 2, 2023 at 5:09 AM Neeraj Vaidya
>>  wrote:
>> 
>> Hi All,
>> I've raised this on StackOverflow as well :
>> https://stackoverflow.com/questions/77208870/kafka-binary-protocol-array-or-compact-array
>> In case someone can help in answering that question.
>> 
>> Regards,
>> Neeraj
>> On Sunday, 1 October, 2023 at 11:32:49 am GMT+11, Neeraj Vaidya <
>> neeraj.vai...@yahoo.co.in> wrote:
>> 
>> Hi All,
>> There are 2 types of arrays specified in the Kafka protocol documentation
>> : ARRAY and COMPACT_ARRAY.
>> But in the protocol details for the different messages, it does not
>> explicitly specify if the array type is which one of the above.
>> 
>> For example, the BNF grammar for the section for MetadataRequest API is as
>> below :
>> 
>> Metadata Request (Version: 0) => [topics]
>>  topics => name
>>name => STRING
>> 
>> What is the type of [topics] ? Is it ARRAY or COMPACT_ARRAY ?
>> 
>> After playing around with the protocol using some tests, I think for
>> Version:0 of this API request, the broker expects this to be of type ARRAY.
>> 
>> But for higher versions, say v9, COMPACT_ARRAY is expected.
>> 
>> I think the protocol really needs to be explicit and is lacking in this
>> respect.
>> 
>> Regards,
>> Neeraj



Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Federico Valeri
Hi Satish, I did the following to verify the release:

- Built from source with Java 17 and Scala 2.13
- Ran all unit and integration tests
- Spot checked documentation
- Ran custom client applications using staging artifacts on a 3-nodes cluster
- Tested tiered storage with one of the available RSM implementations

+1 (non binding)

Thanks
Fede

On Mon, Oct 2, 2023 at 8:50 AM Luke Chen  wrote:
>
> Hi Satish,
>
> I verified with:
> 1. Ran quick start in KRaft for scala 2.12 artifact
> 2. Making sure the checksum are correct
> 3. Browsing release notes, documents, javadocs, protocols.
> 4. Verified the tiered storage feature works well.
>
> +1 (binding).
>
> Thanks.
> Luke
>
>
>
> On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:
>
> > +1 (non-binding). I used the Scala 2.13 binaries and the staged Maven
> > artifacts and run my tests. Everything seems to work fine for me.
> >
> > Thanks
> > Jakub
> >
> > On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana 
> > wrote:
> >
> > > Hello Kafka users, developers and client-developers,
> > >
> > > This is the third candidate for the release of Apache Kafka 3.6.0.
> > > Some of the major features include:
> > >
> > > * KIP-405 : Kafka Tiered Storage
> > > * KIP-868 : KRaft Metadata Transactions
> > > * KIP-875: First-class offsets support in Kafka Connect
> > > * KIP-898: Modernize Connect plugin discovery
> > > * KIP-938: Add more metrics for measuring KRaft performance
> > > * KIP-902: Upgrade Zookeeper to 3.8.1
> > > * KIP-917: Additional custom metadata for remote log segment
> > >
> > > Release notes for the 3.6.0 release:
> > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> > >
> > > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> > >
> > > Kafka's KEYS file containing PGP keys we use to sign the release:
> > > https://kafka.apache.org/KEYS
> > >
> > > * Release artifacts to be voted upon (source and binary):
> > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> > >
> > > * Maven artifacts to be voted upon:
> > > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> > >
> > > * Javadoc:
> > > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> > >
> > > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> > >
> > > * Documentation:
> > > https://kafka.apache.org/36/documentation.html
> > >
> > > * Protocol:
> > > https://kafka.apache.org/36/protocol.html
> > >
> > > * Successful Jenkins builds for the 3.6 branch:
> > > There are a few runs of unit/integration tests. You can see the latest
> > > at https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/. We will
> > > continue running a few more iterations.
> > > System tests:
> > > We will send an update once we have the results.
> > >
> > > Thanks,
> > > Satish.
> > >
> >


Re: Kafka Protocol : Compact Array or Array ?

2023-10-02 Thread Luke Chen
Hi Neeraj,

Yes, for MetadataRequest, version 0 ~ 8, the topic is ARRAY type. After
version 9, it'll be COMPACT_ARRAY.
It's because of this definition: "flexibleVersions": "9+".
You can check KIP-482 for more information:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-482

Thanks.
Luke

On Mon, Oct 2, 2023 at 5:09 AM Neeraj Vaidya
 wrote:

>  Hi All,
> I've raised this on StackOverflow as well :
> https://stackoverflow.com/questions/77208870/kafka-binary-protocol-array-or-compact-array
> In case someone can help in answering that question.
>
> Regards,
> Neeraj
>  On Sunday, 1 October, 2023 at 11:32:49 am GMT+11, Neeraj Vaidya <
> neeraj.vai...@yahoo.co.in> wrote:
>
>  Hi All,
> There are 2 types of arrays specified in the Kafka protocol documentation
> : ARRAY and COMPACT_ARRAY.
> But in the protocol details for the different messages, it does not
> explicitly specify if the array type is which one of the above.
>
> For example, the BNF grammar for the section for MetadataRequest API is as
> below :
>
> Metadata Request (Version: 0) => [topics]
>   topics => name
> name => STRING
>
> What is the type of [topics] ? Is it ARRAY or COMPACT_ARRAY ?
>
> After playing around with the protocol using some tests, I think for
> Version:0 of this API request, the broker expects this to be of type ARRAY.
>
> But for higher versions, say v9, COMPACT_ARRAY is expected.
>
> I think the protocol really needs to be explicit and is lacking in this
> respect.
>
> Regards,
> Neeraj


Re: [VOTE] 3.6.0 RC2

2023-10-02 Thread Luke Chen
Hi Satish,

I verified with:
1. Ran quick start in KRaft for scala 2.12 artifact
2. Making sure the checksum are correct
3. Browsing release notes, documents, javadocs, protocols.
4. Verified the tiered storage feature works well.

+1 (binding).

Thanks.
Luke



On Mon, Oct 2, 2023 at 5:23 AM Jakub Scholz  wrote:

> +1 (non-binding). I used the Scala 2.13 binaries and the staged Maven
> artifacts and run my tests. Everything seems to work fine for me.
>
> Thanks
> Jakub
>
> On Fri, Sep 29, 2023 at 8:17 PM Satish Duggana 
> wrote:
>
> > Hello Kafka users, developers and client-developers,
> >
> > This is the third candidate for the release of Apache Kafka 3.6.0.
> > Some of the major features include:
> >
> > * KIP-405 : Kafka Tiered Storage
> > * KIP-868 : KRaft Metadata Transactions
> > * KIP-875: First-class offsets support in Kafka Connect
> > * KIP-898: Modernize Connect plugin discovery
> > * KIP-938: Add more metrics for measuring KRaft performance
> > * KIP-902: Upgrade Zookeeper to 3.8.1
> > * KIP-917: Additional custom metadata for remote log segment
> >
> > Release notes for the 3.6.0 release:
> > https://home.apache.org/~satishd/kafka-3.6.0-rc2/RELEASE_NOTES.html
> >
> > *** Please download, test and vote by Tuesday, October 3, 12pm PT
> >
> > Kafka's KEYS file containing PGP keys we use to sign the release:
> > https://kafka.apache.org/KEYS
> >
> > * Release artifacts to be voted upon (source and binary):
> > https://home.apache.org/~satishd/kafka-3.6.0-rc2/
> >
> > * Maven artifacts to be voted upon:
> > https://repository.apache.org/content/groups/staging/org/apache/kafka/
> >
> > * Javadoc:
> > https://home.apache.org/~satishd/kafka-3.6.0-rc2/javadoc/
> >
> > * Tag to be voted upon (off 3.6 branch) is the 3.6.0-rc2 tag:
> > https://github.com/apache/kafka/releases/tag/3.6.0-rc2
> >
> > * Documentation:
> > https://kafka.apache.org/36/documentation.html
> >
> > * Protocol:
> > https://kafka.apache.org/36/protocol.html
> >
> > * Successful Jenkins builds for the 3.6 branch:
> > There are a few runs of unit/integration tests. You can see the latest
> > at https://ci-builds.apache.org/job/Kafka/job/kafka/job/3.6/. We will
> > continue running a few more iterations.
> > System tests:
> > We will send an update once we have the results.
> >
> > Thanks,
> > Satish.
> >
>