├── .circleci └── config.yml ├── .gitignore ├── LICENSE ├── README.md ├── build.sbt ├── project ├── build.properties └── plugins.sbt ├── scalastyle-config.xml ├── src ├── main │ └── scala │ │ ├── net │ │ └── heartsavior │ │ │ └── spark │ │ │ ├── KafkaOffsetCommitterListener.scala │ │ │ └── ReflectionHelper.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ └── kafka010 │ │ └── KafkaSourceInspector.scala └── test │ ├── resources │ └── log4j.properties │ └── scala │ └── net │ └── heartsavior │ └── spark │ └── KafkaOffsetCommitterListenerSuite.scala └── version.sbt /.circleci/config.yml: -------------------------------------------------------------------------------- 1 | version: 2.1 2 | jobs: 3 | build: 4 | working_directory: ~/repo 5 | docker: 6 | - image: openjdk:8 7 | environment: 8 | SBT_VERSION: 1.3.10 9 | steps: 10 | - run: echo 'export ARTIFACT_BUILD=$CIRCLE_PROJECT_REPONAME-$CIRCLE_BUILD_NUM.zip' >> $BASH_ENV 11 | - run: 12 | name: Get sbt binary 13 | command: | 14 | apt update && apt install -y curl 15 | curl -L -o sbt-$SBT_VERSION.deb https://dl.bintray.com/sbt/debian/sbt-$SBT_VERSION.deb 16 | dpkg -i sbt-$SBT_VERSION.deb 17 | rm sbt-$SBT_VERSION.deb 18 | apt-get update 19 | apt-get install -y python-pip git 20 | pip install awscli 21 | apt-get clean && apt-get autoclean 22 | - checkout 23 | - restore_cache: 24 | # Read about caching dependencies: https://circleci.com/docs/2.0/caching/ 25 | key: sbt-cache 26 | - run: 27 | name: Compile spark-sql-kafka-offset-committer dist package 28 | command: cat /dev/null | sbt clean +update scalastyle +test +package 29 | - when: 30 | condition: true 31 | steps: 32 | - store_artifacts: 33 | path: ~/repo/target/scalastyle-output.xml 34 | destination: scalastyle-output.xml 35 | 36 | - store_artifacts: 37 | path: ~/repo/target/unit-tests.log 38 | destination: unit-tests.log 39 | - save_cache: 40 | key: sbt-cache 41 | paths: 42 | - "~/.ivy2/cache" 43 | - "~/.sbt" 44 | - "~/.m2" 45 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *#*# 2 | *.#* 3 | *.iml 4 | *.ipr 5 | *.iws 6 | *.pyc 7 | *.pyo 8 | *.swp 9 | *~ 10 | .DS_Store 11 | .cache 12 | .classpath 13 | .idea/ 14 | .idea_modules/ 15 | .project 16 | .pydevproject 17 | .scala_dependencies 18 | .settings 19 | /lib/ 20 | build/*.jar 21 | build/apache-maven* 22 | build/scala* 23 | build/zinc* 24 | cache 25 | dependency-reduced-pom.xml 26 | derby.log 27 | lib_managed/ 28 | log/ 29 | logs/ 30 | out/ 31 | project/boot/ 32 | project/build/target/ 33 | project/plugins/lib_managed/ 34 | project/plugins/project/build.properties 35 | project/plugins/src_managed/ 36 | project/plugins/target/ 37 | scalastyle-on-compile.generated.xml 38 | scalastyle-output.xml 39 | scalastyle.txt 40 | target/ 41 | unit-tests.log 42 | *.crc 43 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "[]" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright [yyyy] [name of copyright owner] 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | 203 | 204 | ------------------------------------------------------------------------------------ 205 | This product bundles various third-party components under other open source licenses. 206 | This section summarizes those components and their licenses. See licenses/ 207 | for text of these licenses. 208 | 209 | 210 | Apache Software Foundation License 2.0 211 | -------------------------------------- 212 | 213 | common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java 214 | core/src/main/java/org/apache/spark/util/collection/TimSort.java 215 | core/src/main/resources/org/apache/spark/ui/static/bootstrap* 216 | core/src/main/resources/org/apache/spark/ui/static/jsonFormatter* 217 | core/src/main/resources/org/apache/spark/ui/static/vis* 218 | docs/js/vendor/bootstrap.js 219 | 220 | 221 | Python Software Foundation License 222 | ---------------------------------- 223 | 224 | pyspark/heapq3.py 225 | 226 | 227 | BSD 3-Clause 228 | ------------ 229 | 230 | python/lib/py4j-*-src.zip 231 | python/pyspark/cloudpickle.py 232 | python/pyspark/join.py 233 | core/src/main/resources/org/apache/spark/ui/static/d3.min.js 234 | 235 | The CSS style for the navigation sidebar of the documentation was originally 236 | submitted by Óscar Nájera for the scikit-learn project. The scikit-learn project 237 | is distributed under the 3-Clause BSD license. 238 | 239 | 240 | MIT License 241 | ----------- 242 | 243 | core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js 244 | core/src/main/resources/org/apache/spark/ui/static/*dataTables* 245 | core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js 246 | ore/src/main/resources/org/apache/spark/ui/static/jquery* 247 | core/src/main/resources/org/apache/spark/ui/static/sorttable.js 248 | docs/js/vendor/anchor.min.js 249 | docs/js/vendor/jquery* 250 | docs/js/vendor/modernizer* 251 | 252 | 253 | Creative Commons CC0 1.0 Universal Public Domain Dedication 254 | ----------------------------------------------------------- 255 | (see LICENSE-CC0.txt) 256 | 257 | data/mllib/images/kittens/29.5.a_b_EGDP022204.jpg 258 | data/mllib/images/kittens/54893.jpg 259 | data/mllib/images/kittens/DP153539.jpg 260 | data/mllib/images/kittens/DP802813.jpg 261 | data/mllib/images/multi-channel/chr30.4.184.jpg -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Kafka offset committer for Spark structured streaming 2 | 3 | [![CircleCI](https://circleci.com/gh/HeartSaVioR/spark-sql-kafka-offset-committer/tree/master.svg?style=svg)](https://circleci.com/gh/HeartSaVioR/spark-sql-kafka-offset-committer/tree/master) 4 | 5 | Kafka offset committer helps structured streaming query which uses Kafka Data Source to commit offsets which batch has been processed. 6 | 7 | This project is not for replacing checkpoint mechanism of Spark with Kafka's one. To provide full of "fault-tolerance" semantic, Spark has to take 100% of control of manipulating checkpoint, and Kafka data source is no exception. This project can be used to leverage Kafka ecosystem tools to track the committed offsets on Spark checkpoint, which is not possible solely with Spark. 8 | 9 | This project is inspired by [SPARK-27549](https://issues.apache.org/jira/browse/SPARK-27549), which proposed to add this feature in Spark codebase, but the decision was taken as not include to Spark. You can call this project as a "follow-up" of SPARK-27549. This project is also inspired by [Spark Atlas Connector](https://github.com/hortonworks-spark/spark-atlas-connector) - SAC leverages Scala reflection to extract topic information from query execution. Kafka offset committer uses the same approach to extract Kafka parameters. Credits to everyone involved SPARK-27549 & SAC. 10 | 11 | ## Supported versions 12 | 13 | Both Spark 3.0.x and 2.4.x is supported: it only means you should use these versions when using this project. 14 | 15 | The project provides cross-compile for Scala 2.11 and 2.12 (thanks [@redsk](https://github.com/redsk)!) for Spark 2.4.x; please pick the right artifact for your Scala version. 16 | 17 | Spark version | Scala versions | artifact version 18 | ------------- | -------------- | ---------------- 19 | 2.4.x | 2.11 / 2.12 | 0.4.0-spark-2.4 20 | 3.0.x | 2.12 | 0.4.0-spark-3.0 21 | 22 | ## How to import 23 | 24 | Add this to your maven pom.xml file. If you're using other builds like groovy or sbt or so, please import the artifact accordingly; groupId: `net.heartsavior.spark`, artifactId: `spark-sql-kafka-offset-committer_`. 25 | 26 | Please replace `{{...}}` with content in above matrix: 27 | 28 | ``` 29 | 30 | net.heartsavior.spark 31 | spark-sql-kafka-offset-committer_{{scala_version}} 32 | {{artifact_version}} 33 | 34 | ``` 35 | 36 | You can dynamically include jar file while submitting, via leveraging `--packages` option. `--packages net.heartsavior.spark:spark-sql-kafka-offset-committer:0.1.0`. You may want to add `--conf spark.sql.streaming.streamingQueryListeners=net.heartsavior.spark.KafkaOffsetCommitterListener` as well, since you're dynamically adding the jar, hence the class is not accessible in your uber jar. 37 | 38 | ## How to use 39 | 40 | Kafka offset committer is implemented as StreamingQueryListener. There're two approaches to enable streaming query listener: 41 | 42 | 1. Attach the instance of `KafkaOffsetCommitterListener` via below: 43 | 44 | ```scala 45 | val listener = new KafkaOffsetCommitterListener() 46 | spark.streams.addListener(listener) 47 | ``` 48 | 49 | 2. Add `net.heartsavior.spark.KafkaOffsetCommitterListener` to the value of `spark.sql.streaming.streamingQueryListeners` in your Spark config. 50 | (The value is separated by `,` so you can add multiple listeners if you have any other listeners.) 51 | 52 | Once the listener is set, you can add special option to Kafka data source options so that Kafka committer can see the `groupId` to commit: 53 | 54 | ```scala 55 | spark.readStream 56 | .format("kafka") 57 | .option("kafka.bootstrap.servers", "localhost:9092") 58 | .option("subscribePattern", "topic[1-3]") 59 | .option("startingOffsets", "earliest") 60 | .option("kafka.consumer.commit.groupid", "groupId1") 61 | .load() 62 | ``` 63 | 64 | "kafka.consumer.commit.groupid" is the new config to specify consumer group ID to commit. Manually specifying consumer group ID is needed, because Spark will 65 | assign unique consumer group ID to avoid multiple queries being conflicted to each other. This also means, you may want to thoughtfully set the option and 66 | decide the name of group ID so that multiple queries don't use the same group ID for committing. 67 | 68 | Due to technical reason, the project uses reflection to extract options from query execution. Given we intercept Kafka parameters instead of source options 69 | of DataSource, adding "kafka." to option key is necessary and it brings unintended warning messages from Kafka side. (Sorry!) You can adjust your log4j config 70 | to hide the warning messages. 71 | 72 | Here's an example of command to run spark-shell with kafka committer listener being set, and simple query to read from Kafka topics and write to Kafka topic. 73 | 74 | > command 75 | 76 | ``` 77 | ./bin/spark-shell --master "local[3]" --packages org.apache.spark:spark-sql-kafka-0-10_2.11:2.4.3 --jars ./spark-sql-kafka-offset-committer-0.1.0-SNAPSHOT.jar --conf spark.sql.streaming.streamingQueryListeners=net.heartsavior.spark.KafkaOffsetCommitterListener 78 | ``` 79 | 80 | > query 81 | 82 | ```scala 83 | val bootstrapServers = "localhost:9092" 84 | val checkpointLocation = "/tmp/mykafkaaaaaaa" 85 | val sourceTopics = Seq("truck_events_stream").mkString(",") 86 | val sourceTopics2 = Seq("truck_speed_events_stream").mkString(",") 87 | 88 | val targetTopic = "sparksinkstreaming" 89 | 90 | val df = spark.readStream.format("kafka").option("kafka.bootstrap.servers", bootstrapServers).option("subscribe", sourceTopics).option("startingOffsets", "earliest").option("kafka.consumer.commit.groupid", "spark-sql-kafka-offset-committer-test-1").load() 91 | 92 | val df2 = spark.readStream.format("kafka").option("kafka.bootstrap.servers", bootstrapServers).option("subscribe", sourceTopics2).option("startingOffsets", "earliest").option("kafka.consumer.commit.groupid", "spark-sql-kafka-offset-committer-test-1").load() 93 | 94 | val query = df.union(df2).writeStream.format("kafka").option("kafka.bootstrap.servers", bootstrapServers).option("checkpointLocation", checkpointLocation).option("topic", targetTopic).option("kafka.atlas.cluster.name", "sink").start() 95 | ``` 96 | 97 | > result 98 | 99 | ``` 100 | $ kafka-consumer-groups --bootstrap-server localhost:9092 --describe --group spark-sql-kafka-offset-committer-test-1 101 | Consumer group 'spark-sql-kafka-offset-committer-test-1' has no active members. 102 | 103 | TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID 104 | truck_speed_events_stream 5 844553 844577 24 - - - 105 | truck_speed_events_stream 2 675521 675540 19 - - - 106 | truck_speed_events_stream 6 168828 168833 5 - - - 107 | truck_speed_events_stream 3 337819 337827 8 - - - 108 | truck_speed_events_stream 7 675566 675585 19 - - - 109 | truck_speed_events_stream 4 168914 168919 5 - - - 110 | truck_speed_events_stream 0 168894 168899 5 - - - 111 | truck_speed_events_stream 8 675570 675589 19 - - - 112 | truck_speed_events_stream 1 168917 168922 5 - - - 113 | truck_events_stream 0 3884586 3884695 109 - - - 114 | truck_speed_events_stream 9 0 0 0 - - - 115 | ``` 116 | 117 | After stopping ingestion of records and waiting for query to fully process the records: 118 | 119 | ``` 120 | $ kafka-consumer-groups --bootstrap-server localhost:9092 --describe --group spark-sql-kafka-offset-committer-test-1 121 | Consumer group 'spark-sql-kafka-offset-committer-test-1' has no active members. 122 | 123 | TOPIC PARTITION CURRENT-OFFSET LOG-END-OFFSET LAG CONSUMER-ID HOST CLIENT-ID 124 | truck_speed_events_stream 5 856338 856338 0 - - - 125 | truck_speed_events_stream 2 684958 684958 0 - - - 126 | truck_speed_events_stream 6 171186 171186 0 - - - 127 | truck_speed_events_stream 3 342534 342534 0 - - - 128 | truck_speed_events_stream 7 684998 684998 0 - - - 129 | truck_speed_events_stream 4 171272 171272 0 - - - 130 | truck_speed_events_stream 0 171255 171255 0 - - - 131 | truck_speed_events_stream 8 684999 684999 0 - - - 132 | truck_speed_events_stream 1 171276 171276 0 - - - 133 | truck_events_stream 0 3938820 3938820 0 - - - 134 | truck_speed_events_stream 9 0 0 0 - - - 135 | ``` 136 | 137 | 138 | ## License 139 | 140 | Copyright 2019-2021 Jungtaek Lim "" 141 | 142 | Licensed under the Apache License, Version 2.0 (the "License"); 143 | you may not use this file except in compliance with the License. 144 | You may obtain a copy of the License at 145 | 146 | http://www.apache.org/licenses/LICENSE-2.0 147 | 148 | Unless required by applicable law or agreed to in writing, software 149 | distributed under the License is distributed on an "AS IS" BASIS, 150 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 151 | See the License for the specific language governing permissions and 152 | limitations under the License. 153 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | lazy val scala212 = "2.12.10" 2 | lazy val supportedScalaVersions = List(scala212) 3 | 4 | scalaVersion := scala212 5 | crossScalaVersions := supportedScalaVersions 6 | 7 | name := "spark-sql-kafka-offset-committer" 8 | 9 | organization := "net.heartsavior.spark" 10 | 11 | description := "Kafka offset committer for Apache Spark which lets you commit the committed " + 12 | "offsets of Kafka data source in structured streaming queries." 13 | 14 | licenses += ("Apache 2.0 License", url("http://www.apache.org/licenses/LICENSE-2.0.html")) 15 | 16 | cancelable := true 17 | 18 | developers := List( 19 | Developer( 20 | "HeartSaVioR", 21 | "Jungtaek Lim", 22 | "kabhwan@gmail.com", 23 | url("https://github.com/HeartSaVioR/") 24 | ) 25 | ) 26 | 27 | javacOptions ++= Seq( 28 | "-Xlint:deprecation", 29 | "-Xlint:unchecked", 30 | "-source", "1.8", 31 | "-target", "1.8", 32 | "-g:vars" 33 | ) 34 | 35 | val sparkVersion = "3.0.0" 36 | 37 | libraryDependencies ++= Seq( 38 | "org.apache.spark" %% "spark-sql" % sparkVersion % "provided", 39 | "org.apache.spark" %% "spark-sql-kafka-0-10" % sparkVersion % "provided", 40 | "org.apache.spark" %% "spark-core" % sparkVersion % Test classifier "tests", 41 | "org.apache.spark" %% "spark-catalyst" % sparkVersion % Test classifier "tests", 42 | "org.apache.spark" %% "spark-sql" % sparkVersion % Test classifier "tests", 43 | "org.apache.spark" %% "spark-sql-kafka-0-10" % sparkVersion % Test classifier "tests", 44 | "org.apache.kafka" %% "kafka" % "2.4.1" % Test, 45 | "org.scalatest" %% "scalatest" % "3.0.3" % Test withSources(), 46 | "junit" % "junit" % "4.12" % Test, 47 | 48 | // update this when updating Spark version 49 | "com.fasterxml.jackson.module" %% "jackson-module-scala" % "2.10.0" % Test, 50 | ) 51 | 52 | 53 | Global / onChangedBuildSource := ReloadOnSourceChanges 54 | 55 | 56 | logBuffered in Test := false 57 | parallelExecution in Test := false 58 | 59 | logLevel := Level.Warn 60 | 61 | // Only show warnings and errors on the screen for compilations. 62 | // This applies to both test:compile and compile and is Info by default 63 | logLevel in compile := Level.Warn 64 | 65 | // Level.INFO is needed to see detailed output when running tests 66 | logLevel in test := Level.Info 67 | 68 | resolvers ++= Seq( 69 | ) 70 | 71 | scalacOptions ++= Seq( 72 | "-deprecation", // Emit warning and location for usages of deprecated APIs. 73 | "-encoding", "utf-8", // Specify character encoding used by source files. 74 | "-explaintypes", // Explain type errors in more detail. 75 | "-feature", // Emit warning and location for usages of features that should be 76 | // imported explicitly. 77 | "-language:existentials", // Existential types (besides wildcard types) can be written 78 | // and inferred 79 | "-language:experimental.macros", // Allow macro definition (besides implementation and 80 | // application) 81 | "-language:higherKinds", // Allow higher-kinded types 82 | "-language:implicitConversions", // Allow definition of implicit functions called views 83 | "-unchecked", // Enable additional warnings where generated code depends on assumptions. 84 | "-Xcheckinit", // Wrap field accessors to throw an exception on uninitialized access. 85 | "-Yno-adapted-args", // Do not adapt an argument list (either by inserting () or 86 | // creating a tuple) to match the receiver. 87 | "-Ypartial-unification", // Enable partial unification in type constructor inference 88 | "-Ywarn-inaccessible", // Warn about inaccessible types in method signatures. 89 | "-Ywarn-infer-any", // Warn when a type argument is inferred to be `Any`. 90 | "-Ywarn-nullary-override", // Warn when non-nullary `def f()' overrides nullary `def f'. 91 | "-Ywarn-nullary-unit", // Warn when nullary methods return Unit. 92 | ) 93 | 94 | scalacOptions ++= 95 | scalaVersion { 96 | case sv if sv.startsWith("2.12") => List( 97 | "-Ywarn-extra-implicit", // Warn when more than one implicit parameter section is defined. 98 | ) 99 | 100 | case _ => Nil 101 | }.value 102 | 103 | // The REPL can’t cope with -Ywarn-unused:imports or -Xfatal-warnings 104 | // so turn them off for the console 105 | scalacOptions in(Compile, console) --= Seq("-Ywarn-unused:imports", "-Xfatal-warnings") 106 | 107 | scalacOptions in(Compile, doc) ++= baseDirectory.map { 108 | bd: File => 109 | Seq[String]( 110 | "-sourcepath", bd.getAbsolutePath, 111 | "-doc-source-url", s"https://github.com/HeartSaVioR/spark-sql-kafka-offset-committer/tree/develop-spark3.0€{FILE_PATH}.scala" 112 | ) 113 | }.value 114 | 115 | homepage := Some(url("https://github.com/HeartSaVioR/spark-sql-kafka-offset-committer")) 116 | scmInfo := Some( 117 | ScmInfo( 118 | url(s"https://github.com/HeartSaVioR/spark-sql-kafka-offset-committer"), 119 | s"git@github.com:HeartSaVioR/spark-sql-kafka-offset-committer.git" 120 | ) 121 | ) 122 | 123 | watchTriggeredMessage in ThisBuild := Watch.clearScreenOnTrigger 124 | 125 | publishTo := { 126 | val nexus = "https://oss.sonatype.org/" 127 | if (isSnapshot.value) 128 | Some("snapshots" at nexus + "content/repositories/snapshots") 129 | else 130 | Some("releases" at nexus + "service/local/staging/deploy/maven2") 131 | } 132 | publishMavenStyle := true 133 | 134 | scalastyleConfig := file("scalastyle-config.xml") 135 | 136 | coverageEnabled in Test := true 137 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.3.10 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") 2 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "2.0.0") 3 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.6.1") 4 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.13") -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 17 | 39 | 40 | 41 | Scalastyle standard configuration 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | " 53 | * 54 | * Licensed under the Apache License, Version 2.0 (the "License"); 55 | * you may not use this file except in compliance with the License. 56 | * You may obtain a copy of the License at 57 | * 58 | * http://www.apache.org/licenses/LICENSE-2.0 59 | * 60 | * Unless required by applicable law or agreed to in writing, software 61 | * distributed under the License is distributed on an "AS IS" BASIS, 62 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 63 | * See the License for the specific language governing permissions and 64 | * limitations under the License. 65 | */]]> 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | true 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW 125 | 126 | 127 | 128 | 129 | 130 | ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW 131 | 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | ^FunSuite[A-Za-z]*$ 140 | Tests must extend org.apache.spark.SparkFunSuite instead. 141 | 142 | 143 | 144 | 145 | ^println$ 146 | 150 | 151 | 152 | 153 | spark(.sqlContext)?.sparkContext.hadoopConfiguration 154 | 163 | 164 | 165 | 166 | @VisibleForTesting 167 | 170 | 171 | 172 | 173 | Runtime\.getRuntime\.addShutdownHook 174 | 182 | 183 | 184 | 185 | mutable\.SynchronizedBuffer 186 | 194 | 195 | 196 | 197 | Class\.forName 198 | 205 | 206 | 207 | 208 | Await\.result 209 | 216 | 217 | 218 | 219 | Await\.ready 220 | 227 | 228 | 229 | 230 | 231 | JavaConversions 232 | Instead of importing implicits in scala.collection.JavaConversions._, import 233 | scala.collection.JavaConverters._ and use .asScala / .asJava methods 234 | 235 | 236 | 237 | org\.apache\.commons\.lang\. 238 | Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead 239 | of Commons Lang 2 (package org.apache.commons.lang.*) 240 | 241 | 242 | 243 | extractOpt 244 | Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter 245 | is slower. 246 | 247 | 248 | 249 | 250 | java,scala,3rdParty,spark 251 | javax?\..* 252 | scala\..* 253 | (?!org\.apache\.spark\.).* 254 | org\.apache\.spark\..* 255 | 256 | 257 | 258 | 259 | 260 | COMMA 261 | 262 | 263 | 264 | 265 | 266 | \)\{ 267 | 270 | 271 | 272 | 273 | (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] 274 | Use Javadoc style indentation for multiline comments 275 | 276 | 277 | 278 | case[^\n>]*=>\s*\{ 279 | Omit braces in case clauses. 280 | 281 | 282 | 283 | 284 | 285 | 286 | 287 | 288 | 289 | 290 | 291 | 292 | 293 | 294 | 295 | 296 | 297 | 298 | 299 | 300 | 301 | 302 | 303 | 304 | 305 | 306 | 307 | 308 | 309 | 310 | 311 | 312 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 320 | 321 | 322 | 323 | 324 | 325 | 326 | 327 | 328 | 329 | 330 | 331 | 332 | 800> 333 | 334 | 335 | 336 | 337 | 30 338 | 339 | 340 | 341 | 342 | 10 343 | 344 | 345 | 346 | 347 | 50 348 | 349 | 350 | 351 | 352 | 353 | 354 | 355 | 356 | 357 | 358 | -1,0,1,2,3 359 | 360 | 361 | 362 | -------------------------------------------------------------------------------- /src/main/scala/net/heartsavior/spark/KafkaOffsetCommitterListener.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 Jungtaek Lim "" 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package net.heartsavior.spark 18 | 19 | import java.time.Duration 20 | 21 | import scala.collection.JavaConverters._ 22 | 23 | import org.apache.kafka.clients.consumer.{KafkaConsumer, OffsetAndMetadata} 24 | 25 | import org.apache.spark.internal.Logging 26 | import org.apache.spark.sql.SparkSession 27 | import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} 28 | import org.apache.spark.sql.kafka010.KafkaSourceInspector 29 | import org.apache.spark.sql.streaming.StreamingQueryListener 30 | 31 | 32 | class KafkaOffsetCommitterListener extends StreamingQueryListener with Logging { 33 | import KafkaOffsetCommitterListener._ 34 | 35 | override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {} 36 | 37 | override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { 38 | val query = SparkSession.active.streams.get(event.progress.id) 39 | if (query != null) { 40 | val exec = query match { 41 | case query: StreamingQueryWrapper => Option(query.streamingQuery.lastExecution) 42 | case query: StreamExecution => Option(query.lastExecution) 43 | case _ => 44 | logWarning(s"Unexpected type of streaming query: ${query.getClass}") 45 | None 46 | } 47 | 48 | exec.foreach { ex => 49 | val inspector = new KafkaSourceInspector(ex.executedPlan) 50 | val idxToKafkaParams = inspector.populateKafkaParams 51 | idxToKafkaParams.foreach { case (idx, params) => 52 | params.get(CONFIG_KEY_GROUP_ID) match { 53 | case Some(groupId) => 54 | val sourceProgress = event.progress.sources(idx) 55 | val tpToOffsets = inspector.partitionOffsets(sourceProgress.endOffset) 56 | 57 | val newParams = new scala.collection.mutable.HashMap[String, Object] 58 | newParams ++= params 59 | newParams += "group.id" -> groupId 60 | 61 | val kafkaConsumer = new KafkaConsumer[String, String](newParams.asJava) 62 | try { 63 | val offsetsToCommit = tpToOffsets.map { case (tp, offset) => 64 | (tp -> new OffsetAndMetadata(offset)) 65 | } 66 | kafkaConsumer.commitSync(offsetsToCommit.asJava, Duration.ofSeconds(10)) 67 | } finally { 68 | kafkaConsumer.close() 69 | } 70 | 71 | case None => 72 | } 73 | } 74 | } 75 | } else { 76 | logWarning(s"Cannot find query ${event.progress.id} from active spark session!") 77 | } 78 | } 79 | 80 | override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {} 81 | } 82 | 83 | object KafkaOffsetCommitterListener { 84 | val CONFIG_KEY_GROUP_ID = "consumer.commit.groupid" 85 | val CONFIG_KEY_GROUP_ID_DATA_SOURCE_OPTION = "kafka." + CONFIG_KEY_GROUP_ID 86 | } 87 | -------------------------------------------------------------------------------- /src/main/scala/net/heartsavior/spark/ReflectionHelper.scala: -------------------------------------------------------------------------------- 1 | // scalastyle:off header 2 | /* 3 | * Licensed to the Apache Software Foundation (ASF) under one or more 4 | * contributor license agreements. See the NOTICE file distributed with 5 | * this work for additional information regarding copyright ownership. 6 | * The ASF licenses this file to You under the Apache License, Version 2.0 7 | * (the "License"); you may not use this file except in compliance with 8 | * the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package net.heartsavior.spark 20 | 21 | import scala.util.control.NonFatal 22 | 23 | import org.apache.spark.internal.Logging 24 | 25 | 26 | object ReflectionHelper extends Logging { 27 | import scala.reflect.runtime.universe.{TermName, TypeTag, runtimeMirror, typeOf} 28 | private val currentMirror = runtimeMirror(getClass.getClassLoader) 29 | 30 | def reflectField[T, OUT](obj: Any, fieldName: String)(implicit ttag: TypeTag[T]): Option[OUT] = { 31 | val relMirror = currentMirror.reflect(obj) 32 | 33 | try { 34 | val method = typeOf[T].decl(TermName(fieldName)).asTerm.accessed.asTerm 35 | 36 | Some(relMirror.reflectField(method).get.asInstanceOf[OUT]) 37 | } catch { 38 | case NonFatal(e) => 39 | logWarning(s"Failed to reflect field $fieldName from $obj. $e") 40 | None 41 | } 42 | } 43 | 44 | def reflectFieldWithContextClassloaderLoosenType(obj: Any, fieldName: String): Option[Any] = { 45 | val typeMirror = runtimeMirror(Thread.currentThread().getContextClassLoader) 46 | val instanceMirror = typeMirror.reflect(obj) 47 | 48 | val members = instanceMirror.symbol.typeSignature.members 49 | val field = members.find(_.name.decodedName.toString == fieldName) 50 | field match { 51 | case Some(f) => 52 | try { 53 | Some(instanceMirror.reflectField(f.asTerm).get) 54 | } catch { 55 | case NonFatal(e) => 56 | logWarning(s"Failed to reflect field $fieldName from $obj. $e") 57 | None 58 | } 59 | 60 | case None => 61 | logWarning(s"Failed to reflect field $fieldName from $obj.") 62 | None 63 | } 64 | } 65 | 66 | def reflectFieldWithContextClassloader[OUT](obj: Any, fieldName: String): Option[OUT] = { 67 | reflectFieldWithContextClassloaderLoosenType(obj, fieldName).map(_.asInstanceOf[OUT]) 68 | } 69 | 70 | def reflectMethodWithContextClassloaderLoosenType( 71 | obj: Any, 72 | methodName: String, 73 | params: Any*): Option[Any] = { 74 | val typeMirror = runtimeMirror(Thread.currentThread().getContextClassLoader) 75 | val instanceMirror = typeMirror.reflect(obj) 76 | 77 | val members = instanceMirror.symbol.typeSignature.members 78 | val method = members.find(_.name.decodedName.toString == methodName) 79 | method match { 80 | case Some(f) => 81 | try { 82 | Some(instanceMirror.reflectMethod(f.asMethod).apply(params)) 83 | } catch { 84 | case NonFatal(_) => 85 | logWarning(s"Failed to call method $methodName from $obj via reflection.") 86 | None 87 | } 88 | 89 | case None => 90 | logWarning(s"Failed to call method $methodName from $obj via reflection.") 91 | None 92 | } 93 | } 94 | 95 | def reflectMethodWithContextClassloader[OUT]( 96 | obj: Any, 97 | fieldName: String, 98 | params: Any*): Option[OUT] = { 99 | reflectMethodWithContextClassloaderLoosenType(obj, fieldName, params: _*) 100 | .map(_.asInstanceOf[OUT]) 101 | } 102 | 103 | def classForName(className: String): Class[_] = { 104 | // scalastyle:off classforname 105 | Class.forName(className, true, getContextOrClassClassLoader) 106 | // scalastyle:on classforname 107 | } 108 | 109 | private def getContextOrClassClassLoader: ClassLoader = 110 | Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) 111 | } 112 | 113 | // scalastyle:on header 114 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceInspector.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 Jungtaek Lim "" 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.kafka010 18 | 19 | import scala.collection.JavaConverters._ 20 | 21 | import net.heartsavior.spark.ReflectionHelper 22 | import org.apache.kafka.common.TopicPartition 23 | 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.sql.execution.{RDDScanExec, RowDataSourceScanExec, SparkPlan} 26 | import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDDPartition, DataSourceV2ScanExecBase} 27 | import org.apache.spark.sql.kafka010.{JsonUtils => KafkaJsonUtils} 28 | import org.apache.spark.sql.sources.BaseRelation 29 | 30 | 31 | class KafkaSourceInspector(sparkPlan: SparkPlan) { 32 | def populateKafkaParams: Map[Int, Map[String, Object]] = { 33 | sparkPlan.collectLeaves().zipWithIndex.flatMap { case (plan, idx) => 34 | val paramsOpt = plan match { 35 | case r: RowDataSourceScanExec if isKafkaRelation(r.relation) => 36 | extractKafkaParamsFromKafkaRelation(r.relation) 37 | case r: RDDScanExec => 38 | extractKafkaParamsFromDataSourceV1(r) 39 | case r: DataSourceV2ScanExecBase => 40 | extractSourceTopicsFromDataSourceV2(r) 41 | case _ => None 42 | } 43 | if (paramsOpt.isDefined) { 44 | Some((idx, paramsOpt.get)) 45 | } else { 46 | None 47 | } 48 | }.map(elem => elem._1 -> elem._2).toMap 49 | } 50 | 51 | private def isKafkaRelation(rel: BaseRelation): Boolean = { 52 | rel match { 53 | case r: KafkaRelation => true 54 | case _ => false 55 | } 56 | } 57 | 58 | private def extractKafkaParamsFromKafkaRelation(rel: BaseRelation) 59 | : Option[Map[String, Object]] = { 60 | require(isKafkaRelation(rel)) 61 | ReflectionHelper.reflectFieldWithContextClassloader[Map[String, String]]( 62 | rel, "specifiedKafkaParams") 63 | } 64 | 65 | private def extractKafkaParamsFromDataSourceV1(r: RDDScanExec): Option[Map[String, Object]] = { 66 | extractKafkaParamsFromDataSourceV1(r.rdd) 67 | } 68 | 69 | def extractKafkaParamsFromDataSourceV1(r: RowDataSourceScanExec) 70 | : Option[Map[String, Object]] = { 71 | extractKafkaParamsFromDataSourceV1(r.rdd) 72 | } 73 | 74 | private def extractKafkaParamsFromDataSourceV1( 75 | rddContainingPartition: RDD[_]): Option[Map[String, Object]] = { 76 | rddContainingPartition.partitions.flatMap { 77 | case _: KafkaSourceRDDPartition => 78 | extractKafkaParamsFromKafkaSourceRDDPartition(rddContainingPartition) 79 | case _ => None 80 | }.headOption 81 | } 82 | 83 | private def extractKafkaParamsFromKafkaSourceRDDPartition( 84 | rddContainingPartition: RDD[_]): Option[Map[String, Object]] = { 85 | def collectLeaves(rdd: RDD[_]): Seq[RDD[_]] = { 86 | // this method is being called with chains of MapPartitionRDDs 87 | // so this recursion won't stack up too much 88 | if (rdd.dependencies.isEmpty) { 89 | Seq(rdd) 90 | } else { 91 | rdd.dependencies.map(_.rdd).flatMap(collectLeaves) 92 | } 93 | } 94 | 95 | collectLeaves(rddContainingPartition).flatMap { 96 | case r: KafkaSourceRDD => Some(r) 97 | case _ => None 98 | }.headOption.flatMap(extractKafkaParamsFromKafkaSourceRDD).orElse(None) 99 | } 100 | 101 | private def extractKafkaParamsFromKafkaSourceRDD( 102 | rdd: KafkaSourceRDD): Option[Map[String, Object]] = { 103 | val map = ReflectionHelper.reflectFieldWithContextClassloader[java.util.Map[String, Object]]( 104 | rdd, "executorKafkaParams") 105 | map.map(_.asScala.toMap).orElse(None) 106 | } 107 | 108 | def extractSourceTopicsFromDataSourceV2( 109 | r: DataSourceV2ScanExecBase): Option[Map[String, Object]] = { 110 | r.inputRDDs().flatMap { rdd => 111 | rdd.partitions.flatMap { 112 | case e: DataSourceRDDPartition => e.inputPartition match { 113 | case part: KafkaBatchInputPartition => 114 | Some(part.executorKafkaParams.asScala.toMap) 115 | case part: KafkaContinuousInputPartition => 116 | Some(part.kafkaParams.asScala.toMap) 117 | case _ => None 118 | } 119 | } 120 | }.headOption 121 | } 122 | 123 | def partitionOffsets(str: String): Map[TopicPartition, Long] = { 124 | KafkaJsonUtils.partitionOffsets(str) 125 | } 126 | } 127 | -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | # Set everything to be logged to the file target/unit-tests.log 19 | test.appender=file 20 | log4j.rootCategory=INFO, ${test.appender} 21 | log4j.appender.file=org.apache.log4j.FileAppender 22 | log4j.appender.file.append=true 23 | log4j.appender.file.file=target/unit-tests.log 24 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 25 | log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n 26 | 27 | # Silence some noisy libraries. 28 | log4j.logger.org.apache.http=WARN 29 | log4j.logger.org.apache.spark=WARN 30 | log4j.logger.org.apache.spark.sql.kafka010=INFO 31 | log4j.logger.net.heartsavior.spark=INFO 32 | log4j.logger.org.eclipse.jetty=WARN 33 | log4j.logger.org.spark-project.jetty=WARN 34 | -------------------------------------------------------------------------------- /src/test/scala/net/heartsavior/spark/KafkaOffsetCommitterListenerSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 Jungtaek Lim "" 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package net.heartsavior.spark 18 | 19 | import java.io.File 20 | import java.nio.file.Files 21 | import java.util.Properties 22 | 23 | import scala.collection.JavaConverters._ 24 | 25 | import net.heartsavior.spark.KafkaOffsetCommitterListener._ 26 | import org.apache.commons.io.FileUtils 27 | import org.apache.kafka.clients.CommonClientConfigs 28 | import org.apache.kafka.clients.admin.AdminClient 29 | import org.apache.kafka.common.TopicPartition 30 | import org.json4s.JsonAST.{JArray, JInt, JObject} 31 | import org.json4s.jackson.JsonMethods.{compact, render} 32 | import org.scalatest.time.SpanSugar._ 33 | 34 | import org.apache.spark.sql.{Dataset, Row} 35 | import org.apache.spark.sql.kafka010.KafkaTestUtils 36 | import org.apache.spark.sql.streaming.{StreamingQuery, StreamTest} 37 | 38 | 39 | class KafkaOffsetCommitterListenerSuite extends StreamTest { 40 | val brokerProps: Map[String, Object] = Map[String, Object]() 41 | var testUtils: KafkaTestUtils = _ 42 | var adminClient: AdminClient = _ 43 | 44 | override def beforeAll(): Unit = { 45 | super.beforeAll() 46 | testUtils = new KafkaTestUtils(brokerProps) 47 | testUtils.setup() 48 | 49 | val props = new Properties() 50 | props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, testUtils.brokerAddress) 51 | adminClient = AdminClient.create(props) 52 | } 53 | 54 | override def afterAll(): Unit = { 55 | if (testUtils != null) { 56 | testUtils.teardown() 57 | testUtils = null 58 | } 59 | if (adminClient != null) { 60 | adminClient.close() 61 | } 62 | super.afterAll() 63 | } 64 | 65 | test("Run micro-batch query with Kafka source(s) - check offsets are committed") { 66 | def assertOffsetsAreCommited( 67 | groupId: String, 68 | topics: Seq[String], 69 | numPartitions: Int, 70 | expectedOffsetsPerPartition: Int): Unit = { 71 | val committedOffsets = adminClient.listConsumerGroupOffsets(groupId) 72 | .partitionsToOffsetAndMetadata().get().asScala 73 | .map { case (tp, offsetAndMetadata) => tp -> offsetAndMetadata.offset() } 74 | .toMap 75 | val expectedOffsets = topics.flatMap { topic => 76 | (0 until numPartitions).map(new TopicPartition(topic, _) -> expectedOffsetsPerPartition) 77 | }.toMap 78 | assert(committedOffsets === expectedOffsets) 79 | } 80 | 81 | def assertOffsetsAreNotCommited(topics: Seq[String]): Unit = { 82 | val groups = adminClient.listConsumerGroups().all().get().asScala 83 | val committedTopics = groups.flatMap { grp => 84 | adminClient.listConsumerGroupOffsets(grp.groupId()) 85 | .partitionsToOffsetAndMetadata().get().asScala 86 | .map { case (tp, offsetAndMetadata) => tp -> offsetAndMetadata.offset() } 87 | .filter { case (_, offset) => offset >= 0 } 88 | .map { case (tp, _) => tp.topic() } 89 | .toSet 90 | }.toSet 91 | assert(Set.empty[String] === committedTopics.intersect(topics.toSet)) 92 | } 93 | 94 | val listener = new KafkaOffsetCommitterListener() 95 | try { 96 | spark.streams.addListener(listener) 97 | 98 | val topicPrefix = "sparkread0" 99 | val brokerAddress = testUtils.brokerAddress 100 | 101 | val groupId1 = "groupId1" 102 | val (df1, topicsToRead1) = kafkaDfSubscribePattern(topicPrefix, brokerAddress, Some(groupId1)) 103 | 104 | val groupId2 = "groupId2" 105 | val (df2, topicsToRead2) = kafkaDfSubscribe(topicPrefix, brokerAddress, Some(groupId2)) 106 | 107 | val (df3, topicsToRead3) = kafkaDfAssign(topicPrefix, brokerAddress, Some(groupId1)) 108 | 109 | val topicPrefix2 = "sparkread1" 110 | val (df4, topicsToRead4) = kafkaDfSubscribe(topicPrefix2, brokerAddress, None) 111 | 112 | val topicsToRead = topicsToRead1 ++ topicsToRead2 ++ topicsToRead3 ++ topicsToRead4 113 | 114 | val topicToWrite = "sparkwrite" 115 | val topics = topicsToRead ++ Seq(topicToWrite) 116 | 117 | val numPartitions = 10 118 | val msgsPerPartition = 100 119 | topics.toSet[String].foreach { ti => 120 | testUtils.createTopic(ti, numPartitions, overwrite = true) 121 | } 122 | 123 | val (_, _, checkpointDir) = createTempDirectories 124 | 125 | val df = df1.union(df2).union(df3).union(df4) 126 | 127 | val query = toKafkaSink(df, brokerAddress, topicToWrite, checkpointDir.getAbsolutePath) 128 | 129 | try { 130 | sendMessages(topicsToRead, numPartitions, msgsPerPartition) 131 | eventually(timeout(30.seconds), interval(100.milliseconds)) { 132 | assertOffsetsAreCommited(groupId1, topicsToRead1 ++ topicsToRead3, numPartitions, 133 | msgsPerPartition) 134 | assertOffsetsAreCommited(groupId2, topicsToRead2, numPartitions, msgsPerPartition) 135 | assertOffsetsAreNotCommited(topicsToRead4) 136 | } 137 | } finally { 138 | query.stop() 139 | } 140 | } finally { 141 | spark.streams.removeListener(listener) 142 | } 143 | } 144 | 145 | private def kafkaDfSubscribePattern( 146 | topicNamePrefix: String, 147 | brokerAddress: String, 148 | commitGroupId: Option[String]): (Dataset[Row], Seq[String]) = { 149 | val topics = (1 to 3).map(idx => topicNamePrefix + idx) 150 | 151 | // test for 'subscribePattern' 152 | val stream = spark.readStream 153 | .format("kafka") 154 | .option("kafka.bootstrap.servers", brokerAddress) 155 | .option("subscribePattern", topicNamePrefix + "[1-3]") 156 | .option("startingOffsets", "earliest") 157 | 158 | val df = commitGroupId match { 159 | case Some(gid) => stream.option(CONFIG_KEY_GROUP_ID_DATA_SOURCE_OPTION, gid).load() 160 | case _ => stream.load() 161 | } 162 | 163 | (df, topics) 164 | } 165 | 166 | private def kafkaDfSubscribe( 167 | topicNamePrefix: String, 168 | brokerAddress: String, 169 | commitGroupId: Option[String]): (Dataset[Row], Seq[String]) = { 170 | val topics = (4 to 5).map(idx => topicNamePrefix + idx) 171 | 172 | // test for 'subscribe' 173 | val stream = spark.readStream 174 | .format("kafka") 175 | .option("kafka.bootstrap.servers", brokerAddress) 176 | .option("subscribe", topics.mkString(",")) 177 | .option("startingOffsets", "earliest") 178 | 179 | val df = commitGroupId match { 180 | case Some(gid) => stream.option(CONFIG_KEY_GROUP_ID_DATA_SOURCE_OPTION, gid).load() 181 | case _ => stream.load() 182 | } 183 | 184 | (df, topics) 185 | } 186 | 187 | private def kafkaDfAssign( 188 | topicNamePrefix: String, 189 | brokerAddress: String, 190 | commitGroupId: Option[String]): (Dataset[Row], Seq[String]) = { 191 | val topics = (6 to 7).map(idx => topicNamePrefix + idx) 192 | // test for 'assign' 193 | val jsonToAssignTopicToRead = { 194 | val r = JObject.apply { 195 | topics.map { 196 | (_, JArray((0 until 10).map(JInt(_)).toList)) 197 | }.toList 198 | } 199 | compact(render(r)) 200 | } 201 | 202 | val stream = spark.readStream 203 | .format("kafka") 204 | .option("kafka.bootstrap.servers", brokerAddress) 205 | .option("assign", jsonToAssignTopicToRead) 206 | .option("startingOffsets", "earliest") 207 | 208 | val df = commitGroupId match { 209 | case Some(gid) => stream.option(CONFIG_KEY_GROUP_ID_DATA_SOURCE_OPTION, gid).load() 210 | case _ => stream.load() 211 | } 212 | 213 | (df, topics) 214 | } 215 | 216 | private def toKafkaSink( 217 | df: Dataset[Row], 218 | brokerAddress: String, 219 | topic: String, 220 | checkpointPath: String): StreamingQuery = { 221 | df.writeStream 222 | .format("kafka") 223 | .option("kafka.bootstrap.servers", brokerAddress) 224 | .option("topic", topic) 225 | .option("checkpointLocation", checkpointPath) 226 | .start() 227 | } 228 | 229 | private def createTempDirectories: (File, File, File) = { 230 | val tempDir = Files.createTempDirectory("kafka-offset-committer-listener-suite-temp") 231 | 232 | val srcDir = new File(tempDir.toFile, "src") 233 | val destDir = new File(tempDir.toFile, "dest") 234 | val checkpointDir = new File(tempDir.toFile, "checkpoint") 235 | 236 | // remove temporary directory in shutdown 237 | org.apache.hadoop.util.ShutdownHookManager.get().addShutdownHook( 238 | new Runnable { 239 | override def run(): Unit = { 240 | FileUtils.forceDelete(tempDir.toFile) 241 | } 242 | }, 10) 243 | 244 | Files.createDirectories(srcDir.toPath) 245 | 246 | (srcDir, destDir, checkpointDir) 247 | } 248 | 249 | private def sendMessages( 250 | topicsToRead: Seq[String], 251 | numPartitions: Int, 252 | msgsPerPartition: Int): Unit = { 253 | topicsToRead.foreach { topic => 254 | (0 until numPartitions).foreach { part => 255 | val messages = (1 to msgsPerPartition).map(_.toString).toArray 256 | testUtils.sendMessages(topic, messages, Some(part)) 257 | } 258 | } 259 | } 260 | } 261 | -------------------------------------------------------------------------------- /version.sbt: -------------------------------------------------------------------------------- 1 | version in ThisBuild := "0.4.1-spark-3.0-SNAPSHOT" 2 | --------------------------------------------------------------------------------