Re: Review Request 27799: Patch for KAFKA-1760

2015-01-21 Thread Jay Kreps


> On Jan. 13, 2015, 10:32 p.m., Onur Karaman wrote:
> > clients/src/main/java/org/apache/kafka/clients/ClientRequest.java, lines 
> > 34-37
> > 
> >
> > It looks like you'd want to replace the attachment docs with new 
> > callback docs.

Good catch.


- Jay


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


On Jan. 19, 2015, 3:10 a.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27799/
> ---
> 
> (Updated Jan. 19, 2015, 3:10 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1760
> https://issues.apache.org/jira/browse/KAFKA-1760
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> New consumer.
> 
> 
> Diffs
> -
> 
>   build.gradle c9ac43378c3bf5443f0f47c8ba76067237ecb348 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
>   clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
> 8aece7e81a804b177a6f2c12e2dc6c89c1613262 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
> ab7e3220f9b76b92ef981d695299656f041ad5ed 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 397695568d3fd8e835d8f923a89b3b00c96d0ead 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> 6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
>   
> clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
> c0c636b3e1ba213033db6d23655032c9bbd5e378 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> 57c1807ccba9f264186f83e91f37c34b959c8060 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
>  e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
> 16af70a5de52cca786fdea147a6a639b7dc4a311 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
> bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> 76efc216c9e6c3ab084461d792877092a189ad0f 
>   clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
> fa88ac1a8b19b4294f211c4467fe68c7707ddbae 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
> ea423ad15eebd262d20d5ec05d592cc115229177 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> fc71710dd5997576d3841a1c3b0f7e19a8c9698e 
>   clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 
> 904976fadf0610982958628eaee810b60a98d725 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
>  dcf46581b912cfb1b5c8d4cbc293d2d1444b7740 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
>  483899d2e69b33655d0e08949f5f64af2519660a 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> ccc03d8447ebba40131a70e16969686ac4aab58a 
>   clients/src/main/java/org/apache/kafka/common/Cluster.java 
> d3299b944062d96852452de455902659ad8af757 
>   clients/src/main/java/org/apache/kafka/common/PartitionInfo.java 
> b15aa2c3ef2d7c4b24618ff42fd4da324237a813 
>   clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
> 98cb79b701918eca3f6ca9823b6c7b7c97b3ecec 
>   clients/src/main/java/org/apache/kafka/common/errors/ApiException.java 
> 7c948b166a8ac07616809f260754116ae7764973 
>   clients/src/main/java/org/apache/kafka/common/network/Selectable.java 
> b68bbf00ab8eba6c5867d346c91188142593ca6e 
>   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
> 74d695ba39de44b6a3d15340ec0114bc4fce2ba2 
>   clients/src/main/java/org/apache/kaf

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-21 Thread Jay Kreps


> On Jan. 20, 2015, 6:11 a.m., Guozhang Wang wrote:
> > clients/src/main/java/org/apache/kafka/common/Cluster.java, line 1
> > 
> >
> > Wondering if we should create a new "metadata" sub-directory of common, 
> > and move Cluster / Node / TopicPartition / PartitionInfo to it.

Personally I find it a bit easier to navigate flatter package 
structures...since these are public classes we probably shouldn't mess with 
them too much.


> On Jan. 20, 2015, 6:11 a.m., Guozhang Wang wrote:
> > clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java, line 
> > 297
> > 
> >
> > Strictly speaking this is a public API since ConfigDef is defined as 
> > public.

Hmm, I disagree. I think our definition of a public classes is any class in a 
package we generate the client javadoc for. Config is not one of those and is 
definitely meant to be an internal helper. I think the public/protected/private 
markers on the class just indicate whether it is public within kafka (i.e. used 
in multiple packages) not whether it is user-facing. Unfortunately java doesn't 
have a way to distinguish these...


> On Jan. 20, 2015, 6:11 a.m., Guozhang Wang wrote:
> > clients/src/main/java/org/apache/kafka/common/protocol/Errors.java, line 59
> > 
> >
> > Should we keep this TODO comment?

I removed it since those are not user-facing errors they are for broker-broker 
communication. I suppose we will add them eventually when we move the broker 
over to use this as source of truth, though...


> On Jan. 20, 2015, 6:11 a.m., Guozhang Wang wrote:
> > clients/src/main/java/org/apache/kafka/common/protocol/Errors.java, lines 
> > 63-65
> > 
> >
> > Shall we define those specific exceptions and replace the general 
> > ApiException here?

Well the issue is that these exceptions are not ones that can actually be 
thrown--those error codes are handled internally by the consumer. So rather 
than create new public exceptions that can't be thrown, which might be 
confusing, I added a dummy exception.


> On Jan. 20, 2015, 6:11 a.m., Guozhang Wang wrote:
> > clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java, 
> > lines 222-224
> > 
> >
> > I think this case is covered by catching EOFException?

No it turns out that buffer.position(val_larger_than_limit) will actually throw 
IllegalArgumentException


- Jay


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


On Jan. 19, 2015, 3:10 a.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27799/
> ---
> 
> (Updated Jan. 19, 2015, 3:10 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1760
> https://issues.apache.org/jira/browse/KAFKA-1760
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> New consumer.
> 
> 
> Diffs
> -
> 
>   build.gradle c9ac43378c3bf5443f0f47c8ba76067237ecb348 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
>   clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
> 8aece7e81a804b177a6f2c12e2dc6c89c1613262 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
> ab7e3220f9b76b92ef981d695299656f041ad5ed 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 397695568d3fd8e835d8f923a89b3b00c96d0ead 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> 6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
>   
> clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
> c0c636b3e1ba213033db6d23655032c9bbd5e378 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> 57c1807ccba9f264186f83e91f37c34b959c8060 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
>  e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
> 16af70a5de52cca786fdea147a6a639b7dc4a311 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-20 Thread Guozhang Wang

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



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java


Magic number.



clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java


Should this be a private class?



clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java


Function names of partitionAutoAssigned and needsPartitionAssignment are a 
bit confusing. Probably rename to partitionAutoAssigned to topicSubscribed?



clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java


Needs updated.


- Guozhang Wang


On Jan. 19, 2015, 3:10 a.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27799/
> ---
> 
> (Updated Jan. 19, 2015, 3:10 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1760
> https://issues.apache.org/jira/browse/KAFKA-1760
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> New consumer.
> 
> 
> Diffs
> -
> 
>   build.gradle c9ac43378c3bf5443f0f47c8ba76067237ecb348 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
>   clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
> 8aece7e81a804b177a6f2c12e2dc6c89c1613262 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
> ab7e3220f9b76b92ef981d695299656f041ad5ed 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 397695568d3fd8e835d8f923a89b3b00c96d0ead 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> 6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
>   
> clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
> c0c636b3e1ba213033db6d23655032c9bbd5e378 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> 57c1807ccba9f264186f83e91f37c34b959c8060 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
>  e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
> 16af70a5de52cca786fdea147a6a639b7dc4a311 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
> bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> 76efc216c9e6c3ab084461d792877092a189ad0f 
>   clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
> fa88ac1a8b19b4294f211c4467fe68c7707ddbae 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
> ea423ad15eebd262d20d5ec05d592cc115229177 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> fc71710dd5997576d3841a1c3b0f7e19a8c9698e 
>   clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 
> 904976fadf0610982958628eaee810b60a98d725 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
>  dcf46581b912cfb1b5c8d4cbc293d2d1444b7740 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
>  483899d2e69b33655d0e08949f5f64af2519660a 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> ccc03d8447ebba40131a70e16969686ac4aab58a 
>   clients/src/main/java/org/apache/kafka/common/Cluster.java 
> d3299b944062d96852452de455902659ad8af757 
>   clients/src/main/java/org/apache/kafka/common/PartitionInfo.java 
> b15aa2c3ef2d7c4b24618ff42fd4da324237a813 
>   clients/src/main/java/org/apache/ka

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-20 Thread Guozhang Wang

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



clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java


"Data will be load balanced ..." I do not understand what does this mean?



clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java


"If not server ..." Remove this sentence?



clients/src/main/java/org/apache/kafka/clients/KafkaClient.java


Wondering why we make newline for @param but keep the same line for @return?



clients/src/main/java/org/apache/kafka/clients/NetworkClient.java


Is this function really private? If yes we do not need keep the javadoc for 
it.



clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java


The javadocs here are a little confusing: users looking for its function 
APIs need to look into KafkaConsumer, an implementation of the interface 
Consumer.



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


"Partitions .." Remove this line as it only make sense when consumer 
subscribe by topics.



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


"if the current offset is smaller than the oldest ..." => "the current 
offset does not exist on the server"



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


Should this inherit from CommonClientConfig?



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


"The maxmium amount of data per-partition ... The maximum total memory used 
..."



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


Could these two be moved to CommonClientConfig?



clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java


"none" => "disable" according to the docs?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Maybe add some comments on why consumerId / generationId are initialized as 
such and when they will be updated and used.



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


There is a potential risk that a topic is deleted and the consumer 
unscribes to it, but not removing it from its metadata topic list, causing the 
underlying network client to keep refreshing metadata.



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Should the ordering of these two "else if" be swapped?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Do we need to back off here?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Incomplete comments.



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Rename this variable?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Should we change the "this.client.inFlightRequestCount .. " condition to 
just "node.ready()"?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Magic number "-1": should we define sth. like "OrdinaryConsumerReplicaId"?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Do not understand the TODO statement.



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java


Rename the function name?



clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java


This kafka exception could be thrown other places besides committed() and 
position(), it could also be thrown in:

private resetOffset() -> 

  private fetchMissingPositionsOrResetThem() ->

public position()
public pool()


Re: Review Request 27799: Patch for KAFKA-1760

2015-01-19 Thread Guozhang Wang

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



clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java


a request is complete => the corresponding response is received for this 
request.



clients/src/main/java/org/apache/kafka/common/Cluster.java


Wondering if we should create a new "metadata" sub-directory of common, and 
move Cluster / Node / TopicPartition / PartitionInfo to it.



clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java


Strictly speaking this is a public API since ConfigDef is defined as public.



clients/src/main/java/org/apache/kafka/common/protocol/Errors.java


Should we keep this TODO comment?



clients/src/main/java/org/apache/kafka/common/protocol/Errors.java


Shall we define those specific exceptions and replace the general 
ApiException here?



clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java


I think this case is covered by catching EOFException?


- Guozhang Wang


On Jan. 19, 2015, 3:10 a.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27799/
> ---
> 
> (Updated Jan. 19, 2015, 3:10 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1760
> https://issues.apache.org/jira/browse/KAFKA-1760
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> New consumer.
> 
> 
> Diffs
> -
> 
>   build.gradle c9ac43378c3bf5443f0f47c8ba76067237ecb348 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
>   clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
> 8aece7e81a804b177a6f2c12e2dc6c89c1613262 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
> ab7e3220f9b76b92ef981d695299656f041ad5ed 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 397695568d3fd8e835d8f923a89b3b00c96d0ead 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> 6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
>   
> clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
> c0c636b3e1ba213033db6d23655032c9bbd5e378 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> 57c1807ccba9f264186f83e91f37c34b959c8060 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
>  e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
> 16af70a5de52cca786fdea147a6a639b7dc4a311 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
> bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> 76efc216c9e6c3ab084461d792877092a189ad0f 
>   clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
> fa88ac1a8b19b4294f211c4467fe68c7707ddbae 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
> ea423ad15eebd262d20d5ec05d592cc115229177 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> fc71710dd5997576d3841a1c3b0f7e19a8c9698e 
>   clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 
> 904976fadf0610982958628eaee810b60a98d725 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
>  dcf46581b912cfb1b5c8d4cbc293d2d1444b7740 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Partit

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-18 Thread Jay Kreps

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

(Updated Jan. 19, 2015, 3:10 a.m.)


Review request for kafka.


Bugs: KAFKA-1760
https://issues.apache.org/jira/browse/KAFKA-1760


Repository: kafka


Description
---

New consumer.


Diffs (updated)
-

  build.gradle c9ac43378c3bf5443f0f47c8ba76067237ecb348 
  clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
  clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
8aece7e81a804b177a6f2c12e2dc6c89c1613262 
  clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
ab7e3220f9b76b92ef981d695299656f041ad5ed 
  clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
397695568d3fd8e835d8f923a89b3b00c96d0ead 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
  clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
c0c636b3e1ba213033db6d23655032c9bbd5e378 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
57c1807ccba9f264186f83e91f37c34b959c8060 
  
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
 e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
16af70a5de52cca786fdea147a6a639b7dc4a311 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
76efc216c9e6c3ab084461d792877092a189ad0f 
  clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
fa88ac1a8b19b4294f211c4467fe68c7707ddbae 
  
clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
 PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
ea423ad15eebd262d20d5ec05d592cc115229177 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
fc71710dd5997576d3841a1c3b0f7e19a8c9698e 
  clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 
904976fadf0610982958628eaee810b60a98d725 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java 
dcf46581b912cfb1b5c8d4cbc293d2d1444b7740 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
 483899d2e69b33655d0e08949f5f64af2519660a 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
ccc03d8447ebba40131a70e16969686ac4aab58a 
  clients/src/main/java/org/apache/kafka/common/Cluster.java 
d3299b944062d96852452de455902659ad8af757 
  clients/src/main/java/org/apache/kafka/common/PartitionInfo.java 
b15aa2c3ef2d7c4b24618ff42fd4da324237a813 
  clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
98cb79b701918eca3f6ca9823b6c7b7c97b3ecec 
  clients/src/main/java/org/apache/kafka/common/errors/ApiException.java 
7c948b166a8ac07616809f260754116ae7764973 
  clients/src/main/java/org/apache/kafka/common/network/Selectable.java 
b68bbf00ab8eba6c5867d346c91188142593ca6e 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
74d695ba39de44b6a3d15340ec0114bc4fce2ba2 
  clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
3316b6a1098311b8603a4a5893bf57b75d2e43cb 
  clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java 
121e880a941fcd3e6392859edba11a94236494cc 
  clients/src/main/java/org/apache/kafka/common/record/LogEntry.java 
e4d688cbe0c61b74ea15fc8dd3b634f9e5ee9b83 
  clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
040e5b91005edb8f015afdfa76fd94e0bf3cb4ca 
  clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java 
2fc471f64f4352eeb128bbd3941779780076fb8c 
  clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java 
99364c1ca464f7b81be7d3da15b40ab66717a659 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java 
3ee5cbad55ce836fd04bb954dcf6ef2f9bc3288f 
  
clients/src/main/java/org/

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-13 Thread Onur Karaman

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



clients/src/main/java/org/apache/kafka/clients/ClientRequest.java


It looks like you'd want to replace the attachment docs with new callback 
docs.


- Onur Karaman


On Jan. 12, 2015, 12:57 a.m., Jay Kreps wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/27799/
> ---
> 
> (Updated Jan. 12, 2015, 12:57 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1760
> https://issues.apache.org/jira/browse/KAFKA-1760
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> New consumer.
> 
> 
> Diffs
> -
> 
>   build.gradle ba52288031e2abc70e35e9297a4423dd5025950b 
>   clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
> d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
>   clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
> 8aece7e81a804b177a6f2c12e2dc6c89c1613262 
>   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
> ab7e3220f9b76b92ef981d695299656f041ad5ed 
>   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
> 397695568d3fd8e835d8f923a89b3b00c96d0ead 
>   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
> 6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
>   
> clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
> PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
> 1bce50185273dbdbc131fbc9c7f5f3e9c346517a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
> 57c1807ccba9f264186f83e91f37c34b959c8060 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
>  e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
> 16af70a5de52cca786fdea147a6a639b7dc4a311 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
> bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
>   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
> a5fedce9ff05ccfdb58ef083118d23bfa7a9bd4a 
>   clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
> 8cab16c0a0bdb671fea1fc2fc2694247f66cc971 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
> ea423ad15eebd262d20d5ec05d592cc115229177 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
>  PRE-CREATION 
>   
> clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
>  PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
>  1d30f9edd95337f86e632a09fc8f4126a67c238b 
>   
> clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
> 84a7a07269c51ccc22ebb4ff9797292d07ba778e 
>   clients/src/main/java/org/apache/kafka/common/Cluster.java 
> d3299b944062d96852452de455902659ad8af757 
>   clients/src/main/java/org/apache/kafka/common/PartitionInfo.java 
> b15aa2c3ef2d7c4b24618ff42fd4da324237a813 
>   clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
> 98cb79b701918eca3f6ca9823b6c7b7c97b3ecec 
>   clients/src/main/java/org/apache/kafka/common/errors/ApiException.java 
> 7c948b166a8ac07616809f260754116ae7764973 
>   clients/src/main/java/org/apache/kafka/common/network/Selectable.java 
> b68bbf00ab8eba6c5867d346c91188142593ca6e 
>   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
> 4dd2cdf773f7eb01a93d7f994383088960303dfc 
>   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
> 3316b6a1098311b8603a4a5893bf57b75d2e43cb 
>   clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java 
> 121e880a941fcd3e6392859edba11a94236494cc 
>   clients/src/main/java/org/apache/kafka/common/record/LogEntry.java 
> e4d688cbe0c61b74ea15fc8dd3b634f9e5ee9b83 
>   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
> 040e5b91005edb8f015afdfa76fd94e0bf3cb4ca 
>   clients/src/main/java/org/apache/kafka/common/requests/F

Re: Review Request 27799: Patch for KAFKA-1760

2015-01-11 Thread Jay Kreps

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

(Updated Jan. 12, 2015, 12:57 a.m.)


Review request for kafka.


Summary (updated)
-

Patch for KAFKA-1760


Bugs: KAFKA-1760
https://issues.apache.org/jira/browse/KAFKA-1760


Repository: kafka


Description (updated)
---

New consumer.


Diffs (updated)
-

  build.gradle ba52288031e2abc70e35e9297a4423dd5025950b 
  clients/src/main/java/org/apache/kafka/clients/ClientRequest.java 
d32c319d8ee4c46dad309ea54b136ea9798e2fd7 
  clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java 
8aece7e81a804b177a6f2c12e2dc6c89c1613262 
  clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/ConnectionState.java 
ab7e3220f9b76b92ef981d695299656f041ad5ed 
  clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
397695568d3fd8e835d8f923a89b3b00c96d0ead 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
6746275d0b2596cd6ff7ce464a3a8225ad75ef00 
  clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java 
PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java 
1bce50185273dbdbc131fbc9c7f5f3e9c346517a 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
57c1807ccba9f264186f83e91f37c34b959c8060 
  
clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
 e4cf7d1cfa01c2844b53213a7b539cdcbcbeaa3a 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java 
16af70a5de52cca786fdea147a6a639b7dc4a311 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java 
bdf4b26942d5a8c8a9503e05908e9a9eff6228a7 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
a5fedce9ff05ccfdb58ef083118d23bfa7a9bd4a 
  clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java 
8cab16c0a0bdb671fea1fc2fc2694247f66cc971 
  
clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
 PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java 
ea423ad15eebd262d20d5ec05d592cc115229177 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 PRE-CREATION 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
  
clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java 
1d30f9edd95337f86e632a09fc8f4126a67c238b 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
84a7a07269c51ccc22ebb4ff9797292d07ba778e 
  clients/src/main/java/org/apache/kafka/common/Cluster.java 
d3299b944062d96852452de455902659ad8af757 
  clients/src/main/java/org/apache/kafka/common/PartitionInfo.java 
b15aa2c3ef2d7c4b24618ff42fd4da324237a813 
  clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java 
98cb79b701918eca3f6ca9823b6c7b7c97b3ecec 
  clients/src/main/java/org/apache/kafka/common/errors/ApiException.java 
7c948b166a8ac07616809f260754116ae7764973 
  clients/src/main/java/org/apache/kafka/common/network/Selectable.java 
b68bbf00ab8eba6c5867d346c91188142593ca6e 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
4dd2cdf773f7eb01a93d7f994383088960303dfc 
  clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
3316b6a1098311b8603a4a5893bf57b75d2e43cb 
  clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java 
121e880a941fcd3e6392859edba11a94236494cc 
  clients/src/main/java/org/apache/kafka/common/record/LogEntry.java 
e4d688cbe0c61b74ea15fc8dd3b634f9e5ee9b83 
  clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
040e5b91005edb8f015afdfa76fd94e0bf3cb4ca 
  clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java 
2fc471f64f4352eeb128bbd3941779780076fb8c 
  clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java 
99364c1ca464f7b81be7d3da15b40ab66717a659 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java 
3ee5cbad55ce836fd04bb954dcf6ef2f9bc3288f 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java 
6b7c269ad7679df57c6bd505516075add39b7534 
  clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java 
3c001d33091c0f04ac3bf49a6731ab9e9f2bb0c4 
  clients/src/main/java/org/apache/kafka/common/utils/Utils.java 
527dd0f9c47