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


Thanks for the patch. A few comments below.


clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java (line 
77)
<https://reviews.apache.org/r/33378/#comment148702>

    With this, we are bounding the constructor to the v1 protocol. This is fine 
for FetchResponse for now since the broker uses the scala objects for dealing 
with the send file api. However, we should add a comment to indicate this is 
the constructor for v1 protocol.



clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
(line 55)
<https://reviews.apache.org/r/33378/#comment148701>

    We actually will need different constructors for different versions. We 
want to reuse those request/response objects on the broker side. So, the broker 
will need to construct different version of the response based on the version 
of the request. You can take a look at OffsetCommitRequest as an example.



core/src/main/scala/kafka/api/FetchResponse.scala (lines 193 - 195)
<https://reviews.apache.org/r/33378/#comment148704>

    No need to wrap single line statement with {}.



core/src/main/scala/kafka/api/ProducerResponse.scala (line 40)
<https://reviews.apache.org/r/33378/#comment148709>

    Perhaps it's worth adding a commment that readFrom() only reads v1 format.



core/src/main/scala/kafka/api/ProducerResponse.scala (lines 49 - 50)
<https://reviews.apache.org/r/33378/#comment148707>

    Coding style: no space before :.
    
    Also, could requestVersion be just version?



core/src/main/scala/kafka/api/ProducerResponse.scala (lines 95 - 97)
<https://reviews.apache.org/r/33378/#comment148706>

    Coding style: no need to wrap single line statement with {}.


- Jun Rao


On July 13, 2015, 8:36 p.m., Aditya Auradkar wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/33378/
> -----------------------------------------------------------
> 
> (Updated July 13, 2015, 8:36 p.m.)
> 
> 
> Review request for kafka, Joel Koshy and Jun Rao.
> 
> 
> Bugs: KAFKA-2136
>     https://issues.apache.org/jira/browse/KAFKA-2136
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> Changes are
> - Addressing Joel's comments
> - protocol changes to the fetch request and response to return the 
> throttle_time_ms to clients
> - New producer/consumer metrics to expose the avg and max delay time for a 
> client
> - Test cases.
> - Addressed Joel's comments
>                       
> For now the patch will publish a zero delay and return a response
> 
> 
> Diffs
> -----
> 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
>  56281ee15cc33dfc96ff64d5b1e596047c7132a4 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> 07e65d4a54ba4eef5b787eba3e71cbe9f6a920bd 
>   clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java 
> 3dc8b015afd2347a41c9a9dbc02b8e367da5f75f 
>   clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java 
> 8686d83aa52e435c6adafbe9ff4bd1602281072a 
>   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
> eb8951fba48c335095cc43fc3672de1c733e07ff 
>   clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java 
> fabeae3083a8ea55cdacbb9568f3847ccd85bab4 
>   clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
> 37ec0b79beafcf5735c386b066eb319fb697eff5 
>   
> clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
>  419541011d652becf0cda7a5e62ce813cddb1732 
>   
> clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
>  8b1805d3d2bcb9fe2bacb37d870c3236aa9532c4 
>   
> clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
>  e3cc1967e407b64cc734548c19e30de700b64ba8 
>   core/src/main/scala/kafka/api/FetchRequest.scala 
> 5b38f8554898e54800abd65a7415dd0ac41fd958 
>   core/src/main/scala/kafka/api/FetchResponse.scala 
> 0b6b33ab6f7a732ff1322b6f48abd4c43e0d7215 
>   core/src/main/scala/kafka/api/ProducerRequest.scala 
> c866180d3680da03e48d374415f10220f6ca68c4 
>   core/src/main/scala/kafka/api/ProducerResponse.scala 
> 5d1fac4cb8943f5bfaa487f8e9d9d2856efbd330 
>   core/src/main/scala/kafka/consumer/SimpleConsumer.scala 
> c16f7edd322709060e54c77eb505c44cbd77a4ec 
>   core/src/main/scala/kafka/server/AbstractFetcherThread.scala 
> 83fc47417dd7fe4edf030217fa7fd69d99b170b0 
>   core/src/main/scala/kafka/server/DelayedFetch.scala 
> de6cf5bdaa0e70394162febc63b50b55ca0a92db 
>   core/src/main/scala/kafka/server/DelayedProduce.scala 
> 05078b24ef28f2f4e099afa943e43f1d00359fda 
>   core/src/main/scala/kafka/server/KafkaApis.scala 
> d63bc18d795a6f0e6994538ca55a9a46f7fb8ffd 
>   core/src/main/scala/kafka/server/OffsetManager.scala 
> 5cca85cf727975f6d3acb2223fd186753ad761dc 
>   core/src/main/scala/kafka/server/ReplicaFetcherThread.scala 
> b31b432a226ba79546dd22ef1d2acbb439c2e9a3 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 
> 59c9bc3ac3a8afc07a6f8c88c5871304db588d17 
>   core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala 
> 5717165f2344823fabe8f7cfafae4bb8af2d949a 
>   core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala 
> 00d59337a99ac135e8689bd1ecd928f7b1423d79 
> 
> Diff: https://reviews.apache.org/r/33378/diff/
> 
> 
> Testing
> -------
> 
> New tests added
> 
> 
> Thanks,
> 
> Aditya Auradkar
> 
>

Reply via email to