[
https://issues.apache.org/jira/browse/KAFKA-3442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15205586#comment-15205586
]
Jun Rao commented on KAFKA-3442:
--------------------------------
[~becket_qin]: It does seem that I can reproduce the issue.
1. Start 0.10.0 broker.
2. Create a topic that allows max message to be 1.3M
bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic test3
--partition 1 --replication-factor 1 --config max.message.bytes=1300000
3.Publish a single message of 1.2M
bin/kafka-producer-perf-test.sh --topic test3 --num-records 1 --record-size
1200000 --throughput 10000000 --producer-props bootstrap.servers=localhost:9092
max.request.size=2000000
4. If I run the following from 0.8.2, the consumer seems to be able to consumer
the oversize message w/o any error.
bin/kafka-console-consumer.sh --from-beginning --topic test3 --zookeeper
localhost:2181
If I run the same command from 0.10.0, the consumer errors out.
bin/kafka-console-consumer.sh --from-beginning --topic test3 --zookeeper
localhost:2181
[2016-03-21 17:22:16,506] ERROR Error processing message, terminating consumer
process: (kafka.tools.ConsoleConsumer$)
kafka.common.MessageSizeTooLargeException: Found a message larger than the
maximum fetch size of this consumer on topic test3 partition 0 at fetch offset
0. Increase the fetch size, or decrease the maximum message size the broker
will allow.
at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:90)
at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33)
at
kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:64)
at kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:56)
at kafka.consumer.OldConsumer.receive(BaseConsumer.scala:102)
at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:113)
at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:73)
at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:51)
at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
> FetchResponse size exceeds max.partition.fetch.bytes
> ----------------------------------------------------
>
> Key: KAFKA-3442
> URL: https://issues.apache.org/jira/browse/KAFKA-3442
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.10.0.0
> Reporter: Dana Powers
> Priority: Blocker
> Fix For: 0.10.0.0
>
>
> Produce 10000 byte message to topic foobar
> Fetch foobar w/ max.partition.fetch.bytes=1024
> Test expects to receive a truncated message (~1024 bytes). 0.8 and 0.9 pass
> this test, but 0.10 FetchResponse has full message, exceeding the max
> specified in the FetchRequest.
> I tested with v0 and v1 apis, both fail. Have not tested w/ v2
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)