[ 
https://issues.apache.org/jira/browse/KAFKA-7549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16690978#comment-16690978
 ] 

ASF GitHub Bot commented on KAFKA-7549:
---------------------------------------

dongjinleekr opened a new pull request #5925: KAFKA-7549: Old ProduceRequest 
with zstd compression does not return error to client
URL: https://github.com/apache/kafka/pull/5925
 
 
   As of current version (2.1.0), zstd-related validations are located in 
following spots:
   
   1. **ProduceRequest**
   
       - `MemoryRecordsBuilder`: can't create `MemoryRecords` with magic < 2 
(`IllegalArgumentException`)
       - `ProduceRequest.Builder`: can't create `ProduceRequest` with api 
version below 7 (`InvalidRecordException`)
   
   2. **FetchRequest**
   
       - `KafkaApis#handleFetchRequest`: Returns `FetchResponse` w/ 
`Errors#UNSUPPORTED_COMPRESSION_TYPE` if ...
           1. `FetchRequest` w/ API version < 10 is delivered to a 
zstd-compressed topic.
           2. Down-conversion failure: `LazyDownConversionRecords#makeNext` → 
`RecordsUtil#downConvert` throws `UnsupportedCompressionTypeException`.
   
   3. **Etc**
   
       - `AbstractLegacyRecordBatch.DeepRecordsIterator`: A boilerplate 
validation for legacy record batches.
   
   **In short, there is no broker-side validation for `ProduceRequest` w/ zstd 
compressed records.** This PR compensates this hole.
   
   There is a reason why this validation can't be located in other class, e.g., 
`LogValidator`: it can't see the API version of `ProduceRequest.` The only 
method that can check both of `CompressionType` and API version is 
`KafkaApis#handleProduceRequest`; it's why.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Old ProduceRequest with zstd compression does not return error to client
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-7549
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7549
>             Project: Kafka
>          Issue Type: Bug
>          Components: compression
>            Reporter: Magnus Edenhill
>            Assignee: Lee Dongjin
>            Priority: Major
>             Fix For: 2.2.0, 2.1.1
>
>
> Kafka broker v2.1.0rc0.
>  
> KIP-110 states that:
> "Zstd will only be allowed for the bumped produce API. That is, for older 
> version clients(=below KAFKA_2_1_IV0), we return UNSUPPORTED_COMPRESSION_TYPE 
> regardless of the message format."
>  
> However, sending a ProduceRequest V3 with zstd compression (which is a client 
> side bug) closes the connection with the following exception rather than 
> returning UNSUPPORTED_COMPRESSION_TYPE in the ProduceResponse:
>  
> {noformat}
> [2018-10-25 11:40:31,813] ERROR Exception while processing request from 
> 127.0.0.1:60723-127.0.0.1:60656-94 (kafka.network.Processor)
> org.apache.kafka.common.errors.InvalidRequestException: Error getting request 
> for apiKey: PRODUCE, apiVersion: 3, connectionId: 
> 127.0.0.1:60723-127.0.0.1:60656-94, listenerName: ListenerName(PLAINTEXT), 
> principal: User:ANONYMOUS
> Caused by: org.apache.kafka.common.record.InvalidRecordException: Produce 
> requests with version 3 are note allowed to use ZStandard compression
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to