Hi Gonzalo,

So, the message conversion is because you enabled server compression with
type zstd.
If you remove the config ["compression.type" = "zstd"] in broker,
everything should be fine.

> I tried disabling zstd, but ProduceMessageConversionsPerSec remained at
the same level
Maybe you need to run longer to see the difference.
Anyway, from the source code, I can confirm if the broker is not enabling
compression, or source compression type == broker compression type, there
will be no message conversion happening.

Please give it another try.

Thank you.

On Fri, Feb 10, 2023 at 4:51 AM Gonzalo Martin Peci <pecigonz...@gmail.com>
wrote:

> Hey Luke, thanks for your reply.
>
> We run:
> - Brokers: 3.3.1
> - Java Clients:
>   - Confluent "7.3.1-ccs"
>   - Kafka official "3.3.2"
>
> We are running on AWS MSK, and using some minimal configuration:
> ```
> # Security
> "allow.everyone.if.no.acl.found" = false
> "auto.create.topics.enable"      = false
>
> # Guardrails
> "default.replication.factor" = 3
> "min.insync.replicas"        = 2
>
> # Cluster Optimizations
> "unclean.leader.election.enable"         = false
> "group.initial.rebalance.delay.ms"       = 3000
> "leader.imbalance.per.broker.percentage" = 10
>
> # Offset Availability
> "offsets.topic.replication.factor" = 3
>
> # Transaction Availability
> "transaction.state.log.replication.factor" = 3
> "transaction.state.log.min.isr"            = 1
>
> # Storage Optimization
> "compression.type" = "zstd"
> ```
>
> I tried disabling zstd, but ProduceMessageConversionsPerSec remained at the
> same level. Client configuration is also default, so nothing about message
> compression or version.
>
> Thanks
>
> PS: Sorry for the delay, for some reason I did not get the reply to my
> emial.
>
> On 2023/02/08 10:46:19 Luke Chen wrote:
> > Hi Gonzalo,
> >
> > For the produce request record version, you should refer to this file:
> >
>
> https://github.com/apache/kafka/blob/trunk/clients/src/main/resources/common/message/ProduceRequest.json#L35
> >
> > But you're right, basically the message conversion happened in a very old
> > produce request version (ex: version 1 or version 2, which I guess is
> > corresponding to Kafka v1.0 or older) sent to a new versioned broker,
> like
> > this comment said in the file above:
> >
> >   // Version 3 adds the transactional ID, which is used for authorization
> > when attempting to write
> >   // transactional data.  *Version 3 also adds support for Kafka Message
> > Format v2.*
> >
> > Could you let us know which kafka version you're using in broker?
> > And which client and version you're using?
> > And the broker configuration and client configuration is also helpful.
> >
> > Thank you
> > Luke
> >
> >
> > On Tue, Feb 7, 2023 at 11:12 PM Gonzalo Martin Peci <pe...@gmail.com>
> > wrote:
> >
> > > Hi! We have been trying to figure out why we see a high value of
> > > ProduceMessageConversionsPerSec and potentially high CPU usage. I was
> > > trying to understand what version our producers/consumers were using,
> but I
> > > was unable to grok this. Any help would be appreciated.
> > >
> > > We are seeing values of `version=8` and `version=9` for RequestsPerSec
> > > "request=produce", and we found:
> > >
> > >
>
> https://github.com/apache/kafka/blob/trunk/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java#L163-L171
> > > which indicate version 3.4 and 3.5.
> > >
> > > Im following what is described in KIP-272. Although I later found
> KIP-511
> > > which seems to expose a new metric, KIP-896 also implies that versions
> are
> > > checked through the RequestsPerSec metric.
> > >
> > > Any guidance will be appreciated.
> > >
> > > References:
> > > -
> > >
> > >
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-272%3A+Add+API+version+tag+to+broker%27s+RequestsPerSec+metric
> > > -
> > >
> > >
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers
> > > -
> > >
> > >
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-896%3A+Remove+old+client+protocol+API+versions+in+Kafka+4.0
> > >
> > > Thanks
> > > Gonzalo
> > >
> >
>

Reply via email to