├── run_tests.sh ├── event-hubs_spark.png ├── core ├── src │ ├── main │ │ ├── resources │ │ │ ├── eventhubs-source-offset-version-2.1.0.txt │ │ │ └── META-INF │ │ │ │ └── services │ │ │ │ └── org.apache.spark.sql.sources.DataSourceRegister │ │ └── scala │ │ │ └── org │ │ │ └── apache │ │ │ └── spark │ │ │ ├── eventhubs │ │ │ ├── client │ │ │ │ ├── ClientThreadPool.scala │ │ │ │ ├── ClientConnectionPool.scala │ │ │ │ ├── Client.scala │ │ │ │ └── EventHubsClient.scala │ │ │ ├── NameAndPartition.scala │ │ │ ├── rdd │ │ │ │ ├── EventHubsRDDPartition.scala │ │ │ │ ├── OffsetRange.scala │ │ │ │ └── EventHubsRDD.scala │ │ │ ├── package.scala │ │ │ ├── EventHubsUtils.scala │ │ │ ├── EventPosition.scala │ │ │ └── utils │ │ │ │ └── EventHubsTestUtils.scala │ │ │ ├── sql │ │ │ └── eventhubs │ │ │ │ ├── EventHubsSink.scala │ │ │ │ ├── EventHubsSourceOffset.scala │ │ │ │ ├── EventHubsWriter.scala │ │ │ │ ├── EventHubsRelation.scala │ │ │ │ ├── JsonUtils.scala │ │ │ │ ├── EventHubsWriteTask.scala │ │ │ │ ├── EventHubsSourceProvider.scala │ │ │ │ └── EventHubsSource.scala │ │ │ └── streaming │ │ │ └── eventhubs │ │ │ └── EventHubsDirectDStream.scala │ └── test │ │ ├── resources │ │ ├── eventhubs-source-offset-version-2.1.0.txt │ │ └── log4j.properties │ │ └── scala │ │ └── org │ │ └── apache │ │ └── spark │ │ ├── eventhubs │ │ ├── client │ │ │ └── EventHubsClientSuite.scala │ │ ├── EventPositionSuite.scala │ │ ├── ConnectionStringBuilderSuite.scala │ │ ├── rdd │ │ │ └── EventHubsRDDSuite.scala │ │ ├── EventHubsConfSuite.scala │ │ └── utils │ │ │ └── EventHubsTestUtilsSuite.scala │ │ ├── sql │ │ └── eventhubs │ │ │ ├── JsonUtilsSuite.scala │ │ │ ├── EventHubsRelationSuite.scala │ │ │ ├── EventHubsSourceOffsetSuite.scala │ │ │ └── EventHubsSinkSuite.scala │ │ └── streaming │ │ └── eventhubs │ │ └── EventHubsDirectDStreamSuite.scala └── pom.xml ├── project └── plugins.sbt ├── .travis.yml ├── .gitignore ├── .scalafmt.conf ├── .github ├── ISSUE_TEMPLATE.md ├── PULL_REQUEST_TEMPLATE.md └── CONTRIBUTING.md ├── docs ├── README.md └── misc │ └── install_spark_on_windows.md ├── README.md ├── pom.xml └── LICENSE /run_tests.sh: -------------------------------------------------------------------------------- 1 | set -e 2 | mvn clean package 3 | -------------------------------------------------------------------------------- /event-hubs_spark.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/tikyau/azure-event-hubs-spark/master/event-hubs_spark.png -------------------------------------------------------------------------------- /core/src/main/resources/eventhubs-source-offset-version-2.1.0.txt: -------------------------------------------------------------------------------- 1 | {"ehName1":{"0":456,"1":789},"ehName2":{"0":0}} -------------------------------------------------------------------------------- /core/src/test/resources/eventhubs-source-offset-version-2.1.0.txt: -------------------------------------------------------------------------------- 1 | {"ehName1":{"0":456,"1":789},"ehName2":{"0":0}} -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | scalaVersion := "2.11.8" 2 | 3 | addSbtPlugin("org.lucidchart" %% "sbt-scalafmt" % "1.12") 4 | -------------------------------------------------------------------------------- /core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | org.apache.spark.sql.eventhubs.EventHubsSourceProvider -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | scala: 3 | - 2.11.8 4 | 5 | jdk: 6 | - oraclejdk8 7 | 8 | script: 9 | - ./run_tests.sh 10 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .git 2 | .classpath 3 | .project 4 | .settings/ 5 | .idea/ 6 | target/ 7 | *.iml 8 | scalafmt-output.xml 9 | dependency-reduced-pom.xml 10 | metastore_db 11 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | maxColumn = 100 2 | project.git = true 3 | project.excludeFilters = [] 4 | 5 | # http://docs.scala-lang.org/style/scaladoc.html recommends the JavaDoc style. 6 | # scala/scala is written that way too https://github.com/scala/scala/blob/v2.12.2/src/library/scala/Predef.scala 7 | docstrings = JavaDoc 8 | 9 | # This also seems more idiomatic to include whitespace in import x.{ yyy } 10 | spaces.inImportCurlyBraces = true 11 | 12 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE.md: -------------------------------------------------------------------------------- 1 | Thanks for filing an issue with us! Below are some guidelines when filing an issue. In general, the more detail the better! 2 | 3 | Feature Requests: 4 | 5 | - What issue are you trying to solve? 6 | - How do you want to solve it? 7 | - What is your use case for this feature? 8 | 9 | Bug Report: 10 | 11 | - Actual behavior 12 | - Expected behavior 13 | - Spark version 14 | - spark-eventhubs artifactId and version 15 | -------------------------------------------------------------------------------- /.github/PULL_REQUEST_TEMPLATE.md: -------------------------------------------------------------------------------- 1 | Thanks for contributing! We appreciate it :) 2 | 3 | For a Pull Request to be accepted, you must: 4 | - Run scalafmt on your code using the `.scalafmt.conf` present in this project 5 | - All tests must pass when you run `mvn clean test` 6 | 7 | Just in case, here are some tips that could prove useful when opening a pull request: 8 | - Read the [Contributor's Guide](CONTRIBUTING.md) 9 | - Make the title of pull request is clear and informative. 10 | - There should be a small number of commits, all with informative messages. 11 | - The pull request shouldn't introduce any breaking changes (unless will occur on the next release) 12 | - Any public code should be properly documented 13 | - Be sure to write tests for any changes in the pull request 14 | - The code should build without any errors -------------------------------------------------------------------------------- /docs/README.md: -------------------------------------------------------------------------------- 1 | # EventHub Connector Documentation 2 | 3 | Hello! This connector supports Strucutred Streaming and Spark Streaming. For documentation on using this connector, please read our integration guides. 4 | 5 | - [Structured Streaming + Event Hubs Integration Guide](structured-streaming-eventhubs-integration.md) 6 | - [Spark Streaming + Event Hubs Integration Guide](spark-streaming-eventhubs-integration.md) 7 | 8 | Additionally, here are some links to documentation on Event Hubs, Spark, and Databricks: 9 | 10 | - [Azure Event Hubs on Databricks](https://docs.azuredatabricks.net/spark/latest/data-sources/azure/eventhubs-connector.html) 11 | - [Structured Streaming Programming Guide](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html) 12 | - [Spark Streaming Programming Guide](https://spark.apache.org/docs/latest/streaming-programming-guide.html) 13 | - [Event Hubs Documentation](https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-what-is-event-hubs) 14 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/client/ClientThreadPool.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.eventhubs.client 19 | 20 | import java.util.concurrent.{ ExecutorService, Executors } 21 | 22 | /** 23 | * Thread pool for EventHub client. We create one thread pool per JVM. 24 | * Threads are created on demand if none are available. 25 | * In future releases, thread pool will be configurable by users. 26 | */ 27 | object ClientThreadPool { 28 | val pool: ExecutorService = Executors.newCachedThreadPool 29 | } 30 | -------------------------------------------------------------------------------- /core/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 -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/NameAndPartition.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.eventhubs 19 | 20 | import org.json4s.jackson.Serialization 21 | 22 | case class NameAndPartition(ehName: String, partitionId: Int) extends Serializable { self => 23 | 24 | override def equals(obj: scala.Any): Boolean = obj match { 25 | case that: NameAndPartition => 26 | this.ehName == that.ehName && 27 | this.partitionId == that.partitionId 28 | case _ => false 29 | } 30 | 31 | override def toString: String = { 32 | Serialization.write(self) 33 | } 34 | 35 | override def hashCode: Rate = { 36 | toTuple.hashCode() 37 | } 38 | 39 | def toTuple: (String, Int) = (ehName, partitionId) 40 | } 41 | 42 | object NameAndPartition { 43 | def fromString(str: String): NameAndPartition = { 44 | Serialization.read[NameAndPartition](str) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/client/EventHubsClientSuite.scala: -------------------------------------------------------------------------------- 1 | /** 2 | * Here we need to test all functionality against an EventHubs instance. 3 | * Add a testing context with EventHub info, and then make calls against that instance. 4 | */ 5 | /* 6 | * Licensed to the Apache Software Foundation (ASF) under one or more 7 | * contributor license agreements. See the NOTICE file distributed with 8 | * this work for additional information regarding copyright ownership. 9 | * The ASF licenses this file to You under the Apache License, Version 2.0 10 | * (the "License"); you may not use this file except in compliance with 11 | * the License. You may obtain a copy of the License at 12 | * 13 | * http://www.apache.org/licenses/LICENSE-2.0 14 | * 15 | * Unless required by applicable law or agreed to in writing, software 16 | * distributed under the License is distributed on an "AS IS" BASIS, 17 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 18 | * See the License for the specific language governing permissions and 19 | * limitations under the License. 20 | */ 21 | 22 | package org.apache.spark.eventhubs.client 23 | 24 | import org.scalatest.mock.MockitoSugar 25 | import org.scalatest.{ BeforeAndAfter, FunSuite } 26 | 27 | /** 28 | * Test suite for EventHubsClient 29 | */ 30 | class EventHubsClientSuite extends FunSuite with BeforeAndAfter with MockitoSugar { 31 | // TODO: add tests for driver-side translation 32 | // Seems we can implement "translate" in our EventHubsTestUtils. I'll do that soon! 33 | 34 | test("EventHubsClient converts parameters correctly when offset was previously saved") {} 35 | 36 | test("EventHubsClient converts parameters for consumergroup") {} 37 | 38 | test("EventHubsClient converts parameters for enqueuetime filter") {} 39 | } 40 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/eventhubs/JsonUtilsSuite.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.sql.eventhubs 19 | 20 | import org.apache.spark.SparkFunSuite 21 | import org.apache.spark.eventhubs.NameAndPartition 22 | 23 | class JsonUtilsSuite extends SparkFunSuite { 24 | 25 | test("parsing partitions") { 26 | val parsed = JsonUtils.partitions("""{"nameA":[0,1],"nameB":[4,6]}""") 27 | val expected = Array( 28 | new NameAndPartition("nameA", 0), 29 | new NameAndPartition("nameA", 1), 30 | new NameAndPartition("nameB", 4), 31 | new NameAndPartition("nameB", 6) 32 | ) 33 | assert(parsed.toSeq === expected.toSeq) 34 | } 35 | 36 | test("parsing partitionSeqNos") { 37 | val parsed = JsonUtils.partitionSeqNos("""{"nameA":{"0":23,"1":-1},"nameB":{"0":-2}}""") 38 | 39 | assert(parsed(new NameAndPartition("nameA", 0)) === 23) 40 | assert(parsed(new NameAndPartition("nameA", 1)) === -1) 41 | assert(parsed(new NameAndPartition("nameB", 0)) === -2) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/rdd/EventHubsRDDPartition.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.eventhubs.rdd 19 | 20 | import org.apache.spark.Partition 21 | import org.apache.spark.eventhubs.NameAndPartition 22 | import org.apache.spark.eventhubs.{ PartitionId, SequenceNumber } 23 | 24 | private class EventHubsRDDPartition(val index: Int, 25 | val nameAndPartition: NameAndPartition, 26 | val fromSeqNo: SequenceNumber, 27 | val untilSeqNo: SequenceNumber, 28 | val preferredLoc: Option[String]) 29 | extends Partition { 30 | 31 | /** Number of messages this partition refers to */ 32 | def count: Long = untilSeqNo - fromSeqNo 33 | 34 | /** The EventHubs name corresponding to this RDD Partition */ 35 | def name: String = nameAndPartition.ehName 36 | 37 | /** The EventHubs partitionId corresponding to this RDD Partition */ 38 | def partitionId: PartitionId = nameAndPartition.partitionId 39 | } 40 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsSink.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.sql.eventhubs 19 | 20 | import org.apache.spark.eventhubs.EventHubsConf 21 | import org.apache.spark.eventhubs.client.Client 22 | import org.apache.spark.internal.Logging 23 | import org.apache.spark.sql.{ DataFrame, SQLContext } 24 | import org.apache.spark.sql.execution.streaming.Sink 25 | 26 | private[eventhubs] class EventHubsSink(sqlContext: SQLContext, 27 | parameters: Map[String, String], 28 | clientFactory: (EventHubsConf) => Client) 29 | extends Sink 30 | with Logging { 31 | 32 | @volatile private var latestBatchId = -1L 33 | 34 | override def toString: String = "EventHubsSink" 35 | 36 | override def addBatch(batchId: Long, data: DataFrame): Unit = { 37 | if (batchId <= latestBatchId) { 38 | logInfo(s"Skipping already committed batch $batchId") 39 | } else { 40 | EventHubsWriter.write(sqlContext.sparkSession, data.queryExecution, parameters, clientFactory) 41 | latestBatchId = batchId 42 | } 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /core/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 22 | 4.0.0 23 | 24 | com.microsoft.azure 25 | azure-eventhubs-spark-parent_2.11 26 | 2.3.1 27 | ../pom.xml 28 | 29 | azure-eventhubs-spark_2.11 30 | jar 31 | 32 | 33 | 34 | org.apache.spark 35 | spark-catalyst_${scala.binary.version} 36 | ${spark.version} 37 | test-jar 38 | test 39 | 40 | 41 | 42 | 43 | 44 | 45 | org.apache.maven.plugins 46 | maven-assembly-plugin 47 | 48 | false 49 | 50 | 51 | 52 | 53 | 54 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsSourceOffset.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.sql.eventhubs 19 | 20 | import org.apache.spark.eventhubs.{ NameAndPartition, SequenceNumber } 21 | import org.apache.spark.sql.execution.streaming.{ Offset, SerializedOffset } 22 | 23 | private[eventhubs] case class EventHubsSourceOffset( 24 | partitionToSeqNos: Map[NameAndPartition, SequenceNumber]) 25 | extends Offset { 26 | 27 | override val json: String = JsonUtils.partitionSeqNos(partitionToSeqNos) 28 | } 29 | 30 | private[eventhubs] object EventHubsSourceOffset { 31 | 32 | def getPartitionSeqNos(offset: Offset): Map[NameAndPartition, SequenceNumber] = { 33 | offset match { 34 | case o: EventHubsSourceOffset => o.partitionToSeqNos 35 | case so: SerializedOffset => EventHubsSourceOffset(so).partitionToSeqNos 36 | case _ => 37 | throw new IllegalArgumentException( 38 | s"Invalid conversion from offset of ${offset.getClass} to EventHubsSourceOffset") 39 | } 40 | } 41 | 42 | def apply(offsetTuples: (String, Int, SequenceNumber)*): EventHubsSourceOffset = { 43 | EventHubsSourceOffset( 44 | offsetTuples.map { case (n, p, s) => (new NameAndPartition(n, p), s) }.toMap) 45 | } 46 | 47 | def apply(offset: SerializedOffset): EventHubsSourceOffset = { 48 | EventHubsSourceOffset(JsonUtils.partitionSeqNos(offset.json)) 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/EventPositionSuite.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.eventhubs 19 | 20 | import java.time.Instant 21 | 22 | import org.scalatest.FunSuite 23 | 24 | class EventPositionSuite extends FunSuite { 25 | 26 | test("convert - offset") { 27 | val actual = EventPosition.fromOffset("123456789").convert 28 | val expected = com.microsoft.azure.eventhubs.EventPosition.fromOffset("123456789", true) 29 | assert(actual.toString === expected.toString) 30 | } 31 | 32 | test("convert - seq no") { 33 | val actual = EventPosition.fromSequenceNumber(42L).convert 34 | val expected = com.microsoft.azure.eventhubs.EventPosition.fromSequenceNumber(42L, true) 35 | assert(actual.toString === expected.toString) 36 | } 37 | 38 | test("convert - enqueued time") { 39 | val instant = Instant.parse("2007-12-03T10:15:30.00Z") 40 | val actual = EventPosition.fromEnqueuedTime(instant).convert 41 | val expected = com.microsoft.azure.eventhubs.EventPosition.fromEnqueuedTime(instant) 42 | assert(actual.toString === expected.toString) 43 | } 44 | 45 | test("convert - start of stream") { 46 | val actual = EventPosition.fromStartOfStream.convert 47 | val expected = com.microsoft.azure.eventhubs.EventPosition.fromStartOfStream() 48 | assert(actual.toString === expected.toString) 49 | } 50 | 51 | test("convert - end of stream") { 52 | val actual = EventPosition.fromEndOfStream.convert 53 | val expected = com.microsoft.azure.eventhubs.EventPosition.fromEndOfStream() 54 | assert(actual.toString === expected.toString) 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/package.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.time.Duration 21 | import java.util.concurrent.{ ExecutorService, Executors } 22 | 23 | import com.microsoft.azure.eventhubs.{ EventHubClient, PartitionReceiver } 24 | import org.json4s.NoTypeHints 25 | import org.json4s.jackson.Serialization 26 | 27 | package object eventhubs { 28 | 29 | implicit val formats = Serialization.formats(NoTypeHints) 30 | 31 | val DefaultEventPosition: EventPosition = EventPosition.fromEndOfStream 32 | val DefaultEndingPosition: EventPosition = EventPosition.fromEndOfStream 33 | val DefaultMaxRatePerPartition: Rate = 1000 34 | val DefaultReceiverTimeout: Duration = Duration.ofSeconds(60) 35 | val DefaultOperationTimeout: Duration = Duration.ofSeconds(60) 36 | val DefaultConsumerGroup: String = EventHubClient.DEFAULT_CONSUMER_GROUP_NAME 37 | val PrefetchCountMinimum: Int = PartitionReceiver.MINIMUM_PREFETCH_COUNT 38 | val DefaultFailOnDataLoss = "true" 39 | val DefaultUseSimulatedClient = "false" 40 | val StartingSequenceNumber = 0L 41 | 42 | type PartitionId = Int 43 | val PartitionId = Int 44 | 45 | type Rate = Int 46 | val Rate = Int 47 | 48 | type Offset = Long 49 | val Offset = Long 50 | 51 | type EnqueueTime = Long 52 | val EnqueueTime = Long 53 | 54 | type SequenceNumber = Long 55 | val SequenceNumber = Long 56 | 57 | // Allow Strings to be converted to types defined in this library. 58 | implicit class EventHubsString(val str: String) extends AnyVal { 59 | def toPartitionId: PartitionId = str.toInt 60 | def toRate: Rate = str.toInt 61 | def toOffset: Offset = str.toLong 62 | def toEnqueueTime: EnqueueTime = str.toLong 63 | def toSequenceNumber: SequenceNumber = str.toLong 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /docs/misc/install_spark_on_windows.md: -------------------------------------------------------------------------------- 1 | Install Spark on Windows 2 | =================== 3 | 4 | This guide will help you install Apache Spark on your local Windows machine. 5 | 6 | ---------- 7 | 8 | ### Install Java 9 | - Install Java 8 or later from Oracle. 10 | - To verify the installation is complete, open a command prompt, type ```java -version```, and hit enter. If you see the Java version print out, you're all set. 11 | 12 | >**Note:** 13 | >- Take a second to make sure ```JAVA_HOME``` is set in your environment variables. The jdk is usually located here: ```C:\Program Files\Java\jdk```. 14 | >- Add ```%JAVA_HOME%\bin``` to your ```PATH``` environment variable if it's not there already. 15 | 16 | 17 | ### Install Scala 18 | - Download and install Scala. 19 | - Set ```SCALA_HOME``` in your environment variables. Typically, scala is installed to ```C:\Program Files (x86)\scala\```. 20 | - Add ```%SCALA_HOME%\bin``` to your ```PATH``` environment variable. 21 | 22 | ### Install Python 23 | - Install Python 2.6 or later 24 | 25 | ### Download SBT 26 | - Download and install SBT. 27 | - Set ```SBT_HOME``` as an environment variable with the SBT installation's path. Typically it's ```C:\Program Files (x86)\sbt\```. 28 | 29 | ### Download ```winutils.exe``` 30 | - Download ```winutils.exe``` from the HortonWorks repo or 31 | git repo. 32 | - Create a folder with the following path: ```C:\Program Files\Hadoop\``` and place ```winutils.exe``` in this new folder. 33 | - Set ```HADOOP_HOME = C:\Program Files\Hadoop\``` in your environment variables. 34 | 35 | ### Download Apache Spark 36 | - Download a **pre-built** Spark package 37 | - Extract your download. We recommend extracting to ```C:\```, and Spark would be located at ```C:\spark-X.X.X-bin-hadoop2.X\``` 38 | - Set ```SPARK_HOME``` to Spark's file path 39 | - Add ```%SPARK_HOME%\bin``` to your PATH environment variable. 40 | 41 | >Note: 42 | >- The Spark-EventHubs Connector only works with Apache Spark 2.1.x and earlier! 43 | >- Your ```SPARK_HOME``` path cannot have any spaces in it - spaces cause issues when starting Spark. 44 | >For instance, ```Program Files``` cannot be apart of ```SPARK_HOME``` due to the space character in the folder name! 45 | 46 | ### You're All Set! 47 | - Open a command prompt and run ```spark-shell``` 48 | - Open http://localhost:4040/ to view the Spark Web UI 49 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsWriter.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.sql.eventhubs 19 | 20 | import org.apache.spark.eventhubs.EventHubsConf 21 | import org.apache.spark.eventhubs.client.Client 22 | import org.apache.spark.internal.Logging 23 | import org.apache.spark.sql.{ AnalysisException, SparkSession } 24 | import org.apache.spark.sql.catalyst.expressions.Attribute 25 | import org.apache.spark.sql.execution.QueryExecution 26 | import org.apache.spark.sql.types.{ BinaryType, StringType } 27 | import org.apache.spark.util.Utils 28 | 29 | /** 30 | * The [[EventHubsWriter]] class is used to write data from a batch query 31 | * or structured streaming query, given by a [[QueryExecution]], to EventHubs. 32 | */ 33 | private[eventhubs] object EventHubsWriter extends Logging { 34 | 35 | val BodyAttributeName = "body" 36 | val PartitionKeyAttributeName = "partitionKey" 37 | val PartitionIdAttributeName = "partitionId" 38 | 39 | override def toString: String = "EventHubsWriter" 40 | 41 | private def validateQuery(schema: Seq[Attribute], parameters: Map[String, String]): Unit = { 42 | schema 43 | .find(_.name == BodyAttributeName) 44 | .getOrElse( 45 | throw new AnalysisException(s"Required attribute '$BodyAttributeName' not found.") 46 | ) 47 | .dataType match { 48 | case StringType | BinaryType => // good 49 | case _ => 50 | throw new AnalysisException( 51 | s"$BodyAttributeName attribute type " + 52 | s"must be a String or BinaryType.") 53 | } 54 | } 55 | 56 | def write( 57 | sparkSession: SparkSession, 58 | queryExecution: QueryExecution, 59 | parameters: Map[String, String], 60 | clientFactory: (EventHubsConf) => Client 61 | ): Unit = { 62 | val schema = queryExecution.analyzed.output 63 | validateQuery(schema, parameters) 64 | queryExecution.toRdd.foreachPartition { iter => 65 | val writeTask = new EventHubsWriteTask(parameters, schema, clientFactory) 66 | Utils.tryWithSafeFinally(block = writeTask.execute(iter))( 67 | finallyBlock = writeTask.close() 68 | ) 69 | } 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/rdd/OffsetRange.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.eventhubs.rdd 19 | 20 | import org.apache.spark.eventhubs.NameAndPartition 21 | import org.apache.spark.eventhubs._ 22 | 23 | import scala.language.implicitConversions 24 | 25 | trait HasOffsetRanges { 26 | def offsetRanges: Array[OffsetRange] 27 | } 28 | 29 | final class OffsetRange(val nameAndPartition: NameAndPartition, 30 | val fromSeqNo: SequenceNumber, 31 | val untilSeqNo: SequenceNumber, 32 | val preferredLoc: Option[String]) 33 | extends Serializable { 34 | import OffsetRange.OffsetRangeTuple 35 | 36 | def name: String = nameAndPartition.ehName 37 | 38 | def partitionId: Int = nameAndPartition.partitionId 39 | 40 | def count: Long = untilSeqNo - fromSeqNo 41 | 42 | override def equals(obj: Any): Boolean = obj match { 43 | case that: OffsetRange => 44 | this.name == that.name && 45 | this.partitionId == that.partitionId && 46 | this.fromSeqNo == that.fromSeqNo && 47 | this.untilSeqNo == that.untilSeqNo 48 | case _ => false 49 | } 50 | 51 | override def hashCode(): Rate = { 52 | toTuple.hashCode() 53 | } 54 | 55 | def toTuple: OffsetRangeTuple = (nameAndPartition, fromSeqNo, untilSeqNo, preferredLoc) 56 | 57 | override def toString = 58 | s"OffsetRange(partitionId: ${nameAndPartition.partitionId} | fromSeqNo: $fromSeqNo | untilSeqNo: $untilSeqNo)" 59 | } 60 | 61 | object OffsetRange { 62 | type OffsetRangeTuple = (NameAndPartition, SequenceNumber, SequenceNumber, Option[String]) 63 | 64 | def apply(name: String, 65 | partitionId: PartitionId, 66 | fromSeq: SequenceNumber, 67 | untilSeq: SequenceNumber, 68 | preferredLoc: Option[String]): OffsetRange = { 69 | OffsetRange(NameAndPartition(name, partitionId), fromSeq, untilSeq, preferredLoc) 70 | } 71 | 72 | def apply(nAndP: NameAndPartition, 73 | fromSeq: SequenceNumber, 74 | untilSeq: SequenceNumber, 75 | preferredLoc: Option[String]): OffsetRange = { 76 | new OffsetRange(nAndP, fromSeq, untilSeq, preferredLoc) 77 | } 78 | 79 | def apply(tuple: OffsetRangeTuple): OffsetRange = { 80 | tupleToOffsetRange(tuple) 81 | } 82 | 83 | implicit def tupleToOffsetRange(tuple: OffsetRangeTuple): OffsetRange = 84 | OffsetRange(tuple._1, tuple._2, tuple._3, tuple._4) 85 | 86 | implicit def tupleListToOffsetRangeList(list: List[OffsetRangeTuple]): List[OffsetRange] = 87 | for { tuple <- list } yield tupleToOffsetRange(tuple) 88 | } 89 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/ConnectionStringBuilderSuite.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.eventhubs 19 | 20 | import java.time.Duration 21 | 22 | import org.scalatest.FunSuite 23 | 24 | class ConnectionStringBuilderSuite extends FunSuite { 25 | 26 | import ConnectionStringBuilderSuite._ 27 | 28 | private val validateConnStrBuilder = (connStrBuilder: ConnectionStringBuilder) => { 29 | assert(connStrBuilder.getEventHubName == CorrectEntityPath) 30 | assert(connStrBuilder.getEndpoint.getHost == CorrectEndpoint) 31 | assert(connStrBuilder.getSasKey == CorrectKey) 32 | assert(connStrBuilder.getSasKeyName == CorrectKeyName) 33 | assert(connStrBuilder.getOperationTimeout == CorrectOperationTimeout) 34 | } 35 | 36 | test("parse invalid connection string") { 37 | intercept[Exception] { 38 | ConnectionStringBuilder("something") 39 | } 40 | } 41 | 42 | test("throw on unrecognized parts") { 43 | intercept[Exception] { 44 | ConnectionStringBuilder(correctConnectionString + ";" + "something") 45 | } 46 | } 47 | 48 | test("parse valid connection string") { 49 | val connStrBuilder = ConnectionStringBuilder(correctConnectionString) 50 | validateConnStrBuilder(connStrBuilder) 51 | } 52 | 53 | test("exchange connection string across constructors") { 54 | val connStrBuilder = ConnectionStringBuilder(correctConnectionString) 55 | val secondConnStr = ConnectionStringBuilder() 56 | .setEndpoint(connStrBuilder.getEndpoint) 57 | .setEventHubName(connStrBuilder.getEventHubName) 58 | .setSasKeyName(connStrBuilder.getSasKeyName) 59 | .setSasKey(connStrBuilder.getSasKey) 60 | secondConnStr.setOperationTimeout(connStrBuilder.getOperationTimeout) 61 | validateConnStrBuilder(ConnectionStringBuilder(secondConnStr.toString)) 62 | } 63 | 64 | test("property setters") { 65 | val connStrBuilder = ConnectionStringBuilder(correctConnectionString) 66 | val testConnStrBuilder = ConnectionStringBuilder(connStrBuilder.toString) 67 | validateConnStrBuilder(testConnStrBuilder) 68 | connStrBuilder.setOperationTimeout(Duration.ofSeconds(8)) 69 | val testConnStrBuilder1 = ConnectionStringBuilder(connStrBuilder.toString) 70 | assert(testConnStrBuilder1.getOperationTimeout.getSeconds == 8) 71 | } 72 | } 73 | 74 | object ConnectionStringBuilderSuite { 75 | private val CorrectEndpoint = "endpoint1" 76 | private val CorrectEntityPath = "eventhub1" 77 | private val CorrectKeyName = "somekeyname" 78 | private val CorrectKey = "somekey" 79 | private val CorrectOperationTimeout = Duration.ofSeconds(5) 80 | 81 | private val correctConnectionString = 82 | s"Endpoint=sb://$CorrectEndpoint;EntityPath=$CorrectEntityPath;SharedAccessKeyName=$CorrectKeyName;" + 83 | s"SharedAccessKey=$CorrectKey;OperationTimeout=$CorrectOperationTimeout;" 84 | } 85 | -------------------------------------------------------------------------------- /.github/CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributor's Guide:
Azure Event Hubs + Apache Spark Connector 2 | 3 | ## Code of Conduct 4 | 5 | This project has adopted the [Microsoft Open Source Code of Conduct] 6 | (https://opensource.microsoft.com/codeofconduct/). For more information 7 | see the [Code of Conduct FAQ](https://opensource.microsoft.com/codeofconduct/faq/). 8 | 9 | ## Getting Started 10 | 11 | To build and test this locally, make sure you install: 12 | - Java 1.8 SDK 13 | - [Maven 3.x](https://maven.apache.org/download.cgi) (or [SBT version 1.x](https://www.scala-sbt.org/1.x/docs/index.html)) 14 | - A supported version of Apache Spark (see [Latest Releases](/README.md#latest-releases) for supported versions). 15 | 16 | After that, cloning the code and running `mvn clean package` should successfully 17 | run all unit/integration tests and build a JAR. 18 | 19 | ## Getting the Staging Version 20 | 21 | We also publish a staging version of the Azure EventHubs + Apache Spark connector 22 | in GitHub. To use the staging version, two things need to be added to your pom.xml. 23 | First add a new repository like so: 24 | 25 | ```XML 26 | 27 | azure-event-hubs-spark 28 | https://raw.github.com/Azure/azure-event-hubs-spark/maven-repo/ 29 | 30 | true 31 | always 32 | 33 | 34 | ``` 35 | 36 | Then add the following dependency declaration: 37 | 38 | ```XML 39 | 40 | com.microsoft.azure 41 | azure-eventhubs-spark_[2.XX] 42 | 2.3.1-SNAPSHOT 43 | 44 | ``` 45 | 46 | ### SBT Dependency 47 | 48 | // https://mvnrepository.com/artifact/com.microsoft.azure/azure-eventhubs-spark_2.11 49 | libraryDependencies += "com.microsoft.azure" %% "azure-eventhubs-spark" %% "2.3.1" 50 | 51 | ## Filing Issues 52 | 53 | You can find all of the issues that have been filed in the [Issues](https://github.com/Azure/spark-eventhubs/issues) 54 | section of the repository. 55 | 56 | If you encounter any bugs, would like to request a feature, or have general 57 | questions/concerns/comments, feel free to file an issue [here](https://github.com/Azure/spark-eventhubs/issues/new). 58 | **Don't hesitate to reach out!** 59 | 60 | ## Pull Requests 61 | 62 | ### Required Guidelines 63 | 64 | When filing a pull request, the following must be true: 65 | 66 | - Tests have been added (if needed) to validate changes 67 | - scalafmt (using the `.scalafmt.conf` in the repo) must be used to style the code 68 | - `mvn clean test` must run successfully 69 | 70 | ### General Guidelines 71 | 72 | If you would like to make changes to this library, **break up the change into small, 73 | logical, testable chunks, and organize your pull requests accordingly**. This makes 74 | for a cleaner, less error-prone development process. 75 | 76 | If you'd like to get involved, but don't know what to work on, then just reach out to 77 | us by opening an issue! All contributions/efforts are welcome :) 78 | 79 | If you're new to opening pull requests - or would like some additional guidance - the 80 | following list is a good set of best practices! 81 | 82 | - Title of the pull request is clear and informative 83 | - There are a small number of commits that each have an informative message 84 | - A description of the changes the pull request makes is included, and a reference to the bug/issue the pull request fixes is included, if applicable 85 | 86 | ### Testing Guidelines 87 | 88 | If you add code, make sure you add tests to validate your changes. Again, below is a 89 | list of best practices when contributing: 90 | 91 | - Pull request includes test coverage for the included changes 92 | - Test code should not contain hard coded values for resource names or similar values 93 | - Test should not use App.config files for settings 94 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/EventHubsUtils.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.eventhubs 19 | 20 | import com.microsoft.azure.eventhubs.EventData 21 | import org.apache.spark.eventhubs.client.EventHubsClient 22 | import org.apache.spark.eventhubs.rdd.{ EventHubsRDD, OffsetRange } 23 | import org.apache.spark.streaming.StreamingContext 24 | import org.apache.spark.streaming.eventhubs.EventHubsDirectDStream 25 | import org.apache.spark.SparkContext 26 | import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } 27 | import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } 28 | 29 | object EventHubsUtils { 30 | 31 | /** 32 | * Creates a Direct DStream which consumes from the Event Hubs instance 33 | * specified in the [[EventHubsConf]]. 34 | * 35 | * @param ssc the StreamingContext this DStream belongs to 36 | * @param ehConf the parameters for your EventHubs instance 37 | * @return An [[EventHubsDirectDStream]] 38 | */ 39 | def createDirectStream(ssc: StreamingContext, ehConf: EventHubsConf): EventHubsDirectDStream = { 40 | new EventHubsDirectDStream(ssc, ehConf, EventHubsClient.apply) 41 | } 42 | 43 | /** 44 | * Creates a Direct DStream which consumes from the Event Hubs instance 45 | * specified in the [[EventHubsConf]]. 46 | * 47 | * @param jssc the JavaStreamingContext this DStream belongs to 48 | * @param ehConf the parameters for your EventHubs instance 49 | * @return A [[JavaInputDStream]] containing [[EventData]] 50 | */ 51 | def createDirectStream(jssc: JavaStreamingContext, 52 | ehConf: EventHubsConf): JavaInputDStream[EventData] = { 53 | new JavaInputDStream(createDirectStream(jssc.ssc, ehConf)) 54 | } 55 | 56 | /** 57 | * Creates an RDD which is contains events from an EventHubs instance. 58 | * Starting and ending offsets are specified in advance. 59 | * 60 | * @param sc the SparkContext the RDD belongs to 61 | * @param ehConf contains EventHubs-specific configurations 62 | * @param offsetRanges offset ranges that define the EventHubs data belonging to this RDD 63 | * @return An [[EventHubsRDD]] 64 | * 65 | */ 66 | def createRDD(sc: SparkContext, 67 | ehConf: EventHubsConf, 68 | offsetRanges: Array[OffsetRange]): EventHubsRDD = { 69 | new EventHubsRDD(sc, ehConf, offsetRanges, EventHubsClient.apply) 70 | } 71 | 72 | /** 73 | * Creates an RDD which is contains events from an EventHubs instance. 74 | * Starting and ending offsets are specified in advance. 75 | * 76 | * @param jsc the JavaSparkContext the RDD belongs to 77 | * @param ehConf contains EventHubs-specific configurations 78 | * @param offsetRanges offset ranges that define the EventHubs data belonging to this RDD 79 | * @return A [[JavaRDD]] containing [[EventData]] 80 | * 81 | */ 82 | def createRDD(jsc: JavaSparkContext, 83 | ehConf: EventHubsConf, 84 | offsetRanges: Array[OffsetRange]): JavaRDD[EventData] = { 85 | new JavaRDD(createRDD(jsc.sc, ehConf, offsetRanges)) 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsRelation.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.sql.eventhubs 19 | 20 | import org.apache.spark.eventhubs.client.Client 21 | import org.apache.spark.eventhubs.rdd.{ EventHubsRDD, OffsetRange } 22 | import org.apache.spark.eventhubs.EventHubsConf 23 | import org.apache.spark.internal.Logging 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.sql.catalyst.InternalRow 26 | import org.apache.spark.sql.catalyst.util.DateTimeUtils 27 | import org.apache.spark.sql.{ Row, SQLContext } 28 | import org.apache.spark.sql.sources.{ BaseRelation, TableScan } 29 | import org.apache.spark.sql.types.StructType 30 | import org.apache.spark.unsafe.types.UTF8String 31 | 32 | import scala.language.postfixOps 33 | 34 | private[eventhubs] class EventHubsRelation(override val sqlContext: SQLContext, 35 | options: Map[String, String], 36 | clientFactory: (EventHubsConf => Client)) 37 | extends BaseRelation 38 | with TableScan 39 | with Logging { 40 | 41 | import org.apache.spark.eventhubs._ 42 | 43 | private val ehConf = EventHubsConf.toConf(options) 44 | 45 | override def schema: StructType = EventHubsSourceProvider.eventHubsSchema 46 | 47 | override def buildScan(): RDD[Row] = { 48 | val client = clientFactory(ehConf) 49 | val partitionCount: Int = client.partitionCount 50 | 51 | val fromSeqNos = client.translate(ehConf, partitionCount) 52 | val untilSeqNos = client.translate(ehConf, partitionCount, useStart = false) 53 | 54 | require(fromSeqNos.forall(f => f._2 >= 0L), 55 | "Currently only sequence numbers can be passed in your starting positions.") 56 | require(untilSeqNos.forall(u => u._2 >= 0L), 57 | "Currently only sequence numbers can be passed in your ending positions.") 58 | 59 | val offsetRanges = untilSeqNos.keySet.map { p => 60 | val fromSeqNo = fromSeqNos 61 | .getOrElse(p, throw new IllegalStateException(s"$p doesn't have a fromSeqNo")) 62 | val untilSeqNo = untilSeqNos(p) 63 | OffsetRange(ehConf.name, p, fromSeqNo, untilSeqNo, None) 64 | }.toArray 65 | client.close() 66 | 67 | logInfo( 68 | "GetBatch generating RDD of with offsetRanges: " + 69 | offsetRanges.sortBy(_.nameAndPartition.toString).mkString(", ")) 70 | 71 | val rdd = new EventHubsRDD(sqlContext.sparkContext, ehConf, offsetRanges, clientFactory) 72 | .mapPartitionsWithIndex { (p, iter) => 73 | { 74 | iter.map { ed => 75 | InternalRow( 76 | ed.getBytes, 77 | UTF8String.fromString(p.toString), 78 | UTF8String.fromString(ed.getSystemProperties.getOffset), 79 | ed.getSystemProperties.getSequenceNumber, 80 | DateTimeUtils.fromJavaTimestamp( 81 | new java.sql.Timestamp(ed.getSystemProperties.getEnqueuedTime.toEpochMilli)), 82 | UTF8String.fromString(ed.getSystemProperties.getPublisher), 83 | UTF8String.fromString(ed.getSystemProperties.getPartitionKey) 84 | ) 85 | } 86 | } 87 | } 88 | 89 | sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = false).rdd 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/JsonUtils.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.sql.eventhubs 19 | 20 | import org.apache.spark.eventhubs.{ NameAndPartition, _ } 21 | import org.json4s.NoTypeHints 22 | import org.json4s.jackson.Serialization 23 | 24 | import scala.collection.mutable 25 | import scala.util.control.NonFatal 26 | 27 | private object JsonUtils { 28 | private implicit val formats = Serialization.formats(NoTypeHints) 29 | 30 | /** 31 | * Read NameAndPartitions from json string 32 | */ 33 | def partitions(str: String): Array[NameAndPartition] = { 34 | try { 35 | Serialization 36 | .read[Map[String, Seq[PartitionId]]](str) 37 | .flatMap { 38 | case (name, parts) => 39 | parts.map { part => 40 | new NameAndPartition(name, part) 41 | } 42 | } 43 | .toArray 44 | } catch { 45 | case NonFatal(_) => 46 | throw new IllegalArgumentException( 47 | s"""Expected e.g. {"ehNameA":[0,1],"ehNameB":[0,1]}, got $str""") 48 | } 49 | } 50 | 51 | /** 52 | * Write NameAndPartitions as json string 53 | */ 54 | def partitions(partitions: Iterable[NameAndPartition]): String = { 55 | val result = new mutable.HashMap[String, List[PartitionId]] 56 | partitions.foreach { nAndP => 57 | val parts: List[PartitionId] = result.getOrElse(nAndP.ehName, Nil) 58 | result += nAndP.ehName -> (nAndP.partitionId :: parts) 59 | } 60 | Serialization.write(result) 61 | } 62 | 63 | /** 64 | * Write per-NameAndPartition seqNos as json string 65 | */ 66 | def partitionSeqNos(partitionSeqNos: Map[NameAndPartition, SequenceNumber]): String = { 67 | val result = new mutable.HashMap[String, mutable.HashMap[PartitionId, SequenceNumber]]() 68 | implicit val ordering = new Ordering[NameAndPartition] { 69 | override def compare(x: NameAndPartition, y: NameAndPartition): Int = { 70 | Ordering 71 | .Tuple2[String, PartitionId] 72 | .compare((x.ehName, x.partitionId), (y.ehName, y.partitionId)) 73 | } 74 | } 75 | val partitions = partitionSeqNos.keySet.toSeq.sorted // sort for more determinism 76 | partitions.foreach { nAndP => 77 | val seqNo = partitionSeqNos(nAndP) 78 | val parts = result.getOrElse(nAndP.ehName, new mutable.HashMap[PartitionId, SequenceNumber]) 79 | parts += nAndP.partitionId -> seqNo 80 | result += nAndP.ehName -> parts 81 | } 82 | Serialization.write(result) 83 | } 84 | 85 | /** 86 | * Read per-NameAndPartition seqNos from json string 87 | */ 88 | def partitionSeqNos(jsonStr: String): Map[NameAndPartition, SequenceNumber] = { 89 | try { 90 | Serialization.read[Map[String, Map[PartitionId, SequenceNumber]]](jsonStr).flatMap { 91 | case (name, partSeqNos) => 92 | partSeqNos.map { 93 | case (part, seqNo) => 94 | NameAndPartition(name, part) -> seqNo 95 | } 96 | } 97 | } catch { 98 | case NonFatal(_) => 99 | throw new IllegalArgumentException( 100 | s"failed to parse $jsonStr" + 101 | s"""Expected e.g. {"ehName":{"0":23,"1":-1},"ehNameB":{"0":-2}}""") 102 | } 103 | } 104 | } 105 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/eventhubs/EventHubsRelationSuite.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.sql.eventhubs 19 | 20 | import java.util.concurrent.atomic.AtomicInteger 21 | 22 | import org.apache.spark.eventhubs.{ EventHubsConf, EventPosition, NameAndPartition } 23 | import org.apache.spark.eventhubs.utils.EventHubsTestUtils 24 | import org.apache.spark.sql.{ DataFrame, QueryTest } 25 | import org.apache.spark.sql.test.SharedSQLContext 26 | import org.scalatest.BeforeAndAfter 27 | 28 | class EventHubsRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { 29 | 30 | import testImplicits._ 31 | 32 | private val eventhubId = new AtomicInteger(0) 33 | 34 | private var testUtils: EventHubsTestUtils = _ 35 | 36 | private def newEventHub(): String = s"eh-${eventhubId.getAndIncrement()}" 37 | 38 | private def getEventHubsConf(eh: String): EventHubsConf = testUtils.getEventHubsConf(eh) 39 | 40 | override def beforeAll(): Unit = { 41 | super.beforeAll() 42 | testUtils = new EventHubsTestUtils 43 | } 44 | 45 | override def afterAll(): Unit = { 46 | if (testUtils != null) { 47 | testUtils.destroyAllEventHubs() 48 | testUtils = null 49 | super.afterAll() 50 | } 51 | } 52 | 53 | private def createDF(ehConf: EventHubsConf): DataFrame = { 54 | spark.read 55 | .format("eventhubs") 56 | .options(ehConf.toMap) 57 | .load() 58 | .select($"body" cast "string") 59 | } 60 | 61 | private def createPositions(seqNo: Long, ehName: String, partitionCount: Int) = { 62 | (for { 63 | p <- 0 until partitionCount 64 | } yield NameAndPartition(ehName, p) -> EventPosition.fromSequenceNumber(seqNo)).toMap 65 | } 66 | 67 | test("default earliest to latest events") { 68 | val eh = newEventHub() 69 | testUtils.createEventHubs(eh, partitionCount = 3) 70 | testUtils.send(eh, 0, 0 to 9) 71 | testUtils.send(eh, 1, 10 to 19) 72 | testUtils.send(eh, 2, 20 to 29) 73 | 74 | val ehConf = getEventHubsConf(eh) 75 | .setStartingPositions(Map.empty) 76 | .setEndingPositions(Map.empty) 77 | 78 | val df = createDF(ehConf) 79 | checkAnswer(df, (0 to 29).map(_.toString).toDF) 80 | } 81 | 82 | test("explicit earliest to latest events") { 83 | val eh = newEventHub() 84 | testUtils.createEventHubs(eh, partitionCount = 3) 85 | testUtils.send(eh, 0, 0 to 9) 86 | testUtils.send(eh, 1, 10 to 19) 87 | testUtils.send(eh, 2, 20 to 29) 88 | 89 | val start = createPositions(0L, eh, partitionCount = 3) 90 | val end = createPositions(10L, eh, partitionCount = 3) 91 | 92 | val ehConf = getEventHubsConf(eh) 93 | .setStartingPositions(start) 94 | .setEndingPositions(end) 95 | 96 | val df = createDF(ehConf) 97 | checkAnswer(df, (0 to 29).map(_.toString).toDF) 98 | } 99 | 100 | test("reuse same dataframe in query") { 101 | val eh = newEventHub() 102 | testUtils.createEventHubs(eh, partitionCount = 1) 103 | testUtils.send(eh, 0, 0 to 10) 104 | 105 | val ehConf = getEventHubsConf(eh) 106 | .setStartingPositions(Map.empty) 107 | .setEndingPositions(Map.empty) 108 | 109 | val df = createDF(ehConf) 110 | checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/client/ClientConnectionPool.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.eventhubs.client 19 | 20 | import java.util.concurrent.ConcurrentLinkedQueue 21 | import java.util.concurrent.atomic.AtomicInteger 22 | 23 | import com.microsoft.azure.eventhubs.EventHubClient 24 | import org.apache.spark.eventhubs._ 25 | import org.apache.spark.internal.Logging 26 | 27 | /** 28 | * A connection pool for EventHubClients. If a connection isn't available in the pool, then 29 | * a new one is created. If a connection idles in the pool for 5 minutes, it will be closed. 30 | * 31 | * @param ehConf The Event Hubs configurations corresponding to this specific connection pool. 32 | */ 33 | private class ClientConnectionPool(val ehConf: EventHubsConf) extends Logging { 34 | 35 | private[this] val pool = new ConcurrentLinkedQueue[EventHubClient]() 36 | private[this] val count = new AtomicInteger(0) 37 | 38 | private def borrowClient: EventHubClient = { 39 | var client = pool.poll() 40 | if (client == null) { 41 | logInfo( 42 | s"No clients left to borrow. EventHub name: ${ehConf.name}. Creating client ${count.incrementAndGet()}") 43 | val connStr = ConnectionStringBuilder(ehConf.connectionString) 44 | connStr.setOperationTimeout(ehConf.operationTimeout.getOrElse(DefaultOperationTimeout)) 45 | while (client == null) { 46 | client = EventHubClient.createSync(connStr.toString, ClientThreadPool.pool) 47 | } 48 | } else { 49 | logInfo(s"Borrowing client. EventHub name: ${ehConf.name}") 50 | } 51 | logInfo(s"Available clients: {${pool.size}}. Total clients: ${count.get}") 52 | client 53 | } 54 | 55 | private def returnClient(client: EventHubClient): Unit = { 56 | pool.offer(client) 57 | logInfo( 58 | s"Client returned. EventHub name: ${ehConf.name}. Total clients: ${count.get}. Available clients: ${pool.size}") 59 | } 60 | } 61 | 62 | object ClientConnectionPool extends Logging { 63 | 64 | private def notInitializedMessage(name: String): String = { 65 | s"Connection pool is not initialized for EventHubs: $name" 66 | } 67 | 68 | type MutableMap[A, B] = scala.collection.mutable.HashMap[A, B] 69 | 70 | private[this] val pools = new MutableMap[String, ClientConnectionPool]() 71 | 72 | def isInitialized(name: String): Boolean = pools.synchronized { 73 | pools.get(name).isDefined 74 | } 75 | 76 | private def ensureInitialized(name: String): Unit = { 77 | if (!isInitialized(name)) { 78 | val message = notInitializedMessage(name) 79 | throw new IllegalStateException(message) 80 | } 81 | } 82 | 83 | private def get(name: String): ClientConnectionPool = pools.synchronized { 84 | pools.getOrElse(name, { 85 | val message = notInitializedMessage(name) 86 | throw new IllegalStateException(message) 87 | }) 88 | } 89 | 90 | def borrowClient(ehConf: EventHubsConf): EventHubClient = { 91 | val name = ehConf.name 92 | 93 | pools.synchronized { 94 | if (!isInitialized(name)) { 95 | pools.update(name, new ClientConnectionPool(ehConf)) 96 | } 97 | } 98 | 99 | val pool = get(name) 100 | pool.borrowClient 101 | } 102 | 103 | def returnClient(client: EventHubClient): Unit = { 104 | val name = client.getEventHubName 105 | ensureInitialized(name) 106 | val pool = get(name) 107 | pool.returnClient(client) 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/EventPosition.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.eventhubs 19 | 20 | import java.time.Instant 21 | import java.util.Date 22 | 23 | import com.microsoft.azure.eventhubs.{ EventPosition => ehep } 24 | 25 | /** 26 | * Defines a position of an event in an event hub partition. 27 | * The position can be an Offset, Sequence Number, or EnqueuedTime. 28 | * 29 | * This event is passed to the EventHubsConf to define a starting point for your Spark job. 30 | */ 31 | case class EventPosition private (offset: String = null, 32 | seqNo: Long = -1L, 33 | enqueuedTime: Date = null, 34 | isInclusive: Boolean = true) 35 | extends Serializable { 36 | 37 | private[eventhubs] def convert: ehep = { 38 | if (offset != null) { 39 | ehep.fromOffset(offset, isInclusive) 40 | } else if (seqNo >= 0L) { 41 | ehep.fromSequenceNumber(seqNo, isInclusive) 42 | } else if (enqueuedTime != null) { 43 | ehep.fromEnqueuedTime(enqueuedTime.toInstant) 44 | } else { 45 | throw new IllegalStateException("No position has been set.") 46 | } 47 | } 48 | 49 | override def equals(obj: Any): Boolean = { 50 | obj match { 51 | case that: EventPosition => 52 | this.offset == that.offset && 53 | this.seqNo == that.seqNo && 54 | this.enqueuedTime == that.enqueuedTime && 55 | this.isInclusive == that.isInclusive 56 | case _ => false 57 | } 58 | } 59 | } 60 | 61 | object EventPosition { 62 | private val StartOfStream: String = "-1" 63 | private val EndOfStream: String = "@latest" 64 | 65 | /** 66 | * Creates a position at the given offset. When using EventHubs with Spark, 67 | * starting positions are always inclusive. Ending positions are always exclusive. 68 | * @param offset is the byte offset of the event. 69 | * @return An [[EventPosition]] instance. 70 | */ 71 | def fromOffset(offset: String): EventPosition = { 72 | EventPosition(offset) 73 | } 74 | 75 | /** 76 | * Creates a position at the given sequence number. When using EventHubs with Spark, 77 | * starting positions are always inclusive. Ending positions are always exclusive. 78 | * 79 | * @param seqNo is the sequence number of the event. 80 | * @return An [[EventPosition]] instance. 81 | */ 82 | def fromSequenceNumber(seqNo: SequenceNumber): EventPosition = { 83 | require(seqNo >= 0L, "Please pass a positive sequence number.") 84 | EventPosition(seqNo = seqNo) 85 | } 86 | 87 | /** 88 | * Creates a position at the given [[Instant]] 89 | * 90 | * @param enqueuedTime is the enqueued time of the specified event. 91 | * @return An [[EventPosition]] instance. 92 | */ 93 | def fromEnqueuedTime(enqueuedTime: Instant): EventPosition = { 94 | EventPosition(enqueuedTime = Date.from(enqueuedTime)) 95 | } 96 | 97 | /** 98 | * Returns the position for the start of a stream. Provide this position to your [[EventHubsConf]] to start 99 | * receiving from the first available event in the partition. 100 | * 101 | * @return An [[EventPosition]] instance. 102 | */ 103 | def fromStartOfStream: EventPosition = { 104 | EventPosition(StartOfStream) 105 | } 106 | 107 | /** 108 | * Returns the position for the end of a stream. Provide this position to your [[EventHubsConf]] to start 109 | * receiving from the next available event in the partition after the receiver is created. 110 | * 111 | * @return An [[EventPosition]] instance. 112 | */ 113 | def fromEndOfStream: EventPosition = { 114 | EventPosition(EndOfStream, isInclusive = false) 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/eventhubs/EventHubsSourceOffsetSuite.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.sql.eventhubs 19 | 20 | import java.io.File 21 | 22 | import org.apache.spark.sql.execution.streaming._ 23 | import org.apache.spark.sql.streaming.OffsetSuite 24 | import org.apache.spark.sql.test.SharedSQLContext 25 | 26 | class EventHubsSourceOffsetSuite extends OffsetSuite with SharedSQLContext { 27 | 28 | compare(one = EventHubsSourceOffset(("t", 0, 1L)), two = EventHubsSourceOffset(("t", 0, 2L))) 29 | 30 | compare(one = EventHubsSourceOffset(("t", 0, 1L), ("t", 1, 0L)), 31 | two = EventHubsSourceOffset(("t", 0, 2L), ("t", 1, 1L))) 32 | 33 | compare(one = EventHubsSourceOffset(("t", 0, 1L), ("T", 0, 0L)), 34 | two = EventHubsSourceOffset(("t", 0, 2L), ("T", 0, 1L))) 35 | 36 | compare(one = EventHubsSourceOffset(("t", 0, 1L)), 37 | two = EventHubsSourceOffset(("t", 0, 2L), ("t", 1, 1L))) 38 | 39 | val ehso1 = EventHubsSourceOffset(("t", 0, 1L)) 40 | val ehso2 = EventHubsSourceOffset(("t", 0, 2L), ("t", 1, 3L)) 41 | val ehso3 = EventHubsSourceOffset(("t", 0, 2L), ("t", 1, 3L), ("t", 1, 4L)) 42 | 43 | compare(EventHubsSourceOffset(SerializedOffset(ehso1.json)), 44 | EventHubsSourceOffset(SerializedOffset(ehso2.json))) 45 | 46 | test("basic serialization - deserialization") { 47 | assert( 48 | EventHubsSourceOffset.getPartitionSeqNos(ehso1) == 49 | EventHubsSourceOffset.getPartitionSeqNos(SerializedOffset(ehso1.json))) 50 | } 51 | 52 | test("OffsetSeqLog serialization - deserialization") { 53 | withTempDir { temp => 54 | // use non-existent directory to test whether log make the dir 55 | val dir = new File(temp, "dir") 56 | val metadataLog = new OffsetSeqLog(spark, dir.getAbsolutePath) 57 | val batch0 = OffsetSeq.fill(ehso1) 58 | val batch1 = OffsetSeq.fill(ehso2, ehso3) 59 | 60 | val batch0Serialized = 61 | OffsetSeq.fill(batch0.offsets.flatMap(_.map(o => SerializedOffset(o.json))): _*) 62 | 63 | val batch1Serialized = 64 | OffsetSeq.fill(batch1.offsets.flatMap(_.map(o => SerializedOffset(o.json))): _*) 65 | 66 | assert(metadataLog.add(0, batch0)) 67 | assert(metadataLog.getLatest() === Some(0 -> batch0Serialized)) 68 | assert(metadataLog.get(0) === Some(batch0Serialized)) 69 | 70 | assert(metadataLog.add(1, batch1)) 71 | assert(metadataLog.get(0) === Some(batch0Serialized)) 72 | assert(metadataLog.get(1) === Some(batch1Serialized)) 73 | assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) 74 | assert( 75 | metadataLog.get(None, Some(1)) === 76 | Array(0 -> batch0Serialized, 1 -> batch1Serialized)) 77 | 78 | // Adding the same batch does nothing 79 | metadataLog.add(1, OffsetSeq.fill(LongOffset(3))) 80 | assert(metadataLog.get(0) === Some(batch0Serialized)) 81 | assert(metadataLog.get(1) === Some(batch1Serialized)) 82 | assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) 83 | assert( 84 | metadataLog.get(None, Some(1)) === 85 | Array(0 -> batch0Serialized, 1 -> batch1Serialized)) 86 | } 87 | } 88 | 89 | test("read Spark 2.1.0 offset format") { 90 | val offset = readFromResource("eventhubs-source-offset-version-2.1.0.txt") 91 | assert( 92 | EventHubsSourceOffset(offset) === 93 | EventHubsSourceOffset(("ehName1", 0, 456L), ("ehName1", 1, 789L), ("ehName2", 0, 0L))) 94 | } 95 | 96 | private def readFromResource(file: String): SerializedOffset = { 97 | import scala.io.Source 98 | val input = getClass.getResource(s"/$file").toURI 99 | val str = Source.fromFile(input).mkString 100 | SerializedOffset(str) 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/client/Client.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.eventhubs.client 19 | 20 | import com.microsoft.azure.eventhubs.EventData 21 | import org.apache.spark.eventhubs.EventHubsConf 22 | import org.apache.spark.eventhubs._ 23 | 24 | private[spark] trait Client extends Serializable { 25 | 26 | /** 27 | * Creates a sender which sends directly to the specified partitionId. 28 | * 29 | * @param partitionId the partition that will receive all events sent from this partition sender. 30 | */ 31 | def createPartitionSender(partitionId: Int) 32 | 33 | /** 34 | * Creates an EventHub receiver. 35 | * 36 | * @param partitionId the partitionId the receiver will consume from. 37 | * @param startingSeqNo the sequence number the receiver will start from. 38 | */ 39 | def createReceiver(partitionId: String, startingSeqNo: SequenceNumber): Unit 40 | 41 | /** 42 | * Sends an [[EventData]] to your EventHubs. 43 | * 44 | * @param event the event that is being sent. 45 | */ 46 | def send(event: EventData): Unit 47 | 48 | /** 49 | * Sends an [[EventData]] to your EventHubs with the provided 50 | * partition key. 51 | * 52 | * @param event the event that is being sent. 53 | * @param partitionKey the partitionKey will be hash'ed to determine the partitionId 54 | * to send the events to. On the Received message this can be accessed 55 | * at [[EventData.SystemProperties#getPartitionKey()]] 56 | */ 57 | def send(event: EventData, partitionKey: String): Unit 58 | 59 | /** 60 | * Sends an [[EventData]] directly to the provided partitionId in your 61 | * EventHubs. 62 | * 63 | * @param event the event that is being sent. 64 | * @param partitionId the partition that will receive all events being sent. 65 | */ 66 | def send(event: EventData, partitionId: Int): Unit 67 | 68 | /** 69 | * Receive events from your EventHubs instance. 70 | * 71 | * @param eventCount the number of events that will be requested from the EventHub partition. 72 | */ 73 | def receive(eventCount: Int): java.lang.Iterable[EventData] 74 | 75 | /** 76 | * When a connection with the service is established, the client will begin to prefetch EventData. 77 | * This number specifies the max number of events that will prefetched. 78 | */ 79 | def setPrefetchCount(count: Int): Unit 80 | 81 | /** 82 | * Provides the earliest (lowest) sequence number that exists in the EventHubs instance 83 | * for the given partition. 84 | * 85 | * @return the earliest sequence number for the specified partition 86 | */ 87 | def earliestSeqNo(partitionId: PartitionId): SequenceNumber 88 | 89 | /** 90 | * Provides the latest (highest) sequence number that exists in the EventHubs 91 | * instance for the given partition. 92 | * 93 | * @return the leatest sequence number for the specified partition 94 | */ 95 | def latestSeqNo(partitionId: PartitionId): SequenceNumber 96 | 97 | /** 98 | * Provides the earliest and the latest sequence numbers in the provided partition. 99 | * 100 | * @return the earliest and latest sequence numbers for the specified partition. 101 | */ 102 | def boundedSeqNos(partitionId: PartitionId): (SequenceNumber, SequenceNumber) 103 | 104 | /** 105 | * Translates any starting point provided by a user to the specific offset and sequence number 106 | * that we need to start from. This ensure that within Spark, we're only dealing with offsets 107 | * and sequence numbers. 108 | */ 109 | def translate[T](ehConf: EventHubsConf, 110 | partitionCount: Int, 111 | useStart: Boolean = true): Map[PartitionId, SequenceNumber] 112 | 113 | /** 114 | * Returns the number of partitions in your EventHubs instance. 115 | */ 116 | def partitionCount: Int 117 | 118 | /** 119 | * Closes the EventHubs client. 120 | */ 121 | def close(): Unit 122 | } 123 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/rdd/EventHubsRDDSuite.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.eventhubs.rdd 19 | 20 | import org.apache.spark.eventhubs.EventHubsConf 21 | import org.apache.spark.eventhubs.utils.{ EventHubsTestUtils, SimulatedClient } 22 | import org.apache.spark.{ SparkConf, SparkContext, SparkFunSuite } 23 | import org.scalatest.BeforeAndAfterAll 24 | 25 | class EventHubsRDDSuite extends SparkFunSuite with BeforeAndAfterAll { 26 | import org.apache.spark.eventhubs.utils.EventHubsTestUtils._ 27 | 28 | private var testUtils: EventHubsTestUtils = _ 29 | 30 | private val sparkConf = 31 | new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) 32 | private var sc: SparkContext = _ 33 | 34 | override def beforeAll { 35 | super.beforeAll() 36 | testUtils = new EventHubsTestUtils 37 | val eventHub = testUtils.createEventHubs(DefaultName, DefaultPartitionCount) 38 | 39 | // Send events to simulated EventHubs 40 | for (i <- 0 until DefaultPartitionCount) { 41 | eventHub.send(i, 0 until 5000) 42 | } 43 | 44 | sc = new SparkContext(sparkConf) 45 | } 46 | 47 | override def afterAll: Unit = { 48 | if (testUtils != null) { 49 | testUtils.destroyAllEventHubs() 50 | testUtils = null 51 | } 52 | 53 | if (sc != null) { 54 | sc.stop 55 | sc = null 56 | } 57 | super.afterAll() 58 | } 59 | 60 | private def getEventHubsConf: EventHubsConf = testUtils.getEventHubsConf() 61 | 62 | test("basic usage") { 63 | val fromSeqNo = 0 64 | val untilSeqNo = 50 65 | val ehConf = getEventHubsConf 66 | 67 | val offsetRanges = (for { 68 | partition <- 0 until DefaultPartitionCount 69 | } yield OffsetRange(ehConf.name, partition, fromSeqNo, untilSeqNo, None)).toArray 70 | 71 | val rdd = new EventHubsRDD(sc, ehConf, offsetRanges, SimulatedClient.apply) 72 | .map(_.getBytes.map(_.toChar).mkString) 73 | 74 | assert(rdd.count == (untilSeqNo - fromSeqNo) * DefaultPartitionCount) 75 | assert(!rdd.isEmpty) 76 | 77 | // Make sure body is still intact 78 | val event = rdd.take(1).head 79 | assert(event contains "0") 80 | } 81 | 82 | test("start from middle of instance") { 83 | val fromSeqNo = 3000 84 | val untilSeqNo = 4000 85 | val ehConf = getEventHubsConf 86 | 87 | val offsetRanges = (for { 88 | partition <- 0 until DefaultPartitionCount 89 | } yield OffsetRange(ehConf.name, partition, fromSeqNo, untilSeqNo, None)).toArray 90 | 91 | val rdd = new EventHubsRDD(sc, ehConf, offsetRanges, SimulatedClient.apply) 92 | .map(_.getBytes.map(_.toChar).mkString) 93 | 94 | assert(rdd.count == (untilSeqNo - fromSeqNo) * DefaultPartitionCount) 95 | assert(!rdd.isEmpty) 96 | 97 | // Make sure body is still intact 98 | val event = rdd.take(1).head 99 | //assert(event contains EventPayload) 100 | } 101 | 102 | test("single partition, make sure seqNos are consecutive") { 103 | val fromSeqNo = 100 104 | val untilSeqNo = 3200 105 | val ehConf = getEventHubsConf 106 | 107 | val offsetRanges = Array(OffsetRange(ehConf.name, 0, fromSeqNo, untilSeqNo, None)) 108 | 109 | val rdd = new EventHubsRDD(sc, ehConf, offsetRanges, SimulatedClient.apply) 110 | .map(_.getSystemProperties.getSequenceNumber) 111 | 112 | assert(rdd.count == (untilSeqNo - fromSeqNo)) // no PartitionCount multiplier b/c we only have one partition 113 | assert(!rdd.isEmpty) 114 | 115 | val received = rdd.collect().sorted.zipWithIndex 116 | 117 | for ((seqNo, index) <- received) { 118 | assert(fromSeqNo + index == seqNo) 119 | } 120 | } 121 | 122 | test("repartition test") { 123 | val fromSeqNo = 100 124 | val untilSeqNo = 4200 125 | val ehConf = getEventHubsConf 126 | 127 | val offsetRanges = (for { 128 | partition <- 0 until DefaultPartitionCount 129 | } yield OffsetRange(ehConf.name, partition, fromSeqNo, untilSeqNo, None)).toArray 130 | 131 | val rdd = new EventHubsRDD(sc, ehConf, offsetRanges, SimulatedClient.apply) 132 | .map(_.getBytes.map(_.toChar).mkString) 133 | .repartition(20) 134 | 135 | assert(rdd.count == (untilSeqNo - fromSeqNo) * DefaultPartitionCount) 136 | assert(!rdd.isEmpty) 137 | 138 | // Make sure body is still intact 139 | val event = rdd.take(1).head 140 | //assert(event contains EventPayload) 141 | } 142 | } 143 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 |

2 | Azure Event Hubs + Apache Spark Connector 3 |

4 | 5 |

Azure Event Hubs Connector for Apache Spark

6 | 7 |

8 | 9 | chat on gitter 10 | 11 | 12 | build status 13 | 14 | 15 | star our repo 16 | 17 |

18 | 19 | This is the source code of the Azure Event Hubs Connector for Apache Spark. 20 | 21 | Azure Event Hubs is a highly scalable publish-subscribe service that can ingest millions of events per second and stream them into multiple applications. 22 | Spark Streaming and Structured Streaming are scalable and fault-tolerant stream processing engines that allow users to process huge amounts of data using 23 | complex algorithms expressed with high-level functions like `map`, `reduce`, `join`, and `window`. This data can then be pushed to 24 | filesystems, databases, or even back to Event Hubs. 25 | 26 | By making Event Hubs and Spark easier to use together, we hope this connector makes building scalable, fault-tolerant applications easier for our users. 27 | 28 | ## Latest Releases 29 | 30 | #### Spark 31 | |Spark Version|Package Name|Package Version| 32 | |-------------|------------|----------------| 33 | |Spark 2.3|azure-eventhubs-spark_2.11|[![Maven Central](https://img.shields.io/badge/maven%20central-2.3.1-brightgreen.svg)](https://search.maven.org/#artifactdetails%7Ccom.microsoft.azure%7Cazure-eventhubs-spark_2.11%7C2.3.1%7Cjar)| 34 | |Spark 2.2|azure-eventhubs-spark_2.11|[![Maven Central](https://img.shields.io/badge/maven%20central-2.2.0-blue.svg)](https://search.maven.org/#artifactdetails%7Ccom.microsoft.azure%7Cazure-eventhubs-spark_2.11%7C2.2.0%7Cjar)| 35 | |Spark 2.1|azure-eventhubs-spark_2.11|[![Maven Central](https://img.shields.io/badge/maven%20central-2.2.0-blue.svg)](https://search.maven.org/#artifactdetails%7Ccom.microsoft.azure%7Cazure-eventhubs-spark_2.11%7C2.2.0%7Cjar)| 36 | 37 | #### Databricks 38 | |Databricks Runtime Version|Artifact Id|Package Version| 39 | |-------------|------------|----------------| 40 | |Databricks Runtime 4.X|azure-eventhubs-spark_2.11|[![Maven Central](https://img.shields.io/badge/maven%20central-2.3.1-brightgreen.svg)](https://search.maven.org/#artifactdetails%7Ccom.microsoft.azure%7Cazure-eventhubs-spark_2.11%7C2.3.1%7Cjar)| 41 | |Databricks Runtime 3.5|azure-eventhubs-spark_2.11|[![Maven Central](https://img.shields.io/badge/maven%20central-2.3.1-brightgreen.svg)](https://search.maven.org/#artifactdetails%7Ccom.microsoft.azure%7Cazure-eventhubs-spark_2.11%7C2.3.1%7Cjar)| 42 | 43 | #### Roadmap 44 | 45 | There is an open issue for each planned feature/enhancement. Additional comments can be found on 46 | our [wiki](https://github.com/Azure/azure-event-hubs-spark/wiki). 47 | 48 | ## Usage 49 | 50 | ### Linking 51 | 52 | For Scala/Java applications using SBT/Maven project definitions, link your application with the artifact below. 53 | **Note:** See [Latest Releases](#latest-releases) to find the correct artifiact for your version of Apache Spark (or Databricks)! 54 | 55 | groupId = com.microsoft.azure 56 | artifactId = azure-eventhubs-spark_2.11 57 | version = 2.3.1 58 | 59 | ### Documentation 60 | 61 | Documentation for our connector can be found [here](docs/). The integration guides there contain all the information you need to use this library. 62 | 63 | **If you're new to Apache Spark and/or Event Hubs, then we highly recommend reading their documentation first.** You can read Event Hubs 64 | documentation [here](https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-what-is-event-hubs), documentation for Spark Streaming 65 | [here](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and, the last but not least, Structured Streaming 66 | [here](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html). 67 | 68 | ### Further Assistance 69 | 70 | **If you need additional assistance, please don't hesitate to ask!** General questions and discussion should happen on our 71 | [gitter chat](https://gitter.im/azure-event-hubs-spark). Please open an issue for bug reports and feature requests! Feedback, feature 72 | requests, bug reports, etc are all welcomed! 73 | 74 | ## Contributing 75 | 76 | If you'd like to help contribute (we'd love to have your help!), then go to our [Contributor's Guide](/.github/CONTRIBUTING.md) for more information. 77 | 78 | ## Build Prerequisites 79 | 80 | In order to use the connector, you need to have: 81 | 82 | - Java 1.8 SDK installed 83 | - [Maven 3.x](https://maven.apache.org/download.cgi) installed (or [SBT version 1.x](https://www.scala-sbt.org/1.x/docs/index.html)) 84 | 85 | More details on building from source and running tests can be found in our [Contributor's Guide](/.github/CONTRIBUTING.md). 86 | 87 | ## Build Command 88 | 89 | // Builds jar and runs all tests 90 | mvn clean package 91 | 92 | // Builds jar, runs all tests, and installs jar to your local maven repository 93 | mvn clean install 94 | 95 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsWriteTask.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.sql.eventhubs 19 | 20 | import com.microsoft.azure.eventhubs.EventData 21 | import com.sun.org.apache.xalan.internal.xsltc.compiler.util.IntType 22 | import org.apache.spark.eventhubs.EventHubsConf 23 | import org.apache.spark.eventhubs.client.Client 24 | import org.apache.spark.sql.catalyst.InternalRow 25 | import org.apache.spark.sql.catalyst.expressions.{ Attribute, Cast, Literal, UnsafeProjection } 26 | import org.apache.spark.sql.types.{ BinaryType, StringType } 27 | import org.apache.spark.unsafe.types.UTF8String.IntWrapper 28 | 29 | /** 30 | * Writes out data in a single Spark task, without any concerns about how 31 | * to commit or abort tasks. Exceptions thrown by the implementation of this 32 | * class will automatically trigger task aborts. 33 | */ 34 | private[eventhubs] class EventHubsWriteTask(parameters: Map[String, String], 35 | inputSchema: Seq[Attribute], 36 | clientFactory: (EventHubsConf) => Client) 37 | extends EventHubsRowWriter(inputSchema) { 38 | 39 | private var sender: Client = _ 40 | private val ehConf = EventHubsConf.toConf(parameters) 41 | 42 | /** 43 | * Writers data out to EventHubs 44 | * 45 | * @param iterator contains all rows to be written to EventHubs 46 | */ 47 | def execute(iterator: Iterator[InternalRow]): Unit = { 48 | sender = clientFactory(ehConf) 49 | while (iterator.hasNext) { 50 | val currentRow = iterator.next 51 | sendRow(currentRow, sender) 52 | } 53 | } 54 | 55 | def close(): Unit = { 56 | if (sender != null) { 57 | sender.close() 58 | sender = null 59 | } 60 | } 61 | } 62 | 63 | private[eventhubs] abstract class EventHubsRowWriter(inputSchema: Seq[Attribute]) { 64 | 65 | protected val projection: UnsafeProjection = createProjection 66 | 67 | /** 68 | * Send the specified row to EventHubs. 69 | */ 70 | protected def sendRow( 71 | row: InternalRow, 72 | sender: Client 73 | ): Unit = { 74 | val projectedRow = projection(row) 75 | val body = projectedRow.getBinary(0) 76 | val partitionKey = projectedRow.getUTF8String(1) 77 | val partitionId = projectedRow.getUTF8String(2) 78 | 79 | require( 80 | partitionId == null || partitionKey == null, 81 | s"Both a partitionKey ($partitionKey) and partitionId ($partitionId) have been detected. Both can not be set.") 82 | 83 | val event = EventData.create(body) 84 | 85 | if (partitionKey != null) { 86 | sender.send(event, partitionKey.toString) 87 | } else if (partitionId != null) { 88 | val wrapper = new IntWrapper 89 | if (partitionId.toInt(wrapper)) { 90 | sender.createPartitionSender(wrapper.value) 91 | sender.send(event, wrapper.value) 92 | } else { 93 | throw new IllegalStateException( 94 | s"partitionId '$partitionId' could not be parsed to an int.") 95 | } 96 | } else { 97 | sender.send(event) 98 | } 99 | } 100 | 101 | private def createProjection = { 102 | val bodyExpression = inputSchema 103 | .find(_.name == EventHubsWriter.BodyAttributeName) 104 | .getOrElse(throw new IllegalStateException( 105 | s"Required attribute '${EventHubsWriter.BodyAttributeName}' not found.")) 106 | 107 | bodyExpression.dataType match { 108 | case StringType | BinaryType => // good 109 | case t => 110 | throw new IllegalStateException( 111 | s"${EventHubsWriter.BodyAttributeName} attribute unsupported type $t") 112 | } 113 | 114 | val partitionKeyExpression = 115 | inputSchema 116 | .find(_.name == EventHubsWriter.PartitionKeyAttributeName) 117 | .getOrElse(Literal(null, StringType)) 118 | 119 | partitionKeyExpression.dataType match { 120 | case StringType => // good 121 | case t => 122 | throw new IllegalStateException( 123 | s"${EventHubsWriter.PartitionKeyAttributeName} attribute unsupported type $t" 124 | ) 125 | } 126 | 127 | val partitionIdExpression = 128 | inputSchema 129 | .find(_.name == EventHubsWriter.PartitionIdAttributeName) 130 | .getOrElse(Literal(null, StringType)) 131 | 132 | partitionIdExpression.dataType match { 133 | case StringType => // good 134 | case t => 135 | throw new IllegalStateException( 136 | s"${EventHubsWriter.PartitionIdAttributeName} attribute unsupported type $t" 137 | ) 138 | } 139 | 140 | UnsafeProjection.create(Seq(Cast(bodyExpression, BinaryType), 141 | Cast(partitionKeyExpression, StringType), 142 | Cast(partitionIdExpression, StringType)), 143 | inputSchema) 144 | } 145 | } 146 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/rdd/EventHubsRDD.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.eventhubs.rdd 19 | 20 | import com.microsoft.azure.eventhubs.EventData 21 | import org.apache.spark.eventhubs.EventHubsConf 22 | import org.apache.spark.eventhubs.client.Client 23 | import org.apache.spark.internal.Logging 24 | import org.apache.spark.rdd.RDD 25 | import org.apache.spark.{ Partition, SparkContext, TaskContext } 26 | 27 | import scala.collection.mutable.ArrayBuffer 28 | 29 | private[spark] class EventHubsRDD(sc: SparkContext, 30 | val ehConf: EventHubsConf, 31 | val offsetRanges: Array[OffsetRange], 32 | receiverFactory: (EventHubsConf => Client)) 33 | extends RDD[EventData](sc, Nil) 34 | with Logging 35 | with HasOffsetRanges { 36 | 37 | import org.apache.spark.eventhubs._ 38 | 39 | override def getPartitions: Array[Partition] = { 40 | for { o <- offsetRanges.sortWith(_.partitionId < _.partitionId) } yield 41 | new EventHubsRDDPartition(o.partitionId, 42 | o.nameAndPartition, 43 | o.fromSeqNo, 44 | o.untilSeqNo, 45 | o.preferredLoc) 46 | } 47 | 48 | override def count: Long = offsetRanges.map(_.count).sum 49 | 50 | override def isEmpty(): Boolean = count == 0L 51 | 52 | override def take(num: Int): Array[EventData] = { 53 | val nonEmptyPartitions = 54 | this.partitions.map(_.asInstanceOf[EventHubsRDDPartition]).filter(_.count > 0) 55 | 56 | if (num < 1 || nonEmptyPartitions.isEmpty) { 57 | return Array() 58 | } 59 | 60 | val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => 61 | val remain = num - result.values.sum 62 | if (remain > 0) { 63 | val taken = Math.min(remain, part.count) 64 | result + (part.index -> taken.toInt) 65 | 66 | } else { 67 | result 68 | } 69 | } 70 | 71 | val buf = new ArrayBuffer[EventData] 72 | val res = context.runJob( 73 | this, 74 | (tc: TaskContext, it: Iterator[EventData]) => it.take(parts(tc.partitionId)).toArray, 75 | parts.keys.toArray) 76 | res.foreach(buf ++= _) 77 | buf.toArray 78 | } 79 | 80 | override def getPreferredLocations(split: Partition): Seq[String] = { 81 | val part = split.asInstanceOf[EventHubsRDDPartition] 82 | part.preferredLoc.map(Seq(_)).getOrElse(Seq.empty) 83 | } 84 | 85 | private def errBeginAfterEnd(part: EventHubsRDDPartition): String = 86 | s"The beginning sequence number ${part.fromSeqNo} is larger than thet ending sequence number ${part.untilSeqNo}" + 87 | s"for EventHubs ${part.name} on partitionId ${part.partitionId}." 88 | 89 | override def compute(partition: Partition, context: TaskContext): Iterator[EventData] = { 90 | val part = partition.asInstanceOf[EventHubsRDDPartition] 91 | assert(part.fromSeqNo <= part.untilSeqNo, errBeginAfterEnd(part)) 92 | 93 | if (part.fromSeqNo == part.untilSeqNo) { 94 | logInfo( 95 | s"Beginning sequence number ${part.fromSeqNo} is equal to the ending sequence number ${part.untilSeqNo}." + 96 | s"Returning empty partition for EH: ${part.name} on partition: ${part.partitionId}") 97 | Iterator.empty 98 | } else { 99 | new EventHubsRDDIterator(part, context) 100 | } 101 | } 102 | 103 | private class EventHubsRDDIterator(part: EventHubsRDDPartition, context: TaskContext) 104 | extends Iterator[EventData] { 105 | 106 | logInfo( 107 | s"Computing EventHubs ${part.name}, partitionId ${part.partitionId} " + 108 | s"sequence numbers ${part.fromSeqNo} => ${part.untilSeqNo}") 109 | 110 | val client: Client = receiverFactory(ehConf) 111 | client.createReceiver(part.partitionId.toString, part.fromSeqNo) 112 | 113 | val prefetchCount = 114 | if (part.count.toInt < PrefetchCountMinimum) PrefetchCountMinimum else part.count.toInt 115 | client.setPrefetchCount(prefetchCount) 116 | 117 | var requestSeqNo: SequenceNumber = part.fromSeqNo 118 | 119 | override def hasNext(): Boolean = requestSeqNo < part.untilSeqNo 120 | 121 | def errWrongSeqNo(part: EventHubsRDDPartition, receivedSeqNo: SequenceNumber): String = 122 | s"requestSeqNo $requestSeqNo does not match the received sequence number $receivedSeqNo" 123 | 124 | override def next(): EventData = { 125 | assert(hasNext(), "Can't call next() once untilSeqNo has been reached.") 126 | 127 | @volatile var event: EventData = null 128 | @volatile var i: java.lang.Iterable[EventData] = null 129 | while (i == null) { 130 | i = client.receive(1) 131 | } 132 | event = i.iterator.next 133 | 134 | assert(requestSeqNo == event.getSystemProperties.getSequenceNumber, 135 | errWrongSeqNo(part, event.getSystemProperties.getSequenceNumber)) 136 | requestSeqNo += 1 137 | event 138 | } 139 | 140 | context.addTaskCompletionListener { _ => 141 | closeIfNeeded() 142 | } 143 | 144 | def closeIfNeeded(): Unit = { 145 | if (client != null) client.close() 146 | } 147 | } 148 | } 149 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsSourceProvider.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.sql.eventhubs 19 | 20 | import java.util.Locale 21 | 22 | import org.apache.spark.eventhubs.{ EventHubsConf, _ } 23 | import org.apache.spark.eventhubs.client.{ Client, EventHubsClient } 24 | import org.apache.spark.eventhubs.utils.SimulatedClient 25 | import org.apache.spark.internal.Logging 26 | import org.apache.spark.sql.{ AnalysisException, DataFrame, SQLContext, SaveMode } 27 | import org.apache.spark.sql.execution.streaming.{ Sink, Source } 28 | import org.apache.spark.sql.sources._ 29 | import org.apache.spark.sql.streaming.OutputMode 30 | import org.apache.spark.sql.types._ 31 | 32 | /** 33 | * The provider class for the [[EventHubsSource]]. 34 | */ 35 | private[sql] class EventHubsSourceProvider 36 | extends DataSourceRegister 37 | with StreamSourceProvider 38 | with StreamSinkProvider 39 | with RelationProvider 40 | with CreatableRelationProvider 41 | with Logging { 42 | 43 | import EventHubsConf._ 44 | 45 | override def shortName(): String = "eventhubs" 46 | 47 | override def sourceSchema(sqlContext: SQLContext, 48 | schema: Option[StructType], 49 | providerName: String, 50 | parameters: Map[String, String]): (String, StructType) = { 51 | (shortName(), EventHubsSourceProvider.eventHubsSchema) 52 | } 53 | 54 | override def createSource(sqlContext: SQLContext, 55 | metadataPath: String, 56 | schema: Option[StructType], 57 | providerName: String, 58 | parameters: Map[String, String]): Source = { 59 | EventHubsClient.userAgent = 60 | s"Structured-Streaming-${sqlContext.sparkSession.sparkContext.version}" 61 | 62 | val caseInsensitiveParameters = parameters.map { 63 | case (k, v) => (k.toLowerCase(Locale.ROOT), v) 64 | } 65 | 66 | new EventHubsSource(sqlContext, 67 | parameters, 68 | clientFactory(caseInsensitiveParameters), 69 | metadataPath) 70 | } 71 | 72 | override def createRelation(sqlContext: SQLContext, 73 | parameters: Map[String, String]): BaseRelation = { 74 | EventHubsClient.userAgent = 75 | s"Structured-Streaming-${sqlContext.sparkSession.sparkContext.version}" 76 | 77 | val caseInsensitiveMap = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } 78 | 79 | new EventHubsRelation(sqlContext, parameters, clientFactory(caseInsensitiveMap)) 80 | } 81 | 82 | override def createSink(sqlContext: SQLContext, 83 | parameters: Map[String, String], 84 | partitionColumns: Seq[String], 85 | outputMode: OutputMode): Sink = { 86 | EventHubsClient.userAgent = 87 | s"Structured-Streaming-${sqlContext.sparkSession.sparkContext.version}" 88 | 89 | val caseInsensitiveMap = parameters.map { 90 | case (k, v) => (k.toLowerCase(Locale.ROOT), v) 91 | } 92 | 93 | new EventHubsSink(sqlContext, caseInsensitiveMap, clientFactory(caseInsensitiveMap)) 94 | } 95 | 96 | override def createRelation(outerSQLContext: SQLContext, 97 | mode: SaveMode, 98 | parameters: Map[String, String], 99 | data: DataFrame): BaseRelation = { 100 | EventHubsClient.userAgent = 101 | s"Structured-Streaming-${outerSQLContext.sparkSession.sparkContext.version}" 102 | 103 | mode match { 104 | case SaveMode.Overwrite | SaveMode.Ignore => 105 | throw new AnalysisException( 106 | s"Save mode $mode not allowed for EventHubs. " + 107 | s"Allowed save modes are ${SaveMode.Append} and " + 108 | s"${SaveMode.ErrorIfExists} (default).") 109 | case _ => // good 110 | } 111 | 112 | val caseInsensitiveMap = parameters.map { 113 | case (k, v) => (k.toLowerCase(Locale.ROOT), v) 114 | } 115 | 116 | EventHubsWriter.write(outerSQLContext.sparkSession, 117 | data.queryExecution, 118 | caseInsensitiveMap, 119 | clientFactory(caseInsensitiveMap)) 120 | 121 | /* This method is suppose to return a relation that reads the data that was written. 122 | * We cannot support this for EventHubs. Therefore, in order to make things consistent, 123 | * we return an empty base relation. 124 | */ 125 | new BaseRelation { 126 | override def sqlContext: SQLContext = unsupportedException 127 | override def schema: StructType = unsupportedException 128 | override def needConversion: Boolean = unsupportedException 129 | override def sizeInBytes: Long = unsupportedException 130 | override def unhandledFilters(filters: Array[Filter]): Array[Filter] = unsupportedException 131 | private def unsupportedException = 132 | throw new UnsupportedOperationException( 133 | "BaseRelation from EventHubs write " + 134 | "operation is not usable.") 135 | } 136 | } 137 | 138 | private def clientFactory(caseInsensitiveParams: Map[String, String]): EventHubsConf => Client = { 139 | if (caseInsensitiveParams 140 | .getOrElse(UseSimulatedClientKey.toLowerCase, DefaultUseSimulatedClient) 141 | .toBoolean) { 142 | SimulatedClient.apply 143 | } else { 144 | EventHubsClient.apply 145 | } 146 | } 147 | } 148 | 149 | private[sql] object EventHubsSourceProvider extends Serializable { 150 | def eventHubsSchema: StructType = { 151 | StructType( 152 | Seq( 153 | StructField("body", BinaryType), 154 | StructField("partition", StringType), 155 | StructField("offset", StringType), 156 | StructField("sequenceNumber", LongType), 157 | StructField("enqueuedTime", TimestampType), 158 | StructField("publisher", StringType), 159 | StructField("partitionKey", StringType) 160 | )) 161 | } 162 | } 163 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/streaming/eventhubs/EventHubsDirectDStream.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.streaming.eventhubs 19 | 20 | import scala.collection.mutable 21 | import com.microsoft.azure.eventhubs.EventData 22 | import org.apache.spark.SparkContext 23 | import org.apache.spark.eventhubs.EventHubsConf 24 | import org.apache.spark.eventhubs.client.Client 25 | import org.apache.spark.eventhubs._ 26 | import org.apache.spark.eventhubs.client.EventHubsClient 27 | import org.apache.spark.eventhubs.rdd.{ EventHubsRDD, OffsetRange } 28 | import org.apache.spark.internal.Logging 29 | import org.apache.spark.rdd.RDD 30 | import org.apache.spark.scheduler.ExecutorCacheTaskLocation 31 | import org.apache.spark.streaming.{ StreamingContext, Time } 32 | import org.apache.spark.streaming.dstream.{ DStreamCheckpointData, InputDStream } 33 | import org.apache.spark.streaming.scheduler.{ RateController, StreamInputInfo } 34 | import org.apache.spark.streaming.scheduler.rate.RateEstimator 35 | 36 | /** 37 | * A DStream where each EventHubs partition corresponds to an RDD partition. 38 | * 39 | * @param _ssc the StreamingContext this stream belongs to 40 | * @param ehConf the configurations related to your EventHubs. See [[EventHubsConf]] for detail. 41 | * @param clientFactory the factory method that creates an EventHubsClient 42 | */ 43 | private[spark] class EventHubsDirectDStream private[spark] ( 44 | _ssc: StreamingContext, 45 | ehConf: EventHubsConf, 46 | clientFactory: (EventHubsConf => Client)) 47 | extends InputDStream[EventData](_ssc) 48 | with Logging { 49 | 50 | import EventHubsDirectDStream._ 51 | 52 | private lazy val partitionCount: Int = ehClient.partitionCount 53 | private lazy val ehName = ehConf.name 54 | 55 | @transient private var _client: Client = _ 56 | private[spark] def ehClient: Client = this.synchronized { 57 | if (_client == null) _client = clientFactory(ehConf) 58 | _client 59 | } 60 | 61 | private var fromSeqNos: Map[PartitionId, SequenceNumber] = _ 62 | 63 | private def init(): Unit = { 64 | fromSeqNos = ehClient.translate(ehConf, partitionCount) 65 | } 66 | init() 67 | 68 | protected[streaming] override val checkpointData = new EventHubDirectDStreamCheckpointData 69 | 70 | override protected[streaming] val rateController: Option[RateController] = { 71 | if (RateController.isBackPressureEnabled(ssc.sparkContext.conf)) { 72 | logWarning("rateController: BackPressure is not currently supported.") 73 | } 74 | None 75 | } 76 | 77 | protected def latestSeqNos(): Map[PartitionId, SequenceNumber] = { 78 | (for { 79 | partitionId <- 0 until partitionCount 80 | endPoint = ehClient.latestSeqNo(partitionId) 81 | } yield partitionId -> endPoint).toMap 82 | } 83 | 84 | protected def clamp( 85 | latestSeqNos: Map[PartitionId, SequenceNumber]): Map[PartitionId, SequenceNumber] = { 86 | (for { 87 | (partitionId, latestSeqNo) <- latestSeqNos 88 | nAndP = NameAndPartition(ehConf.name, partitionId) 89 | defaultMaxRate = ehConf.maxRatePerPartition.getOrElse(DefaultMaxRatePerPartition) 90 | partitionRates = ehConf.maxRatesPerPartition.getOrElse(Map.empty) 91 | maxRate = partitionRates.getOrElse(nAndP, defaultMaxRate) 92 | upperBound = math.min(fromSeqNos(partitionId) + maxRate, latestSeqNo) 93 | untilSeqNo = math.max(fromSeqNos(partitionId), upperBound) 94 | } yield partitionId -> untilSeqNo).toMap 95 | } 96 | 97 | override def compute(validTime: Time): Option[RDD[EventData]] = { 98 | val sortedExecutors = getSortedExecutorList(ssc.sparkContext) 99 | val numExecutors = sortedExecutors.length 100 | logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) 101 | 102 | val untilSeqNos = clamp(latestSeqNos()) 103 | val offsetRanges = (for { 104 | p <- 0 until partitionCount 105 | preferredLoc = if (numExecutors > 0) { 106 | Some(sortedExecutors(Math.floorMod(NameAndPartition(ehName, p).hashCode, numExecutors))) 107 | } else None 108 | } yield 109 | OffsetRange(NameAndPartition(ehName, p), fromSeqNos(p), untilSeqNos(p), preferredLoc)).toArray 110 | 111 | val rdd = new EventHubsRDD(context.sparkContext, ehConf, offsetRanges, clientFactory) 112 | 113 | val description = offsetRanges.map(_.toString).mkString("\n") 114 | logInfo(s"Starting batch at $validTime for EH: $ehName with\n $description") 115 | 116 | val metadata = 117 | Map("seqNos" -> offsetRanges, StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) 118 | val inputInfo = StreamInputInfo(id, rdd.count, metadata) 119 | ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) 120 | 121 | fromSeqNos = untilSeqNos 122 | Some(rdd) 123 | } 124 | 125 | override def start(): Unit = { 126 | EventHubsClient.userAgent = s"Spark-Streaming-${ssc.sc.version}" 127 | } 128 | 129 | override def stop(): Unit = { 130 | logInfo("stop: stopping EventHubDirectDStream") 131 | if (_client != null) _client.close() 132 | } 133 | 134 | private[eventhubs] class EventHubDirectDStreamCheckpointData extends DStreamCheckpointData(this) { 135 | import OffsetRange.OffsetRangeTuple 136 | 137 | def batchForTime: mutable.HashMap[Time, Array[OffsetRangeTuple]] = { 138 | data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRangeTuple]]] 139 | } 140 | 141 | override def update(time: Time): Unit = { 142 | batchForTime.clear() 143 | generatedRDDs.foreach { kv => 144 | val a = kv._2.asInstanceOf[EventHubsRDD].offsetRanges.map(_.toTuple) 145 | batchForTime += kv._1 -> a 146 | } 147 | } 148 | 149 | override def cleanup(time: Time): Unit = {} 150 | 151 | override def restore(): Unit = { 152 | batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { 153 | case (t, b) => 154 | logInfo(s"Restoring EventHubsRDD for time $t ${b.mkString("[", ", ", "]")}") 155 | generatedRDDs += t -> new EventHubsRDD(context.sparkContext, 156 | ehConf, 157 | b.map(OffsetRange(_)), 158 | clientFactory) 159 | } 160 | } 161 | } 162 | 163 | private[eventhubs] class EventHubDirectDStreamRateController(id: Int, estimator: RateEstimator) 164 | extends RateController(id, estimator) { 165 | override protected def publish(rate: Long): Unit = { 166 | // publish nothing as there is no receiver 167 | } 168 | } 169 | } 170 | 171 | private[eventhubs] object EventHubsDirectDStream { 172 | def getSortedExecutorList(sc: SparkContext): Array[String] = { 173 | val bm = sc.env.blockManager 174 | bm.master 175 | .getPeers(bm.blockManagerId) 176 | .toArray 177 | .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) 178 | .sortWith(compare) 179 | .map(_.toString) 180 | } 181 | 182 | private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { 183 | if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } 184 | } 185 | } 186 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/EventHubsConfSuite.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.eventhubs 19 | 20 | import org.apache.spark.eventhubs.utils.EventHubsTestUtils 21 | import org.json4s.NoTypeHints 22 | import org.json4s.jackson.Serialization 23 | import org.json4s.jackson.Serialization.{ read => sread } 24 | import org.json4s.jackson.Serialization.{ write => swrite } 25 | import org.scalatest.{ BeforeAndAfterAll, FunSuite } 26 | 27 | /** 28 | * Tests [[EventHubsConf]] for correctness. 29 | */ 30 | class EventHubsConfSuite extends FunSuite with BeforeAndAfterAll { 31 | 32 | import EventHubsConf._ 33 | import EventHubsTestUtils._ 34 | 35 | private implicit val formats = Serialization.formats(NoTypeHints) 36 | 37 | private var testUtils: EventHubsTestUtils = _ 38 | 39 | override def beforeAll: Unit = { 40 | testUtils = new EventHubsTestUtils 41 | testUtils.createEventHubs("name", partitionCount = 4) 42 | } 43 | 44 | override def afterAll(): Unit = { 45 | if (testUtils != null) { 46 | testUtils.destroyAllEventHubs() 47 | testUtils = null 48 | } 49 | } 50 | 51 | private def expectedConnStr = 52 | ConnectionStringBuilder() 53 | .setNamespaceName("namespace") 54 | .setEventHubName("name") 55 | .setSasKeyName("keyName") 56 | .setSasKey("key") 57 | .build 58 | 59 | test("set throws NullPointerException for null key and value") { 60 | val ehConf = testUtils.getEventHubsConf() 61 | intercept[NullPointerException] { ehConf.set(null, "value") } 62 | intercept[NullPointerException] { ehConf.set("key", null) } 63 | intercept[NullPointerException] { ehConf.set(null, null) } 64 | } 65 | 66 | test("set/apply/get are properly working") { 67 | val ehConf = testUtils.getEventHubsConf().set("some key", "some value") 68 | assert(ehConf("some key") == "some value") 69 | } 70 | 71 | test("toMap") { 72 | val map = testUtils.getEventHubsConf().toMap 73 | val eh = "name" 74 | 75 | val expectedPositions = Serialization.write( 76 | Map( 77 | NameAndPartition(eh, 0) -> EventPosition.fromSequenceNumber(0L), 78 | NameAndPartition(eh, 1) -> EventPosition.fromSequenceNumber(0L), 79 | NameAndPartition(eh, 2) -> EventPosition.fromSequenceNumber(0L), 80 | NameAndPartition(eh, 3) -> EventPosition.fromSequenceNumber(0L) 81 | ).map { case (k, v) => k.toString -> v } 82 | ) 83 | 84 | assert(map(ConnectionStringKey) == expectedConnStr) 85 | assert(map(ConsumerGroupKey) == "consumerGroup") 86 | intercept[Exception] { map(StartingPositionKey) } 87 | assert(map(StartingPositionsKey) == expectedPositions) 88 | assert(map(MaxRatePerPartitionKey).toRate == DefaultMaxRate) 89 | intercept[Exception] { map(MaxRatesPerPartitionKey) } 90 | intercept[Exception] { map(ReceiverTimeoutKey) } 91 | intercept[Exception] { map(OperationTimeoutKey) } 92 | intercept[Exception] { map(MaxEventsPerTriggerKey) } 93 | assert(map(UseSimulatedClientKey).toBoolean) 94 | } 95 | 96 | test("toConf") { 97 | val expectedPosition = EventPosition.fromSequenceNumber(20L) 98 | 99 | val expectedPositions = Map( 100 | NameAndPartition("name", 0) -> EventPosition.fromSequenceNumber(0L), 101 | NameAndPartition("name", 2) -> EventPosition.fromSequenceNumber(0L), 102 | NameAndPartition("name", 3) -> EventPosition.fromSequenceNumber(0L) 103 | ) 104 | 105 | val actualConf = EventHubsConf.toConf( 106 | Map( 107 | ConnectionStringKey -> expectedConnStr, 108 | ConsumerGroupKey -> "consumerGroup", 109 | StartingPositionKey -> Serialization.write(expectedPosition), 110 | StartingPositionsKey -> Serialization.write(expectedPositions.map { 111 | case (k, v) => k.toString -> v 112 | }), 113 | MaxEventsPerTriggerKey -> 4.toString 114 | )) 115 | 116 | val expectedConf = EventHubsConf(expectedConnStr) 117 | .setConsumerGroup("consumerGroup") 118 | .setStartingPosition(expectedPosition) 119 | .setStartingPositions(expectedPositions) 120 | .setMaxEventsPerTrigger(4L) 121 | 122 | assert(expectedConf.equals(actualConf)) 123 | } 124 | 125 | test("toMap, toConf: There and back again") { 126 | val expectedConf = testUtils.getEventHubsConf() 127 | 128 | val actualConf = EventHubsConf.toConf(expectedConf.clone.toMap) 129 | 130 | assert(expectedConf.equals(actualConf)) 131 | } 132 | 133 | test("clone") { 134 | val conf = testUtils.getEventHubsConf() 135 | val cloned = conf.clone 136 | assert(conf.equals(cloned)) 137 | assert(conf ne cloned) 138 | } 139 | 140 | test("name") { 141 | val conf = testUtils.getEventHubsConf() 142 | assert(conf.name == "name") 143 | } 144 | 145 | test("setName") { 146 | val conf = testUtils.getEventHubsConf() 147 | val expected = ConnectionStringBuilder(expectedConnStr) 148 | .setEventHubName("bar") 149 | .build 150 | 151 | conf.setName("bar") 152 | assert(conf.connectionString == expected) 153 | } 154 | 155 | test("EventPosition serialization") { 156 | implicit val formats = Serialization.formats(NoTypeHints) 157 | 158 | val expected = EventPosition.fromSequenceNumber(10L) 159 | val actual = sread[EventPosition](swrite[EventPosition](expected)) 160 | assert(actual.equals(expected)) 161 | } 162 | 163 | test("EventPosition is serialized correctly in EventHubsConf") { 164 | implicit val formats = Serialization.formats(NoTypeHints) 165 | 166 | val expected = EventPosition.fromSequenceNumber(10L) 167 | val conf = testUtils.getEventHubsConf().setStartingPosition(expected) 168 | val actual = conf.startingPosition.get 169 | assert(actual.equals(expected)) 170 | } 171 | 172 | test("Map of EventPositions can be serialized") { 173 | implicit val formats = Serialization.formats(NoTypeHints) 174 | 175 | val expected = Map( 176 | NameAndPartition("name", 0) -> EventPosition.fromSequenceNumber(3L), 177 | NameAndPartition("name", 1) -> EventPosition.fromSequenceNumber(2L), 178 | NameAndPartition("name", 2) -> EventPosition.fromSequenceNumber(1L), 179 | NameAndPartition("name", 3) -> EventPosition.fromSequenceNumber(0L) 180 | ) 181 | 182 | val stringKeys = expected.map { case (k, v) => k.toString -> v } 183 | 184 | val ser = swrite[Map[String, EventPosition]](stringKeys) 185 | val deser = sread[Map[String, EventPosition]](ser) 186 | 187 | val actual = deser map { case (k, v) => NameAndPartition.fromString(k) -> v } 188 | assert(actual.equals(expected)) 189 | } 190 | 191 | test("EventPosition Map is serialized in EventHubsConf") { 192 | implicit val formats = Serialization.formats(NoTypeHints) 193 | 194 | val expected = Map( 195 | NameAndPartition("name", 0) -> EventPosition.fromSequenceNumber(3L), 196 | NameAndPartition("name", 1) -> EventPosition.fromSequenceNumber(2L), 197 | NameAndPartition("name", 2) -> EventPosition.fromSequenceNumber(1L), 198 | NameAndPartition("name", 3) -> EventPosition.fromSequenceNumber(0L) 199 | ) 200 | 201 | val conf = testUtils.getEventHubsConf().setStartingPositions(expected) 202 | val actual = conf.startingPositions.get 203 | 204 | assert(actual.equals(expected)) 205 | } 206 | } 207 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/eventhubs/utils/EventHubsTestUtilsSuite.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.eventhubs.utils 19 | 20 | import java.util.concurrent.atomic.AtomicInteger 21 | 22 | import com.microsoft.azure.eventhubs.EventData 23 | import org.apache.spark.eventhubs.EventHubsConf 24 | import org.apache.spark.internal.Logging 25 | import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll, FunSuite } 26 | 27 | /** 28 | * Tests the functionality of the simulated EventHubs instance used for testing. 29 | */ 30 | class EventHubsTestUtilsSuite 31 | extends FunSuite 32 | with BeforeAndAfter 33 | with BeforeAndAfterAll 34 | with Logging { 35 | 36 | import EventHubsTestUtils._ 37 | 38 | private var testUtils: EventHubsTestUtils = _ 39 | 40 | override def beforeAll: Unit = { 41 | testUtils = new EventHubsTestUtils 42 | } 43 | 44 | override def afterAll(): Unit = { 45 | if (testUtils != null) { 46 | testUtils.destroyAllEventHubs() 47 | testUtils = null 48 | } 49 | } 50 | 51 | private def getEventHubsConf(name: String): EventHubsConf = testUtils.getEventHubsConf(name) 52 | 53 | private val eventHubsId = new AtomicInteger(0) 54 | 55 | def newEventHubs(): String = { 56 | s"eh-${eventHubsId.getAndIncrement()}" 57 | } 58 | 59 | test("Send one event to one partition") { 60 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 61 | eventHub.send(0, Seq(0)) 62 | 63 | val data = eventHub.getPartitions 64 | 65 | assert(data(0).getEvents.size == 1, "Partition 0 didn't have an event.") 66 | 67 | for (i <- 1 to 3) { 68 | assert(data(i).getEvents.isEmpty, "Partitions weren't empty") 69 | } 70 | } 71 | 72 | test("Send 500 events to all partitions") { 73 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 74 | testUtils.populateUniformly(eventHub.name, 500) 75 | 76 | val data = eventHub.getPartitions 77 | 78 | for (i <- 0 until eventHub.partitionCount) { 79 | assert(data(i).getEvents.size === 500) 80 | for (j <- 0 to 499) { 81 | assert(data(i).get(j).getSystemProperties.getSequenceNumber == j, 82 | "Sequence number doesn't match expected value.") 83 | } 84 | } 85 | } 86 | 87 | test("All partitions have different data.") { 88 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 89 | eventHub.send(0, Seq(1, 2, 3)) 90 | eventHub.send(1, Seq(4, 5, 6)) 91 | eventHub.send(2, Seq(7, 8, 9)) 92 | eventHub.send(3, Seq(10, 11, 12)) 93 | 94 | val data = eventHub.getPartitions 95 | 96 | assert(data(0).getEvents.map(_.getBytes.map(_.toChar).mkString.toInt) == Seq(1, 2, 3)) 97 | assert(data(1).getEvents.map(_.getBytes.map(_.toChar).mkString.toInt) == Seq(4, 5, 6)) 98 | assert(data(2).getEvents.map(_.getBytes.map(_.toChar).mkString.toInt) == Seq(7, 8, 9)) 99 | assert(data(3).getEvents.map(_.getBytes.map(_.toChar).mkString.toInt) == Seq(10, 11, 12)) 100 | } 101 | 102 | test("translate") { 103 | val eh = newEventHubs() 104 | testUtils.createEventHubs(eh, DefaultPartitionCount) 105 | val conf = getEventHubsConf(eh) 106 | val client = SimulatedClient(conf) 107 | 108 | val actual = client.translate(conf, client.partitionCount) 109 | val expected = conf.startingPositions.get.map { case (k, v) => k.partitionId -> v.seqNo } 110 | 111 | assert(actual === expected) 112 | } 113 | 114 | test("Test simulated receiver") { 115 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 116 | testUtils.populateUniformly(eventHub.name, 500) 117 | 118 | val data = eventHub.getPartitions 119 | 120 | for (i <- 0 until eventHub.partitionCount) { 121 | assert(data(i).getEvents.size === 500) 122 | for (j <- 0 to 499) { 123 | assert(data(i).get(j).getSystemProperties.getSequenceNumber == j, 124 | "Sequence number doesn't match expected value.") 125 | } 126 | } 127 | 128 | val conf = testUtils.getEventHubsConf(eventHub.name) 129 | val client = SimulatedClient(conf) 130 | client.createReceiver(partitionId = "0", 20) 131 | val event = client.receive(1) 132 | assert(event.iterator.next.getSystemProperties.getSequenceNumber === 20) 133 | } 134 | 135 | test("latestSeqNo") { 136 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 137 | 138 | eventHub.send(0, Seq(1)) 139 | eventHub.send(1, Seq(2, 3)) 140 | eventHub.send(2, Seq(4, 5, 6)) 141 | eventHub.send(3, Seq(7)) 142 | 143 | val conf = testUtils.getEventHubsConf(eventHub.name) 144 | val client = SimulatedClient(conf) 145 | assert(client.latestSeqNo(0) == 1) 146 | assert(client.latestSeqNo(1) == 2) 147 | assert(client.latestSeqNo(2) == 3) 148 | assert(client.latestSeqNo(3) == 1) 149 | } 150 | 151 | test("partitionSize") { 152 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 153 | 154 | assert(eventHub.partitionSize(0) == 0) 155 | assert(eventHub.partitionSize(1) == 0) 156 | assert(eventHub.partitionSize(2) == 0) 157 | assert(eventHub.partitionSize(3) == 0) 158 | 159 | eventHub.send(0, Seq(1)) 160 | eventHub.send(1, Seq(2, 3)) 161 | eventHub.send(2, Seq(4, 5, 6)) 162 | eventHub.send(3, Seq(7)) 163 | 164 | assert(eventHub.partitionSize(0) == 1) 165 | assert(eventHub.partitionSize(1) == 2) 166 | assert(eventHub.partitionSize(2) == 3) 167 | assert(eventHub.partitionSize(3) == 1) 168 | } 169 | 170 | test("totalSize") { 171 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 172 | 173 | assert(eventHub.totalSize == 0) 174 | 175 | eventHub.send(0, Seq(1)) 176 | eventHub.send(1, Seq(2, 3)) 177 | eventHub.send(2, Seq(4, 5, 6)) 178 | eventHub.send(3, Seq(7)) 179 | 180 | assert(eventHub.totalSize == 7) 181 | } 182 | 183 | test("send EventData") { 184 | // events are sent round-robin, so the first event will go to partition 0. 185 | val part = 0 186 | 187 | val eh = newEventHubs() 188 | testUtils.createEventHubs(eh, partitionCount = 10) 189 | 190 | val ehConf = getEventHubsConf(eh) 191 | val client = new SimulatedClient(ehConf) 192 | val event = EventData.create("1".getBytes) 193 | client.send(event) 194 | 195 | assert(testUtils.getEventHubs(eh).getPartitions(part).size == 1) 196 | assert( 197 | testUtils 198 | .getEventHubs(eh) 199 | .getPartitions(part) 200 | .getEvents 201 | .head 202 | .getBytes 203 | .sameElements(event.getBytes)) 204 | 205 | } 206 | 207 | test("send EventData to specific partition") { 208 | // use this partition in the partition sender 209 | val part = 7 210 | 211 | val eh = newEventHubs() 212 | testUtils.createEventHubs(eh, partitionCount = 10) 213 | 214 | val ehConf = getEventHubsConf(eh) 215 | val client = new SimulatedClient(ehConf) 216 | val event = EventData.create("1".getBytes) 217 | client.send(event, part) 218 | 219 | assert(testUtils.getEventHubs(eh).getPartitions(part).size == 1) 220 | assert( 221 | testUtils 222 | .getEventHubs(eh) 223 | .getPartitions(part) 224 | .getEvents 225 | .head 226 | .getBytes 227 | .sameElements(event.getBytes)) 228 | 229 | } 230 | } 231 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 22 | 23 | 24 | 4.0.0 25 | 26 | com.microsoft.azure 27 | azure-eventhubs-spark-parent_2.11 28 | 2.3.1 29 | pom 30 | 31 | EventHubs+Spark Parent POM 32 | Libraries to connect (and demonstrate) Azure Event Hubs with Apache Spark. 33 | https://github.com/Azure/azure-event-hubs-spark 34 | 35 | 36 | 37 | The Apache License, Version 2.0 38 | http://www.apache.org/licenses/LICENSE-2.0.txt 39 | 40 | 41 | 42 | 43 | 44 | Sabee Grewal 45 | sagrewal@microsoft.com 46 | Microsoft Corporation 47 | http://www.microsoft.com 48 | 49 | 50 | 51 | 52 | azure-eventhubs-spark-parent 53 | 2.11 54 | 2.3.0 55 | github 56 | 57 | 58 | 59 | core 60 | 61 | 62 | 63 | 64 | 65 | release-to-github 66 | 67 | 68 | github.repo 69 | Temporary Staging Repository 70 | file://${project.build.directory}/mvn-repo 71 | 72 | 73 | 74 | github 75 | 76 | 77 | 78 | 79 | com.github.github 80 | site-maven-plugin 81 | 0.12 82 | 83 | Maven artifacts for ${project.version} 84 | true 85 | ${project.build.directory}/mvn-repo 86 | refs/heads/maven-repo 87 | **/* 88 | azure-event-hubs-spark 89 | Azure 90 | true 91 | 92 | 93 | 94 | 95 | 96 | site 97 | 98 | deploy 99 | 100 | 101 | 102 | 103 | org.apache.maven.plugins 104 | maven-deploy-plugin 105 | 2.8.2 106 | 107 | internal.repo::default::file://${project.build.directory}/mvn-repo 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | scm:git:git:/github.com/Azure/azure-event-hubs-spark.git 117 | scm:git:ssh://github.com/Azure/azure-event-hubs-spark.git 118 | https://github.com/Azure/azure-event-hubs-spark/tree/master 119 | 120 | 121 | 122 | 123 | org.apache.spark 124 | spark-streaming_${scala.binary.version} 125 | ${spark.version} 126 | provided 127 | 128 | 129 | org.apache.spark 130 | spark-sql_2.11 131 | ${spark.version} 132 | 133 | 134 | com.microsoft.azure 135 | azure-eventhubs 136 | 1.0.1 137 | 138 | 139 | org.apache.spark 140 | spark-core_${scala.binary.version} 141 | ${spark.version} 142 | test-jar 143 | test 144 | 145 | 146 | org.apache.spark 147 | spark-sql_2.11 148 | ${spark.version} 149 | test-jar 150 | test 151 | 152 | 153 | org.mockito 154 | mockito-core 155 | 1.10.8 156 | test 157 | 158 | 159 | org.scalatest 160 | scalatest_${scala.binary.version} 161 | test 162 | 3.0.3 163 | 164 | 165 | 166 | 167 | target/scala-${scala.binary.version}/classes 168 | target/scala-${scala.binary.version}/test-classes 169 | 170 | 171 | org.scala-tools 172 | maven-scala-plugin 173 | 2.15.2 174 | 175 | 176 | 177 | compile 178 | testCompile 179 | 180 | 181 | 182 | 183 | 184 | org.scalatest 185 | scalatest-maven-plugin 186 | 1.0 187 | 188 | ${project.build.directory}/surefire-reports 189 | . 190 | SparkTestSuite.txt 191 | -Xmx512M 192 | 193 | 194 | 195 | test 196 | 197 | test 198 | 199 | 200 | 201 | 202 | 203 | org.apache.maven.plugins 204 | maven-jar-plugin 205 | 3.0.2 206 | 207 | 208 | empty-javadoc-jar 209 | package 210 | 211 | jar 212 | 213 | 214 | javadoc 215 | ${basedir}/javadoc 216 | 217 | 218 | 219 | 220 | 221 | org.apache.maven.plugins 222 | maven-javadoc-plugin 223 | 2.9.1 224 | 225 | 226 | attach-javadocs 227 | 228 | jar 229 | 230 | 231 | 232 | 233 | 234 | 235 | 236 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/client/EventHubsClient.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.eventhubs.client 19 | 20 | import java.time.Duration 21 | import java.util.concurrent.ConcurrentHashMap 22 | 23 | import com.microsoft.azure.eventhubs._ 24 | import com.microsoft.azure.eventhubs.impl.EventHubClientImpl 25 | import org.apache.spark.{ SparkEnv, TaskContext } 26 | import org.apache.spark.eventhubs.EventHubsConf 27 | import org.apache.spark.internal.Logging 28 | import org.json4s.NoTypeHints 29 | import org.json4s.jackson.Serialization 30 | 31 | import scala.collection.JavaConverters._ 32 | import scala.collection.mutable.ArrayBuffer 33 | 34 | /** 35 | * Wraps a raw EventHubReceiver to make it easier for unit tests 36 | */ 37 | @SerialVersionUID(1L) 38 | private[spark] class EventHubsClient(private val ehConf: EventHubsConf) 39 | extends Serializable 40 | with Client 41 | with Logging { 42 | 43 | import org.apache.spark.eventhubs._ 44 | 45 | private implicit val formats = Serialization.formats(NoTypeHints) 46 | 47 | private var _client: EventHubClient = _ 48 | private def client = synchronized { 49 | if (_client == null) { 50 | _client = ClientConnectionPool.borrowClient(ehConf) 51 | } 52 | _client 53 | } 54 | 55 | private var receiver: PartitionReceiver = _ 56 | override def createReceiver(partitionId: String, startingSeqNo: SequenceNumber): Unit = { 57 | if (receiver == null) { 58 | val consumerGroup = ehConf.consumerGroup.getOrElse(DefaultConsumerGroup) 59 | val receiverOptions = new ReceiverOptions 60 | receiverOptions.setReceiverRuntimeMetricEnabled(false) 61 | receiverOptions.setIdentifier(s"${SparkEnv.get.executorId}-${TaskContext.get.taskAttemptId}") 62 | logInfo(s"Starting receiver for partitionId $partitionId from seqNo $startingSeqNo") 63 | receiver = client 64 | .createReceiverSync(consumerGroup, 65 | partitionId, 66 | EventPosition.fromSequenceNumber(startingSeqNo).convert, 67 | receiverOptions) 68 | receiver.setReceiveTimeout(ehConf.receiverTimeout.getOrElse(DefaultReceiverTimeout)) 69 | } 70 | } 71 | 72 | private var partitionSender: PartitionSender = _ 73 | override def createPartitionSender(partitionId: Int): Unit = { 74 | val id = partitionId.toString 75 | if (partitionSender == null) { 76 | logInfo(s"Creating partition sender for $partitionId for EventHub ${client.getEventHubName}") 77 | partitionSender = client.createPartitionSenderSync(id) 78 | } else if (partitionSender.getPartitionId != id) { 79 | logInfo( 80 | s"Closing partition sender for ${partitionSender.getPartitionId} for EventHub ${client.getEventHubName}") 81 | partitionSender.closeSync() 82 | logInfo(s"Creating partition sender for $partitionId for EventHub ${client.getEventHubName}") 83 | partitionSender = client.createPartitionSenderSync(id) 84 | } 85 | } 86 | 87 | override def send(event: EventData): Unit = { 88 | client.sendSync(event) 89 | } 90 | 91 | override def send(event: EventData, partitionKey: String): Unit = { 92 | client.sendSync(event, partitionKey) 93 | } 94 | 95 | override def send(event: EventData, partitionId: Int): Unit = { 96 | require(partitionSender.getPartitionId.toInt == partitionId) 97 | partitionSender.sendSync(event) 98 | } 99 | 100 | override def setPrefetchCount(count: Int): Unit = { 101 | receiver.setPrefetchCount(count) 102 | } 103 | 104 | override def receive(eventCount: Int): java.lang.Iterable[EventData] = { 105 | require(receiver != null, "receive: PartitionReceiver has not been created.") 106 | receiver.receiveSync(eventCount) 107 | } 108 | 109 | // Note: the EventHubs Java Client will retry this API call on failure 110 | private def getRunTimeInfo(partitionId: PartitionId): PartitionRuntimeInformation = { 111 | try { 112 | client.getPartitionRuntimeInformation(partitionId.toString).get 113 | } catch { 114 | case e: Exception => throw e 115 | } 116 | } 117 | 118 | /** 119 | * return the start seq number of each partition 120 | * 121 | * @return a map from eventhubName-partition to seq 122 | */ 123 | override def earliestSeqNo(partitionId: PartitionId): SequenceNumber = { 124 | try { 125 | val runtimeInformation = getRunTimeInfo(partitionId) 126 | val seqNo = runtimeInformation.getBeginSequenceNumber 127 | if (seqNo == -1L) 0L else seqNo 128 | } catch { 129 | case e: Exception => throw e 130 | } 131 | } 132 | 133 | /** 134 | * Returns the end point of each partition 135 | * 136 | * @return a map from eventhubName-partition to (offset, seq) 137 | */ 138 | override def latestSeqNo(partitionId: PartitionId): SequenceNumber = { 139 | try { 140 | val runtimeInfo = getRunTimeInfo(partitionId) 141 | runtimeInfo.getLastEnqueuedSequenceNumber + 1 142 | } catch { 143 | case e: Exception => throw e 144 | } 145 | } 146 | 147 | override def boundedSeqNos(partitionId: PartitionId): (SequenceNumber, SequenceNumber) = { 148 | try { 149 | val runtimeInfo = getRunTimeInfo(partitionId) 150 | val earliest = 151 | if (runtimeInfo.getBeginSequenceNumber == -1L) 0L else runtimeInfo.getBeginSequenceNumber 152 | val latest = runtimeInfo.getLastEnqueuedSequenceNumber + 1 153 | (earliest, latest) 154 | } catch { 155 | case e: Exception => throw e 156 | } 157 | } 158 | 159 | private var _partitionCount = -1 160 | 161 | /** 162 | * The number of partitions in the EventHubs instance. 163 | * 164 | * @return partition count 165 | */ 166 | override def partitionCount: Int = { 167 | if (_partitionCount == -1) { 168 | try { 169 | val runtimeInfo = client.getRuntimeInformation.get 170 | _partitionCount = runtimeInfo.getPartitionCount 171 | } catch { 172 | case e: Exception => throw e 173 | } 174 | } 175 | _partitionCount 176 | } 177 | 178 | override def close(): Unit = { 179 | logInfo("close: Closing EventHubsClient.") 180 | if (receiver != null) { 181 | receiver.closeSync() 182 | receiver = null 183 | } 184 | if (partitionSender != null) { 185 | partitionSender.closeSync() 186 | partitionSender = null 187 | } 188 | if (_client != null) { 189 | ClientConnectionPool.returnClient(_client) 190 | _client = null 191 | } 192 | } 193 | 194 | /** 195 | * Convert any starting positions to the corresponding sequence number. 196 | */ 197 | override def translate[T](ehConf: EventHubsConf, 198 | partitionCount: Int, 199 | useStart: Boolean = true): Map[PartitionId, SequenceNumber] = { 200 | val result = new ConcurrentHashMap[PartitionId, SequenceNumber]() 201 | val needsTranslation = ArrayBuffer[NameAndPartition]() 202 | 203 | logInfo(s"translate: useStart is set to $useStart.") 204 | val positions = if (useStart) { 205 | ehConf.startingPositions.getOrElse(Map.empty).par 206 | } else { 207 | ehConf.endingPositions.getOrElse(Map.empty).par 208 | } 209 | val defaultPos = if (useStart) { 210 | ehConf.startingPosition.getOrElse(DefaultEventPosition) 211 | } else { 212 | ehConf.endingPosition.getOrElse(DefaultEndingPosition) 213 | } 214 | logInfo(s"translate: PerPartitionPositions = $positions") 215 | logInfo(s"translate: Default position = $defaultPos") 216 | 217 | // Partitions which have a sequence number position are put in result. 218 | // All other partitions need to be translated into sequence numbers by the service. 219 | (0 until partitionCount).par.foreach { id => 220 | val nAndP = NameAndPartition(ehConf.name, id) 221 | val position = positions.getOrElse(nAndP, defaultPos) 222 | if (position.seqNo >= 0L) { 223 | result.put(id, position.seqNo) 224 | } else { 225 | synchronized(needsTranslation += nAndP) 226 | } 227 | } 228 | logInfo(s"translate: needsTranslation = $needsTranslation") 229 | 230 | val consumerGroup = ehConf.consumerGroup.getOrElse(DefaultConsumerGroup) 231 | val threads = ArrayBuffer[Thread]() 232 | needsTranslation.foreach(nAndP => { 233 | val partitionId = nAndP.partitionId 234 | threads += new Thread { 235 | override def run(): Unit = { 236 | @volatile var receiver: PartitionReceiver = null 237 | try { 238 | receiver = client 239 | .createReceiverSync(consumerGroup, 240 | partitionId.toString, 241 | positions.getOrElse(nAndP, defaultPos).convert) 242 | receiver.setPrefetchCount(PrefetchCountMinimum) 243 | receiver.setReceiveTimeout(Duration.ofSeconds(5)) 244 | val events = receiver.receiveSync(1) // get the first event that was received. 245 | if (events == null || !events.iterator.hasNext) { 246 | logWarning("translate: failed to receive event.") 247 | // No events to receive can happen in 2 cases: 248 | // 1. Receive from EndOfStream and no new events arrive 249 | // 2. Receive from an empty partition 250 | val (earliest, latest) = boundedSeqNos(partitionId) 251 | if (earliest >= latest) { 252 | result.put(partitionId, earliest) 253 | } else { 254 | result.put(partitionId, latest) 255 | } 256 | } else { 257 | val event = events.iterator.next 258 | result.put(partitionId, event.getSystemProperties.getSequenceNumber) 259 | } 260 | } catch { 261 | case e: IllegalEntityException => 262 | logError("translate: IllegalEntityException. Consumer group may not exist.", e) 263 | throw e 264 | } finally { 265 | if (receiver != null) { 266 | receiver.closeSync() 267 | } 268 | } 269 | } 270 | } 271 | }) 272 | 273 | logInfo("translate: Starting threads to translate to sequence number.") 274 | threads.foreach(_.start()) 275 | threads.foreach(_.join()) 276 | logInfo("translate: Translation complete.") 277 | logInfo(s"translate: result = $result") 278 | 279 | assert(result.size == partitionCount, 280 | s"translate: result size ${result.size} does not equal partition count $partitionCount") 281 | 282 | result.asScala.toMap 283 | .mapValues { seqNo => 284 | { if (seqNo == -1L) 0L else seqNo } 285 | } 286 | .map(identity) 287 | } 288 | } 289 | 290 | private[spark] object EventHubsClient { 291 | private[spark] def apply(ehConf: EventHubsConf): EventHubsClient = 292 | new EventHubsClient(ehConf) 293 | 294 | def userAgent: String = { 295 | EventHubClientImpl.USER_AGENT 296 | } 297 | 298 | def userAgent_=(str: String) { 299 | EventHubClientImpl.USER_AGENT = str 300 | } 301 | } 302 | -------------------------------------------------------------------------------- /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 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/eventhubs/EventHubsSinkSuite.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.sql.eventhubs 19 | 20 | import java.util.Locale 21 | import java.util.concurrent.atomic.AtomicInteger 22 | import org.scalatest.time.SpanSugar._ 23 | 24 | import org.apache.spark.eventhubs._ 25 | import org.apache.spark.eventhubs.utils.EventHubsTestUtils 26 | import org.apache.spark.sql._ 27 | import org.apache.spark.sql.functions._ 28 | import org.apache.spark.sql.execution.streaming.MemoryStream 29 | import org.apache.spark.sql.streaming._ 30 | import org.apache.spark.sql.test.SharedSQLContext 31 | import org.scalatest.time.Span 32 | 33 | class EventHubsSinkSuite extends StreamTest with SharedSQLContext { 34 | import testImplicits._ 35 | import EventHubsTestUtils._ 36 | 37 | protected var testUtils: EventHubsTestUtils = _ 38 | 39 | override val streamingTimeout: Span = 30.seconds 40 | 41 | override def beforeAll(): Unit = { 42 | super.beforeAll() 43 | testUtils = new EventHubsTestUtils 44 | } 45 | 46 | override def afterAll(): Unit = { 47 | if (testUtils != null) { 48 | testUtils.destroyAllEventHubs() 49 | testUtils = null 50 | super.afterAll() 51 | } 52 | } 53 | 54 | private val eventHubId = new AtomicInteger(0) 55 | 56 | private def newEventHub(): String = s"eh-${eventHubId.getAndIncrement}" 57 | 58 | private def getEventHubsConf(name: String) = testUtils.getEventHubsConf(name) 59 | 60 | private def createReader(ehConf: EventHubsConf): DataFrame = { 61 | spark.read 62 | .format("eventhubs") 63 | .options(ehConf.toMap) 64 | .load() 65 | .select($"body" cast "string") 66 | } 67 | 68 | private def createEventHubsWriter( 69 | input: DataFrame, 70 | ehConf: EventHubsConf, 71 | withOutputMode: Option[OutputMode] = None)(withSelectExrp: String*): StreamingQuery = { 72 | var stream: DataStreamWriter[Row] = null 73 | withTempDir { checkpointDir => 74 | var df = input.toDF().withColumnRenamed("value", "body") 75 | if (withSelectExrp.nonEmpty) { 76 | df = df.selectExpr(withSelectExrp: _*) 77 | } 78 | stream = df.writeStream 79 | .format("eventhubs") 80 | .options(ehConf.toMap) 81 | .option("checkpointLocation", checkpointDir.getCanonicalPath) 82 | .queryName("eventHubStream") 83 | withOutputMode.foreach(stream.outputMode(_)) 84 | } 85 | stream.start() 86 | } 87 | 88 | test("batch - write to EventHubs") { 89 | val eh = newEventHub() 90 | testUtils.createEventHubs(eh, DefaultPartitionCount) 91 | val ehConf = getEventHubsConf(eh) 92 | val df = Seq("1", "2", "3", "4", "5").toDF("body") 93 | 94 | df.write 95 | .format("eventhubs") 96 | .options(ehConf.toMap) 97 | .save() 98 | 99 | checkAnswer(createReader(ehConf), 100 | Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) 101 | } 102 | 103 | test("batch - write to specific partition id") { 104 | val eh = newEventHub() 105 | val targetPartition = "0" 106 | testUtils.createEventHubs(eh, DefaultPartitionCount) 107 | 108 | val ehConf = getEventHubsConf(eh) 109 | val df = Seq("1", "2", "3", "4", "5").map(v => (targetPartition, v)).toDF("partitionId", "body") 110 | 111 | df.write 112 | .format("eventhubs") 113 | .options(ehConf.toMap) 114 | .save() 115 | 116 | assert(testUtils.getEventHubs(eh).getPartitions(targetPartition.toInt).size == 5) 117 | checkAnswer(createReader(ehConf), 118 | Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) 119 | } 120 | 121 | test("batch - unsupported save modes") { 122 | val eh = newEventHub() 123 | testUtils.createEventHubs(eh, DefaultPartitionCount) 124 | val ehConf = getEventHubsConf(eh) 125 | val df = Seq[(String, String)](("0", "1")).toDF("partitionId", "body") 126 | 127 | // Test bad save mode Ignore 128 | var ex = intercept[AnalysisException] { 129 | df.write 130 | .format("eventhubs") 131 | .options(ehConf.toMap) 132 | .mode(SaveMode.Ignore) 133 | .save() 134 | } 135 | assert( 136 | ex.getMessage 137 | .toLowerCase(Locale.ROOT) 138 | .contains(s"save mode ignore not allowed for eventhubs")) 139 | 140 | // Test bad save mode Overwrite 141 | ex = intercept[AnalysisException] { 142 | df.write 143 | .format("eventhubs") 144 | .mode(SaveMode.Overwrite) 145 | .save() 146 | } 147 | assert( 148 | ex.getMessage 149 | .toLowerCase(Locale.ROOT) 150 | .contains(s"save mode overwrite not allowed for eventhubs")) 151 | } 152 | 153 | test("SPARK-20496: batch - enforce analyzed plans") { 154 | val inputEvents = 155 | spark 156 | .range(1, 1000) 157 | .select(to_json(struct("*")) as 'body) 158 | 159 | val eh = newEventHub() 160 | testUtils.createEventHubs(eh, DefaultPartitionCount) 161 | val ehConf = getEventHubsConf(eh) 162 | // Should not throw UnresolvedException 163 | inputEvents.write 164 | .format("eventhubs") 165 | .options(ehConf.toMap) 166 | .save() 167 | } 168 | 169 | test("streaming - write to eventhubs") { 170 | val input = MemoryStream[String] 171 | val eh = newEventHub() 172 | testUtils.createEventHubs(eh, DefaultPartitionCount) 173 | val ehConf = getEventHubsConf(eh) 174 | 175 | val writer = createEventHubsWriter( 176 | input.toDF, 177 | ehConf, 178 | withOutputMode = Some(OutputMode.Append) 179 | )("body") 180 | 181 | val reader = (e: EventHubsConf) => createReader(e).as[String].map(_.toInt) 182 | 183 | try { 184 | input.addData("1", "2", "3", "4", "5") 185 | failAfter(streamingTimeout) { 186 | writer.processAllAvailable() 187 | } 188 | checkDatasetUnorderly(reader(ehConf), 1, 2, 3, 4, 5) 189 | input.addData("6", "7", "8", "9", "10") 190 | failAfter(streamingTimeout) { 191 | writer.processAllAvailable() 192 | } 193 | checkDatasetUnorderly(reader(ehConf), 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) 194 | } finally { 195 | writer.stop() 196 | } 197 | } 198 | 199 | test("streaming - write to specific partition") { 200 | val targetPart = "0" 201 | val input = MemoryStream[String] 202 | val eh = newEventHub() 203 | testUtils.createEventHubs(eh, partitionCount = 10) 204 | val ehConf = getEventHubsConf(eh) 205 | 206 | val writer = createEventHubsWriter( 207 | input.toDF, 208 | ehConf, 209 | withOutputMode = Some(OutputMode.Update()) 210 | )(s"'$targetPart' as partitionId", "body") 211 | 212 | val reader = (e: EventHubsConf) => createReader(e).as[String].map(_.toInt) 213 | 214 | try { 215 | input.addData("1", "2", "2", "3", "3", "3") 216 | failAfter(streamingTimeout) { 217 | writer.processAllAvailable() 218 | } 219 | assert(testUtils.getEventHubs(eh).getPartitions(targetPart.toPartitionId).size == 6) 220 | checkDatasetUnorderly(reader(ehConf), 1, 2, 2, 3, 3, 3) 221 | input.addData("1", "2", "3") 222 | failAfter(streamingTimeout) { 223 | writer.processAllAvailable() 224 | } 225 | assert(testUtils.getEventHubs(eh).getPartitions(targetPart.toPartitionId).size == 9) 226 | checkDatasetUnorderly(reader(ehConf), 1, 2, 2, 3, 3, 3, 1, 2, 3) 227 | } finally { 228 | writer.stop() 229 | } 230 | } 231 | 232 | test("streaming - write data with bad schema - no body field") { 233 | val input = MemoryStream[String] 234 | val eh = newEventHub() 235 | testUtils.createEventHubs(eh, partitionCount = 10) 236 | val ehConf = getEventHubsConf(eh) 237 | 238 | var writer: StreamingQuery = null 239 | var ex: Exception = null 240 | try { 241 | ex = intercept[StreamingQueryException] { 242 | writer = createEventHubsWriter(input.toDF(), ehConf)("body as foo") 243 | input.addData("1", "2", "3", "4", "5") 244 | writer.processAllAvailable() 245 | } 246 | } finally { 247 | writer.stop() 248 | } 249 | assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("required attribute 'body' not found.")) 250 | } 251 | 252 | test("streaming - write data with bad schema - partitionKey and partitionId have been set") { 253 | val input = MemoryStream[String] 254 | val eh = newEventHub() 255 | testUtils.createEventHubs(eh, partitionCount = 10) 256 | val ehConf = getEventHubsConf(eh) 257 | 258 | var writer: StreamingQuery = null 259 | var ex: Exception = null 260 | val partitionKey = "foo" 261 | val partitionId = "0" 262 | try { 263 | ex = intercept[StreamingQueryException] { 264 | writer = createEventHubsWriter(input.toDF(), ehConf)(s"'$partitionKey' as partitionKey", 265 | s"'$partitionId' as partitionId", 266 | "body") 267 | input.addData("1", "2", "3", "4", "5") 268 | writer.processAllAvailable() 269 | } 270 | } finally { 271 | writer.stop() 272 | } 273 | assert( 274 | ex.getMessage 275 | .toLowerCase(Locale.ROOT) 276 | .contains( 277 | s"both a partitionkey ($partitionKey) and partitionid ($partitionId) have been detected. both can not be set.")) 278 | } 279 | 280 | test("streaming - write data with valid schema but wrong type - bad body type") { 281 | val input = MemoryStream[String] 282 | val eh = newEventHub() 283 | testUtils.createEventHubs(eh, partitionCount = 10) 284 | val ehConf = getEventHubsConf(eh) 285 | 286 | var writer: StreamingQuery = null 287 | var ex: Exception = null 288 | try { 289 | ex = intercept[StreamingQueryException] { 290 | writer = createEventHubsWriter(input.toDF(), ehConf)("CAST (body as INT) as body") 291 | input.addData("1", "2", "3", "4", "5") 292 | writer.processAllAvailable() 293 | } 294 | } finally { 295 | writer.stop() 296 | } 297 | assert( 298 | ex.getMessage 299 | .toLowerCase(Locale.ROOT) 300 | .contains("body attribute type must be a string or binarytype")) 301 | } 302 | 303 | test("streaming - write data with valid schema but wrong type - bad partitionId type") { 304 | val input = MemoryStream[String] 305 | val eh = newEventHub() 306 | testUtils.createEventHubs(eh, partitionCount = 10) 307 | val ehConf = getEventHubsConf(eh) 308 | 309 | var writer: StreamingQuery = null 310 | var ex: Exception = null 311 | val partitionId = "0" 312 | try { 313 | ex = intercept[StreamingQueryException] { 314 | writer = 315 | createEventHubsWriter(input.toDF(), ehConf)(s"CAST('$partitionId' as INT) as partitionId", 316 | "body") 317 | input.addData("1", "2", "3", "4", "5") 318 | writer.processAllAvailable() 319 | } 320 | } finally { 321 | writer.stop() 322 | } 323 | assert( 324 | ex.getMessage 325 | .toLowerCase(Locale.ROOT) 326 | .contains(s"partitionid attribute unsupported type")) 327 | } 328 | 329 | test("streaming - write data with valid schema but wrong type - bad partitionKey type") { 330 | val input = MemoryStream[String] 331 | val eh = newEventHub() 332 | testUtils.createEventHubs(eh, partitionCount = 10) 333 | val ehConf = getEventHubsConf(eh) 334 | 335 | var writer: StreamingQuery = null 336 | var ex: Exception = null 337 | val partitionKey = "234" 338 | try { 339 | ex = intercept[StreamingQueryException] { 340 | writer = createEventHubsWriter(input.toDF(), ehConf)( 341 | s"CAST('$partitionKey' as INT) as partitionKey", 342 | "body") 343 | input.addData("1", "2", "3", "4", "5") 344 | writer.processAllAvailable() 345 | } 346 | } finally { 347 | writer.stop() 348 | } 349 | assert( 350 | ex.getMessage 351 | .toLowerCase(Locale.ROOT) 352 | .contains(s"partitionkey attribute unsupported type")) 353 | } 354 | } 355 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/eventhubs/utils/EventHubsTestUtils.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.eventhubs.utils 19 | 20 | import java.util.Date 21 | 22 | import com.microsoft.azure.eventhubs.EventData 23 | import com.microsoft.azure.eventhubs.impl.AmqpConstants.{ 24 | ENQUEUED_TIME_UTC, 25 | OFFSET, 26 | SEQUENCE_NUMBER 27 | } 28 | import com.microsoft.azure.eventhubs.impl.EventDataImpl 29 | import org.apache.qpid.proton.amqp.Binary 30 | import org.apache.qpid.proton.amqp.messaging.{ Data, MessageAnnotations } 31 | import org.apache.qpid.proton.message.Message 32 | import org.apache.qpid.proton.message.Message.Factory 33 | import org.apache.spark.eventhubs.{ EventHubsConf, NameAndPartition } 34 | import org.apache.spark.eventhubs.client.Client 35 | import org.apache.spark.eventhubs._ 36 | 37 | import scala.collection.JavaConverters._ 38 | import scala.collection.mutable 39 | 40 | /** 41 | * Test classes used to simulate an EventHubs instance. 42 | * 43 | * In main directory (instead of test) so we can inject SimulatedClient 44 | * in our DStream and Source tests. 45 | */ 46 | private[spark] class EventHubsTestUtils { 47 | 48 | import EventHubsTestUtils._ 49 | 50 | def send(ehName: String, data: Seq[Int]): Seq[Int] = { 51 | eventHubs(ehName).send(data) 52 | } 53 | 54 | def send(ehName: String, partitionId: PartitionId, data: Seq[Int]): Seq[Int] = { 55 | eventHubs(ehName).send(partitionId, data) 56 | } 57 | 58 | def getLatestSeqNos(ehConf: EventHubsConf): Map[NameAndPartition, SequenceNumber] = { 59 | val n = ehConf.name 60 | (for { 61 | p <- 0 until eventHubs(n).partitionCount 62 | seqNo = eventHubs(n).latestSeqNo(p) 63 | } yield NameAndPartition(n, p) -> seqNo).toMap 64 | } 65 | 66 | def getEventHubs(ehName: String): SimulatedEventHubs = { 67 | eventHubs(ehName) 68 | } 69 | 70 | def createEventHubs(ehName: String, partitionCount: Int): SimulatedEventHubs = { 71 | EventHubsTestUtils.eventHubs.put(ehName, new SimulatedEventHubs(ehName, partitionCount)) 72 | eventHubs(ehName) 73 | } 74 | 75 | def destroyEventHubs(ehName: String): Unit = { 76 | eventHubs.remove(ehName) 77 | } 78 | 79 | def destroyAllEventHubs(): Unit = { 80 | eventHubs.clear 81 | } 82 | 83 | def getEventHubsConf(ehName: String = "name"): EventHubsConf = { 84 | val partitionCount = getEventHubs(ehName).partitionCount 85 | 86 | val connectionString = ConnectionStringBuilder() 87 | .setNamespaceName("namespace") 88 | .setEventHubName(ehName) 89 | .setSasKeyName("keyName") 90 | .setSasKey("key") 91 | .build 92 | 93 | val positions: Map[NameAndPartition, EventPosition] = (for { 94 | partitionId <- 0 until partitionCount 95 | } yield NameAndPartition(ehName, partitionId) -> EventPosition.fromSequenceNumber(0L)).toMap 96 | 97 | EventHubsConf(connectionString) 98 | .setConsumerGroup("consumerGroup") 99 | .setStartingPositions(positions) 100 | .setMaxRatePerPartition(DefaultMaxRate) 101 | .setUseSimulatedClient(true) 102 | } 103 | 104 | // Put 'count' events in every simulated EventHubs partition 105 | def populateUniformly(ehName: String, count: Int): Unit = { 106 | val eventHub = eventHubs(ehName) 107 | for (i <- 0 until eventHub.partitionCount) { 108 | eventHub.send(i, 0 until count) 109 | } 110 | } 111 | } 112 | 113 | private[spark] object EventHubsTestUtils { 114 | val DefaultPartitionCount: Int = 4 115 | val DefaultMaxRate: Rate = 5 116 | val DefaultName = "name" 117 | 118 | private[utils] val eventHubs: mutable.Map[String, SimulatedEventHubs] = mutable.Map.empty 119 | 120 | def createEventData(event: Array[Byte], seqNo: Long): EventData = { 121 | val constructor = classOf[EventDataImpl].getDeclaredConstructor(classOf[Message]) 122 | constructor.setAccessible(true) 123 | 124 | val s = seqNo.toLong.asInstanceOf[AnyRef] 125 | // This value is not accurate. However, "offet" is never used in testing. 126 | // Placing dummy value here because one is required in order for EventData 127 | // to serialize/de-serialize properly during tests. 128 | val o = s.toString.asInstanceOf[AnyRef] 129 | val t = new Date(System.currentTimeMillis()).asInstanceOf[AnyRef] 130 | 131 | val msgAnnotations = new MessageAnnotations( 132 | Map(SEQUENCE_NUMBER -> s, OFFSET -> o, ENQUEUED_TIME_UTC -> t).asJava) 133 | 134 | val body = new Data(new Binary(event)) 135 | val msg = Factory.create(null, null, msgAnnotations, null, null, body, null) 136 | constructor.newInstance(msg).asInstanceOf[EventData] 137 | } 138 | } 139 | 140 | /** 141 | * Simulated EventHubs instance. All partitions are empty on creation. 142 | */ 143 | private[spark] class SimulatedEventHubs(val name: String, val partitionCount: Int) { 144 | 145 | private val partitions: Map[PartitionId, SimulatedEventHubsPartition] = 146 | (for { p <- 0 until partitionCount } yield p -> new SimulatedEventHubsPartition).toMap 147 | 148 | private var count = 0 149 | 150 | def partitionSize(partitionId: PartitionId): Int = { 151 | partitions(partitionId).size 152 | } 153 | 154 | def totalSize: Int = { 155 | var totalSize = 0 156 | for (part <- partitions.keySet) { 157 | totalSize += partitions(part).size 158 | } 159 | totalSize 160 | } 161 | 162 | def receive(eventCount: Int, 163 | partitionId: Int, 164 | seqNo: SequenceNumber): java.lang.Iterable[EventData] = { 165 | (for { _ <- 0 until eventCount } yield partitions(partitionId).get(seqNo)).asJava 166 | } 167 | 168 | def send(events: Seq[Int]): Seq[Int] = { 169 | for (event <- events) { 170 | synchronized { 171 | val part = count % this.partitionCount 172 | count += 1 173 | this.send(part, Seq(event)) 174 | } 175 | } 176 | events 177 | } 178 | 179 | def send(partitionId: PartitionId, events: Seq[Int]): Seq[Int] = { 180 | partitions(partitionId).send(events) 181 | events 182 | } 183 | 184 | def send(event: EventData): Unit = { 185 | synchronized { 186 | val part = count % this.partitionCount 187 | count += 1 188 | this.send(part, event) 189 | } 190 | } 191 | 192 | def send(partitionId: PartitionId, event: EventData): Unit = { 193 | synchronized(partitions(partitionId).send(event)) 194 | } 195 | 196 | def earliestSeqNo(partitionId: PartitionId): SequenceNumber = { 197 | partitions(partitionId).earliestSeqNo 198 | } 199 | 200 | def latestSeqNo(partitionId: PartitionId): SequenceNumber = { 201 | partitions(partitionId).latestSeqNo 202 | } 203 | 204 | def getPartitions: Map[PartitionId, SimulatedEventHubsPartition] = { 205 | partitions 206 | } 207 | 208 | override def toString: String = { 209 | print(s""" 210 | |EventHub Name: $name 211 | |Partition Count: $partitionCount 212 | """.stripMargin) 213 | 214 | for (p <- partitions.keySet.toSeq.sorted) { 215 | print(s""" 216 | Partition: $p 217 | ${partitions(p).getEvents.map(_.getBytes.map(_.toChar).mkString)}) 218 | """) 219 | } 220 | "" 221 | } 222 | 223 | /** Specifies the contents of each partition. */ 224 | private[spark] class SimulatedEventHubsPartition { 225 | import com.microsoft.azure.eventhubs.impl.AmqpConstants._ 226 | 227 | private var data: Seq[EventData] = Seq.empty 228 | 229 | def getEvents: Seq[EventData] = data 230 | 231 | // This allows us to invoke the EventData(Message) constructor 232 | private val constructor = classOf[EventDataImpl].getDeclaredConstructor(classOf[Message]) 233 | constructor.setAccessible(true) 234 | 235 | private[spark] def send(events: Seq[Int]): Unit = { 236 | synchronized { 237 | for (event <- events) { 238 | val seqNo = data.size.toLong.asInstanceOf[AnyRef] 239 | 240 | // This value is not accurate. However, "offet" is never used in testing. 241 | // Placing dummy value here because one is required in order for EventData 242 | // to serialize/de-serialize properly during tests. 243 | val offset = data.size.toString.asInstanceOf[AnyRef] 244 | 245 | val time = new Date(System.currentTimeMillis()).asInstanceOf[AnyRef] 246 | 247 | val msgAnnotations = new MessageAnnotations( 248 | Map(SEQUENCE_NUMBER -> seqNo, OFFSET -> offset, ENQUEUED_TIME_UTC -> time).asJava) 249 | 250 | val body = new Data(new Binary(s"$event".getBytes("UTF-8"))) 251 | 252 | val msg = Factory.create(null, null, msgAnnotations, null, null, body, null) 253 | 254 | data = data :+ constructor.newInstance(msg).asInstanceOf[EventData] 255 | } 256 | } 257 | } 258 | 259 | private[spark] def send(event: EventData): Unit = { 260 | // Need to add a Seq No to the EventData to properly simulate the service. 261 | val e = EventHubsTestUtils.createEventData(event.getBytes, data.size.toLong) 262 | synchronized(data = data :+ e) 263 | } 264 | 265 | private[spark] def size = synchronized(data.size) 266 | 267 | private[spark] def get(index: SequenceNumber): EventData = { 268 | data(index.toInt) 269 | } 270 | 271 | private[spark] def earliestSeqNo: SequenceNumber = { 272 | if (data.isEmpty) { 273 | 0L 274 | } else { 275 | data.head.getSystemProperties.getSequenceNumber 276 | } 277 | } 278 | 279 | private[spark] def latestSeqNo: SequenceNumber = { 280 | if (data.isEmpty) { 281 | 0L 282 | } else { 283 | data.size 284 | } 285 | } 286 | } 287 | } 288 | 289 | /** A simulated EventHubs client. */ 290 | private[spark] class SimulatedClient(ehConf: EventHubsConf) extends Client { self => 291 | 292 | import EventHubsTestUtils._ 293 | 294 | private var rPartitionId: Int = _ // used in receivers 295 | private var sPartitionId: Int = _ // used in senders 296 | private var currentSeqNo: SequenceNumber = _ 297 | private val eventHub = eventHubs(ehConf.name) 298 | 299 | override def createReceiver(partitionId: String, startingSeqNo: SequenceNumber): Unit = { 300 | self.rPartitionId = partitionId.toInt 301 | self.currentSeqNo = startingSeqNo 302 | } 303 | 304 | override def createPartitionSender(partitionId: Int): Unit = { 305 | sPartitionId = partitionId.toInt 306 | } 307 | 308 | override def send(event: EventData): Unit = { 309 | eventHub.send(event) 310 | } 311 | 312 | override def send(event: EventData, partitionKey: String): Unit = { 313 | throw new UnsupportedOperationException 314 | } 315 | 316 | override def send(event: EventData, partitionId: Int): Unit = { 317 | eventHub.send(partitionId, event) 318 | } 319 | 320 | override def receive(eventCount: Int): java.lang.Iterable[EventData] = { 321 | val events = eventHub.receive(eventCount, self.rPartitionId, currentSeqNo) 322 | currentSeqNo += eventCount 323 | events 324 | } 325 | 326 | override def setPrefetchCount(count: Int): Unit = { 327 | // not prefetching anything in tests 328 | } 329 | 330 | override def earliestSeqNo(partitionId: PartitionId): SequenceNumber = { 331 | eventHub.earliestSeqNo(partitionId) 332 | } 333 | 334 | override def latestSeqNo(partitionId: PartitionId): SequenceNumber = { 335 | eventHub.latestSeqNo(partitionId) 336 | } 337 | 338 | override def boundedSeqNos(partitionId: PartitionId): (SequenceNumber, SequenceNumber) = { 339 | (earliestSeqNo(partitionId), latestSeqNo(partitionId)) 340 | } 341 | 342 | override def translate[T](ehConf: EventHubsConf, 343 | partitionCount: Int, 344 | useStarting: Boolean = true): Map[PartitionId, SequenceNumber] = { 345 | 346 | val positions = if (useStarting) { ehConf.startingPositions } else { ehConf.endingPositions } 347 | val position = if (useStarting) { ehConf.startingPosition } else { ehConf.endingPosition } 348 | val apiCall = if (useStarting) { earliestSeqNo _ } else { latestSeqNo _ } 349 | 350 | if (positions.isEmpty && position.isEmpty) { 351 | (for { id <- 0 until eventHub.partitionCount } yield id -> apiCall(id)).toMap 352 | } else if (positions.isEmpty) { 353 | require(position.get.seqNo >= 0L) 354 | (for { id <- 0 until partitionCount } yield id -> position.get.seqNo).toMap 355 | } else { 356 | require(positions.get.forall(x => x._2.seqNo >= 0L)) 357 | require(positions.get.size == partitionCount) 358 | positions.get.map { case (k, v) => k.partitionId -> v }.mapValues(_.seqNo).mapValues { 359 | seqNo => 360 | { if (seqNo == -1L) 0L else seqNo } 361 | } 362 | } 363 | } 364 | 365 | override def partitionCount: PartitionId = eventHub.partitionCount 366 | 367 | override def close(): Unit = { 368 | // nothing to close 369 | } 370 | } 371 | 372 | private[spark] object SimulatedClient { 373 | def apply(ehConf: EventHubsConf): SimulatedClient = new SimulatedClient(ehConf) 374 | } 375 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/eventhubs/EventHubsSource.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.sql.eventhubs 19 | 20 | import java.io._ 21 | import java.nio.charset.StandardCharsets 22 | 23 | import org.apache.commons.io.IOUtils 24 | import org.apache.spark.SparkContext 25 | import org.apache.spark.eventhubs.client.Client 26 | import org.apache.spark.eventhubs.rdd.{ EventHubsRDD, OffsetRange } 27 | import org.apache.spark.eventhubs.{ EventHubsConf, NameAndPartition, _ } 28 | import org.apache.spark.internal.Logging 29 | import org.apache.spark.scheduler.ExecutorCacheTaskLocation 30 | import org.apache.spark.sql.catalyst.InternalRow 31 | import org.apache.spark.sql.catalyst.util.DateTimeUtils 32 | import org.apache.spark.sql.execution.streaming.{ 33 | HDFSMetadataLog, 34 | Offset, 35 | SerializedOffset, 36 | Source 37 | } 38 | import org.apache.spark.sql.types._ 39 | import org.apache.spark.sql.{ DataFrame, SQLContext } 40 | import org.apache.spark.unsafe.types.UTF8String 41 | 42 | private[spark] class EventHubsSource private[eventhubs] (sqlContext: SQLContext, 43 | options: Map[String, String], 44 | clientFactory: (EventHubsConf => Client), 45 | metadataPath: String) 46 | extends Source 47 | with Logging { 48 | 49 | import EventHubsConf._ 50 | import EventHubsSource._ 51 | 52 | private lazy val partitionCount: Int = ehClient.partitionCount 53 | 54 | private val ehConf = EventHubsConf.toConf(options) 55 | private val ehName = ehConf.name 56 | 57 | private val sc = sqlContext.sparkContext 58 | 59 | private val maxOffsetsPerTrigger: Option[Long] = 60 | Option(options.get(MaxEventsPerTriggerKey).map(_.toLong).getOrElse(partitionCount * 1000)) 61 | 62 | private var _client: Client = _ 63 | private[spark] def ehClient = { 64 | if (_client == null) _client = clientFactory(ehConf) 65 | _client 66 | } 67 | 68 | private lazy val initialPartitionSeqNos = { 69 | val metadataLog = 70 | new HDFSMetadataLog[EventHubsSourceOffset](sqlContext.sparkSession, metadataPath) { 71 | override def serialize(metadata: EventHubsSourceOffset, out: OutputStream): Unit = { 72 | out.write(0) // SPARK-19517 73 | val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)) 74 | writer.write("v" + VERSION + "\n") 75 | writer.write(metadata.json) 76 | writer.flush() 77 | } 78 | 79 | override def deserialize(in: InputStream): EventHubsSourceOffset = { 80 | in.read() // zero byte is read (SPARK-19517) 81 | val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8)) 82 | // HDFSMetadataLog guarantees that it never creates a partial file. 83 | assert(content.length != 0) 84 | if (content(0) == 'v') { 85 | val indexOfNewLine = content.indexOf("\n") 86 | if (indexOfNewLine > 0) { 87 | val version = 88 | parseVersion(content.substring(0, indexOfNewLine), VERSION) 89 | EventHubsSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) 90 | } else { 91 | throw new IllegalStateException("Log file was malformed.") 92 | } 93 | } else { 94 | EventHubsSourceOffset(SerializedOffset(content)) // Spark 2.1 log file 95 | } 96 | } 97 | } 98 | 99 | metadataLog 100 | .get(0) 101 | .getOrElse { 102 | // translate starting points within ehConf to sequence numbers 103 | val seqNos = ehClient.translate(ehConf, partitionCount).map { 104 | case (pId, seqNo) => 105 | (NameAndPartition(ehName, pId), seqNo) 106 | } 107 | val offset = EventHubsSourceOffset(seqNos) 108 | metadataLog.add(0, offset) 109 | logInfo(s"Initial sequence numbers: $seqNos") 110 | offset 111 | } 112 | .partitionToSeqNos 113 | } 114 | 115 | private var currentSeqNos: Option[Map[NameAndPartition, SequenceNumber]] = None 116 | 117 | private var earliestSeqNos: Option[Map[NameAndPartition, SequenceNumber]] = None 118 | 119 | override def schema: StructType = EventHubsSourceProvider.eventHubsSchema 120 | 121 | override def getOffset: Option[Offset] = { 122 | // Make sure initialPartitionSeqNos is initialized 123 | initialPartitionSeqNos 124 | 125 | // This contains an array of the following elements: 126 | // (partitionId, (earliestSeqNo, latestSeqNo) 127 | val earliestAndLatest = for { 128 | p <- 0 until partitionCount 129 | n = ehName 130 | } yield (p, ehClient.boundedSeqNos(p)) 131 | 132 | // There is a possibility that data from EventHubs will 133 | // expire before it can be consumed from Spark. We collect 134 | // the earliest sequence numbers available in the service 135 | // here. In getBatch, we'll make sure our starting sequence 136 | // numbers are greater than or equal to the earliestSeqNos. 137 | // If not, we'll report possible data loss. 138 | earliestSeqNos = Some(earliestAndLatest.map { 139 | case (p, (e, _)) => NameAndPartition(ehName, p) -> e 140 | }.toMap) 141 | 142 | val latest = earliestAndLatest.map { 143 | case (p, (_, l)) => NameAndPartition(ehName, p) -> l 144 | }.toMap 145 | 146 | val seqNos: Map[NameAndPartition, SequenceNumber] = maxOffsetsPerTrigger match { 147 | case None => 148 | latest 149 | case Some(limit) if currentSeqNos.isEmpty => 150 | rateLimit(limit, initialPartitionSeqNos, latest) 151 | case Some(limit) => 152 | rateLimit(limit, currentSeqNos.get, latest) 153 | } 154 | 155 | currentSeqNos = Some(seqNos) 156 | logDebug(s"GetOffset: ${seqNos.toSeq.map(_.toString).sorted}") 157 | 158 | Some(EventHubsSourceOffset(seqNos)) 159 | } 160 | 161 | def fetchEarliestSeqNos( 162 | nameAndPartitions: Seq[NameAndPartition]): Map[NameAndPartition, SequenceNumber] = { 163 | (for { 164 | nameAndPartition <- nameAndPartitions 165 | seqNo = ehClient.earliestSeqNo(nameAndPartition.partitionId) 166 | } yield nameAndPartition -> seqNo).toMap 167 | } 168 | 169 | /** Proportionally distribute limit number of offsets among partitions */ 170 | private def rateLimit( 171 | limit: Long, 172 | from: Map[NameAndPartition, SequenceNumber], 173 | until: Map[NameAndPartition, SequenceNumber]): Map[NameAndPartition, SequenceNumber] = { 174 | val fromNew = fetchEarliestSeqNos(until.keySet.diff(from.keySet).toSeq) 175 | val sizes = until.flatMap { 176 | case (nameAndPartition, end) => 177 | // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it 178 | from.get(nameAndPartition).orElse(fromNew.get(nameAndPartition)).flatMap { begin => 179 | val size = end - begin 180 | logDebug(s"rateLimit $nameAndPartition size is $size") 181 | if (size > 0) Some(nameAndPartition -> size) else None 182 | } 183 | } 184 | val total = sizes.values.sum.toDouble 185 | if (total < 1) { 186 | until 187 | } else { 188 | until.map { 189 | case (nameAndPartition, end) => 190 | nameAndPartition -> sizes 191 | .get(nameAndPartition) 192 | .map { size => 193 | val begin = from.getOrElse(nameAndPartition, fromNew(nameAndPartition)) 194 | val prorate = limit * (size / total) 195 | logDebug(s"rateLimit $nameAndPartition prorated amount is $prorate") 196 | // Don't completely starve small partitions 197 | val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong 198 | logDebug(s"rateLimit $nameAndPartition new offset is $off") 199 | // Paranoia, make sure not to return an offset that's past end 200 | Math.min(end, off) 201 | } 202 | .getOrElse(end) 203 | } 204 | } 205 | } 206 | 207 | override def getBatch(start: Option[Offset], end: Offset): DataFrame = { 208 | initialPartitionSeqNos 209 | 210 | logInfo(s"getBatch called with start = $start and end = $end") 211 | val untilSeqNos = EventHubsSourceOffset.getPartitionSeqNos(end) 212 | // On recovery, getBatch wil be called before getOffset 213 | if (currentSeqNos.isEmpty) { 214 | currentSeqNos = Some(untilSeqNos) 215 | } 216 | if (start.isDefined && start.get == end) { 217 | return sqlContext.internalCreateDataFrame(sqlContext.sparkContext.emptyRDD, 218 | schema, 219 | isStreaming = true) 220 | } 221 | val fromSeqNos = start match { 222 | case Some(prevBatchEndOffset) => 223 | EventHubsSourceOffset.getPartitionSeqNos(prevBatchEndOffset) 224 | case None => 225 | // we need to 226 | initialPartitionSeqNos 227 | } 228 | if (earliestSeqNos.isEmpty) { 229 | earliestSeqNos = Some(fromSeqNos) 230 | } 231 | fromSeqNos.map { 232 | case (nAndP, seqNo) => 233 | if (seqNo < currentSeqNos.get(nAndP)) { 234 | reportDataLoss( 235 | s"Starting seqNo $seqNo in partition ${nAndP.partitionId} of EventHub ${nAndP.ehName} " + 236 | s"is behind the earliest sequence number ${currentSeqNos.get(nAndP)} " + 237 | s"present in the service. Some events may have expired and been missed.") 238 | nAndP -> currentSeqNos.get(nAndP) 239 | } else { 240 | nAndP -> seqNo 241 | } 242 | } 243 | 244 | // Find the new partitions, and get their earliest offsets 245 | val newPartitions = untilSeqNos.keySet.diff(fromSeqNos.keySet) 246 | val newPartitionSeqNos = fetchEarliestSeqNos(newPartitions.toSeq) 247 | if (newPartitionSeqNos.keySet != newPartitions) { 248 | // We cannot get fromOffsets for some partitions. It means they got deleted. 249 | val deletedPartitions = newPartitions.diff(newPartitionSeqNos.keySet) 250 | reportDataLoss( 251 | s"Cannot find earliest sequence numbers of $deletedPartitions. Some data may have been missed") 252 | } 253 | logInfo(s"Partitions added: $newPartitionSeqNos") 254 | newPartitionSeqNos.filter(_._2 != 0).foreach { 255 | case (p, s) => 256 | reportDataLoss( 257 | s"Added partition $p starts from $s instead of 0. Some data may have been missed") 258 | } 259 | 260 | val deletedPartitions = fromSeqNos.keySet.diff(untilSeqNos.keySet) 261 | if (deletedPartitions.nonEmpty) { 262 | reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") 263 | } 264 | 265 | val nameAndPartitions = untilSeqNos.keySet.filter { p => 266 | // Ignore partitions that we don't know the from offsets. 267 | newPartitionSeqNos.contains(p) || fromSeqNos.contains(p) 268 | }.toSeq 269 | logDebug("Partitions: " + nameAndPartitions.mkString(", ")) 270 | 271 | val sortedExecutors = getSortedExecutorList(sc) 272 | val numExecutors = sortedExecutors.length 273 | logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) 274 | 275 | // Calculate offset ranges 276 | val offsetRanges = (for { 277 | np <- nameAndPartitions 278 | fromSeqNo = fromSeqNos 279 | .getOrElse(np, throw new IllegalStateException(s"$np doesn't have a fromSeqNo")) 280 | untilSeqNo = untilSeqNos(np) 281 | preferredLoc = if (numExecutors > 0) { 282 | Some(sortedExecutors(Math.floorMod(np.hashCode, numExecutors))) 283 | } else None 284 | } yield OffsetRange(np, fromSeqNo, untilSeqNo, preferredLoc)).filter { range => 285 | if (range.untilSeqNo < range.fromSeqNo) { 286 | reportDataLoss( 287 | s"Partition ${range.nameAndPartition}'s sequence number was changed from " + 288 | s"${range.fromSeqNo} to ${range.untilSeqNo}, some data may have been missed") 289 | false 290 | } else { 291 | true 292 | } 293 | }.toArray 294 | 295 | val rdd = new EventHubsRDD(sc, ehConf, offsetRanges, clientFactory) 296 | .mapPartitionsWithIndex { (p, iter) => 297 | { 298 | iter.map { ed => 299 | InternalRow( 300 | ed.getBytes, 301 | UTF8String.fromString(p.toString), 302 | UTF8String.fromString(ed.getSystemProperties.getOffset), 303 | ed.getSystemProperties.getSequenceNumber, 304 | DateTimeUtils.fromJavaTimestamp( 305 | new java.sql.Timestamp(ed.getSystemProperties.getEnqueuedTime.toEpochMilli)), 306 | UTF8String.fromString(ed.getSystemProperties.getPublisher), 307 | UTF8String.fromString(ed.getSystemProperties.getPartitionKey) 308 | ) 309 | } 310 | } 311 | } 312 | 313 | logInfo( 314 | "GetBatch generating RDD of offset range: " + 315 | offsetRanges.sortBy(_.nameAndPartition.toString).mkString(", ")) 316 | 317 | sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true) 318 | } 319 | 320 | override def stop(): Unit = synchronized { 321 | ehClient.close() 322 | } 323 | 324 | /** 325 | * Logs a warning when data may have been missed. 326 | */ 327 | private def reportDataLoss(message: String): Unit = { 328 | logWarning(message + s". $InstructionsForPotentialDataLoss") 329 | } 330 | } 331 | 332 | private[eventhubs] object EventHubsSource { 333 | val InstructionsForPotentialDataLoss = 334 | """ 335 | |Some data may have been lost because they are not available in EventHubs any more; either the 336 | | data was aged out by EventHubs or the EventHubs instance may have been deleted before all the data in the 337 | | instance was processed. 338 | """.stripMargin 339 | 340 | private[eventhubs] val VERSION = 1 341 | 342 | def getSortedExecutorList(sc: SparkContext): Array[String] = { 343 | val bm = sc.env.blockManager 344 | bm.master 345 | .getPeers(bm.blockManagerId) 346 | .toArray 347 | .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) 348 | .sortWith(compare) 349 | .map(_.toString) 350 | } 351 | 352 | private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { 353 | if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } 354 | } 355 | } 356 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/streaming/eventhubs/EventHubsDirectDStreamSuite.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.streaming.eventhubs 19 | 20 | import java.io.File 21 | import java.util 22 | import java.util.concurrent.ConcurrentLinkedQueue 23 | import java.util.concurrent.atomic.{ AtomicInteger, AtomicLong } 24 | 25 | import com.microsoft.azure.eventhubs.EventData 26 | import org.apache.spark.eventhubs.{ EventHubsConf, EventPosition, NameAndPartition } 27 | import org.apache.spark.eventhubs.utils.EventHubsTestUtils._ 28 | import org.apache.spark.eventhubs.rdd.{ HasOffsetRanges, OffsetRange } 29 | import org.apache.spark.eventhubs.utils.{ EventHubsTestUtils, SimulatedClient } 30 | import org.apache.spark.{ SparkConf, SparkFunSuite } 31 | import org.apache.spark.internal.Logging 32 | import org.apache.spark.rdd.RDD 33 | import org.apache.spark.streaming.dstream.DStream 34 | import org.apache.spark.streaming.scheduler.{ 35 | StreamingListener, 36 | StreamingListenerBatchCompleted, 37 | StreamingListenerBatchStarted, 38 | StreamingListenerBatchSubmitted 39 | } 40 | import org.apache.spark.streaming.{ Milliseconds, Seconds, StreamingContext, Time } 41 | import org.apache.spark.util.Utils 42 | import org.scalatest.concurrent.Eventually 43 | import org.scalatest.BeforeAndAfter 44 | 45 | import scala.collection.JavaConverters._ 46 | import scala.concurrent.duration._ 47 | import scala.language.postfixOps 48 | 49 | class EventHubsDirectDStreamSuite 50 | extends SparkFunSuite 51 | with BeforeAndAfter 52 | with Eventually 53 | with Logging { 54 | 55 | import EventHubsDirectDStreamSuite._ 56 | 57 | private var testUtils: EventHubsTestUtils = _ 58 | 59 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) 60 | 61 | private var ssc: StreamingContext = _ 62 | private var testDir: File = _ 63 | 64 | override def beforeAll: Unit = { 65 | super.beforeAll 66 | testUtils = new EventHubsTestUtils 67 | } 68 | 69 | override def afterAll(): Unit = { 70 | if (testUtils != null) { 71 | testUtils.destroyAllEventHubs() 72 | testUtils = null 73 | } 74 | super.afterAll() 75 | } 76 | 77 | after { 78 | if (ssc != null) { 79 | ssc.stop(stopSparkContext = true) 80 | } 81 | if (testDir != null) { 82 | Utils.deleteRecursively(testDir) 83 | } 84 | } 85 | 86 | private val eventHubsId = new AtomicInteger(0) 87 | 88 | def newEventHubs(): String = { 89 | s"eh-${eventHubsId.getAndIncrement()}" 90 | } 91 | 92 | private def getEventHubsConf(ehName: String): EventHubsConf = testUtils.getEventHubsConf(ehName) 93 | 94 | test("basic stream receiving with smallest starting sequence number") { 95 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 96 | testUtils.populateUniformly(eventHub.name, EventsPerPartition) 97 | 98 | val ehConf = getEventHubsConf(eventHub.name) 99 | val batchInterval = 1000 100 | val timeoutAfter = 100000 101 | val expectedTotal = (timeoutAfter / batchInterval) * DefaultMaxRate 102 | 103 | ssc = new StreamingContext(sparkConf, Milliseconds(batchInterval)) 104 | val stream = withClue("Error creating direct stream") { 105 | new EventHubsDirectDStream(ssc, ehConf, SimulatedClient.apply) 106 | } 107 | val allReceived = new ConcurrentLinkedQueue[EventData]() 108 | 109 | // hold a reference to the current offset ranges, so it can be used downstream 110 | var offsetRanges = Array[OffsetRange]() 111 | val tf = stream.transform { rdd => 112 | // Get the offset ranges in the RDD 113 | offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges 114 | rdd 115 | } 116 | 117 | tf.foreachRDD { rdd => 118 | for (o <- offsetRanges) { 119 | logInfo(s"${o.name} ${o.partitionId} ${o.fromSeqNo} ${o.untilSeqNo}") 120 | } 121 | val collected = rdd.mapPartitionsWithIndex { (i, iter) => 122 | // For each partition, get size of the range in the partition, 123 | // and the number of items in the partition 124 | val off = offsetRanges(i) 125 | val all = iter.toSeq 126 | val partSize = all.size 127 | val rangeSize = off.untilSeqNo - off.fromSeqNo 128 | Iterator((partSize, rangeSize)) 129 | }.collect 130 | 131 | // Verify whether number of elements in each partition 132 | // matches with the corresponding offset range 133 | collected.foreach { 134 | case (partSize, rangeSize) => 135 | assert(partSize == rangeSize, "offset ranges are wrong") 136 | } 137 | } 138 | 139 | stream.foreachRDD { rdd => 140 | allReceived.addAll(util.Arrays.asList(rdd.collect(): _*)) 141 | } 142 | ssc.start() 143 | eventually(timeout(timeoutAfter.milliseconds), interval(batchInterval.milliseconds)) { 144 | assert(allReceived.size === expectedTotal, 145 | "didn't get expected number of messages, messages:\n" + 146 | allReceived.asScala.mkString("\n")) 147 | } 148 | ssc.stop() 149 | } 150 | 151 | test("basic stream receiving from random sequence number") { 152 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 153 | testUtils.populateUniformly(eventHub.name, EventsPerPartition) 154 | 155 | val startSeqNo = Math.abs(scala.util.Random.nextInt) % (EventsPerPartition / 2) 156 | 157 | val ehConf = getEventHubsConf(eventHub.name) 158 | .setStartingPositions(Map.empty) 159 | .setStartingPosition(EventPosition.fromSequenceNumber(startSeqNo)) 160 | 161 | val batchInterval = 1000 162 | val timeoutAfter = 10000 163 | val expectedTotal = 164 | (timeoutAfter / batchInterval) * DefaultMaxRate * DefaultPartitionCount 165 | 166 | ssc = new StreamingContext(sparkConf, Milliseconds(batchInterval)) 167 | val stream = withClue("Error creating direct stream") { 168 | new EventHubsDirectDStream(ssc, ehConf, SimulatedClient.apply) 169 | } 170 | val allReceived = new ConcurrentLinkedQueue[EventData]() 171 | 172 | // hold a reference to the current offset ranges so it can be used downstream 173 | var offsetRanges = Array[OffsetRange]() 174 | val tf = stream.transform { rdd => 175 | // Get the offset ranges in the RDD 176 | offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges 177 | rdd 178 | } 179 | 180 | tf.foreachRDD { rdd => 181 | for (o <- offsetRanges) { 182 | logInfo(s"${o.name} ${o.partitionId} ${o.fromSeqNo} ${o.untilSeqNo}") 183 | } 184 | val collected = rdd.mapPartitionsWithIndex { (i, iter) => 185 | // For each partition, get size of the range in the partition 186 | // and the number of items in the partition 187 | val off = offsetRanges(i) 188 | val all = iter.toSeq 189 | val partSize = all.size 190 | val rangeSize = off.untilSeqNo - off.fromSeqNo 191 | Iterator((partSize, rangeSize)) 192 | }.collect 193 | 194 | // Verify whether number of elements in each partition 195 | // matches with the corresponding offset range 196 | collected.foreach { 197 | case (partSize, rangeSize) => 198 | assert(partSize == rangeSize, "offset ranges are wrong") 199 | } 200 | } 201 | 202 | stream.foreachRDD { rdd => 203 | allReceived.addAll(util.Arrays.asList(rdd.collect(): _*)) 204 | } 205 | ssc.start() 206 | eventually(timeout(timeoutAfter milliseconds), interval(batchInterval milliseconds)) { 207 | assert(allReceived.size === expectedTotal, 208 | "didn't get expected number of messages, messages:\n" + 209 | allReceived.asScala.mkString("\n")) 210 | } 211 | ssc.stop() 212 | } 213 | 214 | test("receiving from largest starting offset") { 215 | val eh = newEventHubs() 216 | testUtils.createEventHubs(eh, DefaultPartitionCount) 217 | testUtils.populateUniformly(eh, EventsPerPartition) 218 | 219 | val positions = (for { 220 | id <- 0 until DefaultPartitionCount 221 | } yield NameAndPartition(eh, id) -> EventPosition.fromSequenceNumber(EventsPerPartition)).toMap 222 | 223 | val ehConf = getEventHubsConf(eh).setStartingPositions(positions) 224 | val batchInterval = 1000 225 | val timeoutAfter = 10000 226 | 227 | ssc = new StreamingContext(sparkConf, Milliseconds(batchInterval)) 228 | val stream = withClue("Error creating direct stream") { 229 | new EventHubsDirectDStream(ssc, ehConf, SimulatedClient.apply) 230 | } 231 | 232 | val collectedData = new ConcurrentLinkedQueue[EventData]() 233 | stream.foreachRDD { rdd => 234 | collectedData.addAll(util.Arrays.asList(rdd.collect(): _*)) 235 | } 236 | ssc.start() 237 | eventually(timeout(timeoutAfter.milliseconds), interval(batchInterval.milliseconds)) { 238 | assert(collectedData.isEmpty) 239 | } 240 | assert(collectedData.isEmpty) 241 | ssc.stop() 242 | } 243 | 244 | // Test to verify offset ranges can be recovered from the checkpoints 245 | test("offset recovery") { 246 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 247 | testUtils.populateUniformly(eventHub.name, 25) 248 | testDir = Utils.createTempDir() 249 | 250 | val ehConf = getEventHubsConf(eventHub.name) 251 | .setStartingPositions(Map.empty) 252 | .setStartingPosition(EventPosition.fromSequenceNumber(0L)) 253 | 254 | // Setup the streaming context 255 | ssc = new StreamingContext(sparkConf, Milliseconds(100)) 256 | ssc.remember(Seconds(20)) 257 | val stream = withClue("Error creating direct stream") { 258 | new EventHubsDirectDStream(ssc, ehConf, SimulatedClient.apply) 259 | } 260 | 261 | ssc.checkpoint(testDir.getAbsolutePath) 262 | 263 | //val collectedData = new ConcurrentLinkedQueue[(String, Int)]() 264 | val collectedData = new ConcurrentLinkedQueue[EventData]() 265 | stream.foreachRDD { (rdd: RDD[EventData]) => 266 | collectedData.addAll(util.Arrays.asList(rdd.collect(): _*)) 267 | } 268 | 269 | ssc.start() 270 | 271 | eventually(timeout(20 seconds), interval(50 milliseconds)) { 272 | assert(collectedData.size() === 100) 273 | } 274 | 275 | ssc.stop() 276 | 277 | // Verify that offset ranges were generated 278 | val offsetRangesBeforeStop = getOffsetRanges(stream) 279 | assert(offsetRangesBeforeStop.nonEmpty, "No offset ranges generated") 280 | assert( 281 | offsetRangesBeforeStop.head._2.forall { _.fromSeqNo === 0 }, 282 | "starting sequence number not zero" 283 | ) 284 | 285 | logInfo("====== RESTARTING ======") 286 | 287 | // Recover context from checkpoints 288 | ssc = new StreamingContext(testDir.getAbsolutePath) 289 | val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[EventData]] 290 | 291 | // Verify offset ranges have been recovered 292 | val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => 293 | (x._1, x._2.toSet) 294 | } 295 | assert(recoveredOffsetRanges.nonEmpty, "No offset ranges recovered") 296 | val earlierOffsetRanges = offsetRangesBeforeStop.map { x => 297 | (x._1, x._2.toSet) 298 | } 299 | assert( 300 | recoveredOffsetRanges.forall { or => 301 | earlierOffsetRanges.contains((or._1, or._2)) 302 | }, 303 | "Recovered ranges are not the same as the ones generated\n" + 304 | earlierOffsetRanges + "\n" + recoveredOffsetRanges 305 | ) 306 | 307 | /* 308 | // Send 25 events to every partition 309 | for (i <- 0 until PartitionCount) { 310 | EventHubsTestUtils.eventHubs.send(i, 0 to 25) 311 | } 312 | 313 | // Restart context, give more data and verify the total at the end 314 | // If the total is right that means each record has been received only once. 315 | ssc.start() 316 | 317 | eventually(timeout(20 seconds), interval(50 milliseconds)) { 318 | assert(collectedData.size() === 200) 319 | } 320 | 321 | ssc.stop() 322 | */ 323 | } 324 | 325 | test("Direct EventHubs stream report input information") { 326 | val eventHub = testUtils.createEventHubs(newEventHubs(), DefaultPartitionCount) 327 | testUtils.populateUniformly(eventHub.name, 25) 328 | 329 | val ehConf = getEventHubsConf(eventHub.name) 330 | 331 | val totalSent = 25 * eventHub.partitionCount 332 | 333 | import EventHubsDirectDStreamSuite._ 334 | ssc = new StreamingContext(sparkConf, Milliseconds(200)) 335 | val collector = new InputInfoCollector 336 | ssc.addStreamingListener(collector) 337 | 338 | val stream = withClue("Error creating direct stream") { 339 | new EventHubsDirectDStream(ssc, ehConf, SimulatedClient.apply) 340 | } 341 | 342 | val allReceived = new ConcurrentLinkedQueue[String]() 343 | 344 | stream.map(_.getBytes.map(_.toChar).mkString).foreachRDD { rdd => 345 | allReceived.addAll(util.Arrays.asList(rdd.collect(): _*)) 346 | } 347 | 348 | ssc.start() 349 | 350 | eventually(timeout(20 seconds), interval(200 milliseconds)) { 351 | assert(allReceived.size === totalSent, 352 | "didn't get expected number of messages, messages:\n" + 353 | allReceived.asScala.mkString("\n")) 354 | 355 | // Calculate all the records collected in the StreamingListener. 356 | assert(collector.numRecordsSubmitted.get() === totalSent) 357 | assert(collector.numRecordsStarted.get() === totalSent) 358 | assert(collector.numRecordsCompleted.get() === totalSent) 359 | } 360 | 361 | ssc.stop() 362 | } 363 | 364 | /** Get the generated offset ranges from the EventHubsStream */ 365 | private def getOffsetRanges(stream: DStream[EventData]): Seq[(Time, Array[OffsetRange])] = { 366 | stream.generatedRDDs 367 | .mapValues { rdd => 368 | rdd.asInstanceOf[HasOffsetRanges].offsetRanges 369 | } 370 | .toSeq 371 | .sortBy { _._1 } 372 | } 373 | } 374 | 375 | object EventHubsDirectDStreamSuite { 376 | val EventsPerPartition: Int = 5000 377 | 378 | class InputInfoCollector extends StreamingListener { 379 | val numRecordsSubmitted = new AtomicLong(0L) 380 | val numRecordsStarted = new AtomicLong(0L) 381 | val numRecordsCompleted = new AtomicLong(0L) 382 | 383 | override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { 384 | numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) 385 | } 386 | 387 | override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { 388 | numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) 389 | } 390 | 391 | override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { 392 | numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) 393 | } 394 | } 395 | } 396 | --------------------------------------------------------------------------------