```.
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 |
3 |
4 |
5 | Azure Event Hubs Connector for Apache Spark
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
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|[](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|[](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|[](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|[](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|[](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 |
--------------------------------------------------------------------------------