You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jun Rao <ju...@gmail.com> on 2014/04/15 19:20:12 UTC

Review Request 20372: Patch for KAFKA-1390

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

Review request for kafka.


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


Repository: kafka


Description
-------

1. Fix TestUtils.waitUntilLeaderIsElectedOrChanged to address the following existing issues: 1.1 If we miss a watch, we will wait for the full timeout 1.2 If there are multiple leader changes (in the case of unclean leader election when we first move the leader to -1 and then to a new broker), it's not clear which event the caller wants to wait. 1.3 If the timeout is reached, we should fail instead of silently return.

2. Disable the following tests since there are broken. The reason for the failure are included in the comments in the tests. Will file separate following jiras.
ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled
DeleteTopicTest (all tests)
AutoOffsetResetTest (all tests)
ProducerFailureHandlingTest.testBrokerFailure

3. All unit tests pass after a few runs.


Diffs
-----

  core/src/main/scala/kafka/api/FetchRequest.scala dea118a2e37927d4268e1a2506e698bc17671d75 
  core/src/main/scala/kafka/api/RequestOrResponse.scala 708e547a358c95f1a68fc507a88544fbaa21a938 
  core/src/main/scala/kafka/server/KafkaApis.scala d96229e2d4aa7006b0dbd81055ce5a2459d8758c 
  core/src/main/scala/kafka/utils/ZkUtils.scala 16bf7e3e87b520b31485199b2830494ca5f96c1a 
  core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala ef5604450239d14e3f885b57d095ef9de76eefaa 
  core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 60e68c7dd90abe68bce7a96b1c3877b9e1cdb330 
  core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala 649a1f0d219a88b220b8b839f225efbbdadf3693 
  core/src/test/scala/unit/kafka/admin/AdminTest.scala 00b17c490b338bcb10f00d31f66d401845c3faa1 
  core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala e704290e59dbd42b77c75c70e49a43116119856f 
  core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala 9347ea60f2c02bd01b1c8507e30cb7dfdeaa5d72 
  core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 258dd25a201cd9a9afd96d03a0ad3687c48f6f75 
  core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala e5703bc16b2ce2306395f0c773a0e8a82ac63176 
  core/src/test/scala/unit/kafka/integration/FetcherTest.scala 47130d33c36fabf19ffd94d488fd4281e23b1bdd 
  core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala d44c3ff1d354874ed912dc8df960ff664f652786 
  core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala e86ee803ab9c707820fc2d867212d500da760c3a 
  core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 9998a1156d41d6afcaf5674bb9b45e16652c8167 
  core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala c5f2da993918242389fbb25bd903bf479e26f839 
  core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala 43af649f3297644e8caea176a79cadde745be2a4 
  core/src/test/scala/unit/kafka/producer/ProducerTest.scala 4b2e4ade70f6f3d1804f4627e1eb680f6721c93d 
  core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala 8d63e312d3edac0ee0400b8cf316828c2a5a064f 
  core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala 38e3ae72a87e13f6e1a739cc8c75b781fb004133 
  core/src/test/scala/unit/kafka/server/LogOffsetTest.scala 521d156088911b7b6533b33aed5470846148c54f 
  core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala 4bf0ef69f921d9fb14cccd8f5597258e57e29aad 
  core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala ae9bb3a69d57143b868ead12d736377a001b1536 
  core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala dd85c718167158ad01f00d9aa6f990bf4ec9a6af 
  core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala c7e058fc2e541019fcfb666d556050a13c7d4e66 
  core/src/test/scala/unit/kafka/utils/TestUtils.scala 53d01aa3e501392672ee6bc7f0e225493383c870 

Diff: https://reviews.apache.org/r/20372/diff/


Testing
-------


Thanks,

Jun Rao


Re: Review Request 20372: Patch for KAFKA-1390

Posted by Guozhang Wang <gu...@linkedin.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/20372/#review40411
-----------------------------------------------------------

Ship it!


Apply patch: passed
Unit test: passed

Minor comments below.


core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
<https://reviews.apache.org/r/20372/#comment73398>

    Shall we just comment out this test instead of deleting it?



core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
<https://reviews.apache.org/r/20372/#comment73400>

    Remove */ and the following /*


- Guozhang Wang


On April 15, 2014, 5:20 p.m., Jun Rao wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/20372/
> -----------------------------------------------------------
> 
> (Updated April 15, 2014, 5:20 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1390
>     https://issues.apache.org/jira/browse/KAFKA-1390
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> 1. Fix TestUtils.waitUntilLeaderIsElectedOrChanged to address the following existing issues: 1.1 If we miss a watch, we will wait for the full timeout 1.2 If there are multiple leader changes (in the case of unclean leader election when we first move the leader to -1 and then to a new broker), it's not clear which event the caller wants to wait. 1.3 If the timeout is reached, we should fail instead of silently return.
> 
> 2. Disable the following tests since there are broken. The reason for the failure are included in the comments in the tests. Will file separate following jiras.
> ServerShutdownTest.testCleanShutdownWithDeleteTopicEnabled
> DeleteTopicTest (all tests)
> AutoOffsetResetTest (all tests)
> ProducerFailureHandlingTest.testBrokerFailure
> 
> 3. All unit tests pass after a few runs.
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/api/FetchRequest.scala dea118a2e37927d4268e1a2506e698bc17671d75 
>   core/src/main/scala/kafka/api/RequestOrResponse.scala 708e547a358c95f1a68fc507a88544fbaa21a938 
>   core/src/main/scala/kafka/server/KafkaApis.scala d96229e2d4aa7006b0dbd81055ce5a2459d8758c 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 16bf7e3e87b520b31485199b2830494ca5f96c1a 
>   core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala ef5604450239d14e3f885b57d095ef9de76eefaa 
>   core/src/test/scala/integration/kafka/api/ProducerSendTest.scala 60e68c7dd90abe68bce7a96b1c3877b9e1cdb330 
>   core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala 649a1f0d219a88b220b8b839f225efbbdadf3693 
>   core/src/test/scala/unit/kafka/admin/AdminTest.scala 00b17c490b338bcb10f00d31f66d401845c3faa1 
>   core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala e704290e59dbd42b77c75c70e49a43116119856f 
>   core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala 9347ea60f2c02bd01b1c8507e30cb7dfdeaa5d72 
>   core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala 258dd25a201cd9a9afd96d03a0ad3687c48f6f75 
>   core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala e5703bc16b2ce2306395f0c773a0e8a82ac63176 
>   core/src/test/scala/unit/kafka/integration/FetcherTest.scala 47130d33c36fabf19ffd94d488fd4281e23b1bdd 
>   core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala d44c3ff1d354874ed912dc8df960ff664f652786 
>   core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala e86ee803ab9c707820fc2d867212d500da760c3a 
>   core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 9998a1156d41d6afcaf5674bb9b45e16652c8167 
>   core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala c5f2da993918242389fbb25bd903bf479e26f839 
>   core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala 43af649f3297644e8caea176a79cadde745be2a4 
>   core/src/test/scala/unit/kafka/producer/ProducerTest.scala 4b2e4ade70f6f3d1804f4627e1eb680f6721c93d 
>   core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala 8d63e312d3edac0ee0400b8cf316828c2a5a064f 
>   core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala 38e3ae72a87e13f6e1a739cc8c75b781fb004133 
>   core/src/test/scala/unit/kafka/server/LogOffsetTest.scala 521d156088911b7b6533b33aed5470846148c54f 
>   core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala 4bf0ef69f921d9fb14cccd8f5597258e57e29aad 
>   core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala ae9bb3a69d57143b868ead12d736377a001b1536 
>   core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala dd85c718167158ad01f00d9aa6f990bf4ec9a6af 
>   core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala c7e058fc2e541019fcfb666d556050a13c7d4e66 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 53d01aa3e501392672ee6bc7f0e225493383c870 
> 
> Diff: https://reviews.apache.org/r/20372/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Jun Rao
> 
>