[ https://issues.apache.org/jira/browse/KAFKA-5235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16009814#comment-16009814 ]
ASF GitHub Bot commented on KAFKA-5235: --------------------------------------- GitHub user tashoyan opened a pull request: https://github.com/apache/kafka/pull/3051 KAFKA-5235: GetOffsetShell: new KafkaConsumer API, support for multiple topics, minimized number of requests to server This PR addresses two improvements: [KAFKA-5235 GetOffsetShell: retrieve offsets for all given topics and partitions with single request to the broker](https://issues.apache.org/jira/browse/KAFKA-5235) [KAFKA-5234 GetOffsetShell: retrieve offsets for multiple topics with single request](https://issues.apache.org/jira/browse/KAFKA-5234) 1. Previous implementation used SimpleConsumer to get offsets and old Producer API to get topic/partition metadata. Previous implementation determined a leader broker for each partition and then requested the leader for offsets. In total, it did as many requests to the broker as the number of partitions (plus a request to Zookeeper for metadata). New implementation `kafka-get-offsets.sh` uses KafkaConsumer API. It makes at most two requests to the broker: 1) to query existing topics and partitions, 2) to grab all requested offsets. New implementation correctly handles non-existing topics and partitions asked by user: > kafka-get-offsets.sh --bootstrap-servers vm:9092 --topics AAA,ZZZ --partitions 0,1 > AAA:0:7 > AAA:1:Partition not found > ZZZ:0:Topic not found 2. Previously, user could get offsets for one topic only. Now user can get offsets for many topics at once: `kafka-get-offsets.sh --bootstrap-servers vm:9092 --topics AAA,ZZZ` Moreover, now user is able to retrieve offsets for _all_ topics - this is the default when no topics specified: `kafka-get-offsets.sh --bootstrap-servers vm:9092` Thanks to this feature, there is no need anymore to retrieve all topics by means of `kafka-topics.sh`. When no topics specified, the new `kafka-get-offsets.sh` tool takes into account only user-level topics and ignores Kafka-internal topics (i.e. consumer offsets). This behavior can be altered via a special command line argument: `kafka-get-offsets.sh --bootstrap-servers vm:9092 --include-internal-topics` 3. New `kafka-get-offsets.sh` tool is consistent with other console tools with respect to command line argument names. In addition, `kafka-get-offsets.sh` tool gives the possibility to pass an arbitrary setting to KafkaConsumer via `--consumer-property` argument. I hope, now `kafka-get-offsets.sh` is easier in use and gives performance improvement. @junrao I suppose you may want to review. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tashoyan/kafka trunk Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/3051.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3051 ---- commit ec16d064aac4dfba164aeefaae3950db7b2e35af Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-01T19:52:26Z Add a testable method that retrieves offsets: getOffsets(). Cover it with tests. commit df60a30da3a5c3d0d95c85df2c5eb32a6eeae107 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-01T19:55:13Z Fix some trivial warnings commit 5aa9639666d32f7e036cee4cb42ac9b7223def2e Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-01T21:07:35Z Switch the implementation to getOffsets(). commit 3d772b8fac0c45cbe7631064a57361b7928b9bc2 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-01T22:01:19Z Add a test for a replicated partition commit 15e8b1a83471919c40d56f32ac858a38b7ad7b31 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-10T21:42:33Z Add the implementation based on new KafkaConsumer. Switch tests to this new implementation. Now it works for replicated topics. commit 1cdfce266c217b080410235c671f2764068dc96c Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-11T20:33:34Z Implement support for non-existing topics and partitions commit 7a4bdc4deed9987f348ad76c027bf891d7ef3257 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-11T21:08:26Z Refactor: rename the method. Add documentation. commit d473adb73434b7d4347cf62fbf29e73615ea8a82 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T07:39:13Z Refine the doc commit a21bfab73bac67d2441472f8f73d7a9c956dcc5c Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T09:11:19Z Add more tests for non-existing partitions. Refactor tests. commit 0b8e6f8e02802a5e3fb40156efa656bb91f2211d Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T09:26:34Z Add a test for a non-existing topic. commit 6a65574ac28f86712894e25d6edff1051bbc50ac Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T10:42:08Z Add the ability to retrieve offsets for all partition of all topics. Add the ability to exclude internal topics. commit f23c85305868b2f4f0a00c318cb3a2d0786b467b Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T14:54:55Z Switch the tool to new implementation commit 5eb477fcd6de0eccf51a89f35b9898f60eb21106 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T18:01:38Z Update the test according to changes in the contract commit c1b4b979101b01d29c473a839b823f7d7db7fd5c Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T18:01:54Z Remove old implementation commit 81cc383c3d0640b07926a5ead0a292cdca622029 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T22:42:49Z Update the implementation to support independently empty topic set and empty partition set commit afaed0f91e01a367dc8cb9aebeb79511884313eb Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T22:53:21Z Remove redundant information from error messages, topic and partition are available in the key. Remove unused constant. commit 79eb356f781dcbbdc238d74a1fc69f725382954e Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T23:02:25Z Update test expectations after error messages were changed commit 36f00072d8fed7bc0d629c92d13022bdcb709cf9 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-12T23:10:04Z Add some TODO commit e2b467c882a89bd31747e80557761c3dbd774584 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T13:55:12Z Add an utility to install on my machine - remove it later commit d76672a2611ae7167bc91036fd48121abbc79975 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T13:56:23Z These TODOs are done commit 05a9e04a1f0c440e7cc640773a357ffb4e1f507d Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T14:14:56Z Add an utility to install on my machine - remove it later commit 4c92895f7b083c4f76f48b11a83ccdccbf62d835 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T14:15:53Z Formatting commit 5393d043c1c7e69682b48d4ab17ade72d489dd5f Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T18:14:22Z Make command line args consistent and aligned with other Kafka console tools. Deprecate old args and display warnings. commit 8e47485d7a67ceda1f8949df882d719feb9de099 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T18:23:39Z Add shell wrapper for GetOffsetShell commit e0cf259ee9c6239d0eaaaf6647f82413f144d8ee Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T18:36:04Z Add debug output commit 54086e19a9fcff38fbb16250a65b5687b2fc700d Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T19:14:38Z Pass extra consumer properties specified in command line to the consumer commit 10b0a84c05ccf43ac5d1eae213c10dfbaaa134a5 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T19:27:40Z Refactor: rename tests commit e7c9d197691a3754f6c96476c05937b2860d33d4 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T21:26:36Z Cosmetic commit c2dbfa78900c1c401342dafb95afbcd88dc9ae07 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T22:06:15Z Implement getting offsets for a timestamp commit 1cb3d12d286fe03e8ae8baf980dc385d19bbbf17 Author: Arseniy Tashoyan <tasho...@gmail.com> Date: 2017-05-13T22:35:11Z Fix bug: report non-existing partition when should report non-existing topic. Update tests to distinguish the two error handling paths. ---- > GetOffsetShell: retrieve offsets for all given topics and partitions with > single request to the broker > ------------------------------------------------------------------------------------------------------ > > Key: KAFKA-5235 > URL: https://issues.apache.org/jira/browse/KAFKA-5235 > Project: Kafka > Issue Type: Improvement > Components: tools > Reporter: Arseniy Tashoyan > Labels: tool > Fix For: 0.11.0.0 > > Original Estimate: 168h > Remaining Estimate: 168h > > GetOffsetShell is implemented on old SimpleConsumer. It needs Zookeeper to > retrieve metadata about topics and partitions. At present, GetOffsetShell > does the following: > - get metadata from Zookeeper > - iterate over partitions > - for each partition, connect to its leader broker and request offsets > Instead, GetOffsetShell can use new KafkaConsumer and retrieve offsets by > means of endOffsets(), beginningOffsets() and offsetsForTimes() methods. One > request is sufficient for all topics and partitions. > As far as GetOffsetShell is re-implemented with new KafkaConsumer API, it > will not depend on obsolete API: SimpleConsumer, old producer API. -- This message was sent by Atlassian JIRA (v6.3.15#6346)