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

Maulin Vasavada commented on KAFKA-6315:
----------------------------------------

Hi 

We reproduced this issue and it seems to happen due to message format changes 
during broker upgrades if we are not careful. Example test-case we reproduced 
on,
 # Kafka broker on 1.1.1 with log.message.format.version=0.10.2.0 configured
 # We upgraded to Kafka broker 2.2.1 and made sure we removed 
log.message.format.version configuration with 0.10 value, which mean it 
defaults to '2.2-IV1'
 # We produced some messages to Kafka broker with lz4 compression with Kafka 
producer 0.10.2.1, No consumers running yet
 # We modified the log.message.format.version=0.10.2.0 in the Kafka 2.2.1 
broker and restarted
 # Started Kafka consumer on 0.10.2.1 and got the same exception 
'java.lang.IllegalArgumentException: Unknown compression type id: 5'

In essence, if we are stuck on older message format, the upgrade needs to be 
handled carefully to avoid this issue.

 

> kafka-console-consumer.sh terminating consumer process only for one partition 
> with java.lang.IllegalArgumentException: Unknown compression type id: 5
> -----------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-6315
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6315
>             Project: Kafka
>          Issue Type: Bug
>          Components: compression, consumer, log, security
>    Affects Versions: 0.10.2.1
>            Reporter: VinayKumar
>            Priority: Major
>
> For a topic, having 3 partitions- kafka-console-consumer.sh is able to 
> process messages for two partitions but for one partition, I'm seeing the 
> below error
> [2017-12-06 18:31:49,831] ERROR Error processing message, terminating 
> consumer process:  (kafka.tools.ConsoleConsumer$)
> java.lang.IllegalArgumentException: Unknown compression type id: 5
> Can someone please help on this issue.
> bin/kafka-console-consumer.sh --bootstrap-server xxxxx:9093 --consumer.config 
> /tmp/consumer.poc.properties  --max-messages 1 --from-beginning --partition 1 
> --topic topicx
> [2017-12-06 18:31:49,831] ERROR Error processing message, terminating 
> consumer process:  (kafka.tools.ConsoleConsumer$)
> java.lang.IllegalArgumentException: Unknown compression type id: 5
>       at 
> org.apache.kafka.common.record.CompressionType.forId(CompressionType.java:46)
>       at 
> org.apache.kafka.common.record.Record.compressionType(Record.java:260)
>       at 
> org.apache.kafka.common.record.LogEntry.isCompressed(LogEntry.java:89)
>       at 
> org.apache.kafka.common.record.RecordsIterator.makeNext(RecordsIterator.java:70)
>       at 
> org.apache.kafka.common.record.RecordsIterator.makeNext(RecordsIterator.java:34)
>       at 
> org.apache.kafka.common.utils.AbstractIterator.maybeComputeNext(AbstractIterator.java:79)
>       at 
> org.apache.kafka.common.utils.AbstractIterator.hasNext(AbstractIterator.java:45)
>       at 
> org.apache.kafka.clients.consumer.internals.Fetcher.parseCompletedFetch(Fetcher.java:785)
>       at 
> org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:480)
>       at 
> org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1061)
>       at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:995)
>       at kafka.consumer.NewShinyConsumer.receive(BaseConsumer.scala:88)
>       at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:120)
>       at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:75)
>       at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:50)
>       at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to