Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-06 Thread Guozhang Wang


 On March 5, 2015, 11:39 p.m., Onur Karaman wrote:
  clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java,
   line 83
  https://reviews.apache.org/r/31650/diff/2/?file=886350#file886350line83
 
  I think these scenarios should be split up into separate tests.
 
 Guozhang Wang wrote:
 The general law of defining unit test cases is by functionality instead 
 of scenarios, hence I think it is OK to group them in one test.
 
 Onur Karaman wrote:
 Grouping them means that if one of the earlier scenarios fail, then the 
 later scenarios will not be tested. So we don't know if only that one 
 scenario failed or if multiple later scenarios would have failed.

Hmm.. I think it depends on whether we want to trouble shoot failed tests 
one-at-a-time or just check if one of the tests fail in that test case, fix 
that one and retry. For this case I agree that it may be better to get 
knowledge of all failed cases. Will split them and check-in.


- Guozhang


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


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  edbed5880dc44fc178737a5e298c106a00f38443 
   clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
 a00dcdf15d1c7bac7228be140647bd7d849deb9b 
   clients/src/test/java/org/apache/kafka/clients/MockClient.java 
 8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
   
 

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-06 Thread Guozhang Wang


 On March 5, 2015, 10:42 p.m., Onur Karaman wrote:
  clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java,
   lines 137-138
  https://reviews.apache.org/r/31650/diff/1/?file=882439#file882439line137
 
  This is really minor, but are longs necessary for these time parameters?
  
  Integer.MAX_VALUE translates to a little over 24 days.

These two configs are defined in the common client configs that are used by 
producers also. I think it would be ok to be more conversative on these config 
values.


 On March 5, 2015, 10:42 p.m., Onur Karaman wrote:
  clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java,
   line 183
  https://reviews.apache.org/r/31650/diff/1/?file=882439#file882439line183
 
  This is marking the receivedResponse as the time the request was sent 
  rather than the time we received the response.

Actually we do not need a last heart beat response as consumer client does not 
check for time out expiration at all.


 On March 5, 2015, 10:42 p.m., Onur Karaman wrote:
  clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java,
   lines 218-234
  https://reviews.apache.org/r/31650/diff/1/?file=882439#file882439line218
 
  I think this is simpler as:
  ```java
  boolean done = false;
  while (!done) {
  }
  ```

Actually this is not simpler with this pattern as you need to initialize inside 
the loop as true determine whether to override it to false, not vice versa, 
right?


 On March 5, 2015, 10:42 p.m., Onur Karaman wrote:
  clients/src/main/java/org/apache/kafka/common/protocol/Errors.java, lines 
  71-72
  https://reviews.apache.org/r/31650/diff/1/?file=882446#file882446line71
 
  Using the term consumer implies that generation ids are associated 
  with a consumer, while they're really associated with a group.
  
  Maybe just call this ILLEGAL_GENERATION as stated in the wiki?
  
  
  https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design#Kafka0.9ConsumerRewriteDesign-Groupmanagementprotocol

Good point.


- Guozhang


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


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-06 Thread Guozhang Wang


 On March 5, 2015, 11:39 p.m., Onur Karaman wrote:
  clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java,
   line 83
  https://reviews.apache.org/r/31650/diff/2/?file=886350#file886350line83
 
  I think these scenarios should be split up into separate tests.

The general law of defining unit test cases is by functionality instead of 
scenarios, hence I think it is OK to group them in one test.


- Guozhang


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


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  edbed5880dc44fc178737a5e298c106a00f38443 
   clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
 a00dcdf15d1c7bac7228be140647bd7d849deb9b 
   clients/src/test/java/org/apache/kafka/clients/MockClient.java 
 8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
  090087a319e2697d3a0653ca947d2cfa6d53f6c2 
   
 clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
  c1bc40648479d4c2ae4ac52f40dadc070a6bcf6f 
   
 

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-06 Thread Onur Karaman


 On March 5, 2015, 11:39 p.m., Onur Karaman wrote:
  clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java,
   line 83
  https://reviews.apache.org/r/31650/diff/2/?file=886350#file886350line83
 
  I think these scenarios should be split up into separate tests.
 
 Guozhang Wang wrote:
 The general law of defining unit test cases is by functionality instead 
 of scenarios, hence I think it is OK to group them in one test.

Grouping them means that if one of the earlier scenarios fail, then the later 
scenarios will not be tested. So we don't know if only that one scenario failed 
or if multiple later scenarios would have failed.


- Onur


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


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  edbed5880dc44fc178737a5e298c106a00f38443 
   clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
 a00dcdf15d1c7bac7228be140647bd7d849deb9b 
   clients/src/test/java/org/apache/kafka/clients/MockClient.java 
 8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
  PRE-CREATION 
   
 

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Guozhang Wang

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

(Updated March 5, 2015, 10:55 p.m.)


Review request for kafka.


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


Repository: kafka


Description
---

See comments in KAFKA-1910


Diffs (updated)
-

  clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
06fcfe62cc1fe76f58540221698ef076fe150e96 
  clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
8a3e55aaff7d8c26e56a8407166a4176c1da2644 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
5fb21001abd77cac839bd724afa04e377a3e82aa 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
67ceb754a52c07143c69b053fe128b9e24060b99 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
 ee0751e4949120d114202c2299d49612a89b9d97 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
7397e565fd865214529ffccadd4222d835ac8110 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
122375c473bf73caf05299b9f5174c6b226ca863 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
ed9c63a6679e3aaf83d19fde19268553a4c107c2 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
6baad9366a1975dbaba1786da91efeaa38533319 
  clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
  clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
083e7a39249ab56a73a014b106876244d619f189 
  clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
  clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
0057496228feeeccbc0c009a42f5268fa2cb8611 
  clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
8c50e9be534c61ecf56106bf2b68cf678ea50d66 
  clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
  
clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java 
cfac47a4a05dc8a535595542d93e55237b7d1e93 
  clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
90f31413d7d80a06c0af359009cc271aa0c67be3 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
 4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java 
edbed5880dc44fc178737a5e298c106a00f38443 
  clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
a00dcdf15d1c7bac7228be140647bd7d849deb9b 
  clients/src/test/java/org/apache/kafka/clients/MockClient.java 
8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
 090087a319e2697d3a0653ca947d2cfa6d53f6c2 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
 c1bc40648479d4c2ae4ac52f40dadc070a6bcf6f 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
 ea56c997cb90d5bac8e3706dfc1eaae5b1555ccb 
  clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java 
e343327faf115a901657ec6da8e0c5b8bbf0b76a 
  core/src/main/scala/kafka/common/ErrorMapping.scala 
eedc2f5f21dd8755fba891998456351622e17047 
  core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala 
PRE-CREATION 
  core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 
4cabffeacea09a49913505db19a96a55d58c0909 
  core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala 
21790a5059ee00d6610be6f0389445327b88db1d 
  core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala 
b65c04d0a5d53bf92299d5f67f112be3da3bf77d 
  core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala 
b1248e95d8a648b461f604c154879cc95dc7b1cb 
  core/src/main/scala/kafka/coordinator/GroupRegistry.scala 
7d17e102235134b6312271c4061abd27d7177f7e 
  

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Onur Karaman

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



clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
https://reviews.apache.org/r/31650/#comment122464

I think these scenarios should be split up into separate tests.



clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
https://reviews.apache.org/r/31650/#comment122463

ditto



clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
https://reviews.apache.org/r/31650/#comment122465

ditto



clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
https://reviews.apache.org/r/31650/#comment122469

ditto



clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
https://reviews.apache.org/r/31650/#comment122470

ditto



clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
https://reviews.apache.org/r/31650/#comment122461

This class is missing a test where shouldHeartbeat should return false.



clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
https://reviews.apache.org/r/31650/#comment122443

Rather than add nondeterminism to the unit test with 
System.currentTimeMillis() and make it slower with the sleep call, we can just 
pass in longs.


- Onur Karaman


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Guozhang Wang

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

(Updated March 5, 2015, 10:57 p.m.)


Review request for kafka.


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


Repository: kafka


Description (updated)
---

See comments in KAFKA-1910;

Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and a 
couple changes on MemoryRecords and test utils.


Diffs
-

  clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
06fcfe62cc1fe76f58540221698ef076fe150e96 
  clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
8a3e55aaff7d8c26e56a8407166a4176c1da2644 
  clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
  clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
5fb21001abd77cac839bd724afa04e377a3e82aa 
  clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
67ceb754a52c07143c69b053fe128b9e24060b99 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
 PRE-CREATION 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
 ee0751e4949120d114202c2299d49612a89b9d97 
  
clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
 d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
  clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
7397e565fd865214529ffccadd4222d835ac8110 
  clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
122375c473bf73caf05299b9f5174c6b226ca863 
  clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
ed9c63a6679e3aaf83d19fde19268553a4c107c2 
  clients/src/main/java/org/apache/kafka/common/network/Selector.java 
6baad9366a1975dbaba1786da91efeaa38533319 
  clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
  clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
083e7a39249ab56a73a014b106876244d619f189 
  clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
  clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
0057496228feeeccbc0c009a42f5268fa2cb8611 
  clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
8c50e9be534c61ecf56106bf2b68cf678ea50d66 
  clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
  
clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java 
cfac47a4a05dc8a535595542d93e55237b7d1e93 
  clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
90f31413d7d80a06c0af359009cc271aa0c67be3 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
 4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
  
clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java 
edbed5880dc44fc178737a5e298c106a00f38443 
  clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
a00dcdf15d1c7bac7228be140647bd7d849deb9b 
  clients/src/test/java/org/apache/kafka/clients/MockClient.java 
8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
 PRE-CREATION 
  
clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
 090087a319e2697d3a0653ca947d2cfa6d53f6c2 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
 c1bc40648479d4c2ae4ac52f40dadc070a6bcf6f 
  
clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
 ea56c997cb90d5bac8e3706dfc1eaae5b1555ccb 
  clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java 
e343327faf115a901657ec6da8e0c5b8bbf0b76a 
  core/src/main/scala/kafka/common/ErrorMapping.scala 
eedc2f5f21dd8755fba891998456351622e17047 
  core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala 
PRE-CREATION 
  core/src/main/scala/kafka/common/OffsetMetadataAndError.scala 
4cabffeacea09a49913505db19a96a55d58c0909 
  core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala 
21790a5059ee00d6610be6f0389445327b88db1d 
  core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala 
b65c04d0a5d53bf92299d5f67f112be3da3bf77d 
  core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala 
b1248e95d8a648b461f604c154879cc95dc7b1cb 
  

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Jay Kreps

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



clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
https://reviews.apache.org/r/31650/#comment122448

Not sure if I get this--isn't the serializer specific to the producer and 
deserializer specific to the consumer?



clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
https://reviews.apache.org/r/31650/#comment122455

Do we need to inline all this in the main poll loop? Is it possible to 
break this stuff out into a seperate method just to be able to follow the 
logic. I think this block is basically something like
  reassignPartitions()
?



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122450

Can you move the inner classes to the bottom as we do everywhere else?



clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
https://reviews.apache.org/r/31650/#comment122451

Manager is always such a generic name, what do you think about calling this 
the Fetcher to match the Sender that the Producer has?


I think this refactoring looks reasonable. I'll take a deeper look but I think 
since the goal is to move quick on this stuff, you should consider just 
checking in now and we can do any additional items as follow-ups.

- Jay Kreps


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Onur Karaman

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



clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
https://reviews.apache.org/r/31650/#comment122365

Do key/value serializer/deserializer configs actually belong here? 
CommonClientConfigs is where shared configs for producers and consumers live.

A producer is only interested in key/value serializer config and consumer 
is only interested in key/value deserializer config.

If you are trying to move the addSerializerToConfig and 
addDeserializerToConfig out of the KafkaProducer and KafkaConsumer, maybe the 
ProducerConfig and ConsumerConfig would be a suitable place?



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122377

This class name was very misleading to me. It should be renamed such that 
it's clear that it's used by clients to interact with coordinators, and is not 
the actual coordinator.

Maybe CoordinatorClient, CoordinatorManager, or something else that makes 
this clear.



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122397

This is really minor, but are longs necessary for these time parameters?

Integer.MAX_VALUE translates to a little over 24 days.



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122396

This is marking the receivedResponse as the time the request was sent 
rather than the time we received the response.



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122421

I think this is simpler as:
```java
boolean done = false;
while (!done) {
}
```



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122384

unless you plan on adding a condition to the loop, this can just be a 
while(true) loop.



clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
https://reviews.apache.org/r/31650/#comment122422

`if (responses.isEmpty())`



clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
https://reviews.apache.org/r/31650/#comment122360

Using the term consumer implies that generation ids are associated with a 
consumer, while they're really associated with a group.

Maybe just call this ILLEGAL_GENERATION as stated in the wiki?


https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design#Kafka0.9ConsumerRewriteDesign-Groupmanagementprotocol



core/src/main/scala/kafka/coordinator/GroupRegistry.scala
https://reviews.apache.org/r/31650/#comment122338

var to val for both of these.



core/src/main/scala/kafka/coordinator/GroupRegistry.scala
https://reviews.apache.org/r/31650/#comment122339

the toString is not needed.



core/src/test/scala/integration/kafka/api/ConsumerTest.scala
https://reviews.apache.org/r/31650/#comment122359

It would be nice to see unit tests specifically covering FetchManager.


- Onur Karaman


On March 3, 2015, 12:46 a.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 3, 2015, 12:46 a.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
 

Re: Review Request 31650: Drag Coordinator and FetchManager out of KafkaConsumer, fix a bunch of consumer test issues

2015-03-05 Thread Jay Kreps

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

Ship it!


Ship It!

- Jay Kreps


On March 5, 2015, 10:57 p.m., Guozhang Wang wrote:
 
 ---
 This is an automatically generated e-mail. To reply, visit:
 https://reviews.apache.org/r/31650/
 ---
 
 (Updated March 5, 2015, 10:57 p.m.)
 
 
 Review request for kafka.
 
 
 Bugs: KAFKA-1910
 https://issues.apache.org/jira/browse/KAFKA-1910
 
 
 Repository: kafka
 
 
 Description
 ---
 
 See comments in KAFKA-1910;
 
 Updated RB includes unit test for Coordinator / FetchManager / Heartbeat and 
 a couple changes on MemoryRecords and test utils.
 
 
 Diffs
 -
 
   clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java 
 06fcfe62cc1fe76f58540221698ef076fe150e96 
   clients/src/main/java/org/apache/kafka/clients/KafkaClient.java 
 8a3e55aaff7d8c26e56a8407166a4176c1da2644 
   clients/src/main/java/org/apache/kafka/clients/NetworkClient.java 
 a7fa4a9dfbcfbc4d9e9259630253cbcded158064 
   clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 
 5fb21001abd77cac839bd724afa04e377a3e82aa 
   clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
 67ceb754a52c07143c69b053fe128b9e24060b99 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/FetchManager.java
  PRE-CREATION 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
  ee0751e4949120d114202c2299d49612a89b9d97 
   
 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
  d41d3068c11d4b5c640467dc0ae1b7c20a8d128c 
   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
 7397e565fd865214529ffccadd4222d835ac8110 
   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
 122375c473bf73caf05299b9f5174c6b226ca863 
   
 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java 
 ed9c63a6679e3aaf83d19fde19268553a4c107c2 
   clients/src/main/java/org/apache/kafka/common/network/Selector.java 
 6baad9366a1975dbaba1786da91efeaa38533319 
   clients/src/main/java/org/apache/kafka/common/protocol/Errors.java 
 ad2171f5417c93194f5f234bdc7fdd0b8d59a8a8 
   clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java 
 083e7a39249ab56a73a014b106876244d619f189 
   clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java 
 e67c4c8332cb1dd3d9cde5de687df7760045dfe6 
   
 clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java 
 0057496228feeeccbc0c009a42f5268fa2cb8611 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java 
 8c50e9be534c61ecf56106bf2b68cf678ea50d66 
   
 clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java 
 52b1803d8b558c1eeb978ba8821496c7d3c20a6b 
   
 clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
  cfac47a4a05dc8a535595542d93e55237b7d1e93 
   
 clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java 
 90f31413d7d80a06c0af359009cc271aa0c67be3 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
  4d3b9ececee4b4c0b50ba99da2ddbbb15f9cc08d 
   
 clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
  edbed5880dc44fc178737a5e298c106a00f38443 
   clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java 
 a00dcdf15d1c7bac7228be140647bd7d849deb9b 
   clients/src/test/java/org/apache/kafka/clients/MockClient.java 
 8f1a7a625e4eeafa44bbf9e5cff987de86c949be 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchManagerTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatTest.java
  PRE-CREATION 
   
 clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
  090087a319e2697d3a0653ca947d2cfa6d53f6c2 
   
 clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
  c1bc40648479d4c2ae4ac52f40dadc070a6bcf6f 
   
 clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
  ea56c997cb90d5bac8e3706dfc1eaae5b1555ccb 
   clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java 
 e343327faf115a901657ec6da8e0c5b8bbf0b76a 
   core/src/main/scala/kafka/common/ErrorMapping.scala 
 eedc2f5f21dd8755fba891998456351622e17047 
   core/src/main/scala/kafka/common/NoOffsetsCommittedException.scala 
 PRE-CREATION