├── .gitignore ├── LICENSE ├── README.md ├── pom.xml └── src ├── main ├── resources │ └── META-INF │ │ └── services │ │ └── org.apache.spark.sql.sources.DataSourceRegister └── scala │ ├── com │ └── hortonworks │ │ └── spark │ │ └── sql │ │ └── kafka08 │ │ ├── KafkaSource.scala │ │ ├── KafkaSourceOffset.scala │ │ ├── KafkaSourceProvider.scala │ │ └── util │ │ └── Logging.scala │ └── org │ └── apache │ └── spark │ └── SparkUtils.scala └── test ├── resources └── log4j.properties └── scala └── org └── apache └── spark ├── KafkaSourceSuite.scala └── KafkaTestUtils.scala /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | 4 | # sbt specific 5 | .cache 6 | .history 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | 15 | # Scala-IDE specific 16 | .scala_dependencies 17 | .worksheet 18 | 19 | .idea 20 | *.iml 21 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # spark-kafka-0-8-sql 2 | 3 | Spark Structured Streaming Kafka Source for Kafka 0.8. 4 | 5 | This library is design for Spark Structured Streaming Kafka source, its aim is to provide equal functionalities for users who still use Kafka 0.8/0.9. 6 | 7 | The main differences compared to Kafka 0.10 source are: 8 | 9 | 1. This Kafka 0.8 source uses `SimpleConsumer` rather than new `Consumer` API. 10 | 2. Some configurations (especially the name) are changed in Kafka 0.10, and here we still keep the conventions of Kafka 0.8. 11 | 3. We don't rewrite the whole Kafka connection logics compared to Kafka 0.10 source, instead we still use the existing implementations of Spark Streaming Kafka 0.8 direct approach. 12 | 13 | ## To Use It 14 | 15 | Like other Sources in Spark ecosystem, the simplest way to use is to add the dependencies to Spark by: 16 | 17 | ``` 18 | spark-submit 19 | --master local[*] \ 20 | --packages com.hortonworks.spark:spark-kafka-0-8-sql_2.11:1.0 \ 21 | yourApp 22 | ... 23 | ``` 24 | 25 | Spark will automatically search central and local maven repositories to add dependencies to Spark runtime. Besides you coud use `mvn install` to publish this library to local Maven repo and use `--packages`, which will search local maven repo also. 26 | 27 | To use `KafkaSource`, it is the same as any other Structured Streaming Sources already supported in Spark: 28 | 29 | ```scala 30 | 31 | import spark.implicits 32 | 33 | val reader = spark 34 | .readStream 35 | .format("kafka") 36 | .option("kafka.bootstrap.servers", testUtils.brokerAddress) 37 | .option("startingoffset", "smallest") 38 | .option("topics", topic) 39 | 40 | val kafka = reader.load() 41 | .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") 42 | .writeStream 43 | .format("console") 44 | .trigger(ProcessingTime(2000L)) 45 | .start() 46 | 47 | kafka.awaitTermination() 48 | 49 | ``` 50 | 51 | ### To compile 52 | 53 | This Structured Streaming Kafka 0.8 source is built with Maven, you could build with: 54 | 55 | ``` 56 | mvn clean package 57 | ``` 58 | 59 | ### Compactible Spark Version 60 | 61 | Due to the rigid changes of Structured Streaming component, This Kafka 0.8 Source can only worked with Spark after 2.0.2 and master branch. 62 | 63 | ### Important notes: 64 | 65 | 1. The schema of Kafka 0.8 source is fixed, you cannot change the schema of Kafka 0.8 source, this is different from most of other Sources in Spark. 66 | 67 | ```scala 68 | 69 | StructType(Seq( 70 | StructField("key", BinaryType), 71 | StructField("value", BinaryType), 72 | StructField("topic", StringType), 73 | StructField("partition", IntegerType), 74 | StructField("offset", LongType))) 75 | 76 | ``` 77 | 2. You have to set `kafka.bootstrap.servers` or `kafka.metadata.broker` in Source creation. 78 | 3. You have to specify "topics" in Kafka 0.8 Source options, multiple topics are separated by ",". 79 | 4. All the Kafka related configurations set through Kafka 0.8 Source should be start with "kafka." prefix. 80 | 5. Option "startingoffset" can only be "smallest" or "largest". 81 | 82 | # License 83 | 84 | Apache License, Version 2.0 [http://www.apache.org/licenses/LICENSE-2.0](http://www.apache.org/licenses/LICENSE-2.0) 85 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.hortonworks.spark 8 | spark-kafka-0-8-sql_2.11 9 | 1.0 10 | 11 | 12 | 2.1.0-SNAPSHOT 13 | 2.11.8 14 | 2.11 15 | 512m 16 | 512m 17 | 1.7 18 | 1.7 19 | 1.8 20 | true 21 | false 22 | 23 | 24 | 25 | 26 | 27 | org.apache.spark 28 | spark-core_${scala.binary.version} 29 | ${spark.version} 30 | provided 31 | 32 | 33 | 34 | org.apache.spark 35 | spark-sql_${scala.binary.version} 36 | ${spark.version} 37 | provided 38 | 39 | 40 | 41 | org.apache.spark 42 | spark-catalyst_${scala.binary.version} 43 | ${spark.version} 44 | provided 45 | 46 | 47 | 48 | org.apache.spark 49 | spark-streaming-kafka-0-8_${scala.binary.version} 50 | ${spark.version} 51 | 52 | 53 | 54 | org.apache.spark 55 | spark-streaming_${scala.binary.version} 56 | ${spark.version} 57 | provided 58 | 59 | 60 | 61 | org.apache.kafka 62 | kafka-clients 63 | 0.8.2.1 64 | 65 | 66 | 67 | org.apache.spark 68 | spark-core_${scala.binary.version} 69 | ${spark.version} 70 | test-jar 71 | test 72 | 73 | 74 | 75 | org.apache.spark 76 | spark-catalyst_${scala.binary.version} 77 | ${spark.version} 78 | test-jar 79 | test 80 | 81 | 82 | 83 | org.apache.spark 84 | spark-sql_${scala.binary.version} 85 | ${spark.version} 86 | test-jar 87 | test 88 | 89 | 90 | 91 | org.scalacheck 92 | scalacheck_${scala.binary.version} 93 | 1.12.5 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | org.apache.maven.plugins 104 | maven-enforcer-plugin 105 | 1.4.1 106 | 107 | 108 | enforce-versions 109 | 110 | enforce 111 | 112 | 113 | 114 | 115 | 1.7 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | org.apache.maven.plugins 125 | maven-compiler-plugin 126 | 3.3 127 | 128 | 129 | 130 | org.apache.maven.plugins 131 | maven-antrun-plugin 132 | 1.8 133 | 134 | 135 | 136 | org.apache.maven.plugins 137 | maven-deploy-plugin 138 | 2.8.2 139 | 140 | 141 | 142 | org.codehaus.mojo 143 | build-helper-maven-plugin 144 | 1.10 145 | 146 | 147 | 148 | net.alchim31.maven 149 | scala-maven-plugin 150 | 3.2.2 151 | 152 | 153 | 154 | compile 155 | testCompile 156 | 157 | 158 | 159 | 160 | ${scala.version} 161 | incremental 162 | true 163 | false 164 | 165 | -unchecked 166 | -deprecation 167 | -feature 168 | 169 | 170 | -source 171 | ${java.version} 172 | -target 173 | -Xlint:unchecked 174 | 175 | 176 | -Xms1024m 177 | -Xmx1024m 178 | -XX:MaxPermSize=${MaxPermGen} 179 | -XX:ReservedCodeCacheSize=${CodeCacheSize} 180 | 181 | 182 | 183 | 184 | 185 | org.apache.maven.plugins 186 | maven-install-plugin 187 | 2.5.2 188 | 189 | 190 | 191 | org.apache.maven.plugins 192 | maven-surefire-plugin 193 | 2.19 194 | 195 | 196 | true 197 | ${project.build.directory}/tmp 198 | ${project.version} 199 | 200 | ${test.redirectToFile} 201 | ${test.redirectToFile} 202 | -Xmx2g -XX:MaxPermSize=512m 203 | false 204 | 205 | 206 | 207 | 208 | org.scalatest 209 | scalatest-maven-plugin 210 | 1.0 211 | 212 | 213 | true 214 | ${project.build.directory}/tmp 215 | 216 | D 217 | ${project.build.directory}/surefire-reports 218 | . 219 | WDF TestSuite.txt 220 | -Xmx2g -XX:MaxPermSize=512m 221 | 222 | 223 | 224 | test 225 | 226 | test 227 | 228 | 229 | 230 | 231 | 232 | 233 | 234 | 235 | 236 | 237 | org.apache.maven.plugins 238 | maven-compiler-plugin 239 | 240 | 241 | default-compile 242 | none 243 | 244 | 245 | default-testCompile 246 | none 247 | 248 | 249 | 250 | ${java.version} 251 | ${java.version} 252 | 253 | 254 | 255 | 256 | org.apache.maven.plugins 257 | maven-enforcer-plugin 258 | false 259 | 260 | 261 | 262 | [3.0.0,) 263 | 264 | 265 | [${minJavaVersion}.0,${maxJavaVersion}.1000}] 266 | 267 | 268 | unix 269 | 270 | 271 | 272 | 273 | 274 | clean 275 | 276 | enforce 277 | 278 | pre-clean 279 | 280 | 281 | default 282 | 283 | enforce 284 | 285 | validate 286 | 287 | 288 | site 289 | 290 | enforce 291 | 292 | pre-site 293 | 294 | 295 | 296 | 297 | 298 | net.alchim31.maven 299 | scala-maven-plugin 300 | 301 | 302 | 303 | org.apache.maven.plugins 304 | maven-antrun-plugin 305 | 306 | 307 | 308 | pre-test-clean 309 | generate-test-resources 310 | 311 | run 312 | 313 | 314 | 315 | 316 | 317 | 318 | 319 | 320 | 321 | 322 | create-tmp-dir 323 | generate-test-resources 324 | 325 | run 326 | 327 | 328 | 329 | 330 | 331 | 332 | 333 | 334 | 335 | 336 | 337 | org.apache.maven.plugins 338 | maven-deploy-plugin 339 | 340 | ${skipDeploy} 341 | 342 | 343 | 344 | 345 | org.apache.maven.plugins 346 | maven-surefire-plugin 347 | 348 | 349 | 350 | org.scalatest 351 | scalatest-maven-plugin 352 | 353 | 354 | 355 | 356 | 357 | 358 | 359 | -------------------------------------------------------------------------------- /src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | com.hortonworks.spark.sql.kafka08.KafkaSourceProvider 2 | -------------------------------------------------------------------------------- /src/main/scala/com/hortonworks/spark/sql/kafka08/KafkaSource.scala: -------------------------------------------------------------------------------- 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 | package com.hortonworks.spark.sql.kafka08 19 | 20 | import scala.annotation.tailrec 21 | 22 | import kafka.common.TopicAndPartition 23 | import kafka.message.MessageAndMetadata 24 | import kafka.serializer.DefaultDecoder 25 | import org.apache.spark.SparkException 26 | import org.apache.spark.sql._ 27 | import org.apache.spark.sql.execution.streaming._ 28 | import org.apache.spark.sql.types._ 29 | import org.apache.spark.streaming.kafka.{Broker, KafkaCluster, KafkaUtils, OffsetRange} 30 | import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset 31 | 32 | import com.hortonworks.spark.sql.kafka08.util.Logging 33 | 34 | /** 35 | * A [[Source]] that uses Kafka's SimpleConsumer API to reads data from Kafka. 36 | */ 37 | case class KafkaSource( 38 | sqlContext: SQLContext, 39 | topics: Set[String], 40 | kafkaParams: Map[String, String], 41 | sourceOptions: Map[String, String], 42 | metadataPath: String, 43 | startFromSmallestOffset: Boolean) 44 | extends Source with Logging { 45 | 46 | private val sc = sqlContext.sparkContext 47 | private val kc = new KafkaCluster(kafkaParams) 48 | private val topicPartitions = KafkaCluster.checkErrors(kc.getPartitions(topics)) 49 | 50 | private val maxOffsetFetchAttempts = 51 | sourceOptions.getOrElse("fetchOffset.numRetries", "3").toInt 52 | 53 | private lazy val initialPartitionOffsets = { 54 | val metadataLog = new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) 55 | metadataLog.get(0).getOrElse { 56 | val offsets = for { 57 | leaderOffsets <- (if (startFromSmallestOffset) { 58 | kc.getEarliestLeaderOffsets(topicPartitions) 59 | } else { 60 | kc.getLatestLeaderOffsets(topicPartitions) 61 | }).right 62 | } yield leaderOffsets 63 | 64 | val kafkaSourceOffset = KafkaSourceOffset(KafkaCluster.checkErrors(offsets)) 65 | 66 | metadataLog.add(0, kafkaSourceOffset) 67 | info(s"Initial offsets: $kafkaSourceOffset") 68 | kafkaSourceOffset 69 | }.partitionToOffsets 70 | } 71 | 72 | override def schema: StructType = KafkaSource.kafkaSchema 73 | 74 | /** Returns the maximum available offset for this source. */ 75 | override def getOffset: Option[Offset] = { 76 | // Make sure initialPartitionOffsets is initialized 77 | initialPartitionOffsets 78 | 79 | val offset = KafkaSourceOffset(fetchLatestOffsets(maxOffsetFetchAttempts)) 80 | debug(s"GetOffset: ${offset.partitionToOffsets.toSeq.map(_.toString).sorted}") 81 | Some(offset) 82 | } 83 | 84 | /** 85 | * Returns the data that is between the offsets 86 | * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is 87 | * exclusive. 88 | */ 89 | override def getBatch(start: Option[Offset], end: Offset): DataFrame = { 90 | // Make sure initialPartitionOffsets is initialized 91 | initialPartitionOffsets 92 | 93 | info(s"GetBatch called with start = $start, end = $end") 94 | val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) 95 | val fromPartitionOffsets = start match { 96 | case Some(prevBatchEndOffset) => 97 | KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset) 98 | case None => 99 | initialPartitionOffsets 100 | } 101 | 102 | val offsetRanges = fromPartitionOffsets.map { case (tp, fo) => 103 | val uo = untilPartitionOffsets(tp) 104 | OffsetRange(tp.topic, tp.partition, fo.offset, uo.offset) 105 | }.toArray 106 | 107 | val leaders = untilPartitionOffsets.map { case (tp, lo) => 108 | tp -> Broker(lo.host, lo.port) 109 | } 110 | 111 | val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => { 112 | Row(mmd.key(), mmd.message(), mmd.topic, mmd.partition, mmd.offset) 113 | } 114 | 115 | // Create a RDD that reads from Kafka and get the (key, value) pair as byte arrays. 116 | val rdd = KafkaUtils.createRDD[ 117 | Array[Byte], 118 | Array[Byte], 119 | DefaultDecoder, 120 | DefaultDecoder, 121 | Row](sc, kafkaParams, offsetRanges, leaders, messageHandler) 122 | 123 | info("GetBatch generating RDD of offset range: " + offsetRanges.sortBy(_.topic).mkString(",")) 124 | sqlContext.createDataFrame(rdd, schema) 125 | } 126 | 127 | /** Stop this source and free any resources it has allocated. */ 128 | override def stop(): Unit = { } 129 | 130 | override def toString(): String = s"KafkaSource for topics [${topics.mkString(",")}]" 131 | 132 | /** 133 | * Fetch the latest offset of partitions. 134 | */ 135 | @tailrec 136 | private def fetchLatestOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { 137 | val offsets = kc.getLatestLeaderOffsets(topicPartitions) 138 | if (offsets.isLeft) { 139 | val err = offsets.left.get.toString 140 | if (retries <= 0) { 141 | throw new SparkException(err) 142 | } else { 143 | error(err) 144 | Thread.sleep(kc.config.refreshLeaderBackoffMs) 145 | fetchLatestOffsets(retries - 1) 146 | } 147 | } else { 148 | offsets.right.get 149 | } 150 | } 151 | } 152 | 153 | /** Companion object for the [[KafkaSource]]. */ 154 | private[kafka08] object KafkaSource { 155 | 156 | def kafkaSchema: StructType = StructType(Seq( 157 | StructField("key", BinaryType), 158 | StructField("value", BinaryType), 159 | StructField("topic", StringType), 160 | StructField("partition", IntegerType), 161 | StructField("offset", LongType) 162 | )) 163 | } 164 | 165 | -------------------------------------------------------------------------------- /src/main/scala/com/hortonworks/spark/sql/kafka08/KafkaSourceOffset.scala: -------------------------------------------------------------------------------- 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 | package com.hortonworks.spark.sql.kafka08 19 | 20 | import kafka.common.TopicAndPartition 21 | import org.apache.spark.sql.execution.streaming.Offset 22 | import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset 23 | 24 | /** 25 | * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and 26 | * their offsets. 27 | */ 28 | case class KafkaSourceOffset(partitionToOffsets: Map[TopicAndPartition, LeaderOffset]) 29 | extends Offset { 30 | override def toString(): String = { 31 | partitionToOffsets.toSeq.sortBy(_._1.toString).mkString("[", ", ", "]") 32 | } 33 | } 34 | 35 | /** Companion object of the [[KafkaSourceOffset]] */ 36 | object KafkaSourceOffset { 37 | 38 | def getPartitionOffsets(offset: Offset): Map[TopicAndPartition, LeaderOffset] = { 39 | offset match { 40 | case o: KafkaSourceOffset => o.partitionToOffsets 41 | case _ => 42 | throw new IllegalArgumentException( 43 | s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") 44 | } 45 | } 46 | } 47 | 48 | -------------------------------------------------------------------------------- /src/main/scala/com/hortonworks/spark/sql/kafka08/KafkaSourceProvider.scala: -------------------------------------------------------------------------------- 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 | package com.hortonworks.spark.sql.kafka08 19 | 20 | import java.{util => ju} 21 | 22 | import scala.collection.JavaConverters._ 23 | 24 | import org.apache.kafka.clients.consumer.ConsumerConfig 25 | import org.apache.spark.sql.SQLContext 26 | import org.apache.spark.sql.execution.streaming.Source 27 | import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} 28 | import org.apache.spark.sql.types.StructType 29 | 30 | import com.hortonworks.spark.sql.kafka08.util.Logging 31 | 32 | /** 33 | * The provider class for the [[KafkaSource]]. This provider is designed such that it throws 34 | * IllegalArgumentException when the Kafka Dataset is created, so that it can catch 35 | * missing options even before the query is started. 36 | */ 37 | private[kafka08] class KafkaSourceProvider extends StreamSourceProvider 38 | with DataSourceRegister with Logging { 39 | 40 | import KafkaSourceProvider._ 41 | 42 | /** 43 | * Returns the name and schema of the source. In addition, it also verifies whether the options 44 | * are correct and sufficient to create the [[KafkaSource]] when the query is started. 45 | */ 46 | override def sourceSchema( 47 | sqlContext: SQLContext, 48 | schema: Option[StructType], 49 | providerName: String, 50 | parameters: Map[String, String]): (String, StructType) = { 51 | require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") 52 | validateOptions(parameters) 53 | ("kafka", KafkaSource.kafkaSchema) 54 | } 55 | 56 | override def createSource( 57 | sqlContext: SQLContext, 58 | metadataPath: String, 59 | schema: Option[StructType], 60 | providerName: String, 61 | parameters: Map[String, String]): Source = { 62 | validateOptions(parameters) 63 | val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } 64 | val specifiedKafkaParams = 65 | parameters 66 | .keySet 67 | .filter(_.toLowerCase.startsWith("kafka.")) 68 | .map { k => k.drop(6).toString -> parameters(k) } 69 | .toMap 70 | 71 | val topics = 72 | caseInsensitiveParams.get(TOPICS) match { 73 | case Some(s) => s.split(",").map(_.trim).filter(_.nonEmpty).toSet 74 | case None => throw new IllegalArgumentException(s"$TOPICS should be set.") 75 | } 76 | 77 | val startFromEarliestOffset = 78 | caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY).map(_.trim.toLowerCase) match { 79 | case Some("largest") => false 80 | case Some("smallest") => true 81 | case Some(pos) => 82 | // This should not happen since we have already checked the options. 83 | throw new IllegalStateException(s"Invalid $STARTING_OFFSET_OPTION_KEY: $pos") 84 | case None => false 85 | } 86 | 87 | val kafkaParams = 88 | ConfigUpdater("source", specifiedKafkaParams) 89 | // Set to "largest" to avoid exceptions. However, KafkaSource will fetch the initial offsets 90 | // by itself instead of counting on KafkaConsumer. 91 | .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "largest") 92 | 93 | // So that consumers does not commit offsets unnecessarily 94 | .set("auto.commit.enable", "false") 95 | .setIfUnset(ConsumerConfig.SOCKET_RECEIVE_BUFFER_CONFIG, "65536") 96 | .set(ConsumerConfig.GROUP_ID_CONFIG, "") 97 | .set("zookeeper.connect", "") 98 | .build() 99 | 100 | new KafkaSource( 101 | sqlContext, 102 | topics, 103 | kafkaParams.asScala.toMap, 104 | parameters, 105 | metadataPath, 106 | startFromEarliestOffset) 107 | } 108 | 109 | private def validateOptions(parameters: Map[String, String]): Unit = { 110 | 111 | // Validate source options 112 | val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase, v) } 113 | 114 | caseInsensitiveParams.get(STARTING_OFFSET_OPTION_KEY) match { 115 | case Some(pos) if !STARTING_OFFSET_OPTION_VALUES.contains(pos.trim.toLowerCase) => 116 | throw new IllegalArgumentException( 117 | s"Illegal value '$pos' for option '$STARTING_OFFSET_OPTION_KEY', " + 118 | s"acceptable values are: ${STARTING_OFFSET_OPTION_VALUES.mkString(", ")}") 119 | case _ => 120 | } 121 | 122 | // Validate user-specified Kafka options 123 | if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { 124 | throw new IllegalArgumentException( 125 | s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + 126 | s"user-specified consumer groups is not used to track offsets.") 127 | } 128 | 129 | if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { 130 | throw new IllegalArgumentException( 131 | s""" 132 | |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. 133 | |Instead set the source option '$STARTING_OFFSET_OPTION_KEY' to 'largest' or 134 | |'smallest' to specify where to start. Structured Streaming manages which offsets are 135 | |consumed internally, rather than relying on the kafkaConsumer to do it. This will 136 | |ensure that no data is missed when when new topics/partitions are dynamically 137 | |subscribed. Note that '$STARTING_OFFSET_OPTION_KEY' only applies when a new Streaming 138 | |query is started, and that resuming will always pick up from where the query left 139 | |off. See the docs for more 140 | |details. 141 | """.stripMargin) 142 | } 143 | 144 | if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { 145 | throw new IllegalArgumentException( 146 | s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " 147 | + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " 148 | + "to explicitly deserialize the keys.") 149 | } 150 | 151 | if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) 152 | { 153 | throw new IllegalArgumentException( 154 | s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " 155 | + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " 156 | + "operations to explicitly deserialize the values.") 157 | } 158 | 159 | val otherUnsupportedConfigs = Seq( 160 | "auto.commit.enable", // committing correctly requires new APIs in Source 161 | "zookeeper.connect") 162 | 163 | otherUnsupportedConfigs.foreach { c => 164 | if (caseInsensitiveParams.contains(s"kafka.$c")) { 165 | throw new IllegalArgumentException(s"Kafka option '$c' is not supported") 166 | } 167 | } 168 | 169 | if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}") && 170 | !caseInsensitiveParams.contains(s"kafka.metadata.brokers.list")) { 171 | throw new IllegalArgumentException( 172 | s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' or " + 173 | "'kafka.metadata.broker.list' must be specified for configuring Kafka consumer") 174 | } 175 | } 176 | 177 | override def shortName(): String = "kafka" 178 | 179 | /** Class to conveniently update Kafka config params, while logging the changes */ 180 | private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { 181 | private val map = new ju.HashMap[String, String](kafkaParams.asJava) 182 | 183 | def set(key: String, value: String): this.type = { 184 | map.put(key, value) 185 | info(s"$module: Set $key to $value, earlier value: ${kafkaParams.get(key).getOrElse("")}") 186 | this 187 | } 188 | 189 | def setIfUnset(key: String, value: String): ConfigUpdater = { 190 | if (!map.containsKey(key)) { 191 | map.put(key, value) 192 | info(s"$module: Set $key to $value") 193 | } 194 | this 195 | } 196 | 197 | def build(): ju.Map[String, String] = map 198 | } 199 | } 200 | 201 | private[kafka08] object KafkaSourceProvider { 202 | private val TOPICS = "topics" 203 | private val STARTING_OFFSET_OPTION_KEY = "startingoffset" 204 | private val STARTING_OFFSET_OPTION_VALUES = Set("largest", "smallest") 205 | } 206 | 207 | -------------------------------------------------------------------------------- /src/main/scala/com/hortonworks/spark/sql/kafka08/util/Logging.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to Cloudera, Inc. under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. Cloudera, Inc. licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with 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 com.hortonworks.spark.sql.kafka08.util 20 | 21 | import org.slf4j.LoggerFactory 22 | 23 | trait Logging { 24 | lazy val logger = LoggerFactory.getLogger(this.getClass) 25 | 26 | def trace(message: => Any): Unit = { 27 | if (logger.isTraceEnabled) { 28 | logger.trace(message.toString) 29 | } 30 | } 31 | 32 | def debug(message: => Any): Unit = { 33 | if (logger.isDebugEnabled) { 34 | logger.debug(message.toString) 35 | } 36 | } 37 | 38 | def info(message: => Any): Unit = { 39 | if (logger.isInfoEnabled) { 40 | logger.info(message.toString) 41 | } 42 | } 43 | 44 | def warn(message: => Any): Unit = { 45 | logger.warn(message.toString) 46 | } 47 | 48 | def warn(message: => Any, t: Throwable): Unit = { 49 | logger.warn(message.toString, t) 50 | } 51 | 52 | def error(message: => Any): Unit = { 53 | logger.error(message.toString) 54 | } 55 | 56 | def error(message: => Any, t: Throwable): Unit = { 57 | logger.error(message.toString, t) 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/SparkUtils.scala: -------------------------------------------------------------------------------- 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 | package org.apache.spark 19 | 20 | import org.apache.spark.scheduler.ExecutorCacheTaskLocation 21 | 22 | object SparkUtils { 23 | 24 | /** 25 | * Get sorted executor list. 26 | * @param sc 27 | * @return executor list 28 | */ 29 | def getSortedExecutorList(sc: SparkContext): Array[String] = { 30 | val bm = sc.env.blockManager 31 | bm.master.getPeers(bm.blockManagerId).toArray 32 | .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) 33 | .sortWith(compare) 34 | .map(_.toString) 35 | } 36 | 37 | private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { 38 | if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /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 | log4j.rootCategory=INFO, file 20 | log4j.appender.file=org.apache.log4j.FileAppender 21 | log4j.appender.file.append=true 22 | log4j.appender.file.file=target/unit-tests.log 23 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 24 | log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n 25 | 26 | # Ignore messages below warning level from Jetty, because it's a bit verbose 27 | log4j.logger.org.spark-project.jetty=WARN 28 | 29 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/KafkaSourceSuite.scala: -------------------------------------------------------------------------------- 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 | package org.apache.spark 19 | 20 | import scala.util.Random 21 | 22 | import kafka.common.TopicAndPartition 23 | import org.apache.spark.sql.execution.streaming._ 24 | import org.apache.spark.sql.streaming.StreamTest 25 | import org.apache.spark.sql.test.SharedSQLContext 26 | import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset 27 | import org.scalatest.time.SpanSugar._ 28 | 29 | import com.hortonworks.spark.sql.kafka08.KafkaSource 30 | import com.hortonworks.spark.sql.kafka08.KafkaSourceOffset 31 | 32 | abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { 33 | 34 | protected var testUtils: KafkaTestUtils = _ 35 | 36 | override val streamingTimeout = 30.seconds 37 | 38 | override def beforeAll(): Unit = { 39 | super.beforeAll() 40 | testUtils = new KafkaTestUtils 41 | testUtils.setup() 42 | } 43 | 44 | override def afterAll(): Unit = { 45 | if (testUtils != null) { 46 | testUtils.teardown() 47 | testUtils = null 48 | super.afterAll() 49 | } 50 | } 51 | 52 | protected def makeSureGetOffsetCalled = AssertOnQuery { q => 53 | // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure 54 | // its "getOffset" is called before pushing any data. Otherwise, because of the race condition, 55 | // we don't know which data should be fetched when `startingOffset` is latest. 56 | q.processAllAvailable() 57 | true 58 | } 59 | 60 | /** 61 | * Add data to Kafka. 62 | * 63 | * `topicAction` can be used to run actions for each topic before inserting data. 64 | */ 65 | case class AddKafkaData(topic: String, data: Int*) extends AddData { 66 | 67 | override def addData(query: Option[StreamExecution]): (Source, Offset) = { 68 | if (query.get.isActive) { 69 | // Make sure no Spark job is running when deleting a topic 70 | query.get.processAllAvailable() 71 | } 72 | 73 | val sources = query.get.logicalPlan.collect { 74 | case StreamingExecutionRelation(source, _) if source.isInstanceOf[KafkaSource] => 75 | source.asInstanceOf[KafkaSource] 76 | } 77 | if (sources.isEmpty) { 78 | throw new Exception( 79 | "Could not find Kafka source in the StreamExecution logical plan to add data to") 80 | } else if (sources.size > 1) { 81 | throw new Exception( 82 | "Could not select the Kafka source in the StreamExecution logical plan as there" + 83 | "are multiple Kafka sources:\n\t" + sources.mkString("\n\t")) 84 | } 85 | val kafkaSource = sources.head 86 | testUtils.sendMessages(topic, data.map { _.toString }.toArray) 87 | 88 | val Array(brokerHost, brokerPort) = testUtils.brokerAddress.split(":") 89 | val offset = KafkaSourceOffset(Map(TopicAndPartition(topic, 0) -> 90 | LeaderOffset(brokerHost, brokerPort.toInt, data.size))) 91 | (kafkaSource, offset) 92 | } 93 | 94 | override def toString: String = 95 | s"AddKafkaData(topic = $topic, data = $data)" 96 | } 97 | } 98 | 99 | class KafkaSourceSuite extends KafkaSourceTest { 100 | 101 | test("fetch data from Kafka stream") { 102 | val topic = newTopic() 103 | testUtils.createTopic(topic) 104 | 105 | val implicits = spark.implicits 106 | import implicits._ 107 | val reader = spark 108 | .readStream 109 | .format("kafka") 110 | .option("kafka.bootstrap.servers", testUtils.brokerAddress) 111 | .option("startingoffset", "smallest") 112 | .option("topics", topic) 113 | 114 | val kafka = reader.load() 115 | .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") 116 | .as[(String, String)] 117 | val mapped = kafka.map(kv => kv._2.toInt + 1) 118 | testStream(mapped)( 119 | makeSureGetOffsetCalled, 120 | AddKafkaData(topic, 1, 2, 3), 121 | CheckAnswer(2, 3, 4), 122 | StopStream 123 | ) 124 | } 125 | 126 | test("bad source options") { 127 | def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { 128 | val ex = intercept[IllegalArgumentException] { 129 | val reader = spark 130 | .readStream 131 | .format("kafka") 132 | options.foreach { case (k, v) => reader.option(k, v) } 133 | reader.load() 134 | } 135 | expectedMsgs.foreach { m => 136 | assert(ex.getMessage.toLowerCase.contains(m.toLowerCase)) 137 | } 138 | } 139 | 140 | // no metadata.broker.list or bootstrap.servers specified 141 | testBadOptions()( 142 | """option 'kafka.bootstrap.servers' or 'kafka.metadata.broker.list' must be specified""") 143 | } 144 | 145 | test("unsupported kafka configs") { 146 | def testUnsupportedConfig(key: String, value: String = "someValue"): Unit = { 147 | val ex = intercept[IllegalArgumentException] { 148 | val reader = spark 149 | .readStream 150 | .format("kafka") 151 | .option(s"$key", value) 152 | reader.load() 153 | } 154 | assert(ex.getMessage.toLowerCase.contains("not supported")) 155 | } 156 | 157 | testUnsupportedConfig("kafka.group.id") 158 | testUnsupportedConfig("kafka.auto.offset.reset") 159 | testUnsupportedConfig("kafka.key.deserializer") 160 | testUnsupportedConfig("kafka.value.deserializer") 161 | 162 | testUnsupportedConfig("kafka.auto.offset.reset", "none") 163 | testUnsupportedConfig("kafka.auto.offset.reset", "someValue") 164 | testUnsupportedConfig("kafka.auto.offset.reset", "earliest") 165 | testUnsupportedConfig("kafka.auto.offset.reset", "latest") 166 | } 167 | 168 | private def newTopic(): String = s"topic-${Random.nextInt(10000)}" 169 | } 170 | 171 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/KafkaTestUtils.scala: -------------------------------------------------------------------------------- 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 | package org.apache.spark 19 | 20 | import java.io.File 21 | import java.lang.{Integer => JInt} 22 | import java.net.InetSocketAddress 23 | import java.util.{Properties, Map => JMap} 24 | 25 | import scala.collection.JavaConverters._ 26 | import scala.language.postfixOps 27 | 28 | import kafka.admin.AdminUtils 29 | import kafka.api.Request 30 | import kafka.producer.{KeyedMessage, Producer, ProducerConfig} 31 | import kafka.serializer.StringEncoder 32 | import kafka.server.{KafkaConfig, KafkaServer} 33 | import kafka.utils.{ZKStringSerializer, ZkUtils} 34 | import org.apache.commons.lang3.RandomUtils 35 | import org.apache.spark.util.Utils 36 | import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} 37 | import org.I0Itec.zkclient.ZkClient 38 | import org.scalatest.concurrent.Eventually._ 39 | import org.scalatest.time.SpanSugar._ 40 | 41 | import com.hortonworks.spark.sql.kafka08.util.Logging 42 | 43 | /** 44 | * This is a helper class for Kafka test suites. This has the functionality to set up 45 | * and tear down local Kafka servers, and to push data using Kafka producers. 46 | * 47 | * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. 48 | */ 49 | class KafkaTestUtils extends Logging { 50 | 51 | // Zookeeper related configurations 52 | private val zkHost = "localhost" 53 | private var zkPort: Int = 0 54 | private val zkConnectionTimeout = 60000 55 | private val zkSessionTimeout = 6000 56 | 57 | private var zookeeper: EmbeddedZookeeper = _ 58 | 59 | private var zkClient: ZkClient = _ 60 | 61 | // Kafka broker related configurations 62 | private val brokerHost = "localhost" 63 | private var brokerPort = RandomUtils.nextInt(1024, 65535) 64 | private var brokerConf: KafkaConfig = _ 65 | 66 | // Kafka broker server 67 | private var server: KafkaServer = _ 68 | 69 | // Kafka producer 70 | private var producer: Producer[String, String] = _ 71 | 72 | // Flag to test whether the system is correctly started 73 | private var zkReady = false 74 | private var brokerReady = false 75 | 76 | def zkAddress: String = { 77 | assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") 78 | s"$zkHost:$zkPort" 79 | } 80 | 81 | def brokerAddress: String = { 82 | assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") 83 | s"$brokerHost:$brokerPort" 84 | } 85 | 86 | def zookeeperClient: ZkClient = { 87 | assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") 88 | Option(zkClient).getOrElse( 89 | throw new IllegalStateException("Zookeeper client is not yet initialized")) 90 | } 91 | 92 | // Set up the Embedded Zookeeper server and get the proper Zookeeper port 93 | private def setupEmbeddedZookeeper(): Unit = { 94 | // Zookeeper server startup 95 | zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") 96 | // Get the actual zookeeper binding port 97 | zkPort = zookeeper.actualPort 98 | zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, 99 | ZKStringSerializer) 100 | zkReady = true 101 | } 102 | 103 | // Set up the Embedded Kafka server 104 | private def setupEmbeddedKafkaServer(): Unit = { 105 | assert(zkReady, "Zookeeper should be set up beforehand") 106 | 107 | // Kafka broker startup 108 | Utils.startServiceOnPort(brokerPort, port => { 109 | brokerPort = port 110 | brokerConf = new KafkaConfig(brokerConfiguration) 111 | server = new KafkaServer(brokerConf) 112 | server.startup() 113 | (server, brokerPort) 114 | }, new SparkConf(), "KafkaBroker") 115 | 116 | brokerReady = true 117 | } 118 | 119 | /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ 120 | def setup(): Unit = { 121 | setupEmbeddedZookeeper() 122 | setupEmbeddedKafkaServer() 123 | } 124 | 125 | /** Teardown the whole servers, including Kafka broker and Zookeeper */ 126 | def teardown(): Unit = { 127 | brokerReady = false 128 | zkReady = false 129 | 130 | if (producer != null) { 131 | producer.close() 132 | producer = null 133 | } 134 | 135 | if (server != null) { 136 | server.shutdown() 137 | server = null 138 | } 139 | 140 | brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } 141 | 142 | if (zkClient != null) { 143 | zkClient.close() 144 | zkClient = null 145 | } 146 | 147 | if (zookeeper != null) { 148 | zookeeper.shutdown() 149 | zookeeper = null 150 | } 151 | } 152 | 153 | /** Create a Kafka topic and wait until it is propagated to the whole cluster */ 154 | def createTopic(topic: String, partitions: Int): Unit = { 155 | AdminUtils.createTopic(zkClient, topic, partitions, 1) 156 | // wait until metadata is propagated 157 | (0 until partitions).foreach { p => 158 | waitUntilMetadataIsPropagated(topic, p) 159 | } 160 | } 161 | 162 | /** Create a Kafka topic and wait until it is propagated to the whole cluster */ 163 | def createTopic(topic: String): Unit = { 164 | createTopic(topic, 1) 165 | } 166 | 167 | /** Java-friendly function for sending messages to the Kafka broker */ 168 | def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { 169 | sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) 170 | } 171 | 172 | /** Send the messages to the Kafka broker */ 173 | def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { 174 | val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray 175 | sendMessages(topic, messages) 176 | } 177 | 178 | /** Send the array of messages to the Kafka broker */ 179 | def sendMessages(topic: String, messages: Array[String]): Unit = { 180 | producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) 181 | producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) 182 | producer.close() 183 | producer = null 184 | } 185 | 186 | private def brokerConfiguration: Properties = { 187 | val props = new Properties() 188 | props.put("broker.id", "0") 189 | props.put("host.name", "localhost") 190 | props.put("port", brokerPort.toString) 191 | props.put("log.dir", Utils.createTempDir().getAbsolutePath) 192 | props.put("zookeeper.connect", zkAddress) 193 | props.put("log.flush.interval.messages", "1") 194 | props.put("replica.socket.timeout.ms", "1500") 195 | props 196 | } 197 | 198 | private def producerConfiguration: Properties = { 199 | val props = new Properties() 200 | props.put("metadata.broker.list", brokerAddress) 201 | props.put("serializer.class", classOf[StringEncoder].getName) 202 | props.put("request.required.acks", "-1") 203 | props 204 | } 205 | 206 | private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { 207 | def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { 208 | case Some(partitionState) => 209 | val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr 210 | 211 | ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && 212 | Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && 213 | leaderAndInSyncReplicas.isr.size >= 1 214 | 215 | case _ => 216 | false 217 | } 218 | eventually(timeout(10.seconds)) { 219 | assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") 220 | } 221 | } 222 | 223 | private class EmbeddedZookeeper(val zkConnect: String) { 224 | val snapshotDir = Utils.createTempDir() 225 | val logDir = Utils.createTempDir() 226 | 227 | val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) 228 | val (ip, port) = { 229 | val splits = zkConnect.split(":") 230 | (splits(0), splits(1).toInt) 231 | } 232 | val factory = new NIOServerCnxnFactory() 233 | factory.configure(new InetSocketAddress(ip, port), 16) 234 | factory.startup(zookeeper) 235 | 236 | val actualPort = factory.getLocalPort 237 | 238 | def shutdown() { 239 | factory.shutdown() 240 | Utils.deleteRecursively(snapshotDir) 241 | Utils.deleteRecursively(logDir) 242 | } 243 | } 244 | } 245 | 246 | --------------------------------------------------------------------------------