├── .gitignore
├── README.md
├── core
├── pom.xml
└── src
│ └── main
│ ├── resources
│ └── META-INF
│ │ └── services
│ │ └── org.apache.spark.sql.sources.DataSourceRegister
│ └── scala
│ └── com
│ └── hortonworks
│ └── spark
│ └── hive
│ ├── HiveSourceProvider.scala
│ ├── HiveStreamDataWriter.scala
│ ├── HiveStreamWriter.scala
│ ├── common
│ ├── CachedHiveWriters.scala
│ ├── HiveOptions.scala
│ └── HiveWriter.scala
│ └── utils
│ ├── HiveIsolatedClassLoader.scala
│ └── Logging.scala
├── example
├── pom.xml
└── src
│ └── main
│ └── scala
│ └── com
│ └── hortonworks
│ └── spark
│ └── hive
│ └── example
│ ├── HiveStreamingExample.scala
│ └── RateStreamingExample.scala
├── pom.xml
└── scalastyle-config.xml
/.gitignore:
--------------------------------------------------------------------------------
1 | *.class
2 | *.log
3 | .idea
4 | *.iml
5 | target
6 | metastore_db
7 | tmp
8 | dependency-reduced-pom.xml
9 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | Spark Hive Streaming Sink
2 | ===
3 |
4 | A sink to save Spark Structured Streaming DataFrame into Hive table
5 |
6 |
7 | This sink:
8 |
9 | 1. Saves Structured Streaming micro-batch/continuous-processing(Spark 2.3+) DataFrame into hive table.
10 | 2. Uses isolated classloader to isolate Hive related dependencies, which means it can support different versions of Hive other than Spark's built-in one.
11 | 3. Uses newest DataSource API V2, which means it can only be worked with Spark 2.3+.
12 |
13 | The details of Hive Streaming could be referred [here](https://cwiki.apache.org/confluence/display/Hive/Streaming+Data+Ingest).
14 |
15 | How To Build
16 | ==========
17 |
18 | To use this connector, you will require a latest version of Spark (Spark 2.3+).
19 |
20 | To build this project, please execute:
21 |
22 | ```shell
23 | mvn package -DskipTests
24 | ```
25 |
26 | `mvn package` will generate two jars,including one uber jar. User could use this uber jar at convenience.
27 |
28 | How To Use
29 | ==========
30 |
31 | 1. This Spark hive streaming sink jar should be loaded into Spark's environment by `--jars`.
32 | 2. A required Hive table should be created before ingesting data into this table. The requirement can be checked [here](https://cwiki.apache.org/confluence/display/Hive/Streaming+Data+Ingest#StreamingDataIngest-StreamingRequirements).
33 | 3. A `hive-site.xml` with required configurations should be put into Spark classpath, so that it can be accessed from classloader.
34 | 4. If you're running in a secured environment, then principal and keytab should be provided.
35 |
36 | Please be aware a valid `hive-site.xml` and keytab should be accessible from executor side, which means user should pass it via `--files`.
37 |
38 | To use this library, it is similar to other data source libraries, for example:
39 |
40 | ```scala
41 | val socket = sparkSession.readStream
42 | .format("socket")
43 | .options(Map("host" -> host, "port" -> port))
44 | .load()
45 | .as[String]
46 |
47 | val query = socket.map { s =>
48 | val records = s.split(",")
49 | assert(records.length >= 4)
50 | (records(0).toInt, records(1), records(2), records(3))
51 | }
52 | .selectExpr("_1 as id", "_2 as msg", "_3 as continent", "_4 as country")
53 | .writeStream
54 | .format("hive-streaming")
55 | .option("metastore", metastoreUri)
56 | .option("db", "default")
57 | .option("table", "alerts")
58 | .queryName("socket-hive-streaming")
59 | .start()
60 | ```
61 |
62 | User should convert the data source schema to match the destination table's schema, like above `.selectExpr("_1 as id", "_2 as msg", "_3 as continent", "_4 as country")`.
63 |
64 | User should specify the data source format `hive-streaming` and required options:
65 |
66 | 1. `metastore`, metastore uris for which to connect to.
67 | 2. `db`, db name to write to.
68 | 3. `table`, table name to write to.
69 |
70 | Above 3 options are required to run hive streaming application, for others please check here:
71 |
72 | option | default value | meaning
73 | ------ | ------------- | -------
74 | txn.per.batch | 100 | Hive grants a batch of transactions instead of single transactions to streaming clients.This setting configures the number of desired transactions per Transaction Batch. Data from all transactions in a single batch end up in a single file. Flume will write a maximum of batchSize events in each transaction in the batch. This setting in conjunction with batch.size provides control over the size of each file. Note that eventually Hive will transparently compact these files into larger files.
75 | auto.create.partitions | true | automatically create the necessary Hive partitions to stream to.
76 | principal | none | Kerberos user principal for accessing secure Hive.
77 | keytab | none | Kerberos keytab for accessing secure Hive.
78 | batch.size | 10000 | Max number of events written to Hive in a single Hive transaction.
79 |
80 | License
81 | =======
82 |
83 | Apache License, Version 2.0 http://www.apache.org/licenses/LICENSE-2.0.
84 |
--------------------------------------------------------------------------------
/core/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
21 | 4.0.0
22 |
23 |
24 | com.hortonworks.spark
25 | spark-hive-streaming-sink-main_2.11
26 | 0.1.0-SNAPSHOT
27 | ../pom.xml
28 |
29 |
30 | spark-hive-streaming-sink_2.11
31 | 0.1.0-SNAPSHOT
32 | jar
33 |
34 |
35 | target/scala-${scala.binary.version}/classes
36 | target/scala-${scala.binary.version}/test-classes
37 |
38 |
39 | org.apache.maven.plugins
40 | maven-shade-plugin
41 |
42 | false
43 |
44 |
45 | *:*
46 |
47 |
48 |
49 |
50 | *:*
51 |
52 | META-INF/*.SF
53 | META-INF/*.DSA
54 | META-INF/*.RSA
55 |
56 |
57 |
58 |
59 |
60 |
61 | package
62 |
63 | shade
64 |
65 |
66 |
67 |
68 |
69 | reference.conf
70 |
71 |
72 | log4j.properties
73 |
74 |
75 |
76 |
77 | ${project.build.directory}/${project.artifactId}-assembly-${project.version}.jar
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
--------------------------------------------------------------------------------
/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister:
--------------------------------------------------------------------------------
1 | com.hortonworks.spark.hive.HiveSourceProvider
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/HiveSourceProvider.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive
19 |
20 | import org.apache.spark.sql.SparkSession
21 | import org.apache.spark.sql.sources.DataSourceRegister
22 | import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport}
23 | import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
24 | import org.apache.spark.sql.streaming.OutputMode
25 | import org.apache.spark.sql.types.StructType
26 |
27 | import com.hortonworks.spark.hive.common.HiveOptions
28 |
29 | class HiveSourceProvider extends DataSourceV2 with StreamWriteSupport with DataSourceRegister {
30 |
31 | override def shortName(): String = "hive-streaming"
32 |
33 | override def createStreamWriter(
34 | queryId: String,
35 | schema: StructType,
36 | outputMode: OutputMode,
37 | dataSourceOptions: DataSourceOptions): StreamWriter = {
38 | val localHiveOptions = HiveOptions.fromDataSourceOptions(dataSourceOptions)
39 | val session = SparkSession.getActiveSession.orElse(SparkSession.getDefaultSession)
40 | require(session.isDefined)
41 |
42 | if (outputMode != OutputMode.Append()) {
43 | throw new IllegalStateException("Hive Streaming only supports output with Append mode")
44 | }
45 |
46 | val destTable = try {
47 | session.get.sharedState.externalCatalog.getTable(
48 | localHiveOptions.dbName, localHiveOptions.tableName)
49 | } catch {
50 | case e: Exception => throw new IllegalStateException("Cannot find destination table in " +
51 | s"metastore, please create table ${localHiveOptions.tableName} at first", e)
52 | }
53 | val destSchema = destTable.schema
54 |
55 | if (schema.map(_.name).toSet != destSchema.map(_.name).toSet) {
56 | throw new IllegalStateException(s"Schema $schema transformed from input source is different" +
57 | s" from schema $destSchema for the destination table")
58 | }
59 |
60 | new HiveStreamWriter(schema.map(_.name), destTable.partitionColumnNames, dataSourceOptions)
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/HiveStreamDataWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive
19 |
20 | import java.util.{List => JList, Map => JMap}
21 | import java.util.concurrent.{Executors, TimeUnit}
22 |
23 | import scala.collection.JavaConverters._
24 | import scala.collection.mutable
25 |
26 | import org.apache.spark.sql.Row
27 | import org.apache.spark.sql.sources.v2.DataSourceOptions
28 | import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage}
29 | import org.json4s.{DefaultFormats, Extraction}
30 | import org.json4s.jackson.JsonMethods._
31 |
32 | import com.hortonworks.spark.hive.common.{CachedHiveWriters, CachedKey, HiveOptions, HiveWriter}
33 | import com.hortonworks.spark.hive.utils.Logging
34 |
35 | case object HiveStreamWriterCommitMessage extends WriterCommitMessage
36 |
37 | class HiveStreamDataWriter(
38 | partitionId: Int,
39 | attemptNumber: Int,
40 | columnName: Seq[String],
41 | partitionCols: Seq[String],
42 | dataSourceOptionsMap: JMap[String, String],
43 | initClassLoader: ClassLoader,
44 | isolatedClassLoader: ClassLoader) extends DataWriter[Row] with Logging {
45 |
46 | private implicit def formats = DefaultFormats
47 |
48 | private val hiveOptions =
49 | HiveOptions.fromDataSourceOptions(new DataSourceOptions(dataSourceOptionsMap))
50 | private val ugi = hiveOptions.getUGI()
51 |
52 | private val inUseWriters = new mutable.HashMap[CachedKey, HiveWriter]()
53 |
54 | private val executorService = Executors.newSingleThreadScheduledExecutor()
55 | executorService.scheduleAtFixedRate(new Runnable {
56 | Thread.currentThread().setContextClassLoader(isolatedClassLoader)
57 |
58 | override def run(): Unit = {
59 | inUseWriters.foreach(_._2.heartbeat())
60 | }
61 | }, 10L, 10L, TimeUnit.SECONDS)
62 |
63 | private def withClassLoader[T](func: => T): T = {
64 | try {
65 | Thread.currentThread().setContextClassLoader(isolatedClassLoader)
66 | func
67 | } finally {
68 | Thread.currentThread().setContextClassLoader(initClassLoader)
69 | }
70 | }
71 |
72 | override def write(row: Row): Unit = withClassLoader {
73 | val partitionValues = partitionCols.map { col => stringfyField(row.get(row.fieldIndex(col))) }
74 | val hiveEndPoint =
75 | Class.forName("org.apache.hive.hcatalog.streaming.HiveEndPoint", true, isolatedClassLoader)
76 | .getConstructor(classOf[String], classOf[String], classOf[String], classOf[JList[String]])
77 | .newInstance(
78 | hiveOptions.metastoreUri, hiveOptions.dbName, hiveOptions.tableName, partitionValues.asJava)
79 | .asInstanceOf[Object]
80 |
81 | val key = CachedKey(
82 | hiveOptions.metastoreUri, hiveOptions.dbName, hiveOptions.tableName, partitionValues)
83 |
84 | def getNewWriter(): HiveWriter = {
85 | val writer = CachedHiveWriters.getOrCreate(
86 | key, hiveEndPoint, hiveOptions, ugi, isolatedClassLoader)
87 | writer.beginTransaction()
88 | writer
89 | }
90 | val writer = inUseWriters.getOrElseUpdate(key, {
91 | logDebug(s"writer for $key not found in local cache")
92 | getNewWriter()
93 | })
94 |
95 | val jRow = Extraction.decompose(rowToMap(columnName, row))
96 | val jString = compact(render(jRow))
97 |
98 | logDebug(s"Write JSON row ${pretty(render(jRow))} into Hive Streaming")
99 | writer.write(jString.getBytes("UTF-8"))
100 |
101 | if (writer.totalRecords() >= hiveOptions.batchSize) {
102 | writer.commitTransaction()
103 | writer.beginTransaction()
104 | }
105 | }
106 |
107 | override def abort(): Unit = withClassLoader {
108 | inUseWriters.foreach { case (key, writer) =>
109 | writer.abortTransaction()
110 | CachedHiveWriters.recycle(writer)
111 | logDebug(s"Recycle writer $writer for $key to global cache")
112 | }
113 | inUseWriters.clear()
114 | executorService.shutdown()
115 | }
116 |
117 | override def commit(): WriterCommitMessage = withClassLoader {
118 | inUseWriters.foreach { case (key, writer) =>
119 | writer.commitTransaction()
120 | CachedHiveWriters.recycle(writer)
121 | logDebug(s"Recycle writer $writer for $key to global cache")
122 | }
123 | inUseWriters.clear()
124 | executorService.shutdown()
125 |
126 | HiveStreamWriterCommitMessage
127 | }
128 |
129 | private def stringfyField(col: Any): String = {
130 | col match {
131 | case _: Array[Byte] =>
132 | throw new UnsupportedOperationException("Cannot convert partition column with BinaryType " +
133 | "to String")
134 | case _: Seq[_] =>
135 | throw new UnsupportedOperationException("Cannot convert partition column with ArrayType " +
136 | "to String")
137 | case _: Map[_, _] =>
138 | throw new UnsupportedOperationException("Cannot convert partition column with MapType " +
139 | "to String")
140 | case _: Row =>
141 | throw new UnsupportedOperationException("Cannot convert partition column with StructType " +
142 | "to String")
143 | case i => i.toString
144 | }
145 | }
146 |
147 | private def rowToMap(columnName: Seq[String], row: Row): Map[String, Any] = {
148 | columnName.map { col =>
149 | val field = row.get(row.fieldIndex(col)) match {
150 | case b: java.math.BigDecimal => new BigDecimal(b)
151 | case r: Row => rowToMap(r.schema.map(_.name), r)
152 | case e => e
153 | }
154 | col -> field
155 | }.toMap
156 | }
157 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/HiveStreamWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive
19 |
20 | import java.util.{Map => JMap}
21 |
22 | import org.apache.spark.sql.Row
23 | import org.apache.spark.sql.sources.v2.DataSourceOptions
24 | import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory, WriterCommitMessage}
25 | import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter
26 |
27 | import com.hortonworks.spark.hive.utils.HiveIsolatedClassLoader
28 |
29 | class HiveStreamWriter(
30 | columnNames: Seq[String],
31 | partitionCols: Seq[String],
32 | dataSourceOptions: DataSourceOptions)
33 | extends StreamWriter {
34 |
35 | override def createWriterFactory(): DataWriterFactory[Row] = {
36 | new HiveStreamDataWriterFactory(columnNames, partitionCols, dataSourceOptions.asMap())
37 | }
38 |
39 | override def commit(epochId: Long, writerCommitMessages: Array[WriterCommitMessage]): Unit = {}
40 |
41 | override def abort(epochId: Long, writerCommitMessages: Array[WriterCommitMessage]): Unit = {}
42 | }
43 |
44 | class HiveStreamDataWriterFactory(
45 | columnName: Seq[String],
46 | partitionCols: Seq[String],
47 | dataSourceOptionsMap: JMap[String, String]) extends DataWriterFactory[Row] {
48 |
49 | override def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[Row] = {
50 | val restoredClassLoader = Thread.currentThread().getContextClassLoader
51 | val currentClassLoader = HiveIsolatedClassLoader.isolatedClassLoader()
52 | try {
53 | Thread.currentThread().setContextClassLoader(currentClassLoader)
54 |
55 | currentClassLoader.loadClass(classOf[HiveStreamDataWriter].getName)
56 | .getConstructors.head
57 | .newInstance(partitionId: java.lang.Integer, attemptNumber: java.lang.Integer,
58 | columnName, partitionCols, dataSourceOptionsMap, restoredClassLoader, currentClassLoader)
59 | .asInstanceOf[DataWriter[Row]]
60 | } finally {
61 | Thread.currentThread().setContextClassLoader(restoredClassLoader)
62 | }
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/common/CachedHiveWriters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.common
19 |
20 | import java.util.concurrent.{Executors, TimeUnit}
21 | import javax.annotation.Nullable
22 |
23 | import scala.collection.mutable
24 | import scala.util.Try
25 | import scala.util.control.NonFatal
26 |
27 | import org.apache.hadoop.security.UserGroupInformation
28 |
29 | import com.hortonworks.spark.hive.utils.Logging
30 |
31 | case class CachedKey(metastoreUri: String, db: String, table: String, partitionCols: Seq[String])
32 |
33 | object CachedHiveWriters extends Logging {
34 |
35 | private val cacheExpireTimeout: Long = TimeUnit.MINUTES.toMillis(10)
36 |
37 | private val cache = new mutable.HashMap[CachedKey, mutable.Queue[HiveWriter]]()
38 |
39 | private val executorService = Executors.newSingleThreadScheduledExecutor()
40 | executorService.scheduleAtFixedRate(new Runnable {
41 | override def run(): Unit = {
42 | expireOldestWriters()
43 | }
44 | }, 10L, 10L, TimeUnit.MINUTES)
45 |
46 | Runtime.getRuntime.addShutdownHook(new Thread {
47 | override def run(): Unit = {
48 | try {
49 | clear()
50 | executorService.shutdown()
51 | } catch {
52 | case NonFatal(_) => // swallow exceptions
53 | }
54 | }
55 | })
56 |
57 | def getOrCreate(
58 | key: CachedKey,
59 | hiveEndPoint: Object,
60 | hiveOptions: HiveOptions,
61 | @Nullable ugi: UserGroupInformation,
62 | isolatedClassLoader: ClassLoader): HiveWriter = {
63 | val writer = CachedHiveWriters.synchronized {
64 | val queue = cache.getOrElseUpdate(key, new mutable.Queue[HiveWriter]())
65 | if (queue.isEmpty) {
66 | None
67 | } else {
68 | logDebug(s"Found writer for $key in global cache")
69 | Some(queue.dequeue())
70 | }
71 | }
72 |
73 | writer.getOrElse(new HiveWriter(key, hiveEndPoint, hiveOptions, ugi, isolatedClassLoader))
74 | }
75 |
76 | def recycle(hiveWriter: HiveWriter): Unit = {
77 | CachedHiveWriters.synchronized {
78 | cache.getOrElseUpdate(hiveWriter.key, new mutable.Queue[HiveWriter]())
79 | .enqueue(hiveWriter)
80 | }
81 | }
82 |
83 | private def expireOldestWriters(): Unit = {
84 | val currentTime = System.currentTimeMillis()
85 | val expiredWriters = new mutable.ArrayBuffer[HiveWriter]()
86 |
87 | CachedHiveWriters.synchronized {
88 | val emptyKeys = cache.filter { case (_, queue) =>
89 | while (queue.nonEmpty) {
90 | if (queue.head.lastUsed() + cacheExpireTimeout < currentTime) {
91 | expiredWriters.append(queue.dequeue())
92 | }
93 | }
94 | queue.isEmpty
95 | }.keySet
96 |
97 | emptyKeys.foreach { k => cache.remove(k) }
98 | }
99 |
100 | expiredWriters.foreach { w =>
101 | if (Try { w.close() }.isFailure) {
102 | logWarn("Failed to close writer")
103 | } else {
104 | logInfo(s"Closed expired writer $w")
105 | }
106 | }
107 | }
108 |
109 | private def clear(): Unit = {
110 | val unusedWriters = new mutable.ArrayBuffer[HiveWriter]()
111 |
112 | CachedHiveWriters.synchronized {
113 | cache.foreach { case (_, queue) =>
114 | queue.foreach(unusedWriters.append(_))
115 | }
116 | cache.clear()
117 | }
118 |
119 | unusedWriters.foreach { w =>
120 | if (Try { w.close() }.isFailure) {
121 | logWarn("Failed to close writer")
122 | } else {
123 | logInfo(s"Closed writer $w")
124 | }
125 | }
126 | }
127 | }
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/common/HiveOptions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.common
19 |
20 | import java.io.File
21 | import javax.annotation.Nullable
22 |
23 | import org.apache.hadoop.security.UserGroupInformation
24 | import org.apache.spark.sql.sources.v2.DataSourceOptions
25 |
26 | import com.hortonworks.spark.hive.utils.Logging
27 |
28 | class HiveOptions private (
29 | val metastoreUri: String,
30 | val dbName: String,
31 | val tableName: String) extends Logging {
32 |
33 | var txnPerBatch = 100
34 | var batchSize = 10000
35 | var autoCreatePartitions = true
36 |
37 | private var principal: String = null
38 | private var keytab: String = null
39 |
40 | def withTxnPerBatch(txnPerBatch: Int): HiveOptions = {
41 | this.txnPerBatch = txnPerBatch
42 | this
43 | }
44 |
45 | def withAutoCreatePartitions(autoCreatePartitions: Boolean): HiveOptions = {
46 | this.autoCreatePartitions = autoCreatePartitions
47 | this
48 | }
49 |
50 | def withPrincipalAndKeytab(principal: String, keytab: String): HiveOptions = {
51 | this.principal = principal
52 | this.keytab = keytab
53 | this
54 | }
55 |
56 | def withBatchSize(batchSize: Int): HiveOptions = {
57 | this.batchSize = batchSize
58 | this
59 | }
60 |
61 | @Nullable
62 | def getUGI(): UserGroupInformation = {
63 | if (principal == null || keytab == null) {
64 | null.asInstanceOf[UserGroupInformation]
65 | } else {
66 | val kfile = new File(keytab)
67 | if (!(kfile.exists && kfile.canRead)) {
68 | throw new IllegalArgumentException(s"keytab file $keytab is not existed or unreadable")
69 | }
70 |
71 | val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
72 | logInfo(s"UGI $ugi with principal $principal and keytab $keytab")
73 | ugi
74 | }
75 | }
76 | }
77 |
78 | object HiveOptions {
79 | // Key of hive options (case insensitive).
80 | val METASTORE_URI = "metastore"
81 | val DB_NAME = "db"
82 | val TABLE_NAME = "table"
83 | val TXN_PER_BATCH = "txn.per.batch"
84 | val AUTO_CREATE_PARTITIONS = "auto.create.partitions"
85 | val PRINCIPAL = "principal"
86 | val KEYTAB = "keytab"
87 | val BATCH_SIZE = "batch.size"
88 |
89 | def fromDataSourceOptions(dataSourceOptions: DataSourceOptions): HiveOptions = {
90 | val metastoreUri = dataSourceOptions.get(METASTORE_URI)
91 | if (!metastoreUri.isPresent) {
92 | throw new IllegalArgumentException("metastore URI must be specified")
93 | }
94 |
95 | val dbName = dataSourceOptions.get(DB_NAME)
96 | if (!dbName.isPresent) {
97 | throw new IllegalArgumentException("db name must be specified")
98 | }
99 |
100 | val tblName = dataSourceOptions.get(TABLE_NAME)
101 | if (!tblName.isPresent) {
102 | throw new IllegalArgumentException("table name must be specified")
103 | }
104 |
105 | val option = new HiveOptions(metastoreUri.get(), dbName.get(), tblName.get())
106 |
107 | option.withTxnPerBatch(dataSourceOptions.getInt(TXN_PER_BATCH, option.txnPerBatch))
108 | .withAutoCreatePartitions(
109 | dataSourceOptions.getBoolean(AUTO_CREATE_PARTITIONS, option.autoCreatePartitions))
110 | .withPrincipalAndKeytab(
111 | dataSourceOptions.get(PRINCIPAL).orElse(option.principal),
112 | dataSourceOptions.get(KEYTAB).orElse(option.keytab))
113 | .withBatchSize(dataSourceOptions.getInt(BATCH_SIZE, option.batchSize))
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/common/HiveWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.common
19 |
20 | import java.security.PrivilegedExceptionAction
21 |
22 | import org.apache.hadoop.security.UserGroupInformation
23 |
24 | import com.hortonworks.spark.hive.utils.Logging
25 |
26 | class HiveWriter(
27 | val key: CachedKey,
28 | hiveEndPoint: Object,
29 | hiveOptions: HiveOptions,
30 | ugi: UserGroupInformation,
31 | isolatedClassLoader: ClassLoader) extends Logging {
32 |
33 | private val hiveConf =
34 | Class.forName("org.apache.hadoop.hive.conf.HiveConf", true, isolatedClassLoader)
35 | .newInstance()
36 | .asInstanceOf[Object]
37 | private val txnTimeout = 300 * 1000L
38 |
39 | private val connection = hiveEndPoint.getClass.getMethod(
40 | "newConnection",
41 | classOf[Boolean],
42 | Class.forName("org.apache.hadoop.hive.conf.HiveConf", true, isolatedClassLoader),
43 | classOf[UserGroupInformation])
44 | .invoke(hiveEndPoint, hiveOptions.autoCreatePartitions: java.lang.Boolean, hiveConf, ugi)
45 |
46 | private val writer = if (ugi == null) {
47 | createWriter()
48 | } else {
49 | ugi.doAs(new PrivilegedExceptionAction[Object] {
50 | override def run(): Object = {
51 | createWriter()
52 | }
53 | })
54 | }
55 |
56 | private var txnBatch: Object = null
57 |
58 | // Timestamp to track the activity of this HiveWriter
59 | private var _lastUsed: Long = System.currentTimeMillis()
60 |
61 | // Timestamp to track the last creation time of transaction batch
62 | private var _lastCreated = System.currentTimeMillis()
63 |
64 | // Track the number of records written in this batch
65 | private var _totalRecords = 0
66 |
67 | private var isTransactionBegin = false
68 |
69 | // TODO. for now we only support to write JSON String to Hive Streaming.
70 | private def createWriter(): Object = {
71 | Class.forName("org.apache.hive.hcatalog.streaming.StrictJsonWriter", true, isolatedClassLoader)
72 | .getConstructor(
73 | Class.forName("org.apache.hive.hcatalog.streaming.HiveEndPoint", true, isolatedClassLoader),
74 | Class.forName("org.apache.hadoop.hive.conf.HiveConf", true, isolatedClassLoader))
75 | .newInstance(hiveEndPoint, hiveConf)
76 | .asInstanceOf[Object]
77 | }
78 |
79 | def beginTransaction(): Unit = {
80 | if (txnBatch != null && call[Int](txnBatch, "remainingTransactions") == 0) {
81 | call[Unit](txnBatch, "close")
82 | txnBatch = null
83 | }
84 |
85 | if (txnBatch == null) {
86 | txnBatch = call[Object](connection, "fetchTransactionBatch",
87 | Seq(classOf[Int], Class.forName(
88 | "org.apache.hive.hcatalog.streaming.RecordWriter", true, isolatedClassLoader)),
89 | Seq(hiveOptions.txnPerBatch: java.lang.Integer, writer))
90 | _lastCreated = System.currentTimeMillis()
91 | }
92 |
93 | call[Unit](txnBatch, "beginNextTransaction")
94 | isTransactionBegin = true
95 | _totalRecords = 0
96 |
97 | logDebug(s"Switch to next transaction for $hiveEndPoint")
98 | }
99 |
100 | def write(record: Array[Byte]): Unit = {
101 | require(txnBatch != null, "current transaction is not initialized before writing")
102 | require(isTransactionBegin, "current transaction is not beginning")
103 |
104 | call[Unit](txnBatch, "write", Seq(classOf[Array[Byte]]), Seq(record))
105 | _totalRecords += 1
106 | }
107 |
108 | def commitTransaction(): Unit = {
109 | require(txnBatch != null, "current transaction is not initialized before committing")
110 | require(isTransactionBegin, "current transaction is not beginning")
111 |
112 | call[Unit](txnBatch, "commit")
113 |
114 | _lastUsed = System.currentTimeMillis()
115 | isTransactionBegin = false
116 | _totalRecords = 0
117 | }
118 |
119 | def abortTransaction(): Unit = {
120 | isTransactionBegin = false
121 | _totalRecords = 0
122 |
123 | if (txnBatch != null) {
124 | call[Unit](txnBatch, "abort")
125 | }
126 | }
127 |
128 | def close(): Unit = {
129 | isTransactionBegin = false
130 | _totalRecords = 0
131 |
132 | if (txnBatch != null) {
133 | call[Unit](txnBatch, "commit")
134 | call[Unit](txnBatch, "close")
135 | }
136 |
137 | call[Unit](connection, "close")
138 | }
139 |
140 | def lastUsed(): Long = _lastUsed
141 |
142 | def totalRecords(): Int = _totalRecords
143 |
144 | def heartbeat(): Unit = {
145 | if (System.currentTimeMillis() - _lastCreated > txnTimeout / 2) {
146 | if (txnBatch != null) {
147 | call[Unit](txnBatch, "heartbeat")
148 | }
149 | }
150 | }
151 |
152 | private def call[T](
153 | obj: Object,
154 | method: String,
155 | types: Seq[Class[_]] = Seq.empty,
156 | params: Seq[Object] = Seq.empty): T = {
157 | val mtd = obj.getClass.getMethod(method, types: _*)
158 | mtd.setAccessible(true)
159 | mtd.invoke(obj, params: _*).asInstanceOf[T]
160 | }
161 | }
162 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/utils/HiveIsolatedClassLoader.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.utils
19 |
20 | import java.net.{URL, URLClassLoader}
21 | import java.util
22 |
23 | object HiveIsolatedClassLoader extends Logging {
24 |
25 | def isolatedClassLoader(): ClassLoader = {
26 | val parentClassLoader = Option(Thread.currentThread().getContextClassLoader)
27 | .getOrElse(getClass.getClassLoader)
28 |
29 | // Assume the task parent classloader is either Spark MutableURLClassLoader or
30 | // ExecutorClassLoader
31 | def getAddedURLs(classloader: ClassLoader): Array[URL] = {
32 | classloader match {
33 | case e if e.getClass.getName == "org.apache.spark.repl.ExecutorClassLoader" =>
34 | val method = e.getClass.getMethod("parentLoader")
35 | method.setAccessible(true)
36 | val parent = method.invoke(e).asInstanceOf[ClassLoader].getParent
37 | getAddedURLs(parent)
38 |
39 | case e if e.getClass.getName == "org.apache.spark.util.ChildFirstURLClassLoader" =>
40 | val method = e.getClass.getMethod("parentClassLoader")
41 | method.setAccessible(true)
42 | val parent = method.invoke(e).asInstanceOf[ClassLoader].getParent
43 | getAddedURLs(parent)
44 |
45 | case e if e.getClass.getName == "org.apache.spark.util.MutableURLClassLoader" =>
46 | val method = e.getClass.getMethod("getURLs")
47 | method.setAccessible(true)
48 | method.invoke(e).asInstanceOf[Array[URL]]
49 |
50 | case e: ClassLoader =>
51 | Option(e.getParent).map { getAddedURLs(_) }.getOrElse(
52 | throw new IllegalStateException("Get unexpected classloader"))
53 |
54 | case u =>
55 | throw new IllegalStateException(s"Get unexpected object, $u")
56 | }
57 | }
58 |
59 | val urls = getAddedURLs(parentClassLoader)
60 | new HiveIsolatedClassLoader(urls, parentClassLoader)
61 | }
62 | }
63 |
64 | class HiveIsolatedClassLoader(urls: Array[URL], baseClassLoader: ClassLoader)
65 | extends URLClassLoader(urls, ClassLoader.getSystemClassLoader.getParent.getParent)
66 | with Logging {
67 |
68 | override def loadClass(name: String, resolve: Boolean): Class[_] = {
69 | val loaded = findLoadedClass(name)
70 | if (loaded == null) doLoadClass(name, resolve) else loaded
71 | }
72 |
73 | override def getResource(name: String): URL = {
74 | baseClassLoader.getResource(name)
75 | }
76 |
77 | override def getResources(name: String): util.Enumeration[URL] = {
78 | baseClassLoader.getResources(name)
79 | }
80 |
81 | def doLoadClass(name: String, resolve: Boolean): Class[_] = {
82 | if (isHiveClass(name)) {
83 | logTrace(s"hive class: $name - ${super.getResource(classToPath(name))}")
84 | super.loadClass(name, resolve)
85 | } else {
86 | try {
87 | baseClassLoader.loadClass(name)
88 | } catch {
89 | case _: ClassNotFoundException =>
90 | super.loadClass(name, resolve)
91 | }
92 | }
93 | }
94 |
95 | private def isHiveClass(name: String): Boolean = {
96 | name.startsWith("org.apache.hadoop.hive.") ||
97 | name.startsWith("org.apache.hive.") ||
98 | name.startsWith("org.apache.orc.")
99 | }
100 |
101 | private def classToPath(name: String): String =
102 | name.replaceAll("\\.", "/") + ".class"
103 | }
104 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/hortonworks/spark/hive/utils/Logging.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.utils
19 |
20 | import org.slf4j.LoggerFactory
21 |
22 | trait Logging {
23 | lazy val logger = LoggerFactory.getLogger(this.getClass)
24 |
25 | def logTrace(message: => Any): Unit = {
26 | if (logger.isTraceEnabled) {
27 | logger.trace(message.toString)
28 | }
29 | }
30 |
31 | def logDebug(message: => Any): Unit = {
32 | if (logger.isDebugEnabled) {
33 | logger.debug(message.toString)
34 | }
35 | }
36 |
37 | def logInfo(message: => Any): Unit = {
38 | if (logger.isInfoEnabled) {
39 | logger.info(message.toString)
40 | }
41 | }
42 |
43 | def logWarn(message: => Any): Unit = {
44 | logger.warn(message.toString)
45 | }
46 |
47 | def logWarn(message: => Any, t: Throwable): Unit = {
48 | logger.warn(message.toString, t)
49 | }
50 |
51 | def logError(message: => Any, t: Throwable): Unit = {
52 | logger.error(message.toString, t)
53 | }
54 |
55 | def logError(message: => Any): Unit = {
56 | logger.error(message.toString)
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/example/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
21 | 4.0.0
22 |
23 |
24 | com.hortonworks.spark
25 | spark-hive-streaming-sink-main_2.11
26 | 0.1.0-SNAPSHOT
27 | ../pom.xml
28 |
29 |
30 | spark-hive-streaming-sink-example_2.11
31 | 0.1.0-SNAPSHOT
32 | jar
33 |
34 |
35 |
36 | com.hortonworks.spark
37 | spark-hive-streaming-sink_2.11
38 | 0.1.0-SNAPSHOT
39 |
40 |
41 |
42 |
43 |
--------------------------------------------------------------------------------
/example/src/main/scala/com/hortonworks/spark/hive/example/HiveStreamingExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.example
19 |
20 | import org.apache.spark.SparkConf
21 | import org.apache.spark.sql.SparkSession
22 |
23 | /**
24 | * A Hive Streaming example to ingest data from socket and push into hive table.
25 | *
26 | * Assumed HIVE table Schema:
27 | * create table alerts ( id int , msg string )
28 | * partitioned by (continent string, country string)
29 | * clustered by (id) into 5 buckets
30 | * stored as orc tblproperties("transactional"="true");
31 | */
32 | object HiveStreamingExample {
33 |
34 | def main(args: Array[String]): Unit = {
35 | if (args.length < 3 || args.length > 5) {
36 | // scalastyle:off println
37 | System.err.println(s"Usage: HiveStreamingExample " +
38 | s" [principal] [keytab]")
39 | // scalastyle:on println
40 | System.exit(1)
41 | }
42 |
43 | val host = args(0)
44 | val port = args(1)
45 | val metastoreUri = args(2)
46 |
47 | val (principal, keytab) = if (args.length == 5) {
48 | (args(3), args(4))
49 | } else {
50 | (null, null)
51 | }
52 |
53 | val sparkConf = new SparkConf()
54 | .set("spark.sql.streaming.checkpointLocation", "./checkpoint")
55 | val sparkSession = SparkSession.builder()
56 | .appName("HiveStreamingExample")
57 | .config(sparkConf)
58 | .enableHiveSupport()
59 | .getOrCreate()
60 |
61 | import sparkSession.implicits._
62 |
63 | val socket = sparkSession.readStream
64 | .format("socket")
65 | .options(Map("host" -> host, "port" -> port))
66 | .load()
67 | .as[String]
68 |
69 | val writer = socket.map { s =>
70 | val records = s.split(",")
71 | assert(records.length >= 4)
72 | (records(0).toInt, records(1), records(2), records(3))
73 | }
74 | .selectExpr("_1 as id", "_2 as msg", "_3 as continent", "_4 as country")
75 | .writeStream
76 | .format("hive-streaming")
77 | .option("metastore", metastoreUri)
78 | .option("db", "default")
79 | .option("table", "alerts")
80 |
81 | if (principal != null && keytab != null) {
82 | writer.option("principal", principal)
83 | .option("keytab", keytab)
84 | }
85 |
86 | val query = writer.start()
87 | query.awaitTermination()
88 |
89 | query.stop()
90 | sparkSession.stop()
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/example/src/main/scala/com/hortonworks/spark/hive/example/RateStreamingExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package com.hortonworks.spark.hive.example
19 |
20 | import org.apache.spark.SparkConf
21 | import org.apache.spark.sql.SparkSession
22 | import org.apache.spark.sql.streaming.Trigger
23 |
24 | /**
25 | * A Hive Streaming example to ingest data from rate stream and push into hive table.
26 | *
27 | * Assumed HIVE table Schema:
28 | * create table rate (value bigint)
29 | * clustered by (value) into 5 buckets
30 | * stored as orc tblproperties("transactional"="true");
31 | */
32 | object RateStreamingExample {
33 |
34 | def main(args: Array[String]): Unit = {
35 | if (args.length < 1 || args.length > 4) {
36 | // scalastyle:off println
37 | System.err.println(s"Usage: RateStreamingExample [principal] " +
38 | s"[keytab] [continuous?]")
39 | // scalastyle:on println
40 | System.exit(1)
41 | }
42 |
43 | val metastoreUri = args(0)
44 | val continuous = if (args.length == 2) {
45 | args(1) == "continuous"
46 | } else if (args.length == 4) {
47 | args(3) == "continuous"
48 | } else {
49 | false
50 | }
51 |
52 | val principal = if (args.length >= 3) args(1) else null
53 | val keytab = if (args.length >= 3) args(2) else null
54 |
55 | val sparkConf = new SparkConf()
56 | .set("spark.sql.streaming.checkpointLocation", "./checkpoint")
57 | val sparkSession = SparkSession.builder()
58 | .appName("RateStreamingExample")
59 | .config(sparkConf)
60 | .enableHiveSupport()
61 | .getOrCreate()
62 |
63 | val rate = sparkSession.readStream
64 | .format("rate")
65 | .option("rowsPerSecond", "1")
66 | .load()
67 |
68 | val writer = rate.select("value")
69 | .writeStream
70 | .format("hive-streaming")
71 | .option("metastore", metastoreUri)
72 | .option("db", "default")
73 | .option("table", "rate")
74 |
75 | if (principal != null && keytab != null) {
76 | writer.option("principal", principal)
77 | writer.option("keytab", keytab)
78 | }
79 |
80 | val query = writer
81 | .trigger(if (continuous) Trigger.Continuous(3000L) else Trigger.ProcessingTime(3000L))
82 | .start()
83 |
84 | query.awaitTermination()
85 |
86 | query.stop()
87 | sparkSession.stop()
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
21 | 4.0.0
22 |
23 | com.hortonworks.spark
24 | spark-hive-streaming-sink-main_2.11
25 | 0.1.0-SNAPSHOT
26 | pom
27 |
28 |
29 | 2.3.0
30 | 1.2.1
31 | 2.11.8
32 | 2.11
33 | 512m
34 | 512m
35 | 1.8
36 | 1.8
37 | 1.8
38 | true
39 | 2.2.4
40 | 1.9.5
41 |
42 |
43 |
44 | core
45 | example
46 |
47 |
48 |
49 |
50 |
51 | hortonworks-repo
52 | Hortonworks Repository
53 | http://repo.hortonworks.com/content/repositories/releases/
54 |
55 | true
56 |
57 |
58 | false
59 |
60 |
61 |
62 |
63 | hortonworks-snapshot-repo
64 | Hortonworks Snapshot Repository
65 | http://nexus-private.hortonworks.com/nexus/content/groups/public/
66 |
67 |
68 |
69 |
70 |
71 |
72 |
73 | org.apache.spark
74 | spark-sql_${scala.binary.version}
75 | ${spark.version}
76 | provided
77 |
78 |
79 |
80 | org.apache.spark
81 | spark-catalyst_${scala.binary.version}
82 | ${spark.version}
83 | provided
84 |
85 |
86 |
87 | org.scalacheck
88 | scalacheck_${scala.binary.version}
89 | 1.12.5
90 |
91 |
92 |
93 | org.apache.hive.hcatalog
94 | hive-hcatalog-streaming
95 | ${hive.version}
96 |
97 |
98 | org.slf4j
99 | slf4j-log4j12
100 |
101 |
102 | org.apache.hadoop
103 | *
104 |
105 |
106 | org.apache.hbase
107 | *
108 |
109 |
110 | org.apache.hive
111 | hive-llap-tez
112 |
113 |
114 | org.apache.hive
115 | hive-llap-server
116 |
117 |
118 |
119 |
120 |
121 | org.apache.hive.hcatalog
122 | hive-hcatalog-core
123 | ${hive.version}
124 |
125 |
126 | org.slf4j
127 | slf4j-log4j12
128 |
129 |
130 | org.apache.hadoop
131 | *
132 |
133 |
134 |
135 |
136 |
137 | log4j
138 | log4j
139 | 1.2.16
140 | provided
141 |
142 |
143 |
144 | org.scalatest
145 | scalatest_${scala.binary.version}
146 | ${scalatest.version}
147 | test
148 |
149 |
150 |
151 | org.mockito
152 | mockito-all
153 | ${mockito.version}
154 |
155 |
156 |
157 |
158 |
159 |
160 |
161 |
162 |
163 | org.apache.maven.plugins
164 | maven-enforcer-plugin
165 | 1.4.1
166 |
167 |
168 | enforce-versions
169 |
170 | enforce
171 |
172 |
173 |
174 |
175 | 1.8
176 |
177 |
178 |
179 |
180 |
181 |
182 |
183 |
184 | org.apache.maven.plugins
185 | maven-compiler-plugin
186 | 3.3
187 |
188 |
189 |
190 | org.apache.maven.plugins
191 | maven-antrun-plugin
192 | 1.8
193 |
194 |
195 |
196 | org.apache.maven.plugins
197 | maven-deploy-plugin
198 | 2.8.2
199 |
200 |
201 |
202 | org.codehaus.mojo
203 | build-helper-maven-plugin
204 | 1.10
205 |
206 |
207 |
208 | net.alchim31.maven
209 | scala-maven-plugin
210 | 3.2.2
211 |
212 |
213 |
214 | compile
215 | testCompile
216 |
217 |
218 |
219 |
220 | ${scala.version}
221 | incremental
222 | true
223 | false
224 |
225 | -unchecked
226 | -deprecation
227 | -feature
228 |
229 |
230 | -source
231 | ${java.version}
232 | -target
233 | -Xlint:unchecked
234 |
235 |
236 | -Xms1024m
237 | -Xmx1024m
238 | -XX:MaxPermSize=${MaxPermGen}
239 | -XX:ReservedCodeCacheSize=${CodeCacheSize}
240 |
241 |
242 |
243 |
244 |
245 | org.apache.maven.plugins
246 | maven-install-plugin
247 | 2.5.2
248 |
249 |
250 |
251 | org.apache.maven.plugins
252 | maven-surefire-plugin
253 | 2.19
254 |
255 |
256 | true
257 | ${project.build.directory}/tmp
258 | ${project.version}
259 |
260 | ${test.redirectToFile}
261 | ${test.redirectToFile}
262 | -Xmx2g -XX:MaxPermSize=512m
263 | false
264 |
265 |
266 |
267 |
268 | org.scalatest
269 | scalatest-maven-plugin
270 | 1.0
271 |
272 |
273 | true
274 | ${project.build.directory}/tmp
275 |
276 | D
277 | ${project.build.directory}/surefire-reports
278 | .
279 | WDF TestSuite.txt
280 | -Xmx2g -XX:MaxPermSize=512m
281 |
282 |
283 |
284 | test
285 |
286 | test
287 |
288 |
289 |
290 |
291 |
292 |
293 | org.apache.maven.plugins
294 | maven-shade-plugin
295 | 3.1.0
296 |
297 |
298 |
299 |
300 |
301 |
302 |
303 | org.apache.maven.plugins
304 | maven-compiler-plugin
305 |
306 |
307 | default-compile
308 | none
309 |
310 |
311 | default-testCompile
312 | none
313 |
314 |
315 |
316 | ${java.version}
317 | ${java.version}
318 |
319 |
320 |
321 |
322 | org.apache.maven.plugins
323 | maven-enforcer-plugin
324 | false
325 |
326 |
327 |
328 | [3.0.0,)
329 |
330 |
331 | [${minJavaVersion}.0,${maxJavaVersion}.1000}]
332 |
333 |
334 | unix
335 |
336 |
337 |
338 |
339 |
340 | clean
341 |
342 | enforce
343 |
344 | pre-clean
345 |
346 |
347 | default
348 |
349 | enforce
350 |
351 | validate
352 |
353 |
354 | site
355 |
356 | enforce
357 |
358 | pre-site
359 |
360 |
361 |
362 |
363 |
364 | net.alchim31.maven
365 | scala-maven-plugin
366 |
367 |
368 |
369 | org.apache.maven.plugins
370 | maven-antrun-plugin
371 |
372 |
373 |
374 | pre-test-clean
375 | generate-test-resources
376 |
377 | run
378 |
379 |
380 |
381 |
382 |
383 |
384 |
385 |
386 |
387 |
388 | create-tmp-dir
389 | generate-test-resources
390 |
391 | run
392 |
393 |
394 |
395 |
396 |
397 |
398 |
399 |
400 |
401 |
402 |
403 | org.apache.maven.plugins
404 | maven-surefire-plugin
405 |
406 |
407 |
408 | org.scalatest
409 | scalatest-maven-plugin
410 |
411 |
412 |
413 | org.scalastyle
414 | scalastyle-maven-plugin
415 | 1.0.0
416 |
417 | false
418 | true
419 | false
420 | false
421 | ${basedir}/src/main/scala
422 | ${basedir}/src/test/scala
423 | scalastyle-config.xml
424 | ${basedir}/target/scalastyle-output.xml
425 | ${project.build.sourceEncoding}
426 | ${project.reporting.outputEncoding}
427 |
428 |
429 |
430 |
431 | check
432 |
433 |
434 |
435 |
436 |
437 |
438 |
439 |
440 |
441 |
442 | hortonworks
443 |
444 |
445 | hortonworks
446 |
447 |
448 |
449 | 3.0.0.3.0.0.2-SNAPSHOT
450 | 2.3.0.3.0.0.2-SNAPSHOT
451 |
452 |
453 |
454 |
455 |
456 |
457 |
--------------------------------------------------------------------------------
/scalastyle-config.xml:
--------------------------------------------------------------------------------
1 |
17 |
39 |
40 |
41 | Scalastyle standard configuration
42 |
43 |
44 |
45 |
46 |
47 |
48 |
49 |
69 |
70 |
71 |
72 |
73 |
74 |
75 |
76 |
77 |
78 |
79 |
80 | true
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 |
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 |
97 |
98 |
99 |
100 |
101 |
102 |
103 |
104 |
105 |
106 |
107 |
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
118 |
119 |
120 |
121 |
122 |
123 |
124 |
125 | ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW
126 |
127 |
128 |
129 |
130 |
131 | ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW
132 |
133 |
134 |
135 |
136 |
137 |
138 |
139 |
140 | ^println$
141 |
145 |
146 |
147 |
148 | @VisibleForTesting
149 |
152 |
153 |
154 |
155 | mutable\.SynchronizedBuffer
156 |
164 |
165 |
166 |
167 | Class\.forName
168 |
175 |
176 |
177 |
178 | Await\.result
179 |
186 |
187 |
188 |
189 | Await\.ready
190 |
197 |
198 |
199 |
200 |
201 | JavaConversions
202 | Instead of importing implicits in scala.collection.JavaConversions._, import
203 | scala.collection.JavaConverters._ and use .asScala / .asJava methods
204 |
205 |
206 |
207 | org\.apache\.commons\.lang\.
208 | Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead
209 | of Commons Lang 2 (package org.apache.commons.lang.*)
210 |
211 |
212 |
213 | extractOpt
214 | Use Utils.jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter
215 | is slower.
216 |
217 |
218 |
219 |
220 | java,scala,3rdParty,spark,hortonworks
221 | javax?\..*
222 | scala\..*
223 | (?!org\.apache\.spark\.).*
224 | org\.apache\.spark\..*
225 | com\.hortonworks\..*
226 |
227 |
228 |
229 |
230 |
231 | COMMA
232 |
233 |
234 |
235 |
236 |
237 | \)\{
238 |
241 |
242 |
243 |
244 | (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*]
245 | Use Javadoc style indentation for multiline comments
246 |
247 |
248 |
249 | case[^\n>]*=>\s*\{
250 | Omit braces in case clauses.
251 |
252 |
253 |
254 |
255 |
256 |
257 |
258 |
259 |
260 |
261 |
262 |
263 |
264 |
265 |
266 |
267 |
268 |
269 |
270 |
271 |
272 |
273 |
274 |
275 |
276 |
277 |
278 |
279 |
280 |
281 |
282 |
283 |
284 |
285 |
286 |
287 |
288 |
289 |
290 |
291 |
292 |
293 |
294 |
295 |
296 |
297 |
298 |
299 |
300 |
301 |
302 |
303 | 800>
304 |
305 |
306 |
307 |
308 | 30
309 |
310 |
311 |
312 |
313 | 10
314 |
315 |
316 |
317 |
318 | 50
319 |
320 |
321 |
322 |
323 |
324 |
325 |
326 |
327 |
328 |
329 | -1,0,1,2,3
330 |
331 |
332 |
333 |
--------------------------------------------------------------------------------