├── .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 | "metastore, please create table 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 | // We assumed the type of partition column is String. 74 | val partitionValues = partitionCols.map { col => row.getAs[String](col) } 75 | val hiveEndPoint = 76 | Class.forName("org.apache.hive.hcatalog.streaming.HiveEndPoint", true, isolatedClassLoader) 77 | .getConstructor(classOf[String], classOf[String], classOf[String], classOf[JList[String]]) 78 | .newInstance( 79 | hiveOptions.metastoreUri, hiveOptions.dbName, hiveOptions.tableName, partitionValues.asJava) 80 | .asInstanceOf[Object] 81 | 82 | val key = CachedKey( 83 | hiveOptions.metastoreUri, hiveOptions.dbName, hiveOptions.tableName, partitionValues) 84 | 85 | def getNewWriter(): HiveWriter = { 86 | val writer = CachedHiveWriters.getOrCreate( 87 | key, hiveEndPoint, hiveOptions, ugi, isolatedClassLoader) 88 | writer.beginTransaction() 89 | writer 90 | } 91 | val writer = inUseWriters.getOrElseUpdate(key, { 92 | logDebug(s"writer for $key not found in local cache") 93 | getNewWriter() 94 | }) 95 | 96 | val jRow = Extraction.decompose(columnName.map { col => col -> row.getAs(col) }.toMap) 97 | val jString = compact(render(jRow)) 98 | 99 | logDebug(s"Write JSON row ${pretty(render(jRow))} into Hive Streaming") 100 | writer.write(jString.getBytes("UTF-8")) 101 | 102 | if (writer.totalRecords() >= hiveOptions.batchSize) { 103 | writer.commitTransaction() 104 | writer.beginTransaction() 105 | } 106 | } 107 | 108 | override def abort(): Unit = withClassLoader { 109 | inUseWriters.foreach { case (_, writer) => 110 | writer.abortTransaction() 111 | CachedHiveWriters.recycle(writer) 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 in global cache") 122 | } 123 | inUseWriters.clear() 124 | executorService.shutdown() 125 | 126 | HiveStreamWriterCommitMessage 127 | } 128 | } -------------------------------------------------------------------------------- /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 | 50 | 51 | 67 | 68 | 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 | --------------------------------------------------------------------------------