-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/36578/#review92374
-----------------------------------------------------------


In addition to inline comments, the patch fell out of sync recently and will 
need a rebase.


core/src/main/scala/kafka/admin/TopicCommand.scala (line 90)
<https://reviews.apache.org/r/36578/#comment146503>

    This format call isn't working because it's being called on the second 
string due to the split across lines and +. Needs parens added or switched back 
to a single string.



core/src/main/scala/kafka/server/AbstractFetcherThread.scala (line 166)
<https://reviews.apache.org/r/36578/#comment146524>

    My quick test indicates this doesn't work -- I don't think this is the code 
that would be returned anyway since I don't think max message size is checked 
for fetch requests, fetch.message.max.bytes for consumers and  
replica.fetch.max.bytes for brokers are both for aggregate data size. The 
problem occurs when the total aggregate size permitted per request is smaller 
than a single message. I think in that case we're just returning 0 messages, 
but not an error code. After enough time the result is that the ISR shrinks 
(which is what my test showed in the broker logs).
    
    I think to properly log this we might need to log it on the leader node not 
on the fetcher -- probably somewhere in ReplicaManager. If you're not familiar 
with this code, you'll want to start in KafkaApis.scala in handleOffsetRequest. 
However, I'm not sure what this means about issuing a useful warning to 
consumers since they wouldn't have easy access to broker logs. On the other 
hand, a stalled consumer is less of a problem than the ISR being forced down to 
a single replica.
    
    For reference, I tested this with a simple config with two brokers, 
adjusting message.max.bytes and replica.fetch.max.bytes. Then I created a topic 
with replication factor 2 and used console producer to send data of different 
sizes to test the output.


- Ewen Cheslack-Postava


On July 18, 2015, 3:37 a.m., Edward Ribeiro wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/36578/
> -----------------------------------------------------------
> 
> (Updated July 18, 2015, 3:37 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-2338
>     https://issues.apache.org/jira/browse/KAFKA-2338
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-2338 Warn users if they change max.message.bytes that they also need to 
> update broker and consumer settings
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/admin/TopicCommand.scala 
> a90aa8787ff21b963765a547980154363c1c93c6 
>   core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
> f84306143c43049e3aa44e42beaffe7eb2783163 
> 
> Diff: https://reviews.apache.org/r/36578/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Edward Ribeiro
> 
>

Reply via email to