Driver state information for driver id {driverId}
- Back to Drivers + Back to DriversDriver state: {driverState.state}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 0a1d60f58bc58..45a3f43045437 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import org.apache.spark.Logging +import org.apache.spark.SecurityManager import org.apache.spark.deploy.Command import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils @@ -40,12 +41,14 @@ object CommandUtils extends Logging { */ def buildProcessBuilder( command: Command, + securityMgr: SecurityManager, memory: Int, sparkHome: String, substituteArguments: String => String, classPaths: Seq[String] = Seq[String](), env: Map[String, String] = sys.env): ProcessBuilder = { - val localCommand = buildLocalCommand(command, substituteArguments, classPaths, env) + val localCommand = buildLocalCommand( + command, securityMgr, substituteArguments, classPaths, env) val commandSeq = buildCommandSeq(localCommand, memory, sparkHome) val builder = new ProcessBuilder(commandSeq: _*) val environment = builder.environment() @@ -69,6 +72,7 @@ object CommandUtils extends Logging { */ private def buildLocalCommand( command: Command, + securityMgr: SecurityManager, substituteArguments: String => String, classPath: Seq[String] = Seq[String](), env: Map[String, String]): Command = { @@ -76,20 +80,26 @@ object CommandUtils extends Logging { val libraryPathEntries = command.libraryPathEntries val cmdLibraryPath = command.environment.get(libraryPathName) - val newEnvironment = if (libraryPathEntries.nonEmpty && libraryPathName.nonEmpty) { + var newEnvironment = if (libraryPathEntries.nonEmpty && libraryPathName.nonEmpty) { val libraryPaths = libraryPathEntries ++ cmdLibraryPath ++ env.get(libraryPathName) command.environment + ((libraryPathName, libraryPaths.mkString(File.pathSeparator))) } else { command.environment } + // set auth secret to env variable if needed + if (securityMgr.isAuthenticationEnabled) { + newEnvironment += (SecurityManager.ENV_AUTH_SECRET -> securityMgr.getSecretKey) + } + Command( command.mainClass, command.arguments.map(substituteArguments), newEnvironment, command.classPathEntries ++ classPath, Seq[String](), // library path already captured in environment variable - command.javaOpts) + // filter out auth secret from java options + command.javaOpts.filterNot(_.startsWith("-D" + SecurityManager.SPARK_AUTH_SECRET_CONF))) } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index ef7a703bffe67..1386055eb8c48 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -85,8 +85,8 @@ private[deploy] class DriverRunner( } // TODO: If we add ability to submit multiple jars they should also be added here - val builder = CommandUtils.buildProcessBuilder(driverDesc.command, driverDesc.mem, - sparkHome.getAbsolutePath, substituteVariables) + val builder = CommandUtils.buildProcessBuilder(driverDesc.command, securityManager, + driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables) launchDriver(builder, driverDir, driverDesc.supervise) } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 7aa85b732fc87..fff17e1095042 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -25,7 +25,7 @@ import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.util.Utils @@ -125,8 +125,8 @@ private[deploy] class ExecutorRunner( private def fetchAndRunExecutor() { try { // Launch the process - val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, - sparkHome.getAbsolutePath, substituteVariables) + val builder = CommandUtils.buildProcessBuilder(appDesc.command, new SecurityManager(conf), + memory, sparkHome.getAbsolutePath, substituteVariables) val command = builder.command() logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index dc2bee6f2bdca..5a1d06eb87db9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.worker.ui +import java.io.File +import java.net.URI import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -135,6 +137,13 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with return ("Error: Log type must be one of " + supportedLogTypes.mkString(", "), 0, 0, 0) } + // Verify that the normalized path of the log directory is in the working directory + val normalizedUri = new URI(logDirectory).normalize() + val normalizedLogDir = new File(normalizedUri.getPath) + if (!Utils.isInDirectory(workDir, normalizedLogDir)) { + return ("Error: invalid log directory " + logDirectory, 0, 0, 0) + } + try { val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") @@ -150,7 +159,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with offset } } - val endIndex = math.min(startIndex + totalLength, totalLength) + val endIndex = math.min(startIndex + byteLength, totalLength) logDebug(s"Getting log from $startIndex to $endIndex") val logText = Utils.offsetBytes(files, startIndex, endIndex) logDebug(s"Got log of length ${logText.length} bytes") diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 38b61d7242fce..a3b4561b07e7f 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -94,8 +94,8 @@ class TaskMetrics extends Serializable { */ private var _diskBytesSpilled: Long = _ def diskBytesSpilled: Long = _diskBytesSpilled - def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value - def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value + private[spark] def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value + private[spark] def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 8edf493780687..d7495551ad233 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -23,10 +23,10 @@ import java.util.Properties import scala.collection.mutable import scala.util.matching.Regex -import org.apache.spark.Logging import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf} -private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { +private[spark] class MetricsConfig(conf: SparkConf) extends Logging { private val DEFAULT_PREFIX = "*" private val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r @@ -46,23 +46,14 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi // Add default properties in case there's no properties file setDefaultProperties(properties) - // If spark.metrics.conf is not set, try to get file in class path - val isOpt: Option[InputStream] = configFile.map(new FileInputStream(_)).orElse { - try { - Option(Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_METRICS_CONF_FILENAME)) - } catch { - case e: Exception => - logError("Error loading default configuration file", e) - None - } - } + loadPropertiesFromFile(conf.getOption("spark.metrics.conf")) - isOpt.foreach { is => - try { - properties.load(is) - } finally { - is.close() - } + // Also look for the properties in provided Spark configuration + val prefix = "spark.metrics.conf." + conf.getAll.foreach { + case (k, v) if k.startsWith(prefix) => + properties.setProperty(k.substring(prefix.length()), v) + case _ => } propertyCategories = subProperties(properties, INSTANCE_REGEX) @@ -97,5 +88,31 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) } } -} + /** + * Loads configuration from a config file. If no config file is provided, try to get file + * in class path. + */ + private[this] def loadPropertiesFromFile(path: Option[String]): Unit = { + var is: InputStream = null + try { + is = path match { + case Some(f) => new FileInputStream(f) + case None => Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_METRICS_CONF_FILENAME) + } + + if (is != null) { + properties.load(is) + } + } catch { + case e: Exception => + val file = path.getOrElse(DEFAULT_METRICS_CONF_FILENAME) + logError(s"Error loading configuration file $file", e) + } finally { + if (is != null) { + is.close() + } + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 9150ad35712a1..ed5131c79fdc5 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -70,8 +70,7 @@ private[spark] class MetricsSystem private ( securityMgr: SecurityManager) extends Logging { - private[this] val confFile = conf.get("spark.metrics.conf", null) - private[this] val metricsConfig = new MetricsConfig(Option(confFile)) + private[this] val metricsConfig = new MetricsConfig(conf) private val sinks = new mutable.ArrayBuffer[Sink] private val sources = new mutable.ArrayBuffer[Source] diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2ab41ba488ff6..8ae76c5f72f2e 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -43,5 +43,5 @@ package org.apache package object spark { // For package docs only - val SPARK_VERSION = "1.4.0-SNAPSHOT" + val SPARK_VERSION = "1.5.0-SNAPSHOT" } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index a4715e3437d94..33e6998b2cb10 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -21,13 +21,12 @@ import java.io.IOException import scala.reflect.ClassTag -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableConfiguration, Utils} private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -38,7 +37,7 @@ private[spark] class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) extends RDD[T](sc, Nil) { - val broadcastedConf = sc.broadcast(new SerializableWritable(sc.hadoopConfiguration)) + val broadcastedConf = sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration)) @transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration) @@ -87,7 +86,7 @@ private[spark] object CheckpointRDD extends Logging { def writeToFile[T: ClassTag]( path: String, - broadcastedConf: Broadcast[SerializableWritable[Configuration]], + broadcastedConf: Broadcast[SerializableConfiguration], blockSize: Int = -1 )(ctx: TaskContext, iterator: Iterator[T]) { val env = SparkEnv.get @@ -135,7 +134,7 @@ private[spark] object CheckpointRDD extends Logging { def readFromFile[T]( path: Path, - broadcastedConf: Broadcast[SerializableWritable[Configuration]], + broadcastedConf: Broadcast[SerializableConfiguration], context: TaskContext ): Iterator[T] = { val env = SparkEnv.get @@ -164,7 +163,7 @@ private[spark] object CheckpointRDD extends Logging { val path = new Path(hdfsPath, "temp") val conf = SparkHadoopUtil.get.newConfiguration(new SparkConf()) val fs = path.getFileSystem(conf) - val broadcastedConf = sc.broadcast(new SerializableWritable(conf)) + val broadcastedConf = sc.broadcast(new SerializableConfiguration(conf)) sc.runJob(rdd, CheckpointRDD.writeToFile[Int](path.toString, broadcastedConf, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 2cefe63d44b20..bee59a437f120 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -44,7 +44,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD -import org.apache.spark.util.{NextIterator, Utils} +import org.apache.spark.util.{SerializableConfiguration, NextIterator, Utils} import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} import org.apache.spark.storage.StorageLevel @@ -100,7 +100,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp @DeveloperApi class HadoopRDD[K, V]( @transient sc: SparkContext, - broadcastedConf: Broadcast[SerializableWritable[Configuration]], + broadcastedConf: Broadcast[SerializableConfiguration], initLocalJobConfFuncOpt: Option[JobConf => Unit], inputFormatClass: Class[_ <: InputFormat[K, V]], keyClass: Class[K], @@ -121,8 +121,8 @@ class HadoopRDD[K, V]( minPartitions: Int) = { this( sc, - sc.broadcast(new SerializableWritable(conf)) - .asInstanceOf[Broadcast[SerializableWritable[Configuration]]], + sc.broadcast(new SerializableConfiguration(conf)) + .asInstanceOf[Broadcast[SerializableConfiguration]], None /* initLocalJobConfFuncOpt */, inputFormatClass, keyClass, diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 84456d6d868dc..f827270ee6a44 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -33,7 +33,7 @@ import org.apache.spark._ import org.apache.spark.executor.DataReadMethod import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableConfiguration, Utils} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.storage.StorageLevel @@ -74,7 +74,7 @@ class NewHadoopRDD[K, V]( with Logging { // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it - private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) + private val confBroadcast = sc.broadcast(new SerializableConfiguration(conf)) // private val serializableConf = new SerializableWritable(conf) private val jobTrackerId: String = { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index cfd3e26faf2b9..91a6a2d039852 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -44,7 +44,7 @@ import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils +import org.apache.spark.util.{SerializableConfiguration, Utils} import org.apache.spark.util.collection.CompactBuffer import org.apache.spark.util.random.StratifiedSamplingUtils @@ -1002,7 +1002,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id - val wrappedConf = new SerializableWritable(job.getConfiguration) + val wrappedConf = new SerializableConfiguration(job.getConfiguration) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance @@ -1065,7 +1065,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf - val wrappedConf = new SerializableWritable(hadoopConf) + val wrappedConf = new SerializableConfiguration(hadoopConf) val outputFormatInstance = hadoopConf.getOutputFormat val keyClass = hadoopConf.getOutputKeyClass val valueClass = hadoopConf.getOutputValueClass diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 1722c27e55003..acbd31aacdf59 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} +import org.apache.spark.util.SerializableConfiguration /** * Enumeration to manage state transitions of an RDD through checkpointing @@ -91,7 +92,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( - new SerializableWritable(rdd.context.hadoopConfiguration)) + new SerializableConfiguration(rdd.context.hadoopConfiguration)) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { rdd.context.cleaner.foreach { cleaner => diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala index 6b09dfafc889c..44667281c1063 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -95,10 +95,9 @@ private[spark] object RDDOperationScope extends Logging { private[spark] def withScope[T]( sc: SparkContext, allowNesting: Boolean = false)(body: => T): T = { - val stackTrace = Thread.currentThread.getStackTrace().tail // ignore "Thread#getStackTrace" - val ourMethodName = stackTrace(1).getMethodName // i.e. withScope - // Climb upwards to find the first method that's called something different - val callerMethodName = stackTrace + val ourMethodName = "withScope" + val callerMethodName = Thread.currentThread.getStackTrace() + .dropWhile(_.getMethodName != ourMethodName) .find(_.getMethodName != ourMethodName) .map(_.getMethodName) .getOrElse { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 75a567fb31520..aea6674ed20be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -137,6 +137,22 @@ class DAGScheduler( private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) + // Flag to control if reduce tasks are assigned preferred locations + private val shuffleLocalityEnabled = + sc.getConf.getBoolean("spark.shuffle.reduceLocality.enabled", true) + // Number of map, reduce tasks above which we do not assign preferred locations + // based on map output sizes. We limit the size of jobs for which assign preferred locations + // as computing the top locations by size becomes expensive. + private[this] val SHUFFLE_PREF_MAP_THRESHOLD = 1000 + // NOTE: This should be less than 2000 as we use HighlyCompressedMapStatus beyond that + private[this] val SHUFFLE_PREF_REDUCE_THRESHOLD = 1000 + + // Fraction of total map output that must be at a location for it to considered as a preferred + // location for a reduce task. + // Making this larger will focus on fewer locations where most data can be read locally, but + // may lead to more delay in scheduling if those locations are busy. + private[scheduler] val REDUCER_PREF_LOCS_FRACTION = 0.2 + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessLoop.post(BeginEvent(task, taskInfo)) @@ -1384,17 +1400,32 @@ class DAGScheduler( if (rddPrefs.nonEmpty) { return rddPrefs.map(TaskLocation(_)) } - // If the RDD has narrow dependencies, pick the first partition of the first narrow dep - // that has any placement preferences. Ideally we would choose based on transfer sizes, - // but this will do for now. + rdd.dependencies.foreach { case n: NarrowDependency[_] => + // If the RDD has narrow dependencies, pick the first partition of the first narrow dep + // that has any placement preferences. Ideally we would choose based on transfer sizes, + // but this will do for now. for (inPart <- n.getParents(partition)) { val locs = getPreferredLocsInternal(n.rdd, inPart, visited) if (locs != Nil) { return locs } } + case s: ShuffleDependency[_, _, _] => + // For shuffle dependencies, pick locations which have at least REDUCER_PREF_LOCS_FRACTION + // of data as preferred locations + if (shuffleLocalityEnabled && + rdd.partitions.size < SHUFFLE_PREF_REDUCE_THRESHOLD && + s.rdd.partitions.size < SHUFFLE_PREF_MAP_THRESHOLD) { + // Get the preferred map output locations for this reducer + val topLocsForReducer = mapOutputTracker.getLocationsWithLargestOutputs(s.shuffleId, + partition, rdd.partitions.size, REDUCER_PREF_LOCS_FRACTION) + if (topLocsForReducer.nonEmpty) { + return topLocsForReducer.get.map(loc => TaskLocation(loc.host, loc.executorId)) + } + } + case _ => } Nil diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 673cd0e19eba2..82455b0426a5d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -46,7 +46,7 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * * @param sched the TaskSchedulerImpl associated with the TaskSetManager * @param taskSet the TaskSet to manage scheduling for - * @param maxTaskFailures if any particular task fails more than this number of times, the entire + * @param maxTaskFailures if any particular task fails this number of times, the entire * task set will be aborted */ private[spark] class TaskSetManager( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index fcad959540f5a..7c7f70d8a193b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -103,7 +103,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case None => // Ignoring the update since we don't know about the executor. logWarning(s"Ignored task status update ($taskId state $state) " + - "from unknown executor $sender with ID $executorId") + s"from unknown executor with ID $executorId") } } diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index bb5db545531d2..cc2f0506817d3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -17,7 +17,7 @@ package org.apache.spark.serializer -import java.io.{NotSerializableException, ObjectOutput, ObjectStreamClass, ObjectStreamField} +import java.io._ import java.lang.reflect.{Field, Method} import java.security.AccessController @@ -62,7 +62,7 @@ private[spark] object SerializationDebugger extends Logging { * * It does not yet handle writeObject override, but that shouldn't be too hard to do either. */ - def find(obj: Any): List[String] = { + private[serializer] def find(obj: Any): List[String] = { new SerializationDebugger().visit(obj, List.empty) } @@ -125,6 +125,12 @@ private[spark] object SerializationDebugger extends Logging { return List.empty } + /** + * Visit an externalizable object. + * Since writeExternal() can choose to add arbitrary objects at the time of serialization, + * the only way to capture all the objects it will serialize is by using a + * dummy ObjectOutput that collects all the relevant objects for further testing. + */ private def visitExternalizable(o: java.io.Externalizable, stack: List[String]): List[String] = { val fieldList = new ListObjectOutput @@ -145,17 +151,50 @@ private[spark] object SerializationDebugger extends Logging { // An object contains multiple slots in serialization. // Get the slots and visit fields in all of them. val (finalObj, desc) = findObjectAndDescriptor(o) + + // If the object has been replaced using writeReplace(), + // then call visit() on it again to test its type again. + if (!finalObj.eq(o)) { + return visit(finalObj, s"writeReplace data (class: ${finalObj.getClass.getName})" :: stack) + } + + // Every class is associated with one or more "slots", each slot refers to the parent + // classes of this class. These slots are used by the ObjectOutputStream + // serialization code to recursively serialize the fields of an object and + // its parent classes. For example, if there are the following classes. + // + // class ParentClass(parentField: Int) + // class ChildClass(childField: Int) extends ParentClass(1) + // + // Then serializing the an object Obj of type ChildClass requires first serializing the fields + // of ParentClass (that is, parentField), and then serializing the fields of ChildClass + // (that is, childField). Correspondingly, there will be two slots related to this object: + // + // 1. ParentClass slot, which will be used to serialize parentField of Obj + // 2. ChildClass slot, which will be used to serialize childField fields of Obj + // + // The following code uses the description of each slot to find the fields in the + // corresponding object to visit. + // val slotDescs = desc.getSlotDescs var i = 0 while (i < slotDescs.length) { val slotDesc = slotDescs(i) if (slotDesc.hasWriteObjectMethod) { - // TODO: Handle classes that specify writeObject method. + // If the class type corresponding to current slot has writeObject() defined, + // then its not obvious which fields of the class will be serialized as the writeObject() + // can choose arbitrary fields for serialization. This case is handled separately. + val elem = s"writeObject data (class: ${slotDesc.getName})" + val childStack = visitSerializableWithWriteObjectMethod(finalObj, elem :: stack) + if (childStack.nonEmpty) { + return childStack + } } else { + // Visit all the fields objects of the class corresponding to the current slot. val fields: Array[ObjectStreamField] = slotDesc.getFields val objFieldValues: Array[Object] = new Array[Object](slotDesc.getNumObjFields) val numPrims = fields.length - objFieldValues.length - desc.getObjFieldValues(finalObj, objFieldValues) + slotDesc.getObjFieldValues(finalObj, objFieldValues) var j = 0 while (j < objFieldValues.length) { @@ -169,18 +208,54 @@ private[spark] object SerializationDebugger extends Logging { } j += 1 } - } i += 1 } return List.empty } + + /** + * Visit a serializable object which has the writeObject() defined. + * Since writeObject() can choose to add arbitrary objects at the time of serialization, + * the only way to capture all the objects it will serialize is by using a + * dummy ObjectOutputStream that collects all the relevant fields for further testing. + * This is similar to how externalizable objects are visited. + */ + private def visitSerializableWithWriteObjectMethod( + o: Object, stack: List[String]): List[String] = { + val innerObjectsCatcher = new ListObjectOutputStream + var notSerializableFound = false + try { + innerObjectsCatcher.writeObject(o) + } catch { + case io: IOException => + notSerializableFound = true + } + + // If something was not serializable, then visit the captured objects. + // Otherwise, all the captured objects are safely serializable, so no need to visit them. + // As an optimization, just added them to the visited list. + if (notSerializableFound) { + val innerObjects = innerObjectsCatcher.outputArray + var k = 0 + while (k < innerObjects.length) { + val childStack = visit(innerObjects(k), stack) + if (childStack.nonEmpty) { + return childStack + } + k += 1 + } + } else { + visited ++= innerObjectsCatcher.outputArray + } + return List.empty + } } /** * Find the object to serialize and the associated [[ObjectStreamClass]]. This method handles * writeReplace in Serializable. It starts with the object itself, and keeps calling the - * writeReplace method until there is no more + * writeReplace method until there is no more. */ @tailrec private def findObjectAndDescriptor(o: Object): (Object, ObjectStreamClass) = { @@ -220,6 +295,31 @@ private[spark] object SerializationDebugger extends Logging { override def writeByte(i: Int): Unit = {} } + /** An output stream that emulates /dev/null */ + private class NullOutputStream extends OutputStream { + override def write(b: Int) { } + } + + /** + * A dummy [[ObjectOutputStream]] that saves the list of objects written to it and returns + * them through `outputArray`. This works by using the [[ObjectOutputStream]]'s `replaceObject()` + * method which gets called on every object, only if replacing is enabled. So this subclass + * of [[ObjectOutputStream]] enabled replacing, and uses replaceObject to get the objects that + * are being serializabled. The serialized bytes are ignored by sending them to a + * [[NullOutputStream]], which acts like a /dev/null. + */ + private class ListObjectOutputStream extends ObjectOutputStream(new NullOutputStream) { + private val output = new mutable.ArrayBuffer[Any] + this.enableReplaceObject(true) + + def outputArray: Array[Any] = output.toArray + + override def replaceObject(obj: Object): Object = { + output += obj + obj + } + } + /** An implicit class that allows us to call private methods of ObjectStreamClass. */ implicit class ObjectStreamClassMethods(val desc: ObjectStreamClass) extends AnyVal { def getSlotDescs: Array[ObjectStreamClass] = { diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index f1bdff96d3df1..bd2704dc81871 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -182,6 +182,7 @@ abstract class DeserializationStream { } catch { case eof: EOFException => finished = true + null } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index f2bfef376d3ca..df7bbd64247dd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -56,9 +56,6 @@ private[spark] object UnsafeShuffleManager extends Logging { } else if (dependency.aggregator.isDefined) { log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because an aggregator is defined") false - } else if (dependency.keyOrdering.isDefined) { - log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because a key ordering is defined") - false } else if (dependency.partitioner.numPartitions > MAX_SHUFFLE_OUTPUT_PARTITIONS) { log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because it has more than " + s"$MAX_SHUFFLE_OUTPUT_PARTITIONS partitions") diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index f73c742732dec..50b6ba67e9931 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.status.api.v1 +import java.util.zip.ZipOutputStream import javax.servlet.ServletContext import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} @@ -164,6 +165,18 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { } } + @Path("applications/{appId}/logs") + def getEventLogs( + @PathParam("appId") appId: String): EventLogDownloadResource = { + new EventLogDownloadResource(uiRoot, appId, None) + } + + @Path("applications/{appId}/{attemptId}/logs") + def getEventLogs( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { + new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) + } } private[spark] object ApiRootResource { @@ -193,6 +206,17 @@ private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] + /** + * Write the event logs for the given app to the [[ZipOutputStream]] instance. If attemptId is + * [[None]], event logs for all attempts of this application will be written out. + */ + def writeEventLogs(appId: String, attemptId: Option[String], zipStream: ZipOutputStream): Unit = { + Response.serverError() + .entity("Event logs are only available through the history server.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() + } + /** * Get the spark UI with the given appID, and apply a function * to it. If there is no such app, throw an appropriate exception diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala new file mode 100644 index 0000000000000..22e21f0c62a29 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import java.io.OutputStream +import java.util.zip.ZipOutputStream +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.{MediaType, Response, StreamingOutput} + +import scala.util.control.NonFatal + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil + +@Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) +private[v1] class EventLogDownloadResource( + val uIRoot: UIRoot, + val appId: String, + val attemptId: Option[String]) extends Logging { + val conf = SparkHadoopUtil.get.newConfiguration(new SparkConf) + + @GET + def getEventLogs(): Response = { + try { + val fileName = { + attemptId match { + case Some(id) => s"eventLogs-$appId-$id.zip" + case None => s"eventLogs-$appId.zip" + } + } + + val stream = new StreamingOutput { + override def write(output: OutputStream): Unit = { + val zipStream = new ZipOutputStream(output) + try { + uIRoot.writeEventLogs(appId, attemptId, zipStream) + } finally { + zipStream.close() + } + + } + } + + Response.ok(stream) + .header("Content-Disposition", s"attachment; filename=$fileName") + .header("Content-Type", MediaType.APPLICATION_OCTET_STREAM) + .build() + } catch { + case NonFatal(e) => + Response.serverError() + .entity(s"Event logs are not available for app: $appId.") + .status(Response.Status.SERVICE_UNAVAILABLE) + .build() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5048c7dab240b..1beafa1771448 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -83,8 +83,10 @@ private[spark] class BlockManager( private var externalBlockStoreInitialized = false private[spark] val memoryStore = new MemoryStore(this, maxMemory) private[spark] val diskStore = new DiskStore(this, diskBlockManager) - private[spark] lazy val externalBlockStore: ExternalBlockStore = + private[spark] lazy val externalBlockStore: ExternalBlockStore = { + externalBlockStoreInitialized = true new ExternalBlockStore(this, executorId) + } private[spark] val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 89c1f86647922..f70f701494dbf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -202,6 +202,14 @@ class BlockManagerMaster( timeout.awaitResult(future) } + /** + * Find out if the executor has cached blocks. This method does not consider broadcast blocks, + * since they are not reported the master. + */ + def hasCachedBlocks(executorId: String): Boolean = { + driverEndpoint.askWithRetry[Boolean](HasCachedBlocks(executorId)) + } + /** Stop the driver endpoint, called only on the Spark driver node */ def stop() { if (driverEndpoint != null && isDriver) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 2cd8c5297b741..68ed9096731c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.util.{HashMap => JHashMap} +import scala.collection.immutable.HashSet import scala.collection.mutable import scala.collection.JavaConversions._ import scala.concurrent.{ExecutionContext, Future} @@ -112,6 +113,17 @@ class BlockManagerMasterEndpoint( case BlockManagerHeartbeat(blockManagerId) => context.reply(heartbeatReceived(blockManagerId)) + case HasCachedBlocks(executorId) => + blockManagerIdByExecutor.get(executorId) match { + case Some(bm) => + if (blockManagerInfo.contains(bm)) { + val bmInfo = blockManagerInfo(bm) + context.reply(bmInfo.cachedBlocks.nonEmpty) + } else { + context.reply(false) + } + case None => context.reply(false) + } } private def removeRdd(rddId: Int): Future[Seq[Int]] = { @@ -418,6 +430,9 @@ private[spark] class BlockManagerInfo( // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] + // Cached blocks held by this BlockManager. This does not include broadcast blocks. + private val _cachedBlocks = new mutable.HashSet[BlockId] + def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId)) def updateLastSeenMs() { @@ -451,27 +466,35 @@ private[spark] class BlockManagerInfo( * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ + var blockStatus: BlockStatus = null if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0, 0)) + blockStatus = BlockStatus(storageLevel, memSize, 0, 0) + _blocks.put(blockId, blockStatus) _remainingMem -= memSize logInfo("Added %s in memory on %s (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), Utils.bytesToString(_remainingMem))) } if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize, 0)) + blockStatus = BlockStatus(storageLevel, 0, diskSize, 0) + _blocks.put(blockId, blockStatus) logInfo("Added %s on disk on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } if (storageLevel.useOffHeap) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, externalBlockStoreSize)) + blockStatus = BlockStatus(storageLevel, 0, 0, externalBlockStoreSize) + _blocks.put(blockId, blockStatus) logInfo("Added %s on ExternalBlockStore on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(externalBlockStoreSize))) } + if (!blockId.isBroadcast && blockStatus.isCached) { + _cachedBlocks += blockId + } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. val blockStatus: BlockStatus = _blocks.get(blockId) _blocks.remove(blockId) + _cachedBlocks -= blockId if (blockStatus.storageLevel.useMemory) { logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), @@ -494,6 +517,7 @@ private[spark] class BlockManagerInfo( _remainingMem += _blocks.get(blockId).memSize _blocks.remove(blockId) } + _cachedBlocks -= blockId } def remainingMem: Long = _remainingMem @@ -502,6 +526,9 @@ private[spark] class BlockManagerInfo( def blocks: JHashMap[BlockId, BlockStatus] = _blocks + // This does not include broadcast blocks. + def cachedBlocks: collection.Set[BlockId] = _cachedBlocks + override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem def clear() { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 1683576067fe8..376e9eb48843d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -42,7 +42,6 @@ private[spark] object BlockManagerMessages { case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) extends ToBlockManagerSlave - ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. ////////////////////////////////////////////////////////////////////////////////// @@ -108,4 +107,6 @@ private[spark] object BlockManagerMessages { extends ToBlockManagerMaster case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + + case class HasCachedBlocks(executorId: String) extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 65162f4fdcd62..7898039519201 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -362,7 +362,7 @@ private[spark] object UIUtils extends Logging { { g.incomingEdges.map { e =>/applications/[app-id]/storage/rdd/[rdd-id]
/applications/[app-id]/logs
/applications/[app-id]/[attempt-id]/logs
spark.sql.parquet.cacheMetadata
spark.sql.parquet.output.committer.class
org.apache.parquet.hadoop.
ParquetOutputCommitter
+ The output committer class used by Parquet. The specified class needs to be a subclass of
+ org.apache.hadoop.
. Typically, it's also a
+ subclass of
mapreduce.OutputCommitterorg.apache.parquet.hadoop.ParquetOutputCommitter
.
+
+ Note: +
-
+
-
+ This option must be set via Hadoop
Configuration
rather than Spark +SQLConf
. +
+ -
+ This option overrides
spark.sql.sources.
. +
outputCommitterClass
+
+ Spark SQL comes with a builtin
+ org.apache.spark.sql.
, which can be more
+ efficient then the default Parquet output committer when writing data to S3.
+
parquet.DirectParquetOutputCommitter
spark.sql.planner.externalSort
Note: The default value of valueContainsNull is True. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 6a2048121f8bf..a75587a92adc7 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the one's for which it has in-built support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). +the ones for which it has built-in support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. Note that custom receivers can be implemented @@ -21,15 +21,15 @@ A custom receiver must extend this abstract class by implementing two methods - `onStop()`: Things to do to stop receiving data. Both `onStart()` and `onStop()` must not block indefinitely. Typically, `onStart()` would start the threads -that responsible for receiving the data and `onStop()` would ensure that the receiving by those threads +that are responsible for receiving the data, and `onStop()` would ensure that these threads receiving the data are stopped. The receiving threads can also use `isStopped()`, a `Receiver` method, to check whether they should stop receiving data. Once the data is received, that data can be stored inside Spark by calling `store(data)`, which is a method provided by the Receiver class. -There are number of flavours of `store()` which allow you store the received data -record-at-a-time or as whole collection of objects / serialized bytes. Note that the flavour of -`store()` used to implemented a receiver affects its reliability and fault-tolerance semantics. +There are a number of flavors of `store()` which allow one to store the received data +record-at-a-time or as whole collection of objects / serialized bytes. Note that the flavor of +`store()` used to implement a receiver affects its reliability and fault-tolerance semantics. This is discussed [later](#receiver-reliability) in more detail. Any exception in the receiving threads should be caught and handled properly to avoid silent @@ -60,7 +60,7 @@ class CustomReceiver(host: String, port: Int) def onStop() { // There is nothing much to do as the thread calling receive() - // is designed to stop by itself isStopped() returns false + // is designed to stop by itself if isStopped() returns false } /** Create a socket connection and receive data until receiver is stopped */ @@ -123,7 +123,7 @@ public class JavaCustomReceiver extends Receiver
@@ -1009,11 +1008,11 @@ window. As shown in the figure, every time the window *slides* over a source DStream, the source RDDs that fall within the window are combined and operated upon to produce the -RDDs of the windowed DStream. In this specific case, the operation is applied over last 3 time +RDDs of the windowed DStream. In this specific case, the operation is applied over the last 3 time units of data, and slides by 2 time units. This shows that any window operation needs to specify two parameters. - * window length - The duration of the window (3 in the figure) + * window length - The duration of the window (3 in the figure). * sliding interval - The interval at which the window operation is performed (2 in the figure). @@ -1021,7 +1020,7 @@ These two parameters must be multiples of the batch interval of the source DStre figure). Let's illustrate the window operations with an example. Say, you want to extend the -[earlier example](#a-quick-example) by generating word counts over last 30 seconds of data, +[earlier example](#a-quick-example) by generating word counts over the last 30 seconds of data, every 10 seconds. To do this, we have to apply the `reduceByKey` operation on the `pairs` DStream of `(word, 1)` pairs over the last 30 seconds of data. This is done using the operation `reduceByKeyAndWindow`. @@ -1096,13 +1095,13 @@ said two parameters - windowLength and slideInterval.
reduceByKeyAndWindow()
where the reduce
+ reduceByKeyAndWindow()
where the reduce
value of each window is calculated incrementally using the reduce values of the previous window.
- This is done by reducing the new data that enter the sliding window, and "inverse reducing" the
- old data that leave the window. An example would be that of "adding" and "subtracting" counts
- of keys as the window slides. However, it is applicable to only "invertible reduce functions",
+ This is done by reducing the new data that enters the sliding window, and "inverse reducing" the
+ old data that leaves the window. An example would be that of "adding" and "subtracting" counts
+ of keys as the window slides. However, it is applicable only to "invertible reduce functions",
that is, those reduce functions which have a corresponding "inverse reduce" function (taken as
- parameter invFunc. Like in reduceByKeyAndWindow
, the number of reduce tasks
+ parameter invFunc). Like in reduceByKeyAndWindow
, the number of reduce tasks
is configurable through an optional argument. Note that [checkpointing](#checkpointing) must be
enabled for using this operation.
Python API This is called @@ -1242,12 +1241,12 @@ Currently, the following output operations are defined:
SequenceFile
of serialized Java objects. The file
+ SequenceFiles
of serialized Java objects. The file
name at each batch interval is generated based on prefix and
suffix: "prefix-TIME_IN_MS[.suffix]".
@@ -1257,7 +1256,7 @@ Currently, the following output operations are defined:
Python API This is not available in @@ -1267,7 +1266,7 @@ Currently, the following output operations are defined:
ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan
has been run.")
+
+ val DEFAULT_SIZE_IN_BYTES = longConf("spark.sql.defaultSizeInBytes", isPublic = false)
+
+ val SHUFFLE_PARTITIONS = intConf("spark.sql.shuffle.partitions",
+ defaultValue = Some(200),
+ doc = "Configures the number of partitions to use when shuffling data for joins or " +
+ "aggregations.")
+
+ val CODEGEN_ENABLED = booleanConf("spark.sql.codegen",
+ defaultValue = Some(true),
+ doc = "When true, code will be dynamically generated at runtime for expression evaluation in" +
+ " a specific query. For some queries with complicated expression this option can lead to " +
+ "significant speed-ups. However, for simple queries this can actually slow down query " +
+ "execution.")
+
+ val UNSAFE_ENABLED = booleanConf("spark.sql.unsafe.enabled",
+ defaultValue = Some(false),
+ doc = "-Phive
is enabled. When this option is chosen, " +
+ "spark.sql.hive.metastore.version must be either 0.13.1
or not defined. " +
+ "2. \"maven\" Use Hive jars of specified version downloaded from Maven repositories." +
+ "3. A classpath in the standard format for both Hive and Hadoop.")
+
+ val CONVERT_METASTORE_PARQUET = booleanConf("spark.sql.hive.convertMetastoreParquet",
+ defaultValue = Some(true),
+ doc = "When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of " +
+ "the built in support.")
+
+ val CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING = booleanConf(
+ "spark.sql.hive.convertMetastoreParquet.mergeSchema",
+ defaultValue = Some(false),
+ doc = "TODO")
+
+ val CONVERT_CTAS = booleanConf("spark.sql.hive.convertCTAS",
+ defaultValue = Some(false),
+ doc = "TODO")
+
+ val HIVE_METASTORE_SHARED_PREFIXES = stringSeqConf("spark.sql.hive.metastore.sharedPrefixes",
+ defaultValue = Some(jdbcPrefixes),
+ doc = "A comma separated list of class prefixes that should be loaded using the classloader " +
+ "that is shared between Spark SQL and a specific version of Hive. An example of classes " +
+ "that should be shared is JDBC drivers that are needed to talk to the metastore. Other " +
+ "classes that need to be shared are those that interact with classes that are already " +
+ "shared. For example, custom appenders that are used by log4j.")
+
+ private def jdbcPrefixes = Seq(
+ "com.mysql.jdbc", "org.postgresql", "com.microsoft.sqlserver", "oracle.jdbc")
+
+ val HIVE_METASTORE_BARRIER_PREFIXES = stringSeqConf("spark.sql.hive.metastore.barrierPrefixes",
+ defaultValue = Some(Seq()),
+ doc = "A comma separated list of class prefixes that should explicitly be reloaded for each " +
+ "version of Hive that Spark SQL is communicating with. For example, Hive UDFs that are " +
+ "declared in a prefix that typically would be shared (i.e. org.apache.spark.*
).")
+
+ val HIVE_THRIFT_SERVER_ASYNC = booleanConf("spark.sql.hive.thriftServer.async",
+ defaultValue = Some(true),
+ doc = "TODO")
/** Constructs a configuration for hive, where the metastore is located in a temp directory. */
def newTemporaryConfiguration(): Map[String, String] = {
@@ -564,7 +603,7 @@ private[hive] object HiveContext {
case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8")
case (decimal: java.math.BigDecimal, DecimalType()) =>
// Hive strips trailing zeros so use its toString
- HiveShim.createDecimal(decimal).toString
+ HiveDecimal.create(decimal).toString
case (other, tpe) if primitiveTypes contains tpe => other.toString
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 24cd335082639..864c888ab073d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -20,13 +20,15 @@ package org.apache.spark.sql.hive
import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
import org.apache.hadoop.hive.serde2.objectinspector.{StructField => HiveStructField, _}
+import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfoFactory}
import org.apache.hadoop.hive.serde2.{io => hiveIo}
import org.apache.hadoop.{io => hadoopIo}
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateUtils
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types
import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -241,15 +243,16 @@ private[hive] trait HiveInspectors {
def unwrap(data: Any, oi: ObjectInspector): Any = oi match {
case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null
case poi: WritableConstantStringObjectInspector =>
- UTF8String(poi.getWritableConstantValue.toString)
+ UTF8String.fromString(poi.getWritableConstantValue.toString)
case poi: WritableConstantHiveVarcharObjectInspector =>
- UTF8String(poi.getWritableConstantValue.getHiveVarchar.getValue)
+ UTF8String.fromString(poi.getWritableConstantValue.getHiveVarchar.getValue)
case poi: WritableConstantHiveDecimalObjectInspector =>
HiveShim.toCatalystDecimal(
PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector,
poi.getWritableConstantValue.getHiveDecimal)
case poi: WritableConstantTimestampObjectInspector =>
- poi.getWritableConstantValue.getTimestamp.clone()
+ val t = poi.getWritableConstantValue
+ t.getSeconds * 10000000L + t.getNanos / 100L
case poi: WritableConstantIntObjectInspector =>
poi.getWritableConstantValue.get()
case poi: WritableConstantDoubleObjectInspector =>
@@ -270,7 +273,7 @@ private[hive] trait HiveInspectors {
System.arraycopy(writable.getBytes, 0, temp, 0, temp.length)
temp
case poi: WritableConstantDateObjectInspector =>
- DateUtils.fromJavaDate(poi.getWritableConstantValue.get())
+ DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get())
case mi: StandardConstantMapObjectInspector =>
// take the value from the map inspector object, rather than the input data
mi.getWritableConstantValue.map { case (k, v) =>
@@ -286,13 +289,13 @@ private[hive] trait HiveInspectors {
case pi: PrimitiveObjectInspector => pi match {
// We think HiveVarchar is also a String
case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() =>
- UTF8String(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue)
+ UTF8String.fromString(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue)
case hvoi: HiveVarcharObjectInspector =>
- UTF8String(hvoi.getPrimitiveJavaObject(data).getValue)
+ UTF8String.fromString(hvoi.getPrimitiveJavaObject(data).getValue)
case x: StringObjectInspector if x.preferWritable() =>
- UTF8String(x.getPrimitiveWritableObject(data).toString)
+ UTF8String.fromString(x.getPrimitiveWritableObject(data).toString)
case x: StringObjectInspector =>
- UTF8String(x.getPrimitiveJavaObject(data))
+ UTF8String.fromString(x.getPrimitiveJavaObject(data))
case x: IntObjectInspector if x.preferWritable() => x.get(data)
case x: BooleanObjectInspector if x.preferWritable() => x.get(data)
case x: FloatObjectInspector if x.preferWritable() => x.get(data)
@@ -310,13 +313,13 @@ private[hive] trait HiveInspectors {
System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength())
result
case x: DateObjectInspector if x.preferWritable() =>
- DateUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get())
- case x: DateObjectInspector => DateUtils.fromJavaDate(x.getPrimitiveJavaObject(data))
- // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object
- // if next timestamp is null, so Timestamp object is cloned
+ DateTimeUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get())
+ case x: DateObjectInspector => DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data))
case x: TimestampObjectInspector if x.preferWritable() =>
- x.getPrimitiveWritableObject(data).getTimestamp.clone()
- case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone()
+ val t = x.getPrimitiveWritableObject(data)
+ t.getSeconds * 10000000L + t.getNanos / 100
+ case ti: TimestampObjectInspector =>
+ DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data))
case _ => pi.getPrimitiveJavaObject(data)
}
case li: ListObjectInspector =>
@@ -350,17 +353,20 @@ private[hive] trait HiveInspectors {
new HiveVarchar(s, s.size)
case _: JavaHiveDecimalObjectInspector =>
- (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal)
+ (o: Any) => HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal)
case _: JavaDateObjectInspector =>
- (o: Any) => DateUtils.toJavaDate(o.asInstanceOf[Int])
+ (o: Any) => DateTimeUtils.toJavaDate(o.asInstanceOf[Int])
+
+ case _: JavaTimestampObjectInspector =>
+ (o: Any) => DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long])
case soi: StandardStructObjectInspector =>
val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector))
(o: Any) => {
if (o != null) {
val struct = soi.create()
- (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row].toSeq).zipped.foreach {
+ (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[InternalRow].toSeq).zipped.foreach {
(field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data))
}
struct
@@ -439,36 +445,36 @@ private[hive] trait HiveInspectors {
case _ if a == null => null
case x: PrimitiveObjectInspector => x match {
// TODO we don't support the HiveVarcharObjectInspector yet.
- case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a)
+ case _: StringObjectInspector if x.preferWritable() => getStringWritable(a)
case _: StringObjectInspector => a.asInstanceOf[UTF8String].toString()
- case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a)
+ case _: IntObjectInspector if x.preferWritable() => getIntWritable(a)
case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer]
- case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a)
+ case _: BooleanObjectInspector if x.preferWritable() => getBooleanWritable(a)
case _: BooleanObjectInspector => a.asInstanceOf[java.lang.Boolean]
- case _: FloatObjectInspector if x.preferWritable() => HiveShim.getFloatWritable(a)
+ case _: FloatObjectInspector if x.preferWritable() => getFloatWritable(a)
case _: FloatObjectInspector => a.asInstanceOf[java.lang.Float]
- case _: DoubleObjectInspector if x.preferWritable() => HiveShim.getDoubleWritable(a)
+ case _: DoubleObjectInspector if x.preferWritable() => getDoubleWritable(a)
case _: DoubleObjectInspector => a.asInstanceOf[java.lang.Double]
- case _: LongObjectInspector if x.preferWritable() => HiveShim.getLongWritable(a)
+ case _: LongObjectInspector if x.preferWritable() => getLongWritable(a)
case _: LongObjectInspector => a.asInstanceOf[java.lang.Long]
- case _: ShortObjectInspector if x.preferWritable() => HiveShim.getShortWritable(a)
+ case _: ShortObjectInspector if x.preferWritable() => getShortWritable(a)
case _: ShortObjectInspector => a.asInstanceOf[java.lang.Short]
- case _: ByteObjectInspector if x.preferWritable() => HiveShim.getByteWritable(a)
+ case _: ByteObjectInspector if x.preferWritable() => getByteWritable(a)
case _: ByteObjectInspector => a.asInstanceOf[java.lang.Byte]
case _: HiveDecimalObjectInspector if x.preferWritable() =>
- HiveShim.getDecimalWritable(a.asInstanceOf[Decimal])
+ getDecimalWritable(a.asInstanceOf[Decimal])
case _: HiveDecimalObjectInspector =>
- HiveShim.createDecimal(a.asInstanceOf[Decimal].toJavaBigDecimal)
- case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a)
+ HiveDecimal.create(a.asInstanceOf[Decimal].toJavaBigDecimal)
+ case _: BinaryObjectInspector if x.preferWritable() => getBinaryWritable(a)
case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]]
- case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a)
- case _: DateObjectInspector => DateUtils.toJavaDate(a.asInstanceOf[Int])
- case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a)
- case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp]
+ case _: DateObjectInspector if x.preferWritable() => getDateWritable(a)
+ case _: DateObjectInspector => DateTimeUtils.toJavaDate(a.asInstanceOf[Int])
+ case _: TimestampObjectInspector if x.preferWritable() => getTimestampWritable(a)
+ case _: TimestampObjectInspector => DateTimeUtils.toJavaTimestamp(a.asInstanceOf[Long])
}
case x: SettableStructObjectInspector =>
val fieldRefs = x.getAllStructFieldRefs
- val row = a.asInstanceOf[Row]
+ val row = a.asInstanceOf[InternalRow]
// 1. create the pojo (most likely) object
val result = x.create()
var i = 0
@@ -484,7 +490,7 @@ private[hive] trait HiveInspectors {
result
case x: StructObjectInspector =>
val fieldRefs = x.getAllStructFieldRefs
- val row = a.asInstanceOf[Row]
+ val row = a.asInstanceOf[InternalRow]
val result = new java.util.ArrayList[AnyRef](fieldRefs.length)
var i = 0
while (i < fieldRefs.length) {
@@ -511,7 +517,7 @@ private[hive] trait HiveInspectors {
}
def wrap(
- row: Row,
+ row: InternalRow,
inspectors: Seq[ObjectInspector],
cache: Array[AnyRef]): Array[AnyRef] = {
var i = 0
@@ -574,31 +580,31 @@ private[hive] trait HiveInspectors {
*/
def toInspector(expr: Expression): ObjectInspector = expr match {
case Literal(value, StringType) =>
- HiveShim.getStringWritableConstantObjectInspector(value)
+ getStringWritableConstantObjectInspector(value)
case Literal(value, IntegerType) =>
- HiveShim.getIntWritableConstantObjectInspector(value)
+ getIntWritableConstantObjectInspector(value)
case Literal(value, DoubleType) =>
- HiveShim.getDoubleWritableConstantObjectInspector(value)
+ getDoubleWritableConstantObjectInspector(value)
case Literal(value, BooleanType) =>
- HiveShim.getBooleanWritableConstantObjectInspector(value)
+ getBooleanWritableConstantObjectInspector(value)
case Literal(value, LongType) =>
- HiveShim.getLongWritableConstantObjectInspector(value)
+ getLongWritableConstantObjectInspector(value)
case Literal(value, FloatType) =>
- HiveShim.getFloatWritableConstantObjectInspector(value)
+ getFloatWritableConstantObjectInspector(value)
case Literal(value, ShortType) =>
- HiveShim.getShortWritableConstantObjectInspector(value)
+ getShortWritableConstantObjectInspector(value)
case Literal(value, ByteType) =>
- HiveShim.getByteWritableConstantObjectInspector(value)
+ getByteWritableConstantObjectInspector(value)
case Literal(value, BinaryType) =>
- HiveShim.getBinaryWritableConstantObjectInspector(value)
+ getBinaryWritableConstantObjectInspector(value)
case Literal(value, DateType) =>
- HiveShim.getDateWritableConstantObjectInspector(value)
+ getDateWritableConstantObjectInspector(value)
case Literal(value, TimestampType) =>
- HiveShim.getTimestampWritableConstantObjectInspector(value)
+ getTimestampWritableConstantObjectInspector(value)
case Literal(value, DecimalType()) =>
- HiveShim.getDecimalWritableConstantObjectInspector(value)
+ getDecimalWritableConstantObjectInspector(value)
case Literal(_, NullType) =>
- HiveShim.getPrimitiveNullWritableConstantObjectInspector
+ getPrimitiveNullWritableConstantObjectInspector
case Literal(value, ArrayType(dt, _)) =>
val listObjectInspector = toInspector(dt)
if (value == null) {
@@ -658,8 +664,8 @@ private[hive] trait HiveInspectors {
case _: JavaFloatObjectInspector => FloatType
case _: WritableBinaryObjectInspector => BinaryType
case _: JavaBinaryObjectInspector => BinaryType
- case w: WritableHiveDecimalObjectInspector => HiveShim.decimalTypeInfoToCatalyst(w)
- case j: JavaHiveDecimalObjectInspector => HiveShim.decimalTypeInfoToCatalyst(j)
+ case w: WritableHiveDecimalObjectInspector => decimalTypeInfoToCatalyst(w)
+ case j: JavaHiveDecimalObjectInspector => decimalTypeInfoToCatalyst(j)
case _: WritableDateObjectInspector => DateType
case _: JavaDateObjectInspector => DateType
case _: WritableTimestampObjectInspector => TimestampType
@@ -668,10 +674,136 @@ private[hive] trait HiveInspectors {
case _: JavaVoidObjectInspector => NullType
}
+ private def decimalTypeInfoToCatalyst(inspector: PrimitiveObjectInspector): DecimalType = {
+ val info = inspector.getTypeInfo.asInstanceOf[DecimalTypeInfo]
+ DecimalType(info.precision(), info.scale())
+ }
+
+ private def getStringWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.stringTypeInfo, getStringWritable(value))
+
+ private def getIntWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.intTypeInfo, getIntWritable(value))
+
+ private def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.doubleTypeInfo, getDoubleWritable(value))
+
+ private def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.booleanTypeInfo, getBooleanWritable(value))
+
+ private def getLongWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.longTypeInfo, getLongWritable(value))
+
+ private def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.floatTypeInfo, getFloatWritable(value))
+
+ private def getShortWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.shortTypeInfo, getShortWritable(value))
+
+ private def getByteWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.byteTypeInfo, getByteWritable(value))
+
+ private def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.binaryTypeInfo, getBinaryWritable(value))
+
+ private def getDateWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.dateTypeInfo, getDateWritable(value))
+
+ private def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value))
+
+ private def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.decimalTypeInfo, getDecimalWritable(value))
+
+ private def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector =
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.voidTypeInfo, null)
+
+ private def getStringWritable(value: Any): hadoopIo.Text =
+ if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].getBytes)
+
+ private def getIntWritable(value: Any): hadoopIo.IntWritable =
+ if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])
+
+ private def getDoubleWritable(value: Any): hiveIo.DoubleWritable =
+ if (value == null) {
+ null
+ } else {
+ new hiveIo.DoubleWritable(value.asInstanceOf[Double])
+ }
+
+ private def getBooleanWritable(value: Any): hadoopIo.BooleanWritable =
+ if (value == null) {
+ null
+ } else {
+ new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])
+ }
+
+ private def getLongWritable(value: Any): hadoopIo.LongWritable =
+ if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])
+
+ private def getFloatWritable(value: Any): hadoopIo.FloatWritable =
+ if (value == null) {
+ null
+ } else {
+ new hadoopIo.FloatWritable(value.asInstanceOf[Float])
+ }
+
+ private def getShortWritable(value: Any): hiveIo.ShortWritable =
+ if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])
+
+ private def getByteWritable(value: Any): hiveIo.ByteWritable =
+ if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])
+
+ private def getBinaryWritable(value: Any): hadoopIo.BytesWritable =
+ if (value == null) {
+ null
+ } else {
+ new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
+ }
+
+ private def getDateWritable(value: Any): hiveIo.DateWritable =
+ if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int])
+
+ private def getTimestampWritable(value: Any): hiveIo.TimestampWritable =
+ if (value == null) {
+ null
+ } else {
+ new hiveIo.TimestampWritable(DateTimeUtils.toJavaTimestamp(value.asInstanceOf[Long]))
+ }
+
+ private def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable =
+ if (value == null) {
+ null
+ } else {
+ // TODO precise, scale?
+ new hiveIo.HiveDecimalWritable(
+ HiveDecimal.create(value.asInstanceOf[Decimal].toJavaBigDecimal))
+ }
+
implicit class typeInfoConversions(dt: DataType) {
import org.apache.hadoop.hive.serde2.typeinfo._
import TypeInfoFactory._
+ private def decimalTypeInfo(decimalType: DecimalType): TypeInfo = decimalType match {
+ case DecimalType.Fixed(precision, scale) => new DecimalTypeInfo(precision, scale)
+ case _ => new DecimalTypeInfo(
+ HiveShim.UNLIMITED_DECIMAL_PRECISION,
+ HiveShim.UNLIMITED_DECIMAL_SCALE)
+ }
+
def toTypeInfo: TypeInfo = dt match {
case ArrayType(elemType, _) =>
getListTypeInfo(elemType.toTypeInfo)
@@ -690,7 +822,7 @@ private[hive] trait HiveInspectors {
case LongType => longTypeInfo
case ShortType => shortTypeInfo
case StringType => stringTypeInfo
- case d: DecimalType => HiveShim.decimalTypeInfo(d)
+ case d: DecimalType => decimalTypeInfo(d)
case DateType => dateTypeInfo
case TimestampType => timestampTypeInfo
case NullType => voidTypeInfo
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index ca1f49b546bd7..f35ae96ee0b50 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -19,11 +19,13 @@ package org.apache.spark.sql.hive
import com.google.common.base.Objects
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
+
import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.common.StatsSetupConst
import org.apache.hadoop.hive.metastore.Warehouse
import org.apache.hadoop.hive.metastore.api.FieldSchema
import org.apache.hadoop.hive.ql.metadata._
-import org.apache.hadoop.hive.serde2.Deserializer
+import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog}
@@ -37,7 +39,6 @@ import org.apache.spark.sql.parquet.ParquetRelation2
import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource}
import org.apache.spark.sql.types._
import org.apache.spark.sql.{AnalysisException, SQLContext, SaveMode, sources}
-import org.apache.spark.util.Utils
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -142,7 +143,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive
provider: String,
options: Map[String, String],
isExternal: Boolean): Unit = {
- val (dbName, tblName) = processDatabaseAndTableName("default", tableName)
+ val (dbName, tblName) = processDatabaseAndTableName(client.currentDatabase, tableName)
val tableProperties = new scala.collection.mutable.HashMap[String, String]
tableProperties.put("spark.sql.sources.provider", provider)
@@ -301,7 +302,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive
val partitionColumnDataTypes = partitionSchema.map(_.dataType)
val partitions = metastoreRelation.hiveQlPartitions.map { p =>
val location = p.getLocation
- val values = Row.fromSeq(p.getValues.zip(partitionColumnDataTypes).map {
+ val values = InternalRow.fromSeq(p.getValues.zip(partitionColumnDataTypes).map {
case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null)
})
ParquetPartition(values, location)
@@ -670,8 +671,8 @@ private[hive] case class MetastoreRelation
@transient override lazy val statistics: Statistics = Statistics(
sizeInBytes = {
- val totalSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)
- val rawDataSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize)
+ val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)
+ val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE)
// TODO: check if this estimate is valid for tables after partition pruning.
// NOTE: getting `totalSize` directly from params is kind of hacky, but this should be
// relatively cheap if parameters for the table are populated into the metastore. An
@@ -697,11 +698,7 @@ private[hive] case class MetastoreRelation
}
}
- val tableDesc = HiveShim.getTableDesc(
- Class.forName(
- hiveQlTable.getSerializationLib,
- true,
- Utils.getContextOrSparkClassLoader).asInstanceOf[Class[Deserializer]],
+ val tableDesc = new TableDesc(
hiveQlTable.getInputFormatClass,
// The class of table should be org.apache.hadoop.hive.ql.metadata.Table because
// getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to
@@ -743,6 +740,11 @@ private[hive] case class MetastoreRelation
private[hive] object HiveMetastoreTypes {
def toDataType(metastoreType: String): DataType = DataTypeParser.parse(metastoreType)
+ def decimalMetastoreString(decimalType: DecimalType): String = decimalType match {
+ case DecimalType.Fixed(precision, scale) => s"decimal($precision,$scale)"
+ case _ => s"decimal($HiveShim.UNLIMITED_DECIMAL_PRECISION,$HiveShim.UNLIMITED_DECIMAL_SCALE)"
+ }
+
def toMetastoreType(dt: DataType): String = dt match {
case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>"
case StructType(fields) =>
@@ -759,7 +761,7 @@ private[hive] object HiveMetastoreTypes {
case BinaryType => "binary"
case BooleanType => "boolean"
case DateType => "date"
- case d: DecimalType => HiveShim.decimalMetastoreString(d)
+ case d: DecimalType => decimalMetastoreString(d)
case TimestampType => "timestamp"
case NullType => "void"
case udt: UserDefinedType[_] => toMetastoreType(udt.sqlType)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index a5ca3613c5e00..7c4620952ba4b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -19,8 +19,6 @@ package org.apache.spark.sql.hive
import java.sql.Date
-import scala.collection.mutable.ArrayBuffer
-
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.hadoop.hive.ql.{ErrorMsg, Context}
@@ -39,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
import org.apache.spark.sql.execution.ExplainCommand
import org.apache.spark.sql.sources.DescribeCommand
+import org.apache.spark.sql.hive.HiveShim._
import org.apache.spark.sql.hive.client._
import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema}
import org.apache.spark.sql.types._
@@ -46,6 +45,7 @@ import org.apache.spark.util.random.RandomSampler
/* Implicit conversions */
import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
/**
* Used when we need to start parsing the AST before deciding that we are going to pass the command
@@ -415,13 +415,6 @@ private[hive] object HiveQl {
throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ")
}
- protected def nameExpressions(exprs: Seq[Expression]): Seq[NamedExpression] = {
- exprs.zipWithIndex.map {
- case (ne: NamedExpression, _) => ne
- case (e, i) => Alias(e, s"_c$i")()
- }
- }
-
protected def extractDbNameTableName(tableNameParts: Node): (Option[String], String) = {
val (db, tableName) =
tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) } match {
@@ -942,7 +935,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
// (if there is a group by) or a script transformation.
val withProject: LogicalPlan = transformation.getOrElse {
val selectExpressions =
- nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq)
+ select.getChildren.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)).toSeq
Seq(
groupByClause.map(e => e match {
case Token("TOK_GROUPBY", children) =>
@@ -1307,16 +1300,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
HiveParser.DecimalLiteral)
/* Case insensitive matches */
- val ARRAY = "(?i)ARRAY".r
- val COALESCE = "(?i)COALESCE".r
val COUNT = "(?i)COUNT".r
- val AVG = "(?i)AVG".r
val SUM = "(?i)SUM".r
- val MAX = "(?i)MAX".r
- val MIN = "(?i)MIN".r
- val UPPER = "(?i)UPPER".r
- val LOWER = "(?i)LOWER".r
- val RAND = "(?i)RAND".r
val AND = "(?i)AND".r
val OR = "(?i)OR".r
val NOT = "(?i)NOT".r
@@ -1330,8 +1315,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
val BETWEEN = "(?i)BETWEEN".r
val WHEN = "(?i)WHEN".r
val CASE = "(?i)CASE".r
- val SUBSTR = "(?i)SUBSTR(?:ING)?".r
- val SQRT = "(?i)SQRT".r
protected def nodeToExpr(node: Node): Expression = node match {
/* Attribute References */
@@ -1353,18 +1336,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
UnresolvedStar(Some(name))
/* Aggregate Functions */
- case Token("TOK_FUNCTION", Token(AVG(), Nil) :: arg :: Nil) => Average(nodeToExpr(arg))
- case Token("TOK_FUNCTION", Token(COUNT(), Nil) :: arg :: Nil) => Count(nodeToExpr(arg))
case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => Count(Literal(1))
case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr))
- case Token("TOK_FUNCTION", Token(SUM(), Nil) :: arg :: Nil) => Sum(nodeToExpr(arg))
case Token("TOK_FUNCTIONDI", Token(SUM(), Nil) :: arg :: Nil) => SumDistinct(nodeToExpr(arg))
- case Token("TOK_FUNCTION", Token(MAX(), Nil) :: arg :: Nil) => Max(nodeToExpr(arg))
- case Token("TOK_FUNCTION", Token(MIN(), Nil) :: arg :: Nil) => Min(nodeToExpr(arg))
-
- /* System functions about string operations */
- case Token("TOK_FUNCTION", Token(UPPER(), Nil) :: arg :: Nil) => Upper(nodeToExpr(arg))
- case Token("TOK_FUNCTION", Token(LOWER(), Nil) :: arg :: Nil) => Lower(nodeToExpr(arg))
/* Casts */
case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) =>
@@ -1414,7 +1388,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right))
case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right))
case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right))
- case Token("TOK_FUNCTION", Token(SQRT(), Nil) :: arg :: Nil) => Sqrt(nodeToExpr(arg))
/* Comparisons */
case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right))
@@ -1469,17 +1442,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
case Token("[", child :: ordinal :: Nil) =>
UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal))
- /* Other functions */
- case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) =>
- CreateArray(children.map(nodeToExpr))
- case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand()
- case Token("TOK_FUNCTION", Token(RAND(), Nil) :: seed :: Nil) => Rand(seed.toString.toLong)
- case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) =>
- Substring(nodeToExpr(string), nodeToExpr(pos), Literal.create(Integer.MAX_VALUE, IntegerType))
- case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) =>
- Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length))
- case Token("TOK_FUNCTION", Token(COALESCE(), Nil) :: list) => Coalesce(list.map(nodeToExpr))
-
/* Window Functions */
case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) =>
val function = UnresolvedWindowFunction(name, args.map(nodeToExpr))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
new file mode 100644
index 0000000000000..d08c594151654
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.io.{InputStream, OutputStream}
+import java.rmi.server.UID
+
+/* Implicit conversions */
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+import scala.reflect.ClassTag
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.ql.exec.{UDF, Utilities}
+import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
+import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector
+import org.apache.hadoop.io.Writable
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.util.Utils
+
+private[hive] object HiveShim {
+ // Precision and scale to pass for unlimited decimals; these are the same as the precision and
+ // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs)
+ val UNLIMITED_DECIMAL_PRECISION = 38
+ val UNLIMITED_DECIMAL_SCALE = 18
+
+ /*
+ * This function in hive-0.13 become private, but we have to do this to walkaround hive bug
+ */
+ private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) {
+ val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "")
+ val result: StringBuilder = new StringBuilder(old)
+ var first: Boolean = old.isEmpty
+
+ for (col <- cols) {
+ if (first) {
+ first = false
+ } else {
+ result.append(',')
+ }
+ result.append(col)
+ }
+ conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString)
+ }
+
+ /*
+ * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty
+ */
+ def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
+ if (ids != null && ids.nonEmpty) {
+ ColumnProjectionUtils.appendReadColumns(conf, ids)
+ }
+ if (names != null && names.nonEmpty) {
+ appendReadColumnNames(conf, names)
+ }
+ }
+
+ /*
+ * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that
+ * is needed to initialize before serialization.
+ */
+ def prepareWritable(w: Writable): Writable = {
+ w match {
+ case w: AvroGenericRecordWritable =>
+ w.setRecordReaderID(new UID())
+ case _ =>
+ }
+ w
+ }
+
+ def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = {
+ if (hdoi.preferWritable()) {
+ Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue,
+ hdoi.precision(), hdoi.scale())
+ } else {
+ Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale())
+ }
+ }
+
+ /**
+ * This class provides the UDF creation and also the UDF instance serialization and
+ * de-serialization cross process boundary.
+ *
+ * Detail discussion can be found at https://github.com/apache/spark/pull/3640
+ *
+ * @param functionClassName UDF class name
+ */
+ private[hive] case class HiveFunctionWrapper(var functionClassName: String)
+ extends java.io.Externalizable {
+
+ // for Serialization
+ def this() = this(null)
+
+ @transient
+ def deserializeObjectByKryo[T: ClassTag](
+ kryo: Kryo,
+ in: InputStream,
+ clazz: Class[_]): T = {
+ val inp = new Input(in)
+ val t: T = kryo.readObject(inp, clazz).asInstanceOf[T]
+ inp.close()
+ t
+ }
+
+ @transient
+ def serializeObjectByKryo(
+ kryo: Kryo,
+ plan: Object,
+ out: OutputStream) {
+ val output: Output = new Output(out)
+ kryo.writeObject(output, plan)
+ output.close()
+ }
+
+ def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = {
+ deserializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), is, clazz)
+ .asInstanceOf[UDFType]
+ }
+
+ def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = {
+ serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out)
+ }
+
+ private var instance: AnyRef = null
+
+ def writeExternal(out: java.io.ObjectOutput) {
+ // output the function name
+ out.writeUTF(functionClassName)
+
+ // Write a flag if instance is null or not
+ out.writeBoolean(instance != null)
+ if (instance != null) {
+ // Some of the UDF are serializable, but some others are not
+ // Hive Utilities can handle both cases
+ val baos = new java.io.ByteArrayOutputStream()
+ serializePlan(instance, baos)
+ val functionInBytes = baos.toByteArray
+
+ // output the function bytes
+ out.writeInt(functionInBytes.length)
+ out.write(functionInBytes, 0, functionInBytes.length)
+ }
+ }
+
+ def readExternal(in: java.io.ObjectInput) {
+ // read the function name
+ functionClassName = in.readUTF()
+
+ if (in.readBoolean()) {
+ // if the instance is not null
+ // read the function in bytes
+ val functionInBytesLength = in.readInt()
+ val functionInBytes = new Array[Byte](functionInBytesLength)
+ in.read(functionInBytes, 0, functionInBytesLength)
+
+ // deserialize the function object via Hive Utilities
+ instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes),
+ Utils.getContextOrSparkClassLoader.loadClass(functionClassName))
+ }
+ }
+
+ def createFunction[UDFType <: AnyRef](): UDFType = {
+ if (instance != null) {
+ instance.asInstanceOf[UDFType]
+ } else {
+ val func = Utils.getContextOrSparkClassLoader
+ .loadClass(functionClassName).newInstance.asInstanceOf[UDFType]
+ if (!func.isInstanceOf[UDF]) {
+ // We cache the function if it's no the Simple UDF,
+ // as we always have to create new instance for Simple UDF
+ instance = func
+ }
+ func
+ }
+ }
+ }
+
+ /*
+ * Bug introduced in hive-0.13. FileSinkDesc is serializable, but its member path is not.
+ * Fix it through wrapper.
+ */
+ implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = {
+ val f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed)
+ f.setCompressCodec(w.compressCodec)
+ f.setCompressType(w.compressType)
+ f.setTableInfo(w.tableInfo)
+ f.setDestTableId(w.destTableId)
+ f
+ }
+
+ /*
+ * Bug introduced in hive-0.13. FileSinkDesc is serializable, but its member path is not.
+ * Fix it through wrapper.
+ */
+ private[hive] class ShimFileSinkDesc(
+ var dir: String,
+ var tableInfo: TableDesc,
+ var compressed: Boolean)
+ extends Serializable with Logging {
+ var compressCodec: String = _
+ var compressType: String = _
+ var destTableId: Int = _
+
+ def setCompressed(compressed: Boolean) {
+ this.compressed = compressed
+ }
+
+ def getDirName(): String = dir
+
+ def setDestTableId(destTableId: Int) {
+ this.destTableId = destTableId
+ }
+
+ def setTableInfo(tableInfo: TableDesc) {
+ this.tableInfo = tableInfo
+ }
+
+ def setCompressCodec(intermediateCompressorCodec: String) {
+ compressCodec = intermediateCompressorCodec
+ }
+
+ def setCompressType(intermediateCompressType: String) {
+ compressType = intermediateCompressType
+ }
+ }
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index c6b65106452bf..452b7f0bcc749 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
-import org.apache.spark.sql.catalyst.expressions.{Row, _}
+import org.apache.spark.sql.catalyst.expressions.{InternalRow, _}
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -137,7 +137,7 @@ private[hive] trait HiveStrategies {
val partitionLocations = partitions.map(_.getLocation)
if (partitionLocations.isEmpty) {
- PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil
+ PhysicalRDD(plan.output, sparkContext.emptyRDD[InternalRow]) :: Nil
} else {
hiveContext
.read.parquet(partitionLocations: _*)
@@ -165,7 +165,7 @@ private[hive] trait HiveStrategies {
// TODO: Remove this hack for Spark 1.3.
case iae: java.lang.IllegalArgumentException
if iae.getMessage.contains("Can not create a Path from an empty string") =>
- PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil
+ PhysicalRDD(plan.output, sparkContext.emptyRDD[InternalRow]) :: Nil
}
case _ => Nil
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index 294fc3bd7d5e9..00e61e35d4354 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.hive
-import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._
@@ -25,26 +24,25 @@ import org.apache.hadoop.hive.ql.exec.Utilities
import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable}
import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc}
import org.apache.hadoop.hive.serde2.Deserializer
-import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector}
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
+import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf}
-import org.apache.spark.SerializableWritable
-import org.apache.spark.broadcast.Broadcast
import org.apache.spark.Logging
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.util.DateUtils
-import org.apache.spark.util.Utils
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.util.{SerializableConfiguration, Utils}
/**
* A trait for subclasses that handle table scans.
*/
private[hive] sealed trait TableReader {
- def makeRDDForTable(hiveTable: HiveTable): RDD[Row]
+ def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow]
- def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row]
+ def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow]
}
@@ -73,9 +71,9 @@ class HadoopTableReader(
// TODO: set aws s3 credentials.
private val _broadcastedHiveConf =
- sc.sparkContext.broadcast(new SerializableWritable(hiveExtraConf))
+ sc.sparkContext.broadcast(new SerializableConfiguration(hiveExtraConf))
- override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] =
+ override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
makeRDDForTable(
hiveTable,
Class.forName(
@@ -95,7 +93,7 @@ class HadoopTableReader(
def makeRDDForTable(
hiveTable: HiveTable,
deserializerClass: Class[_ <: Deserializer],
- filterOpt: Option[PathFilter]): RDD[Row] = {
+ filterOpt: Option[PathFilter]): RDD[InternalRow] = {
assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table,
since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""")
@@ -126,7 +124,7 @@ class HadoopTableReader(
deserializedHadoopRDD
}
- override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] = {
+ override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow] = {
val partitionToDeserializer = partitions.map(part =>
(part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap
makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None)
@@ -145,7 +143,7 @@ class HadoopTableReader(
def makeRDDForPartitionedTable(
partitionToDeserializer: Map[HivePartition,
Class[_ <: Deserializer]],
- filterOpt: Option[PathFilter]): RDD[Row] = {
+ filterOpt: Option[PathFilter]): RDD[InternalRow] = {
// SPARK-5068:get FileStatus and do the filtering locally when the path is not exists
def verifyPartitionPath(
@@ -172,7 +170,7 @@ class HadoopTableReader(
path.toString + tails
}
- val partPath = HiveShim.getDataLocationPath(partition)
+ val partPath = partition.getDataLocation
val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size();
var pathPatternStr = getPathPatternByPath(partNum, partPath)
if (!pathPatternSet.contains(pathPatternStr)) {
@@ -187,7 +185,7 @@ class HadoopTableReader(
val hivePartitionRDDs = verifyPartitionPath(partitionToDeserializer)
.map { case (partition, partDeserializer) =>
val partDesc = Utilities.getPartitionDesc(partition)
- val partPath = HiveShim.getDataLocationPath(partition)
+ val partPath = partition.getDataLocation
val inputPathStr = applyFilterIfNeeded(partPath, filterOpt)
val ifc = partDesc.getInputFileFormatClass
.asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]]
@@ -244,7 +242,7 @@ class HadoopTableReader(
// Even if we don't use any partitions, we still need an empty RDD
if (hivePartitionRDDs.size == 0) {
- new EmptyRDD[Row](sc.sparkContext)
+ new EmptyRDD[InternalRow](sc.sparkContext)
} else {
new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs)
}
@@ -277,7 +275,7 @@ class HadoopTableReader(
val rdd = new HadoopRDD(
sc.sparkContext,
- _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableWritable[Configuration]]],
+ _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableConfiguration]],
Some(initializeJobConfFunc),
inputFormatClass,
classOf[Writable],
@@ -320,12 +318,12 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
rawDeser: Deserializer,
nonPartitionKeyAttrs: Seq[(Attribute, Int)],
mutableRow: MutableRow,
- tableDeser: Deserializer): Iterator[Row] = {
+ tableDeser: Deserializer): Iterator[InternalRow] = {
val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) {
rawDeser.getObjectInspector.asInstanceOf[StructObjectInspector]
} else {
- HiveShim.getConvertedOI(
+ ObjectInspectorConverters.getConvertedOI(
rawDeser.getObjectInspector,
tableDeser.getObjectInspector).asInstanceOf[StructObjectInspector]
}
@@ -364,10 +362,10 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
row.update(ordinal, HiveShim.toCatalystDecimal(oi, value))
case oi: TimestampObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) =>
- row.update(ordinal, oi.getPrimitiveJavaObject(value).clone())
+ row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value)))
case oi: DateObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) =>
- row.update(ordinal, DateUtils.fromJavaDate(oi.getPrimitiveJavaObject(value)))
+ row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value)))
case oi: BinaryObjectInspector =>
(value: Any, row: MutableRow, ordinal: Int) =>
row.update(ordinal, oi.getPrimitiveJavaObject(value))
@@ -392,7 +390,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
i += 1
}
- mutableRow: Row
+ mutableRow: InternalRow
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index 99aa0f1ded3f8..2f771d76793e5 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.client
import java.io.{BufferedReader, InputStreamReader, File, PrintStream}
import java.net.URI
import java.util.{ArrayList => JArrayList, Map => JMap, List => JList, Set => JSet}
+import javax.annotation.concurrent.GuardedBy
import scala.collection.JavaConversions._
import scala.language.reflectiveCalls
@@ -27,7 +28,7 @@ import scala.language.reflectiveCalls
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.metastore.api.Database
import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.metastore.TableType
+import org.apache.hadoop.hive.metastore.{TableType => HTableType}
import org.apache.hadoop.hive.metastore.api
import org.apache.hadoop.hive.metastore.api.FieldSchema
import org.apache.hadoop.hive.ql.metadata
@@ -54,13 +55,15 @@ import org.apache.spark.sql.execution.QueryExecutionException
* @param version the version of hive used when pick function calls that are not compatible.
* @param config a collection of configuration options that will be added to the hive conf before
* opening the hive client.
+ * @param initClassLoader the classloader used when creating the `state` field of
+ * this ClientWrapper.
*/
private[hive] class ClientWrapper(
version: HiveVersion,
- config: Map[String, String])
+ config: Map[String, String],
+ initClassLoader: ClassLoader)
extends ClientInterface
- with Logging
- with ReflectionMagic {
+ with Logging {
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
private val outputBuffer = new java.io.OutputStream {
@@ -90,13 +93,27 @@ private[hive] class ClientWrapper(
}
}
+ private val shim = version match {
+ case hive.v12 => new Shim_v0_12()
+ case hive.v13 => new Shim_v0_13()
+ case hive.v14 => new Shim_v0_14()
+ }
+
+ // Create an internal session state for this ClientWrapper.
val state = {
val original = Thread.currentThread().getContextClassLoader
- Thread.currentThread().setContextClassLoader(getClass.getClassLoader)
+ // Switch to the initClassLoader.
+ Thread.currentThread().setContextClassLoader(initClassLoader)
val ret = try {
val oldState = SessionState.get()
if (oldState == null) {
val initialConf = new HiveConf(classOf[SessionState])
+ // HiveConf is a Hadoop Configuration, which has a field of classLoader and
+ // the initial value will be the current thread's context class loader
+ // (i.e. initClassLoader at here).
+ // We call initialConf.setClassLoader(initClassLoader) at here to make
+ // this action explicit.
+ initialConf.setClassLoader(initClassLoader)
config.foreach { case (k, v) =>
logDebug(s"Hive Config: $k=$v")
initialConf.set(k, v)
@@ -119,23 +136,70 @@ private[hive] class ClientWrapper(
def conf: HiveConf = SessionState.get().getConf
// TODO: should be a def?s
- private val client = Hive.get(conf)
+ // When we create this val client, the HiveConf of it (conf) is the one associated with state.
+ @GuardedBy("this")
+ private var client = Hive.get(conf)
+
+ // We use hive's conf for compatibility.
+ private val retryLimit = conf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES)
+ private val retryDelayMillis = shim.getMetastoreClientConnectRetryDelayMillis(conf)
+
+ /**
+ * Runs `f` with multiple retries in case the hive metastore is temporarily unreachable.
+ */
+ private def retryLocked[A](f: => A): A = synchronized {
+ // Hive sometimes retries internally, so set a deadline to avoid compounding delays.
+ val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong
+ var numTries = 0
+ var caughtException: Exception = null
+ do {
+ numTries += 1
+ try {
+ return f
+ } catch {
+ case e: Exception if causedByThrift(e) =>
+ caughtException = e
+ logWarning(
+ "HiveClientWrapper got thrift exception, destroying client and retrying " +
+ s"(${retryLimit - numTries} tries remaining)", e)
+ Thread.sleep(retryDelayMillis)
+ try {
+ client = Hive.get(state.getConf, true)
+ } catch {
+ case e: Exception if causedByThrift(e) =>
+ logWarning("Failed to refresh hive client, will retry.", e)
+ }
+ }
+ } while (numTries <= retryLimit && System.nanoTime < deadline)
+ if (System.nanoTime > deadline) {
+ logWarning("Deadline exceeded")
+ }
+ throw caughtException
+ }
+
+ private def causedByThrift(e: Throwable): Boolean = {
+ var target = e
+ while (target != null) {
+ val msg = target.getMessage()
+ if (msg != null && msg.matches("(?s).*(TApplication|TProtocol|TTransport)Exception.*")) {
+ return true
+ }
+ target = target.getCause()
+ }
+ false
+ }
/**
* Runs `f` with ThreadLocal session state and classloaders configured for this version of hive.
*/
- private def withHiveState[A](f: => A): A = synchronized {
+ private def withHiveState[A](f: => A): A = retryLocked {
val original = Thread.currentThread().getContextClassLoader
- Thread.currentThread().setContextClassLoader(getClass.getClassLoader)
+ // Set the thread local metastore client to the client associated with this ClientWrapper.
Hive.set(client)
- version match {
- case hive.v12 =>
- classOf[SessionState]
- .callStatic[SessionState, SessionState]("start", state)
- case hive.v13 =>
- classOf[SessionState]
- .callStatic[SessionState, SessionState]("setCurrentSessionState", state)
- }
+ // setCurrentSessionState will use the classLoader associated
+ // with the HiveConf in `state` to override the context class loader of the current
+ // thread.
+ shim.setCurrentSessionState(state)
val ret = try f finally {
Thread.currentThread().setContextClassLoader(original)
}
@@ -193,15 +257,12 @@ private[hive] class ClientWrapper(
properties = h.getParameters.toMap,
serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap,
tableType = h.getTableType match {
- case TableType.MANAGED_TABLE => ManagedTable
- case TableType.EXTERNAL_TABLE => ExternalTable
- case TableType.VIRTUAL_VIEW => VirtualView
- case TableType.INDEX_TABLE => IndexTable
- },
- location = version match {
- case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString)
- case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString)
+ case HTableType.MANAGED_TABLE => ManagedTable
+ case HTableType.EXTERNAL_TABLE => ExternalTable
+ case HTableType.VIRTUAL_VIEW => VirtualView
+ case HTableType.INDEX_TABLE => IndexTable
},
+ location = shim.getDataLocation(h),
inputFormat = Option(h.getInputFormatClass).map(_.getName),
outputFormat = Option(h.getOutputFormatClass).map(_.getName),
serde = Option(h.getSerializationLib),
@@ -231,14 +292,7 @@ private[hive] class ClientWrapper(
// set create time
qlTable.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int])
- version match {
- case hive.v12 =>
- table.location.map(new URI(_)).foreach(u => qlTable.call[URI, Unit]("setDataLocation", u))
- case hive.v13 =>
- table.location
- .map(new org.apache.hadoop.fs.Path(_))
- .foreach(qlTable.call[Path, Unit]("setDataLocation", _))
- }
+ table.location.foreach { loc => shim.setDataLocation(qlTable, loc) }
table.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass)
table.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass)
table.serde.foreach(qlTable.setSerializationLib)
@@ -279,13 +333,7 @@ private[hive] class ClientWrapper(
override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState {
val qlTable = toQlTable(hTable)
- val qlPartitions = version match {
- case hive.v12 =>
- client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsForPruner", qlTable)
- case hive.v13 =>
- client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsOf", qlTable)
- }
- qlPartitions.toSeq.map(toHivePartition)
+ shim.getAllPartitions(client, qlTable).map(toHivePartition)
}
override def listTables(dbName: String): Seq[String] = withHiveState {
@@ -315,15 +363,7 @@ private[hive] class ClientWrapper(
val tokens: Array[String] = cmd_trimmed.split("\\s+")
// The remainder of the command.
val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
- val proc: CommandProcessor = version match {
- case hive.v12 =>
- classOf[CommandProcessorFactory]
- .callStatic[String, HiveConf, CommandProcessor]("get", tokens(0), conf)
- case hive.v13 =>
- classOf[CommandProcessorFactory]
- .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf)
- }
-
+ val proc = shim.getCommandProcessor(tokens(0), conf)
proc match {
case driver: Driver =>
val response: CommandProcessorResponse = driver.run(cmd)
@@ -334,21 +374,7 @@ private[hive] class ClientWrapper(
}
driver.setMaxRows(maxRows)
- val results = version match {
- case hive.v12 =>
- val res = new JArrayList[String]
- driver.call[JArrayList[String], Boolean]("getResults", res)
- res.toSeq
- case hive.v13 =>
- val res = new JArrayList[Object]
- driver.call[JList[Object], Boolean]("getResults", res)
- res.map { r =>
- r match {
- case s: String => s
- case a: Array[Object] => a(0).asInstanceOf[String]
- }
- }
- }
+ val results = shim.getDriverResults(driver)
driver.close()
results
@@ -382,8 +408,8 @@ private[hive] class ClientWrapper(
holdDDLTime: Boolean,
inheritTableSpecs: Boolean,
isSkewedStoreAsSubdir: Boolean): Unit = withHiveState {
-
- client.loadPartition(
+ shim.loadPartition(
+ client,
new Path(loadPath), // TODO: Use URI
tableName,
partSpec,
@@ -398,7 +424,8 @@ private[hive] class ClientWrapper(
tableName: String,
replace: Boolean,
holdDDLTime: Boolean): Unit = withHiveState {
- client.loadTable(
+ shim.loadTable(
+ client,
new Path(loadPath),
tableName,
replace,
@@ -413,7 +440,8 @@ private[hive] class ClientWrapper(
numDP: Int,
holdDDLTime: Boolean,
listBucketingEnabled: Boolean): Unit = withHiveState {
- client.loadDynamicPartitions(
+ shim.loadDynamicPartitions(
+ client,
new Path(loadPath),
tableName,
partSpec,
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
new file mode 100644
index 0000000000000..e7c1779f80ce6
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
@@ -0,0 +1,381 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.client
+
+import java.lang.{Boolean => JBoolean, Integer => JInteger}
+import java.lang.reflect.{Method, Modifier}
+import java.net.URI
+import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JSet}
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConversions._
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.Driver
+import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
+import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory}
+import org.apache.hadoop.hive.ql.session.SessionState
+
+/**
+ * A shim that defines the interface between ClientWrapper and the underlying Hive library used to
+ * talk to the metastore. Each Hive version has its own implementation of this class, defining
+ * version-specific version of needed functions.
+ *
+ * The guideline for writing shims is:
+ * - always extend from the previous version unless really not possible
+ * - initialize methods in lazy vals, both for quicker access for multiple invocations, and to
+ * avoid runtime errors due to the above guideline.
+ */
+private[client] sealed abstract class Shim {
+
+ /**
+ * Set the current SessionState to the given SessionState. Also, set the context classloader of
+ * the current thread to the one set in the HiveConf of this given `state`.
+ * @param state
+ */
+ def setCurrentSessionState(state: SessionState): Unit
+
+ /**
+ * This shim is necessary because the return type is different on different versions of Hive.
+ * All parameters are the same, though.
+ */
+ def getDataLocation(table: Table): Option[String]
+
+ def setDataLocation(table: Table, loc: String): Unit
+
+ def getAllPartitions(hive: Hive, table: Table): Seq[Partition]
+
+ def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor
+
+ def getDriverResults(driver: Driver): Seq[String]
+
+ def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long
+
+ def loadPartition(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ holdDDLTime: Boolean,
+ inheritTableSpecs: Boolean,
+ isSkewedStoreAsSubdir: Boolean): Unit
+
+ def loadTable(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ replace: Boolean,
+ holdDDLTime: Boolean): Unit
+
+ def loadDynamicPartitions(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ numDP: Int,
+ holdDDLTime: Boolean,
+ listBucketingEnabled: Boolean): Unit
+
+ protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = {
+ val method = findMethod(klass, name, args: _*)
+ require(Modifier.isStatic(method.getModifiers()),
+ s"Method $name of class $klass is not static.")
+ method
+ }
+
+ protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = {
+ klass.getMethod(name, args: _*)
+ }
+
+}
+
+private[client] class Shim_v0_12 extends Shim {
+
+ private lazy val startMethod =
+ findStaticMethod(
+ classOf[SessionState],
+ "start",
+ classOf[SessionState])
+ private lazy val getDataLocationMethod = findMethod(classOf[Table], "getDataLocation")
+ private lazy val setDataLocationMethod =
+ findMethod(
+ classOf[Table],
+ "setDataLocation",
+ classOf[URI])
+ private lazy val getAllPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "getAllPartitionsForPruner",
+ classOf[Table])
+ private lazy val getCommandProcessorMethod =
+ findStaticMethod(
+ classOf[CommandProcessorFactory],
+ "get",
+ classOf[String],
+ classOf[HiveConf])
+ private lazy val getDriverResultsMethod =
+ findMethod(
+ classOf[Driver],
+ "getResults",
+ classOf[JArrayList[String]])
+ private lazy val loadPartitionMethod =
+ findMethod(
+ classOf[Hive],
+ "loadPartition",
+ classOf[Path],
+ classOf[String],
+ classOf[JMap[String, String]],
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadTableMethod =
+ findMethod(
+ classOf[Hive],
+ "loadTable",
+ classOf[Path],
+ classOf[String],
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadDynamicPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "loadDynamicPartitions",
+ classOf[Path],
+ classOf[String],
+ classOf[JMap[String, String]],
+ JBoolean.TYPE,
+ JInteger.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+
+ override def setCurrentSessionState(state: SessionState): Unit = {
+ // Starting from Hive 0.13, setCurrentSessionState will internally override
+ // the context class loader of the current thread by the class loader set in
+ // the conf of the SessionState. So, for this Hive 0.12 shim, we add the same
+ // behavior and make shim.setCurrentSessionState of all Hive versions have the
+ // consistent behavior.
+ Thread.currentThread().setContextClassLoader(state.getConf.getClassLoader)
+ startMethod.invoke(null, state)
+ }
+
+ override def getDataLocation(table: Table): Option[String] =
+ Option(getDataLocationMethod.invoke(table)).map(_.toString())
+
+ override def setDataLocation(table: Table, loc: String): Unit =
+ setDataLocationMethod.invoke(table, new URI(loc))
+
+ override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] =
+ getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq
+
+ override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor =
+ getCommandProcessorMethod.invoke(null, token, conf).asInstanceOf[CommandProcessor]
+
+ override def getDriverResults(driver: Driver): Seq[String] = {
+ val res = new JArrayList[String]()
+ getDriverResultsMethod.invoke(driver, res)
+ res.toSeq
+ }
+
+ override def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long = {
+ conf.getIntVar(HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY) * 1000
+ }
+
+ override def loadPartition(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ holdDDLTime: Boolean,
+ inheritTableSpecs: Boolean,
+ isSkewedStoreAsSubdir: Boolean): Unit = {
+ loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean,
+ holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean)
+ }
+
+ override def loadTable(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ replace: Boolean,
+ holdDDLTime: Boolean): Unit = {
+ loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean)
+ }
+
+ override def loadDynamicPartitions(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ numDP: Int,
+ holdDDLTime: Boolean,
+ listBucketingEnabled: Boolean): Unit = {
+ loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean,
+ numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean)
+ }
+
+}
+
+private[client] class Shim_v0_13 extends Shim_v0_12 {
+
+ private lazy val setCurrentSessionStateMethod =
+ findStaticMethod(
+ classOf[SessionState],
+ "setCurrentSessionState",
+ classOf[SessionState])
+ private lazy val setDataLocationMethod =
+ findMethod(
+ classOf[Table],
+ "setDataLocation",
+ classOf[Path])
+ private lazy val getAllPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "getAllPartitionsOf",
+ classOf[Table])
+ private lazy val getCommandProcessorMethod =
+ findStaticMethod(
+ classOf[CommandProcessorFactory],
+ "get",
+ classOf[Array[String]],
+ classOf[HiveConf])
+ private lazy val getDriverResultsMethod =
+ findMethod(
+ classOf[Driver],
+ "getResults",
+ classOf[JList[Object]])
+
+ override def setCurrentSessionState(state: SessionState): Unit =
+ setCurrentSessionStateMethod.invoke(null, state)
+
+ override def setDataLocation(table: Table, loc: String): Unit =
+ setDataLocationMethod.invoke(table, new Path(loc))
+
+ override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] =
+ getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq
+
+ override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor =
+ getCommandProcessorMethod.invoke(null, Array(token), conf).asInstanceOf[CommandProcessor]
+
+ override def getDriverResults(driver: Driver): Seq[String] = {
+ val res = new JArrayList[Object]()
+ getDriverResultsMethod.invoke(driver, res)
+ res.map { r =>
+ r match {
+ case s: String => s
+ case a: Array[Object] => a(0).asInstanceOf[String]
+ }
+ }
+ }
+
+}
+
+private[client] class Shim_v0_14 extends Shim_v0_13 {
+
+ private lazy val loadPartitionMethod =
+ findMethod(
+ classOf[Hive],
+ "loadPartition",
+ classOf[Path],
+ classOf[String],
+ classOf[JMap[String, String]],
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadTableMethod =
+ findMethod(
+ classOf[Hive],
+ "loadTable",
+ classOf[Path],
+ classOf[String],
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+ private lazy val loadDynamicPartitionsMethod =
+ findMethod(
+ classOf[Hive],
+ "loadDynamicPartitions",
+ classOf[Path],
+ classOf[String],
+ classOf[JMap[String, String]],
+ JBoolean.TYPE,
+ JInteger.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE,
+ JBoolean.TYPE)
+ private lazy val getTimeVarMethod =
+ findMethod(
+ classOf[HiveConf],
+ "getTimeVar",
+ classOf[HiveConf.ConfVars],
+ classOf[TimeUnit])
+
+ override def loadPartition(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ holdDDLTime: Boolean,
+ inheritTableSpecs: Boolean,
+ isSkewedStoreAsSubdir: Boolean): Unit = {
+ loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean,
+ holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean,
+ JBoolean.TRUE, JBoolean.FALSE)
+ }
+
+ override def loadTable(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ replace: Boolean,
+ holdDDLTime: Boolean): Unit = {
+ loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean,
+ JBoolean.TRUE, JBoolean.FALSE, JBoolean.FALSE)
+ }
+
+ override def loadDynamicPartitions(
+ hive: Hive,
+ loadPath: Path,
+ tableName: String,
+ partSpec: JMap[String, String],
+ replace: Boolean,
+ numDP: Int,
+ holdDDLTime: Boolean,
+ listBucketingEnabled: Boolean): Unit = {
+ loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean,
+ numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean, JBoolean.FALSE)
+ }
+
+ override def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long = {
+ getTimeVarMethod.invoke(
+ conf,
+ HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY,
+ TimeUnit.MILLISECONDS).asInstanceOf[Long]
+ }
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
index 16851fdd71a98..0934ad5034671 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.hive.client
import java.io.File
+import java.lang.reflect.InvocationTargetException
import java.net.{URL, URLClassLoader}
import java.util
@@ -28,6 +29,7 @@ import org.apache.commons.io.{FileUtils, IOUtils}
import org.apache.spark.Logging
import org.apache.spark.deploy.SparkSubmitUtils
+import org.apache.spark.util.Utils
import org.apache.spark.sql.catalyst.util.quietly
import org.apache.spark.sql.hive.HiveContext
@@ -48,29 +50,27 @@ private[hive] object IsolatedClientLoader {
def hiveVersion(version: String): HiveVersion = version match {
case "12" | "0.12" | "0.12.0" => hive.v12
case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13
+ case "14" | "0.14" | "0.14.0" => hive.v14
}
private def downloadVersion(version: HiveVersion): Seq[URL] = {
- val hiveArtifacts =
- (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++
- (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil))
- .map(a => s"org.apache.hive:$a:${version.fullVersion}") :+
- "com.google.guava:guava:14.0.1" :+
- "org.apache.hadoop:hadoop-client:2.4.0"
+ val hiveArtifacts = version.extraDeps ++
+ Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde")
+ .map(a => s"org.apache.hive:$a:${version.fullVersion}") ++
+ Seq("com.google.guava:guava:14.0.1",
+ "org.apache.hadoop:hadoop-client:2.4.0")
val classpath = quietly {
SparkSubmitUtils.resolveMavenCoordinates(
hiveArtifacts.mkString(","),
Some("http://www.datanucleus.org/downloads/maven2"),
- None)
+ None,
+ exclusions = version.exclusions)
}
val allFiles = classpath.split(",").map(new File(_)).toSet
// TODO: Remove copy logic.
- val tempDir = File.createTempFile("hive", "v" + version.toString)
- tempDir.delete()
- tempDir.mkdir()
-
+ val tempDir = Utils.createTempDir(namePrefix = s"hive-${version}")
allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir))
tempDir.listFiles().map(_.toURL)
}
@@ -95,9 +95,8 @@ private[hive] object IsolatedClientLoader {
* @param config A set of options that will be added to the HiveConf of the constructed client.
* @param isolationOn When true, custom versions of barrier classes will be constructed. Must be
* true unless loading the version of hive that is on Sparks classloader.
- * @param rootClassLoader The system root classloader.
- * @param baseClassLoader The spark classloader that is used to load shared classes. Must not know
- * about Hive classes.
+ * @param rootClassLoader The system root classloader. Must not know about Hive classes.
+ * @param baseClassLoader The spark classloader that is used to load shared classes.
*/
private[hive] class IsolatedClientLoader(
val version: HiveVersion,
@@ -110,8 +109,8 @@ private[hive] class IsolatedClientLoader(
val barrierPrefixes: Seq[String] = Seq.empty)
extends Logging {
- // Check to make sure that the base classloader does not know about Hive.
- assert(Try(baseClassLoader.loadClass("org.apache.hive.HiveConf")).isFailure)
+ // Check to make sure that the root classloader does not know about Hive.
+ assert(Try(rootClassLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")).isFailure)
/** All jars used by the hive specific classloader. */
protected def allJars = execJars.toArray
@@ -129,7 +128,7 @@ private[hive] class IsolatedClientLoader(
/** True if `name` refers to a spark class that must see specific version of Hive. */
protected def isBarrierClass(name: String): Boolean =
name.startsWith(classOf[ClientWrapper].getName) ||
- name.startsWith(classOf[ReflectionMagic].getName) ||
+ name.startsWith(classOf[Shim].getName) ||
barrierPrefixes.exists(name.startsWith)
protected def classToPath(name: String): String =
@@ -145,6 +144,7 @@ private[hive] class IsolatedClientLoader(
def doLoadClass(name: String, resolve: Boolean): Class[_] = {
val classFileName = name.replaceAll("\\.", "/") + ".class"
if (isBarrierClass(name) && isolationOn) {
+ // For barrier classes, we construct a new copy of the class.
val bytes = IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName))
logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}")
defineClass(name, bytes, 0, bytes.length)
@@ -152,6 +152,7 @@ private[hive] class IsolatedClientLoader(
logDebug(s"hive class: $name - ${getResource(classToPath(name))}")
super.loadClass(name, resolve)
} else {
+ // For shared classes, we delegate to baseClassLoader.
logDebug(s"shared class: $name")
baseClassLoader.loadClass(name)
}
@@ -167,14 +168,19 @@ private[hive] class IsolatedClientLoader(
classLoader
.loadClass(classOf[ClientWrapper].getName)
.getConstructors.head
- .newInstance(version, config)
+ .newInstance(version, config, classLoader)
.asInstanceOf[ClientInterface]
} catch {
- case ReflectionException(cnf: NoClassDefFoundError) =>
- throw new ClassNotFoundException(
- s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" +
- "Please make sure that jars for your version of hive and hadoop are included in the " +
- s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.")
+ case e: InvocationTargetException =>
+ if (e.getCause().isInstanceOf[NoClassDefFoundError]) {
+ val cnf = e.getCause().asInstanceOf[NoClassDefFoundError]
+ throw new ClassNotFoundException(
+ s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" +
+ "Please make sure that jars for your version of hive and hadoop are included in the " +
+ s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.")
+ } else {
+ throw e
+ }
} finally {
Thread.currentThread.setContextClassLoader(baseClassLoader)
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala
deleted file mode 100644
index 4d053ae42c2ea..0000000000000
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive.client
-
-import scala.reflect._
-
-/** Unwraps reflection exceptions. */
-private[client] object ReflectionException {
- def unapply(a: Throwable): Option[Throwable] = a match {
- case ite: java.lang.reflect.InvocationTargetException => Option(ite.getCause)
- case _ => None
- }
-}
-
-/**
- * Provides implicit functions on any object for calling methods reflectively.
- */
-private[client] trait ReflectionMagic {
- /** code for InstanceMagic
- println(
- (1 to 22).map { n =>
- def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ")
- val types = repeat(n => s"A$n <: AnyRef : ClassTag")
- val inArgs = repeat(n => s"a$n: A$n")
- val erasure = repeat(n => s"classTag[A$n].erasure")
- val outArgs = repeat(n => s"a$n")
- s"""|def call[$types, R](name: String, $inArgs): R = {
- | clazz.getMethod(name, $erasure).invoke(a, $outArgs).asInstanceOf[R]
- |}""".stripMargin
- }.mkString("\n")
- )
- */
-
- // scalastyle:off
- protected implicit class InstanceMagic(a: Any) {
- private val clazz = a.getClass
-
- def call[R](name: String): R = {
- clazz.getMethod(name).invoke(a).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = {
- clazz.getMethod(name, classTag[A1].erasure).invoke(a, a1).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(a, a1, a2).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(a, a1, a2, a3).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(a, a1, a2, a3, a4).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(a, a1, a2, a3, a4, a5).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(a, a1, a2, a3, a4, a5, a6).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R]
- }
- def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = {
- clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R]
- }
- }
-
- /** code for StaticMagic
- println(
- (1 to 22).map { n =>
- def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ")
- val types = repeat(n => s"A$n <: AnyRef : ClassTag")
- val inArgs = repeat(n => s"a$n: A$n")
- val erasure = repeat(n => s"classTag[A$n].erasure")
- val outArgs = repeat(n => s"a$n")
- s"""|def callStatic[$types, R](name: String, $inArgs): R = {
- | c.getDeclaredMethod(name, $erasure).invoke(c, $outArgs).asInstanceOf[R]
- |}""".stripMargin
- }.mkString("\n")
- )
- */
-
- protected implicit class StaticMagic(c: Class[_]) {
- def callStatic[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure).invoke(c, a1).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(c, a1, a2).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(c, a1, a2, a3).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(c, a1, a2, a3, a4).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(c, a1, a2, a3, a4, a5).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(c, a1, a2, a3, a4, a5, a6).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R]
- }
- def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = {
- c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R]
- }
- }
- // scalastyle:on
-}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
index 410d9881ac214..27a3d8f5896cc 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala
@@ -19,15 +19,27 @@ package org.apache.spark.sql.hive
/** Support for interacting with different versions of the HiveMetastoreClient */
package object client {
- private[client] abstract class HiveVersion(val fullVersion: String, val hasBuiltinsJar: Boolean)
+ private[client] abstract class HiveVersion(
+ val fullVersion: String,
+ val extraDeps: Seq[String] = Nil,
+ val exclusions: Seq[String] = Nil)
// scalastyle:off
private[client] object hive {
- case object v10 extends HiveVersion("0.10.0", true)
- case object v11 extends HiveVersion("0.11.0", false)
- case object v12 extends HiveVersion("0.12.0", false)
- case object v13 extends HiveVersion("0.13.1", false)
+ case object v12 extends HiveVersion("0.12.0")
+ case object v13 extends HiveVersion("0.13.1")
+
+ // Hive 0.14 depends on calcite 0.9.2-incubating-SNAPSHOT which does not exist in
+ // maven central anymore, so override those with a version that exists.
+ //
+ // org.pentaho:pentaho-aggdesigner-algorithm is also nowhere to be found, so exclude
+ // it explicitly. If it's needed by the metastore client, users will have to dig it
+ // out of somewhere and use configuration to point Spark at the correct jars.
+ case object v14 extends HiveVersion("0.14.0",
+ Seq("org.apache.calcite:calcite-core:1.3.0-incubating",
+ "org.apache.calcite:calcite-avatica:1.3.0-incubating"),
+ Seq("org.pentaho:pentaho-aggdesigner-algorithm"))
}
// scalastyle:on
-}
\ No newline at end of file
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 7d3ec12c4eb05..0e4a2427a9c15 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution
import org.apache.spark.annotation.Experimental
import org.apache.spark.sql.{AnalysisException, SQLContext}
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan}
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.hive.client.{HiveTable, HiveColumn}
@@ -42,7 +42,7 @@ case class CreateTableAsSelect(
def database: String = tableDesc.database
def tableName: String = tableDesc.name
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
lazy val metastoreRelation: MetastoreRelation = {
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
@@ -50,17 +50,25 @@ case class CreateTableAsSelect(
import org.apache.hadoop.io.Text
import org.apache.hadoop.mapred.TextInputFormat
- val withSchema =
+ val withFormat =
tableDesc.copy(
- schema =
- query.output.map(c =>
- HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null)),
inputFormat =
tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)),
outputFormat =
tableDesc.outputFormat
.orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)),
serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName())))
+
+ val withSchema = if (withFormat.schema.isEmpty) {
+ // Hive doesn't support specifying the column list for target table in CTAS
+ // However we don't think SparkSQL should follow that.
+ tableDesc.copy(schema =
+ query.output.map(c =>
+ HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null)))
+ } else {
+ withFormat
+ }
+
hiveContext.catalog.client.createTable(withSchema)
// Get the Metastore Relation
@@ -81,7 +89,7 @@ case class CreateTableAsSelect(
hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd
}
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
override def argString: String = {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index 6fce69b58b85e..a89381000ad5f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -21,12 +21,10 @@ import scala.collection.JavaConversions._
import org.apache.hadoop.hive.metastore.api.FieldSchema
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
-import org.apache.spark.sql.execution.{SparkPlan, RunnableCommand}
-import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation}
-import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.hive.MetastoreRelation
/**
* Implementation for "describe [extended] table".
@@ -37,7 +35,7 @@ case class DescribeHiveTableCommand(
override val output: Seq[Attribute],
isExtended: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
// Trying to mimic the format of Hive's output. But not exactly the same.
var results: Seq[(String, String, String)] = Nil
@@ -59,7 +57,7 @@ case class DescribeHiveTableCommand(
}
results.map { case (name, dataType, comment) =>
- Row(name, dataType, comment)
+ InternalRow(name, dataType, comment)
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 60a9bb630d0d9..87f8e3f7fcfcc 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -1,34 +1,34 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive.execution
-
-import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
-import org.apache.spark.sql.execution.RunnableCommand
-import org.apache.spark.sql.hive.HiveContext
-import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.types.StringType
-
-private[hive]
-case class HiveNativeCommand(sql: String) extends RunnableCommand {
-
- override def output: Seq[AttributeReference] =
- Seq(AttributeReference("result", StringType, nullable = false)())
-
- override def run(sqlContext: SQLContext): Seq[Row] =
- sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_))
-}
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, InternalRow}
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.hive.HiveContext
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.types.StringType
+
+private[hive]
+case class HiveNativeCommand(sql: String) extends RunnableCommand {
+
+ override def output: Seq[AttributeReference] =
+ Seq(AttributeReference("result", StringType, nullable = false)())
+
+ override def run(sqlContext: SQLContext): Seq[InternalRow] =
+ sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(InternalRow(_))
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
index 11ee5503146b9..1f5e4af2e4746 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
@@ -129,7 +129,7 @@ case class HiveTableScan(
}
}
- protected override def doExecute(): RDD[Row] = if (!relation.hiveQlTable.isPartitioned) {
+ protected override def doExecute(): RDD[InternalRow] = if (!relation.hiveQlTable.isPartitioned) {
hadoopReader.makeRDDForTable(relation.hiveQlTable)
} else {
hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index 8613332186f28..05f425f2b65f3 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -19,26 +19,26 @@ package org.apache.spark.sql.hive.execution
import java.util
-import scala.collection.JavaConversions._
-
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.metastore.MetaStoreUtils
-import org.apache.hadoop.hive.ql.metadata.Hive
import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.ql.{Context, ErrorMsg}
import org.apache.hadoop.hive.serde2.Serializer
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
import org.apache.hadoop.hive.serde2.objectinspector._
-import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf}
+import org.apache.hadoop.mapred.{FileOutputFormat, JobConf}
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
import org.apache.spark.sql.execution.{UnaryNode, SparkPlan}
+import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
import org.apache.spark.sql.hive._
-import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc}
-import org.apache.spark.sql.hive.HiveShim._
-import org.apache.spark.{SerializableWritable, SparkException, TaskContext}
+import org.apache.spark.{SparkException, TaskContext}
+
+import scala.collection.JavaConversions._
+import org.apache.spark.util.SerializableJobConf
private[hive]
case class InsertIntoHiveTable(
@@ -62,10 +62,10 @@ case class InsertIntoHiveTable(
def output: Seq[Attribute] = child.output
def saveAsHiveFile(
- rdd: RDD[Row],
+ rdd: RDD[InternalRow],
valueClass: Class[_],
fileSinkConf: FileSinkDesc,
- conf: SerializableWritable[JobConf],
+ conf: SerializableJobConf,
writerContainer: SparkHiveWriterContainer): Unit = {
assert(valueClass != null, "Output value class not set")
conf.value.setOutputValueClass(valueClass)
@@ -84,7 +84,7 @@ case class InsertIntoHiveTable(
writerContainer.commitJob()
// Note that this function is executed on executor side
- def writeToFile(context: TaskContext, iterator: Iterator[Row]): Unit = {
+ def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = {
val serializer = newSerializer(fileSinkConf.getTableInfo)
val standardOI = ObjectInspectorUtils
.getStandardObjectInspector(
@@ -121,12 +121,12 @@ case class InsertIntoHiveTable(
*
* Note: this is run once and then kept to avoid double insertions.
*/
- protected[sql] lazy val sideEffectResult: Seq[Row] = {
+ protected[sql] lazy val sideEffectResult: Seq[InternalRow] = {
// Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer
// instances within the closure, since Serializer is not serializable while TableDesc is.
val tableDesc = table.tableDesc
val tableLocation = table.hiveQlTable.getDataLocation
- val tmpLocation = HiveShim.getExternalTmpPath(hiveContext, tableLocation)
+ val tmpLocation = hiveContext.getExternalTmpPath(tableLocation.toUri)
val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false)
val isCompressed = sc.hiveconf.getBoolean(
ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal)
@@ -173,7 +173,7 @@ case class InsertIntoHiveTable(
}
val jobConf = new JobConf(sc.hiveconf)
- val jobConfSer = new SerializableWritable(jobConf)
+ val jobConfSer = new SerializableJobConf(jobConf)
val writerContainer = if (numDynamicPartitions > 0) {
val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions)
@@ -198,7 +198,6 @@ case class InsertIntoHiveTable(
table.hiveQlTable.getPartCols().foreach { entry =>
orderedPartitionSpec.put(entry.getName, partitionSpec.get(entry.getName).getOrElse(""))
}
- val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec)
// inheritTableSpecs is set to true. It should be set to false for a IMPORT query
// which is currently considered as a Hive native command.
@@ -252,12 +251,13 @@ case class InsertIntoHiveTable(
// however for now we return an empty list to simplify compatibility checks with hive, which
// does not return anything for insert operations.
// TODO: implement hive compatibility as rules.
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
- override def executeCollect(): Array[Row] = sideEffectResult.toArray
+ override def executeCollect(): Array[Row] =
+ sideEffectResult.toArray
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
sqlContext.sparkContext.parallelize(sideEffectResult, 1)
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
index fd623370cc407..9d8872aa47d1f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.hive.execution
import java.io.{BufferedReader, DataInputStream, DataOutputStream, EOFException, InputStreamReader}
+import java.lang.ProcessBuilder.Redirect
import java.util.Properties
import scala.collection.JavaConversions._
@@ -54,10 +55,16 @@ case class ScriptTransformation(
override def otherCopyArgs: Seq[HiveContext] = sc :: Nil
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().mapPartitions { iter =>
val cmd = List("/bin/bash", "-c", script)
val builder = new ProcessBuilder(cmd)
+ // redirectError(Redirect.INHERIT) would consume the error output from buffer and
+ // then print it to stderr (inherit the target from the current Scala process).
+ // If without this there would be 2 issues:
+ // 1) The error msg generated by the script process would be hidden.
+ // 2) If the error msg is too big to chock up the buffer, the input logic would be hung
+ builder.redirectError(Redirect.INHERIT)
val proc = builder.start()
val inputStream = proc.getInputStream
val outputStream = proc.getOutputStream
@@ -65,8 +72,8 @@ case class ScriptTransformation(
val (outputSerde, outputSoi) = ioschema.initOutputSerDe(output)
- val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors {
- var cacheRow: Row = null
+ val iterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors {
+ var cacheRow: InternalRow = null
var curLine: String = null
var eof: Boolean = false
@@ -83,7 +90,7 @@ case class ScriptTransformation(
}
}
- def deserialize(): Row = {
+ def deserialize(): InternalRow = {
if (cacheRow != null) return cacheRow
val mutableRow = new SpecificMutableRow(output.map(_.dataType))
@@ -113,7 +120,7 @@ case class ScriptTransformation(
}
}
- override def next(): Row = {
+ override def next(): InternalRow = {
if (!hasNext) {
throw new NoSuchElementException
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index 0ba94d7b7c649..aad58bfa2e6e0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.sources._
import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext}
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.hive.HiveContext
@@ -39,9 +39,9 @@ import org.apache.spark.util.Utils
private[hive]
case class AnalyzeTable(tableName: String) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
sqlContext.asInstanceOf[HiveContext].analyze(tableName)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -53,7 +53,7 @@ case class DropTable(
tableName: String,
ifExists: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
val ifExistsClause = if (ifExists) "IF EXISTS " else ""
try {
@@ -70,7 +70,7 @@ case class DropTable(
hiveContext.invalidateTable(tableName)
hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")
hiveContext.catalog.unregisterTable(Seq(tableName))
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -83,7 +83,7 @@ case class AddJar(path: String) extends RunnableCommand {
schema.toAttributes
}
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
val currentClassLoader = Utils.getContextOrSparkClassLoader
@@ -91,26 +91,32 @@ case class AddJar(path: String) extends RunnableCommand {
val jarURL = new java.io.File(path).toURL
val newClassLoader = new java.net.URLClassLoader(Array(jarURL), currentClassLoader)
Thread.currentThread.setContextClassLoader(newClassLoader)
- org.apache.hadoop.hive.ql.metadata.Hive.get().getConf().setClassLoader(newClassLoader)
-
- // Add jar to isolated hive classloader
+ // We need to explicitly set the class loader associated with the conf in executionHive's
+ // state because this class loader will be used as the context class loader of the current
+ // thread to execute any Hive command.
+ // We cannot use `org.apache.hadoop.hive.ql.metadata.Hive.get().getConf()` because Hive.get()
+ // returns the value of a thread local variable and its HiveConf may not be the HiveConf
+ // associated with `executionHive.state` (for example, HiveContext is created in one thread
+ // and then add jar is called from another thread).
+ hiveContext.executionHive.state.getConf.setClassLoader(newClassLoader)
+ // Add jar to isolated hive (metadataHive) class loader.
hiveContext.runSqlHive(s"ADD JAR $path")
// Add jar to executors
hiveContext.sparkContext.addJar(path)
- Seq(Row(0))
+ Seq(InternalRow(0))
}
}
private[hive]
case class AddFile(path: String) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
hiveContext.runSqlHive(s"ADD FILE $path")
hiveContext.sparkContext.addFile(path)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -123,12 +129,12 @@ case class CreateMetastoreDataSource(
allowExisting: Boolean,
managedIfNoPath: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
if (hiveContext.catalog.tableExists(tableName :: Nil)) {
if (allowExisting) {
- return Seq.empty[Row]
+ return Seq.empty[InternalRow]
} else {
throw new AnalysisException(s"Table $tableName already exists.")
}
@@ -151,7 +157,7 @@ case class CreateMetastoreDataSource(
optionsWithPath,
isExternal)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -164,7 +170,7 @@ case class CreateMetastoreDataSourceAsSelect(
options: Map[String, String],
query: LogicalPlan) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
var createMetastoreTable = false
var isExternal = true
@@ -188,7 +194,7 @@ case class CreateMetastoreDataSourceAsSelect(
s"Or, if you are using SQL CREATE TABLE, you need to drop $tableName first.")
case SaveMode.Ignore =>
// Since the table already exists and the save mode is Ignore, we will just return.
- return Seq.empty[Row]
+ return Seq.empty[InternalRow]
case SaveMode.Append =>
// Check if the specified data source match the data source of the existing table.
val resolved = ResolvedDataSource(
@@ -230,7 +236,7 @@ case class CreateMetastoreDataSourceAsSelect(
val data = DataFrame(hiveContext, query)
val df = existingSchema match {
// If we are inserting into an existing table, just use the existing schema.
- case Some(schema) => sqlContext.createDataFrame(data.queryExecution.toRdd, schema)
+ case Some(schema) => sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, schema)
case None => data
}
@@ -253,6 +259,6 @@ case class CreateMetastoreDataSourceAsSelect(
// Refresh the cache of the table in the catalog.
hiveContext.refreshTable(tableName)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 1658bb93b0b79..4986b1ea9d906 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -17,11 +17,9 @@
package org.apache.spark.sql.hive
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
-import org.apache.spark.sql.AnalysisException
-
import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConversions._
+import scala.util.Try
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector}
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions
@@ -30,47 +28,55 @@ import org.apache.hadoop.hive.ql.exec._
import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType}
import org.apache.hadoop.hive.ql.udf.generic._
import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
import org.apache.spark.Logging
+import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.hive.HiveShim._
import org.apache.spark.sql.types._
-/* Implicit conversions */
-import scala.collection.JavaConversions._
-private[hive] abstract class HiveFunctionRegistry
+private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
extends analysis.FunctionRegistry with HiveInspectors {
def getFunctionInfo(name: String): FunctionInfo = FunctionRegistry.getFunctionInfo(name)
- def lookupFunction(name: String, children: Seq[Expression]): Expression = {
- // We only look it up to see if it exists, but do not include it in the HiveUDF since it is
- // not always serializable.
- val functionInfo: FunctionInfo =
- Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse(
- sys.error(s"Couldn't find function $name"))
-
- val functionClassName = functionInfo.getFunctionClass.getName
-
- if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveSimpleUdf(new HiveFunctionWrapper(functionClassName), children)
- } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdf(new HiveFunctionWrapper(functionClassName), children)
- } else if (
- classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdaf(new HiveFunctionWrapper(functionClassName), children)
- } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveUdaf(new HiveFunctionWrapper(functionClassName), children)
- } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), children)
- } else {
- sys.error(s"No handler for udf ${functionInfo.getFunctionClass}")
+ override def lookupFunction(name: String, children: Seq[Expression]): Expression = {
+ Try(underlying.lookupFunction(name, children)).getOrElse {
+ // We only look it up to see if it exists, but do not include it in the HiveUDF since it is
+ // not always serializable.
+ val functionInfo: FunctionInfo =
+ Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse(
+ throw new AnalysisException(s"undefined function $name"))
+
+ val functionClassName = functionInfo.getFunctionClass.getName
+
+ if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
+ HiveSimpleUdf(new HiveFunctionWrapper(functionClassName), children)
+ } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
+ HiveGenericUdf(new HiveFunctionWrapper(functionClassName), children)
+ } else if (
+ classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) {
+ HiveGenericUdaf(new HiveFunctionWrapper(functionClassName), children)
+ } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) {
+ HiveUdaf(new HiveFunctionWrapper(functionClassName), children)
+ } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) {
+ HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), children)
+ } else {
+ sys.error(s"No handler for udf ${functionInfo.getFunctionClass}")
+ }
}
}
+
+ override def registerFunction(name: String, builder: FunctionBuilder): Unit =
+ throw new UnsupportedOperationException
}
private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
@@ -114,8 +120,10 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre
@transient
protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length)
+ override def isThreadSafe: Boolean = false
+
// TODO: Finish input output types.
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
unwrap(
FunctionRegistry.invoke(method, function, conversionHelper
.convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*),
@@ -172,7 +180,9 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr
lazy val dataType: DataType = inspectorToDataType(returnInspector)
- override def eval(input: Row): Any = {
+ override def isThreadSafe: Boolean = false
+
+ override def eval(input: InternalRow): Any = {
returnInspector // Make sure initialized.
var i = 0
@@ -339,7 +349,7 @@ private[hive] case class HiveWindowFunction(
def nullable: Boolean = true
- override def eval(input: Row): Any =
+ override def eval(input: InternalRow): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
@transient
@@ -363,7 +373,7 @@ private[hive] case class HiveWindowFunction(
evaluator.reset(hiveEvaluatorBuffer)
}
- override def prepareInputParameters(input: Row): AnyRef = {
+ override def prepareInputParameters(input: InternalRow): AnyRef = {
wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length))
}
// Add input parameters for a single row.
@@ -506,7 +516,7 @@ private[hive] case class HiveGenericUdtf(
field => (inspectorToDataType(field.getFieldObjectInspector), true)
}
- override def eval(input: Row): TraversableOnce[Row] = {
+ override def eval(input: InternalRow): TraversableOnce[InternalRow] = {
outputInspector // Make sure initialized.
val inputProjection = new InterpretedProjection(children)
@@ -516,23 +526,23 @@ private[hive] case class HiveGenericUdtf(
}
protected class UDTFCollector extends Collector {
- var collected = new ArrayBuffer[Row]
+ var collected = new ArrayBuffer[InternalRow]
override def collect(input: java.lang.Object) {
// We need to clone the input here because implementations of
// GenericUDTF reuse the same object. Luckily they are always an array, so
// it is easy to clone.
- collected += unwrap(input, outputInspector).asInstanceOf[Row]
+ collected += unwrap(input, outputInspector).asInstanceOf[InternalRow]
}
- def collectRows(): Seq[Row] = {
+ def collectRows(): Seq[InternalRow] = {
val toCollect = collected
- collected = new ArrayBuffer[Row]
+ collected = new ArrayBuffer[InternalRow]
toCollect
}
}
- override def terminate(): TraversableOnce[Row] = {
+ override def terminate(): TraversableOnce[InternalRow] = {
outputInspector // Make sure initialized.
function.close()
collector.collectRows()
@@ -572,7 +582,7 @@ private[hive] case class HiveUdafFunction(
private val buffer =
function.getNewAggregationBuffer
- override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector)
+ override def eval(input: InternalRow): Any = unwrap(function.evaluate(buffer), returnInspector)
@transient
val inputProjection = new InterpretedProjection(exprs)
@@ -580,7 +590,7 @@ private[hive] case class HiveUdafFunction(
@transient
protected lazy val cached = new Array[AnyRef](exprs.length)
- def update(input: Row): Unit = {
+ def update(input: InternalRow): Unit = {
val inputs = inputProjection(input)
function.iterate(buffer, wrap(inputs, inspectors, cached))
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
index 2bb526b14be34..ab75b12e2a2e7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
@@ -34,10 +34,10 @@ import org.apache.hadoop.hive.common.FileUtils
import org.apache.spark.mapred.SparkHadoopMapRedUtil
import org.apache.spark.sql.Row
import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter}
-import org.apache.spark.sql.catalyst.util.DateUtils
-import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc}
-import org.apache.spark.sql.hive.HiveShim._
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
import org.apache.spark.sql.types._
+import org.apache.spark.util.SerializableJobConf
/**
* Internal helper class that saves an RDD using a Hive OutputFormat.
@@ -58,7 +58,7 @@ private[hive] class SparkHiveWriterContainer(
PlanUtils.configureOutputJobPropertiesForStorageHandler(tableDesc)
Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf)
}
- protected val conf = new SerializableWritable(jobConf)
+ protected val conf = new SerializableJobConf(jobConf)
private var jobID = 0
private var splitID = 0
@@ -201,7 +201,7 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer(
def convertToHiveRawString(col: String, value: Any): String = {
val raw = String.valueOf(value)
schema(col).dataType match {
- case DateType => DateUtils.toString(raw.toInt)
+ case DateType => DateTimeUtils.toString(raw.toInt)
case _: DecimalType => BigDecimal(raw).toString()
case _ => raw
}
@@ -228,12 +228,11 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer(
newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec)
newFileSinkDesc.setCompressType(fileSinkConf.getCompressType)
- val path = {
- val outputPath = FileOutputFormat.getOutputPath(conf.value)
- assert(outputPath != null, "Undefined job output-path")
- val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/"))
- new Path(workPath, getOutputName)
- }
+ // use the path like ${hive_tmp}/_temporary/${attemptId}/
+ // to avoid write to the same file when `spark.speculation=true`
+ val path = FileOutputFormat.getTaskOutputPath(
+ conf.value,
+ dynamicPartPath.stripPrefix("/") + "/" + getOutputName)
HiveFileFormatUtils.getHiveRecordWriter(
conf.value,
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
index 1e51173a19882..e3ab9442b4821 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala
@@ -27,13 +27,13 @@ import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.sql.hive.HiveMetastoreTypes
import org.apache.spark.sql.types.StructType
-private[orc] object OrcFileOperator extends Logging{
+private[orc] object OrcFileOperator extends Logging {
def getFileReader(pathStr: String, config: Option[Configuration] = None ): Reader = {
val conf = config.getOrElse(new Configuration)
val fspath = new Path(pathStr)
val fs = fspath.getFileSystem(conf)
val orcFiles = listOrcFiles(pathStr, conf)
-
+ logDebug(s"Creating ORC Reader from ${orcFiles.head}")
// TODO Need to consider all files when schema evolution is taken into account.
OrcFile.createReader(fs, orcFiles.head)
}
@@ -42,6 +42,7 @@ private[orc] object OrcFileOperator extends Logging{
val reader = getFileReader(path, conf)
val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector]
val schema = readerInspector.getTypeName
+ logDebug(s"Reading schema from file $path, got Hive schema string: $schema")
HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType]
}
@@ -52,14 +53,14 @@ private[orc] object OrcFileOperator extends Logging{
def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = {
val origPath = new Path(pathStr)
val fs = origPath.getFileSystem(conf)
- val path = origPath.makeQualified(fs)
+ val path = origPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
val paths = SparkHadoopUtil.get.listLeafStatuses(fs, origPath)
.filterNot(_.isDir)
.map(_.getPath)
.filterNot(_.getName.startsWith("_"))
.filterNot(_.getName.startsWith("."))
- if (paths == null || paths.size == 0) {
+ if (paths == null || paths.isEmpty) {
throw new IllegalArgumentException(
s"orcFileOperator: path $path does not have valid orc files matching the pattern")
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
index f03c4cd54e7e6..705f48f1cd9f0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapred.{InputFormat => MapRedInputFormat, JobConf, Reco
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
+import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.mapred.SparkHadoopMapRedUtil
import org.apache.spark.rdd.{HadoopRDD, RDD}
@@ -39,7 +40,7 @@ import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveMetastoreType
import org.apache.spark.sql.sources.{Filter, _}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.{Logging, SerializableWritable}
+import org.apache.spark.util.SerializableConfiguration
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -104,13 +105,14 @@ private[orc] class OrcOutputWriter(
recordWriterInstantiated = true
val conf = context.getConfiguration
+ val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID")
val partition = context.getTaskAttemptID.getTaskID.getId
- val filename = f"part-r-$partition%05d-${System.currentTimeMillis}%015d.orc"
+ val filename = f"part-r-$partition%05d-$uniqueWriteJobId.orc"
new OrcOutputFormat().getRecordWriter(
new Path(path, filename).getFileSystem(conf),
conf.asInstanceOf[JobConf],
- new Path(path, filename).toUri.getPath,
+ new Path(path, filename).toString,
Reporter.NULL
).asInstanceOf[RecordWriter[NullWritable, Writable]]
}
@@ -283,7 +285,7 @@ private[orc] case class OrcTableScan(
classOf[Writable]
).asInstanceOf[HadoopRDD[NullWritable, Writable]]
- val wrappedConf = new SerializableWritable(conf)
+ val wrappedConf = new SerializableConfiguration(conf)
rdd.mapPartitionsWithInputSplit { case (split: OrcSplit, iterator) =>
val mutableRow = new SpecificMutableRow(attributes.map(_.dataType))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 7c7afc824d7a6..ea325cc93cb85 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -49,7 +49,7 @@ import scala.collection.JavaConversions._
object TestHive
extends TestHiveContext(
new SparkContext(
- "local[2]",
+ System.getProperty("spark.sql.test.master", "local[32]"),
"TestSQLContext",
new SparkConf()
.set("spark.sql.test", "")
@@ -112,12 +112,11 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
protected[hive] class SQLSession extends super.SQLSession {
/** Fewer partitions to speed up testing. */
protected[sql] override lazy val conf: SQLConf = new SQLConf {
- override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt
+ override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, 5)
// TODO as in unit test, conf.clear() probably be called, all of the value will be cleared.
// The super.getConf(SQLConf.DIALECT) is "sql" by default, we need to set it as "hiveql"
override def dialect: String = super.getConf(SQLConf.DIALECT, "hiveql")
- override def caseSensitiveAnalysis: Boolean =
- getConf(SQLConf.CASE_SENSITIVE, "false").toBoolean
+ override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false)
}
}
diff --git a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247
index 27de46fdf22ac..84a31a5a6970b 100644
--- a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247
+++ b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247
@@ -1 +1 @@
--0.0010000000000000009
+-0.001
diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609
new file mode 100644
index 0000000000000..7e5fceeddeeeb
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609
@@ -0,0 +1,97 @@
+Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 2 66619.10876874991 0.811328754177887 2801.7074999999995
+Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 6 66619.10876874991 0.811328754177887 2801.7074999999995
+Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 34 66619.10876874991 0.811328754177887 2801.7074999999995
+Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 2 74912.8826888888 1.0 4128.782222222221
+Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 34 74912.8826888888 1.0 4128.782222222221
+Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 2 53315.51002399992 0.695639377397664 2210.7864
+Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 6 53315.51002399992 0.695639377397664 2210.7864
+Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 28 53315.51002399992 0.695639377397664 2210.7864
+Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 34 53315.51002399992 0.695639377397664 2210.7864
+Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 2 41099.896184 0.630785977101214 2009.9536000000007
+Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 6 41099.896184 0.630785977101214 2009.9536000000007
+Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 28 41099.896184 0.630785977101214 2009.9536000000007
+Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 34 41099.896184 0.630785977101214 2009.9536000000007
+Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 42 41099.896184 0.630785977101214 2009.9536000000007
+Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 6 14788.129118750014 0.2036684720435979 331.1337500000004
+Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 28 14788.129118750014 0.2036684720435979 331.1337500000004
+Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 34 14788.129118750014 0.2036684720435979 331.1337500000004
+Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 42 14788.129118750014 0.2036684720435979 331.1337500000004
+Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 6 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502
+Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 28 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502
+Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 42 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502
+Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 2 20231.169866666663 -0.49369526554523185 -1113.7466666666658
+Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 14 20231.169866666663 -0.49369526554523185 -1113.7466666666658
+Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 40 20231.169866666663 -0.49369526554523185 -1113.7466666666658
+Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 2 18978.662075 -0.5205630897335946 -1004.4812499999995
+Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 14 18978.662075 -0.5205630897335946 -1004.4812499999995
+Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 25 18978.662075 -0.5205630897335946 -1004.4812499999995
+Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 40 18978.662075 -0.5205630897335946 -1004.4812499999995
+Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 2 16910.329504000005 -0.46908967495720255 -766.1791999999995
+Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 14 16910.329504000005 -0.46908967495720255 -766.1791999999995
+Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 18 16910.329504000005 -0.46908967495720255 -766.1791999999995
+Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 25 16910.329504000005 -0.46908967495720255 -766.1791999999995
+Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 40 16910.329504000005 -0.46908967495720255 -766.1791999999995
+Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 2 18374.07627499999 -0.6091405874714462 -1128.1787499999987
+Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 18 18374.07627499999 -0.6091405874714462 -1128.1787499999987
+Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 25 18374.07627499999 -0.6091405874714462 -1128.1787499999987
+Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 40 18374.07627499999 -0.6091405874714462 -1128.1787499999987
+Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 2 24473.534488888927 -0.9571686373491608 -1441.4466666666676
+Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 18 24473.534488888927 -0.9571686373491608 -1441.4466666666676
+Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 25 24473.534488888927 -0.9571686373491608 -1441.4466666666676
+Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 14 38720.09628888887 0.5557168646224995 224.6944444444446
+Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 17 38720.09628888887 0.5557168646224995 224.6944444444446
+Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 19 38720.09628888887 0.5557168646224995 224.6944444444446
+Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 1 75702.81305 -0.6720833036576083 -1296.9000000000003
+Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 14 75702.81305 -0.6720833036576083 -1296.9000000000003
+Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 17 75702.81305 -0.6720833036576083 -1296.9000000000003
+Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 19 75702.81305 -0.6720833036576083 -1296.9000000000003
+Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 1 67722.117896 -0.5703526513979519 -2129.0664
+Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 14 67722.117896 -0.5703526513979519 -2129.0664
+Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 17 67722.117896 -0.5703526513979519 -2129.0664
+Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 19 67722.117896 -0.5703526513979519 -2129.0664
+Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 45 67722.117896 -0.5703526513979519 -2129.0664
+Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 1 76128.53331875012 -0.577476899644802 -2547.7868749999993
+Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 14 76128.53331875012 -0.577476899644802 -2547.7868749999993
+Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 19 76128.53331875012 -0.577476899644802 -2547.7868749999993
+Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 45 76128.53331875012 -0.577476899644802 -2547.7868749999993
+Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 1 67902.76602222225 -0.8710736366736884 -4099.731111111111
+Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 19 67902.76602222225 -0.8710736366736884 -4099.731111111111
+Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 45 67902.76602222225 -0.8710736366736884 -4099.731111111111
+Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 10 28944.25735555559 -0.6656975320098423 -1347.4777777777779
+Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 27 28944.25735555559 -0.6656975320098423 -1347.4777777777779
+Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 39 28944.25735555559 -0.6656975320098423 -1347.4777777777779
+Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 7 58693.95151875002 -0.8051852719193339 -2537.328125
+Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 10 58693.95151875002 -0.8051852719193339 -2537.328125
+Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 27 58693.95151875002 -0.8051852719193339 -2537.328125
+Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 39 58693.95151875002 -0.8051852719193339 -2537.328125
+Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 7 54802.817784000035 -0.6046935574240581 -1719.8079999999995
+Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 10 54802.817784000035 -0.6046935574240581 -1719.8079999999995
+Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 12 54802.817784000035 -0.6046935574240581 -1719.8079999999995
+Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 27 54802.817784000035 -0.6046935574240581 -1719.8079999999995
+Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 39 54802.817784000035 -0.6046935574240581 -1719.8079999999995
+Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 7 61174.24181875003 -0.5508665654707869 -1719.0368749999975
+Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 12 61174.24181875003 -0.5508665654707869 -1719.0368749999975
+Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 27 61174.24181875003 -0.5508665654707869 -1719.0368749999975
+Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 39 61174.24181875003 -0.5508665654707869 -1719.0368749999975
+Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 7 80278.40095555557 -0.7755740084632333 -1867.4888888888881
+Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 12 80278.40095555557 -0.7755740084632333 -1867.4888888888881
+Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 27 80278.40095555557 -0.7755740084632333 -1867.4888888888881
+Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 2 7005.487488888913 0.39004303087285047 418.9233333333353
+Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 6 7005.487488888913 0.39004303087285047 418.9233333333353
+Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 31 7005.487488888913 0.39004303087285047 418.9233333333353
+Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 2 100286.53662500004 -0.713612911776183 -4090.853749999999
+Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 6 100286.53662500004 -0.713612911776183 -4090.853749999999
+Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 31 100286.53662500004 -0.713612911776183 -4090.853749999999
+Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 46 100286.53662500004 -0.713612911776183 -4090.853749999999
+Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 2 81456.04997600002 -0.712858514567818 -3297.2011999999986
+Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 6 81456.04997600002 -0.712858514567818 -3297.2011999999986
+Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 23 81456.04997600002 -0.712858514567818 -3297.2011999999986
+Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 31 81456.04997600002 -0.712858514567818 -3297.2011999999986
+Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 46 81456.04997600002 -0.712858514567818 -3297.2011999999986
+Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 2 81474.56091875004 -0.984128787153391 -4871.028125000002
+Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 6 81474.56091875004 -0.984128787153391 -4871.028125000002
+Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 23 81474.56091875004 -0.984128787153391 -4871.028125000002
+Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 46 81474.56091875004 -0.984128787153391 -4871.028125000002
+Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 2 99807.08486666664 -0.9978877469246936 -5664.856666666666
+Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 23 99807.08486666664 -0.9978877469246936 -5664.856666666666
+Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 46 99807.08486666664 -0.9978877469246936 -5664.856666666666
diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838
deleted file mode 100644
index 1f7e8a5d67036..0000000000000
--- a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-da0e0cca69e42118a96b8609b8fa5838
+++ /dev/null
@@ -1,26 +0,0 @@
-Manufacturer#1 almond antique burnished rose metallic 2 273.70217881648074 273.70217881648074 [34,2] 74912.8826888888 1.0 4128.782222222221
-Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 [34,2,6] 66619.10876874991 0.811328754177887 2801.7074999999995
-Manufacturer#1 almond antique chartreuse lavender yellow 34 230.90151585470358 230.90151585470358 [34,2,6,28] 53315.51002399992 0.695639377397664 2210.7864
-Manufacturer#1 almond antique salmon chartreuse burlywood 6 202.73109328368946 202.73109328368946 [34,2,6,42,28] 41099.896184 0.630785977101214 2009.9536000000007
-Manufacturer#1 almond aquamarine burnished black steel 28 121.6064517973862 121.6064517973862 [34,6,42,28] 14788.129118750014 0.2036684720435979 331.1337500000004
-Manufacturer#1 almond aquamarine pink moccasin thistle 42 96.5751586416853 96.5751586416853 [6,42,28] 9326.761266666683 -1.4442181184933883E-4 -0.20666666666708502
-Manufacturer#2 almond antique violet chocolate turquoise 14 142.2363169751898 142.2363169751898 [2,40,14] 20231.169866666663 -0.49369526554523185 -1113.7466666666658
-Manufacturer#2 almond antique violet turquoise frosted 40 137.76306498840682 137.76306498840682 [2,25,40,14] 18978.662075 -0.5205630897335946 -1004.4812499999995
-Manufacturer#2 almond aquamarine midnight light salmon 2 130.03972279269132 130.03972279269132 [2,18,25,40,14] 16910.329504000005 -0.46908967495720255 -766.1791999999995
-Manufacturer#2 almond aquamarine rose maroon antique 25 135.55100986344584 135.55100986344584 [2,18,25,40] 18374.07627499999 -0.6091405874714462 -1128.1787499999987
-Manufacturer#2 almond aquamarine sandy cyan gainsboro 18 156.44019460768044 156.44019460768044 [2,18,25] 24473.534488888927 -0.9571686373491608 -1441.4466666666676
-Manufacturer#3 almond antique chartreuse khaki white 17 196.7742266885805 196.7742266885805 [17,19,14] 38720.09628888887 0.5557168646224995 224.6944444444446
-Manufacturer#3 almond antique forest lavender goldenrod 14 275.14144189852607 275.14144189852607 [17,1,19,14] 75702.81305 -0.6720833036576083 -1296.9000000000003
-Manufacturer#3 almond antique metallic orange dim 19 260.23473614412046 260.23473614412046 [17,1,19,14,45] 67722.117896 -0.5703526513979519 -2129.0664
-Manufacturer#3 almond antique misty red olive 1 275.9139962356932 275.9139962356932 [1,19,14,45] 76128.53331875012 -0.577476899644802 -2547.7868749999993
-Manufacturer#3 almond antique olive coral navajo 45 260.5815918713796 260.5815918713796 [1,19,45] 67902.76602222225 -0.8710736366736884 -4099.731111111111
-Manufacturer#4 almond antique gainsboro frosted violet 10 170.13011889596618 170.13011889596618 [39,27,10] 28944.25735555559 -0.6656975320098423 -1347.4777777777779
-Manufacturer#4 almond antique violet mint lemon 39 242.26834609323197 242.26834609323197 [39,7,27,10] 58693.95151875002 -0.8051852719193339 -2537.328125
-Manufacturer#4 almond aquamarine floral ivory bisque 27 234.10001662537326 234.10001662537326 [39,7,27,10,12] 54802.817784000035 -0.6046935574240581 -1719.8079999999995
-Manufacturer#4 almond aquamarine yellow dodger mint 7 247.3342714197732 247.3342714197732 [39,7,27,12] 61174.24181875003 -0.5508665654707869 -1719.0368749999975
-Manufacturer#4 almond azure aquamarine papaya violet 12 283.3344330566893 283.3344330566893 [7,27,12] 80278.40095555557 -0.7755740084632333 -1867.4888888888881
-Manufacturer#5 almond antique blue firebrick mint 31 83.69879024746363 83.69879024746363 [2,6,31] 7005.487488888913 0.39004303087285047 418.9233333333353
-Manufacturer#5 almond antique medium spring khaki 6 316.68049612345885 316.68049612345885 [2,6,46,31] 100286.53662500004 -0.713612911776183 -4090.853749999999
-Manufacturer#5 almond antique sky peru orange 2 285.40506298242155 285.40506298242155 [2,23,6,46,31] 81456.04997600002 -0.712858514567818 -3297.2011999999986
-Manufacturer#5 almond aquamarine dodger light gainsboro 46 285.43749038756283 285.43749038756283 [2,23,6,46] 81474.56091875004 -0.984128787153391 -4871.028125000002
-Manufacturer#5 almond azure blanched chiffon midnight 23 315.9225931564038 315.9225931564038 [2,23,46] 99807.08486666664 -0.9978877469246936 -5664.856666666666
diff --git a/sql/hive/src/test/resources/hive-contrib-0.13.1.jar b/sql/hive/src/test/resources/hive-contrib-0.13.1.jar
new file mode 100644
index 0000000000000..ce0740d9245a7
Binary files /dev/null and b/sql/hive/src/test/resources/hive-contrib-0.13.1.jar differ
diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala
new file mode 100644
index 0000000000000..e1715177e3f1b
--- /dev/null
+++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.hive.HiveContext
+
+/**
+ * Entry point in test application for SPARK-8489.
+ *
+ * This file is not meant to be compiled during tests. It is already included
+ * in a pre-built "test.jar" located in the same directory as this file.
+ * This is included here for reference only and should NOT be modified without
+ * rebuilding the test jar itself.
+ *
+ * This is used in org.apache.spark.sql.hive.HiveSparkSubmitSuite.
+ */
+object Main {
+ def main(args: Array[String]) {
+ println("Running regression test for SPARK-8489.")
+ val sc = new SparkContext("local", "testing")
+ val hc = new HiveContext(sc)
+ // This line should not throw scala.reflect.internal.MissingRequirementError.
+ // See SPARK-8470 for more detail.
+ val df = hc.createDataFrame(Seq(MyCoolClass("1", "2", "3")))
+ df.collect()
+ println("Regression test for SPARK-8489 success!")
+ }
+}
+
diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/MyCoolClass.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/MyCoolClass.scala
new file mode 100644
index 0000000000000..b1681745c2ef7
--- /dev/null
+++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/MyCoolClass.scala
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Dummy class used in regression test SPARK-8489. */
+case class MyCoolClass(past: String, present: String, future: String)
+
diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar
new file mode 100644
index 0000000000000..4f59fba9eab55
Binary files /dev/null and b/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar differ
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
index df137e7b2b333..aff0456b37ed5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
@@ -28,8 +28,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
import org.apache.hadoop.io.LongWritable
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.expressions.{Literal, Row}
+import org.apache.spark.sql.catalyst.expressions.{Literal, InternalRow}
import org.apache.spark.sql.types._
+import org.apache.spark.sql.Row
class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
test("Test wrap SettableStructObjectInspector") {
@@ -45,7 +46,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
classOf[UDAFPercentile.State],
ObjectInspectorOptions.JAVA).asInstanceOf[StructObjectInspector]
- val a = unwrap(state, soi).asInstanceOf[Row]
+ val a = unwrap(state, soi).asInstanceOf[InternalRow]
val b = wrap(a, soi).asInstanceOf[UDAFPercentile.State]
val sfCounts = soi.getStructFieldRef("counts")
@@ -127,7 +128,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
}
}
- def checkValues(row1: Seq[Any], row2: Row): Unit = {
+ def checkValues(row1: Seq[Any], row2: InternalRow): Unit = {
row1.zip(row2.toSeq).foreach { case (r1, r2) =>
checkValue(r1, r2)
}
@@ -203,7 +204,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
})
checkValues(row,
- unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row])
+ unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[InternalRow])
checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt)))
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
index 5a5ea10e3c82e..af68615e8e9d6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
@@ -17,10 +17,9 @@
package org.apache.spark.sql.hive
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.parquet.ParquetTest
-import org.apache.spark.sql.{QueryTest, SQLConf}
+import org.apache.spark.sql.{QueryTest, Row, SQLConf}
case class Cases(lower: String, UPPER: String)
@@ -82,11 +81,11 @@ class HiveParquetSuite extends QueryTest with ParquetTest {
}
}
- withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") {
+ withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "true") {
run("Parquet data source enabled")
}
- withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") {
+ withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "false") {
run("Parquet data source disabled")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
new file mode 100644
index 0000000000000..d85516ab0878e
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.io.File
+
+import scala.sys.process.{ProcessLogger, Process}
+
+import org.apache.spark._
+import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext}
+import org.apache.spark.util.{ResetSystemProperties, Utils}
+import org.scalatest.Matchers
+import org.scalatest.concurrent.Timeouts
+import org.scalatest.time.SpanSugar._
+
+/**
+ * This suite tests spark-submit with applications using HiveContext.
+ */
+class HiveSparkSubmitSuite
+ extends SparkFunSuite
+ with Matchers
+ with ResetSystemProperties
+ with Timeouts {
+
+ // TODO: rewrite these or mark them as slow tests to be run sparingly
+
+ def beforeAll() {
+ System.setProperty("spark.testing", "true")
+ }
+
+ test("SPARK-8368: includes jars passed in through --jars") {
+ val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
+ val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA"))
+ val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB"))
+ val jar3 = TestHive.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath()
+ val jar4 = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath()
+ val jarsString = Seq(jar1, jar2, jar3, jar4).map(j => j.toString).mkString(",")
+ val args = Seq(
+ "--class", SparkSubmitClassLoaderTest.getClass.getName.stripSuffix("$"),
+ "--name", "SparkSubmitClassLoaderTest",
+ "--master", "local-cluster[2,1,512]",
+ "--jars", jarsString,
+ unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB")
+ runSparkSubmit(args)
+ }
+
+ test("SPARK-8020: set sql conf in spark conf") {
+ val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
+ val args = Seq(
+ "--class", SparkSQLConfTest.getClass.getName.stripSuffix("$"),
+ "--name", "SparkSQLConfTest",
+ "--master", "local-cluster[2,1,512]",
+ unusedJar.toString)
+ runSparkSubmit(args)
+ }
+
+ test("SPARK-8489: MissingRequirementError during reflection") {
+ // This test uses a pre-built jar to test SPARK-8489. In a nutshell, this test creates
+ // a HiveContext and uses it to create a data frame from an RDD using reflection.
+ // Before the fix in SPARK-8470, this results in a MissingRequirementError because
+ // the HiveContext code mistakenly overrides the class loader that contains user classes.
+ // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala.
+ val testJar = "sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar"
+ val args = Seq("--class", "Main", testJar)
+ runSparkSubmit(args)
+ }
+
+ // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
+ // This is copied from org.apache.spark.deploy.SparkSubmitSuite
+ private def runSparkSubmit(args: Seq[String]): Unit = {
+ val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+ val process = Process(
+ Seq("./bin/spark-submit") ++ args,
+ new File(sparkHome),
+ "SPARK_TESTING" -> "1",
+ "SPARK_HOME" -> sparkHome
+ ).run(ProcessLogger(
+ (line: String) => { println(s"out> $line") },
+ (line: String) => { println(s"err> $line") }
+ ))
+
+ try {
+ val exitCode = failAfter(120 seconds) { process.exitValue() }
+ if (exitCode != 0) {
+ fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.")
+ }
+ } finally {
+ // Ensure we still kill the process in case it timed out
+ process.destroy()
+ }
+ }
+}
+
+// This object is used for testing SPARK-8368: https://issues.apache.org/jira/browse/SPARK-8368.
+// We test if we can load user jars in both driver and executors when HiveContext is used.
+object SparkSubmitClassLoaderTest extends Logging {
+ def main(args: Array[String]) {
+ Utils.configTestLog4j("INFO")
+ val conf = new SparkConf()
+ val sc = new SparkContext(conf)
+ val hiveContext = new TestHiveContext(sc)
+ val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j")
+ // First, we load classes at driver side.
+ try {
+ Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+ } catch {
+ case t: Throwable =>
+ throw new Exception("Could not load user class from jar:\n", t)
+ }
+ // Second, we load classes at the executor side.
+ val result = df.mapPartitions { x =>
+ var exception: String = null
+ try {
+ Class.forName(args(0), true, Thread.currentThread().getContextClassLoader)
+ Class.forName(args(1), true, Thread.currentThread().getContextClassLoader)
+ } catch {
+ case t: Throwable =>
+ exception = t + "\n" + t.getStackTraceString
+ exception = exception.replaceAll("\n", "\n\t")
+ }
+ Option(exception).toSeq.iterator
+ }.collect()
+ if (result.nonEmpty) {
+ throw new Exception("Could not load user class from jar:\n" + result(0))
+ }
+
+ // Load a Hive UDF from the jar.
+ hiveContext.sql(
+ """
+ |CREATE TEMPORARY FUNCTION example_max
+ |AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax'
+ """.stripMargin)
+ val source =
+ hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val")
+ source.registerTempTable("sourceTable")
+ // Load a Hive SerDe from the jar.
+ hiveContext.sql(
+ """
+ |CREATE TABLE t1(key int, val string)
+ |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'
+ """.stripMargin)
+ // Actually use the loaded UDF and SerDe.
+ hiveContext.sql(
+ "INSERT INTO TABLE t1 SELECT example_max(key) as key, val FROM sourceTable GROUP BY val")
+ val count = hiveContext.table("t1").orderBy("key", "val").count()
+ if (count != 10) {
+ throw new Exception(s"table t1 should have 10 rows instead of $count rows")
+ }
+ }
+}
+
+// This object is used for testing SPARK-8020: https://issues.apache.org/jira/browse/SPARK-8020.
+// We test if we can correctly set spark sql configurations when HiveContext is used.
+object SparkSQLConfTest extends Logging {
+ def main(args: Array[String]) {
+ Utils.configTestLog4j("INFO")
+ // We override the SparkConf to add spark.sql.hive.metastore.version and
+ // spark.sql.hive.metastore.jars to the beginning of the conf entry array.
+ // So, if metadataHive get initialized after we set spark.sql.hive.metastore.version but
+ // before spark.sql.hive.metastore.jars get set, we will see the following exception:
+ // Exception in thread "main" java.lang.IllegalArgumentException: Builtin jars can only
+ // be used when hive execution version == hive metastore version.
+ // Execution: 0.13.1 != Metastore: 0.12. Specify a vaild path to the correct hive jars
+ // using $HIVE_METASTORE_JARS or change spark.sql.hive.metastore.version to 0.13.1.
+ val conf = new SparkConf() {
+ override def getAll: Array[(String, String)] = {
+ def isMetastoreSetting(conf: String): Boolean = {
+ conf == "spark.sql.hive.metastore.version" || conf == "spark.sql.hive.metastore.jars"
+ }
+ // If there is any metastore settings, remove them.
+ val filteredSettings = super.getAll.filterNot(e => isMetastoreSetting(e._1))
+
+ // Always add these two metastore settings at the beginning.
+ ("spark.sql.hive.metastore.version" -> "0.12") +:
+ ("spark.sql.hive.metastore.jars" -> "maven") +:
+ filteredSettings
+ }
+
+ // For this simple test, we do not really clone this object.
+ override def clone: SparkConf = this
+ }
+ val sc = new SparkContext(conf)
+ val hiveContext = new TestHiveContext(sc)
+ // Run a simple command to make sure all lazy vals in hiveContext get instantiated.
+ hiveContext.tables().collect()
+ }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 58e2d1fbfa73e..cc294bc3e8bc3 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -456,7 +456,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
withTable("savedJsonTable") {
val df = (1 to 10).map(i => i -> s"str$i").toDF("a", "b")
- withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
+ withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") {
// Save the df as a managed table (by not specifying the path).
df.write.saveAsTable("savedJsonTable")
@@ -484,7 +484,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
}
// Create an external table by specifying the path.
- withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+ withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") {
df.write
.format("org.apache.spark.sql.json")
.mode(SaveMode.Append)
@@ -508,7 +508,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
s"""{ "a": $i, "b": "str$i" }"""
}))
- withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+ withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") {
df.write
.format("json")
.mode(SaveMode.Append)
@@ -516,7 +516,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
.saveAsTable("savedJsonTable")
}
- withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
+ withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") {
createExternalTable("createdJsonTable", tempPath.toString)
assert(table("createdJsonTable").schema === df.schema)
checkAnswer(sql("SELECT * FROM createdJsonTable"), df)
@@ -533,7 +533,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
checkAnswer(read.json(tempPath.toString), df)
// Try to specify the schema.
- withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+ withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") {
val schema = StructType(StructField("b", StringType, true) :: Nil)
createExternalTable(
"createdJsonTable",
@@ -561,30 +561,28 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
}
}
- if (HiveShim.version == "0.13.1") {
- test("scan a parquet table created through a CTAS statement") {
- withSQLConf(
- "spark.sql.hive.convertMetastoreParquet" -> "true",
- SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") {
-
- withTempTable("jt") {
- (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt")
-
- withTable("test_parquet_ctas") {
- sql(
- """CREATE TABLE test_parquet_ctas STORED AS PARQUET
- |AS SELECT tmp.a FROM jt tmp WHERE tmp.a < 5
- """.stripMargin)
-
- checkAnswer(
- sql(s"SELECT a FROM test_parquet_ctas WHERE a > 2 "),
- Row(3) :: Row(4) :: Nil)
-
- table("test_parquet_ctas").queryExecution.optimizedPlan match {
- case LogicalRelation(p: ParquetRelation2) => // OK
- case _ =>
- fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation2]}")
- }
+ test("scan a parquet table created through a CTAS statement") {
+ withSQLConf(
+ HiveContext.CONVERT_METASTORE_PARQUET.key -> "true",
+ SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "true") {
+
+ withTempTable("jt") {
+ (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt")
+
+ withTable("test_parquet_ctas") {
+ sql(
+ """CREATE TABLE test_parquet_ctas STORED AS PARQUET
+ |AS SELECT tmp.a FROM jt tmp WHERE tmp.a < 5
+ """.stripMargin)
+
+ checkAnswer(
+ sql(s"SELECT a FROM test_parquet_ctas WHERE a > 2 "),
+ Row(3) :: Row(4) :: Nil)
+
+ table("test_parquet_ctas").queryExecution.optimizedPlan match {
+ case LogicalRelation(p: ParquetRelation2) => // OK
+ case _ =>
+ fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation2]}")
}
}
}
@@ -708,7 +706,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
}
test("SPARK-6024 wide schema support") {
- withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD -> "4000") {
+ withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") {
withTable("wide_schema") {
// We will need 80 splits for this schema if the threshold is 4000.
val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType, true)))
@@ -835,4 +833,21 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeA
(70 to 79).map(i => Row(i, s"str$i")))
}
}
+
+ test("SPARK-8156:create table to specific database by 'use dbname' ") {
+
+ val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c")
+ sqlContext.sql("""create database if not exists testdb8156""")
+ sqlContext.sql("""use testdb8156""")
+ df.write
+ .format("parquet")
+ .mode(SaveMode.Overwrite)
+ .saveAsTable("ttt3")
+
+ checkAnswer(
+ sqlContext.sql("show TABLES in testdb8156").filter("tableName = 'ttt3'"),
+ Row("ttt3", false))
+ sqlContext.sql("""use default""")
+ sqlContext.sql("""drop database if exists testdb8156 CASCADE""")
+ }
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
index 4990092df6a99..017bc2adc103b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala
@@ -20,16 +20,17 @@ package org.apache.spark.sql.hive
import com.google.common.io.Files
import org.apache.spark.sql.{QueryTest, _}
-import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.util.Utils
class QueryPartitionSuite extends QueryTest {
- import org.apache.spark.sql.hive.test.TestHive.implicits._
+
+ private lazy val ctx = org.apache.spark.sql.hive.test.TestHive
+ import ctx.implicits._
+ import ctx.sql
test("SPARK-5068: query data when path doesn't exist"){
- val testData = TestHive.sparkContext.parallelize(
+ val testData = ctx.sparkContext.parallelize(
(1 to 10).map(i => TestData(i, i.toString))).toDF()
testData.registerTempTable("testData")
@@ -48,8 +49,8 @@ class QueryPartitionSuite extends QueryTest {
// test for the exist path
checkAnswer(sql("select key,value from table_with_partition"),
- testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect
- ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect)
+ testData.toDF.collect ++ testData.toDF.collect
+ ++ testData.toDF.collect ++ testData.toDF.collect)
// delete the path of one partition
tmpDir.listFiles
@@ -58,8 +59,7 @@ class QueryPartitionSuite extends QueryTest {
// test for after delete the path
checkAnswer(sql("select key,value from table_with_partition"),
- testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect
- ++ testData.toSchemaRDD.collect)
+ testData.toDF.collect ++ testData.toDF.collect ++ testData.toDF.collect)
sql("DROP TABLE table_with_partition")
sql("DROP TABLE createAndInsertTest")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala
index a492ecf203d17..93dcb10f7a296 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala
@@ -19,12 +19,11 @@ package org.apache.spark.sql.hive
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.serializer.JavaSerializer
-import org.apache.spark.sql.hive.test.TestHive
class SerializationSuite extends SparkFunSuite {
test("[SPARK-5840] HiveContext should be serializable") {
- val hiveContext = TestHive
+ val hiveContext = org.apache.spark.sql.hive.test.TestHive
hiveContext.hiveconf
val serializer = new JavaSerializer(new SparkConf()).newInstance()
val bytes = serializer.serialize(hiveContext)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index 00a69de9e4262..f067ea0d4fc75 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -23,13 +23,18 @@ import scala.reflect.ClassTag
import org.apache.spark.sql.{Row, SQLConf, QueryTest}
import org.apache.spark.sql.execution.joins._
-import org.apache.spark.sql.hive.test.TestHive
-import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.execution._
class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
- TestHive.reset()
- TestHive.cacheTables = false
+
+ private lazy val ctx: HiveContext = {
+ val ctx = org.apache.spark.sql.hive.test.TestHive
+ ctx.reset()
+ ctx.cacheTables = false
+ ctx
+ }
+
+ import ctx.sql
test("parse analyze commands") {
def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) {
@@ -72,17 +77,13 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
test("analyze MetastoreRelations") {
def queryTotalSize(tableName: String): BigInt =
- catalog.lookupRelation(Seq(tableName)).statistics.sizeInBytes
+ ctx.catalog.lookupRelation(Seq(tableName)).statistics.sizeInBytes
// Non-partitioned table
sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect()
sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect()
sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect()
- // TODO: How does it works? needs to add it back for other hive version.
- if (HiveShim.version =="0.12.0") {
- assert(queryTotalSize("analyzeTable") === conf.defaultSizeInBytes)
- }
sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan")
assert(queryTotalSize("analyzeTable") === BigInt(11624))
@@ -110,7 +111,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
|SELECT * FROM src
""".stripMargin).collect()
- assert(queryTotalSize("analyzeTable_part") === conf.defaultSizeInBytes)
+ assert(queryTotalSize("analyzeTable_part") === ctx.conf.defaultSizeInBytes)
sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan")
@@ -121,9 +122,9 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
// Try to analyze a temp table
sql("""SELECT * FROM src""").registerTempTable("tempTable")
intercept[UnsupportedOperationException] {
- analyze("tempTable")
+ ctx.analyze("tempTable")
}
- catalog.unregisterTable(Seq("tempTable"))
+ ctx.catalog.unregisterTable(Seq("tempTable"))
}
test("estimates the size of a test MetastoreRelation") {
@@ -151,8 +152,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
val sizes = df.queryExecution.analyzed.collect {
case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes
}
- assert(sizes.size === 2 && sizes(0) <= conf.autoBroadcastJoinThreshold
- && sizes(1) <= conf.autoBroadcastJoinThreshold,
+ assert(sizes.size === 2 && sizes(0) <= ctx.conf.autoBroadcastJoinThreshold
+ && sizes(1) <= ctx.conf.autoBroadcastJoinThreshold,
s"query should contain two relations, each of which has size smaller than autoConvertSize")
// Using `sparkPlan` because for relevant patterns in HashJoin to be
@@ -163,10 +164,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
checkAnswer(df, expectedAnswer) // check correctness of output
- TestHive.conf.settings.synchronized {
- val tmp = conf.autoBroadcastJoinThreshold
+ ctx.conf.settings.synchronized {
+ val tmp = ctx.conf.autoBroadcastJoinThreshold
- sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""")
+ sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""")
df = sql(query)
bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j }
assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off")
@@ -175,7 +176,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
assert(shj.size === 1,
"ShuffledHashJoin should be planned when BroadcastHashJoin is turned off")
- sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""")
+ sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp""")
}
after()
@@ -207,8 +208,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
.isAssignableFrom(r.getClass) =>
r.statistics.sizeInBytes
}
- assert(sizes.size === 2 && sizes(1) <= conf.autoBroadcastJoinThreshold
- && sizes(0) <= conf.autoBroadcastJoinThreshold,
+ assert(sizes.size === 2 && sizes(1) <= ctx.conf.autoBroadcastJoinThreshold
+ && sizes(0) <= ctx.conf.autoBroadcastJoinThreshold,
s"query should contain two relations, each of which has size smaller than autoConvertSize")
// Using `sparkPlan` because for relevant patterns in HashJoin to be
@@ -221,10 +222,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
checkAnswer(df, answer) // check correctness of output
- TestHive.conf.settings.synchronized {
- val tmp = conf.autoBroadcastJoinThreshold
+ ctx.conf.settings.synchronized {
+ val tmp = ctx.conf.autoBroadcastJoinThreshold
- sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1")
+ sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1")
df = sql(leftSemiJoinQuery)
bhj = df.queryExecution.sparkPlan.collect {
case j: BroadcastLeftSemiJoinHash => j
@@ -237,7 +238,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll {
assert(shj.size === 1,
"LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off")
- sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp")
+ sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
index 8245047626d57..4056dee777574 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala
@@ -17,20 +17,20 @@
package org.apache.spark.sql.hive
-/* Implicits */
-
import org.apache.spark.sql.QueryTest
-import org.apache.spark.sql.hive.test.TestHive._
case class FunctionResult(f1: String, f2: String)
class UDFSuite extends QueryTest {
+
+ private lazy val ctx = org.apache.spark.sql.hive.test.TestHive
+
test("UDF case insensitive") {
- udf.register("random0", () => { Math.random() })
- udf.register("RANDOM1", () => { Math.random() })
- udf.register("strlenScala", (_: String).length + (_: Int))
- assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
- assert(sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
- assert(sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5)
+ ctx.udf.register("random0", () => { Math.random() })
+ ctx.udf.register("RANDOM1", () => { Math.random() })
+ ctx.udf.register("strlenScala", (_: String).length + (_: Int))
+ assert(ctx.sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
+ assert(ctx.sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
+ assert(ctx.sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5)
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index 7eb4842726665..9a571650b6e25 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -72,7 +72,7 @@ class VersionsSuite extends SparkFunSuite with Logging {
assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'")
}
- private val versions = Seq("12", "13")
+ private val versions = Seq("12", "13", "14")
private var client: ClientInterface = null
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index 440b7c87b0da2..51dabc67fa7c1 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -874,15 +874,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
|WITH serdeproperties('s1'='9')
""".stripMargin)
}
- // Now only verify 0.12.0, and ignore other versions due to binary compatibility
- // current TestSerDe.jar is from 0.12.0
- if (HiveShim.version == "0.12.0") {
- sql(s"ADD JAR $testJar")
- sql(
- """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe'
- |WITH serdeproperties('s1'='9')
- """.stripMargin)
- }
sql("DROP TABLE alter1")
}
@@ -890,15 +881,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
// this is a test case from mapjoin_addjar.q
val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath
val testData = TestHive.getHiveFile("data/files/sample.json").getCanonicalPath
- if (HiveShim.version == "0.13.1") {
- sql(s"ADD JAR $testJar")
- sql(
- """CREATE TABLE t1(a string, b string)
- |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
- sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
- sql("select * from src join t1 on src.key = t1.a")
- sql("DROP TABLE t1")
- }
+ sql(s"ADD JAR $testJar")
+ sql(
+ """CREATE TABLE t1(a string, b string)
+ |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin)
+ sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""")
+ sql("select * from src join t1 on src.key = t1.a")
+ sql("DROP TABLE t1")
}
test("ADD FILE command") {
@@ -1095,14 +1084,16 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
val testKey = "spark.sql.key.usedfortestonly"
val testVal = "test.val.0"
val nonexistentKey = "nonexistent"
- val KV = "([^=]+)=([^=]*)".r
- def collectResults(df: DataFrame): Set[(String, String)] =
+ def collectResults(df: DataFrame): Set[Any] =
df.collect().map {
case Row(key: String, value: String) => key -> value
- case Row(KV(key, value)) => key -> value
+ case Row(key: String, defaultValue: String, doc: String) => (key, defaultValue, doc)
}.toSet
conf.clear()
+ val expectedConfs = conf.getAllDefinedConfs.toSet
+ assertResult(expectedConfs)(collectResults(sql("SET -v")))
+
// "SET" itself returns all config variables currently specified in SQLConf.
// TODO: Should we be listing the default here always? probably...
assert(sql("SET").collect().size == 0)
@@ -1113,16 +1104,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
assert(hiveconf.get(testKey, "") == testVal)
assertResult(Set(testKey -> testVal))(collectResults(sql("SET")))
- assertResult(Set(testKey -> testVal))(collectResults(sql("SET -v")))
sql(s"SET ${testKey + testKey}=${testVal + testVal}")
assert(hiveconf.get(testKey + testKey, "") == testVal + testVal)
assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) {
collectResults(sql("SET"))
}
- assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) {
- collectResults(sql("SET -v"))
- }
// "SET key"
assertResult(Set(testKey -> testVal)) {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index aba3becb1bce2..a2e666586c186 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
-import org.apache.spark.sql.hive.{HiveQLDialect, HiveShim, MetastoreRelation}
+import org.apache.spark.sql.hive.{HiveContext, HiveQLDialect, MetastoreRelation}
import org.apache.spark.sql.parquet.ParquetRelation2
import org.apache.spark.sql.sources.LogicalRelation
import org.apache.spark.sql.types._
@@ -191,9 +191,9 @@ class SQLQuerySuite extends QueryTest {
}
}
- val originalConf = getConf("spark.sql.hive.convertCTAS", "false")
+ val originalConf = convertCTAS
- setConf("spark.sql.hive.convertCTAS", "true")
+ setConf(HiveContext.CONVERT_CTAS, true)
sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
@@ -235,7 +235,7 @@ class SQLQuerySuite extends QueryTest {
checkRelation("ctas1", false)
sql("DROP TABLE ctas1")
- setConf("spark.sql.hive.convertCTAS", originalConf)
+ setConf(HiveContext.CONVERT_CTAS, originalConf)
}
test("SQL Dialect Switching") {
@@ -330,38 +330,54 @@ class SQLQuerySuite extends QueryTest {
"serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22", "MANAGED_TABLE"
)
- if (HiveShim.version =="0.13.1") {
- val origUseParquetDataSource = conf.parquetUseDataSourceApi
- try {
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
- sql(
- """CREATE TABLE ctas5
- | STORED AS parquet AS
- | SELECT key, value
- | FROM src
- | ORDER BY key, value""".stripMargin).collect()
-
- checkExistence(sql("DESC EXTENDED ctas5"), true,
- "name:key", "type:string", "name:value", "ctas5",
- "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
- "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
- "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
- "MANAGED_TABLE"
- )
-
- val default = getConf("spark.sql.hive.convertMetastoreParquet", "true")
- // use the Hive SerDe for parquet tables
- sql("set spark.sql.hive.convertMetastoreParquet = false")
- checkAnswer(
- sql("SELECT key, value FROM ctas5 ORDER BY key, value"),
- sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
- sql(s"set spark.sql.hive.convertMetastoreParquet = $default")
- } finally {
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, origUseParquetDataSource.toString)
- }
+ val origUseParquetDataSource = conf.parquetUseDataSourceApi
+ try {
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
+ sql(
+ """CREATE TABLE ctas5
+ | STORED AS parquet AS
+ | SELECT key, value
+ | FROM src
+ | ORDER BY key, value""".stripMargin).collect()
+
+ checkExistence(sql("DESC EXTENDED ctas5"), true,
+ "name:key", "type:string", "name:value", "ctas5",
+ "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
+ "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
+ "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
+ "MANAGED_TABLE"
+ )
+
+ val default = convertMetastoreParquet
+ // use the Hive SerDe for parquet tables
+ sql("set spark.sql.hive.convertMetastoreParquet = false")
+ checkAnswer(
+ sql("SELECT key, value FROM ctas5 ORDER BY key, value"),
+ sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
+ sql(s"set spark.sql.hive.convertMetastoreParquet = $default")
+ } finally {
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, origUseParquetDataSource)
}
}
+ test("specifying the column list for CTAS") {
+ Seq((1, "111111"), (2, "222222")).toDF("key", "value").registerTempTable("mytable1")
+
+ sql("create table gen__tmp(a int, b string) as select key, value from mytable1")
+ checkAnswer(
+ sql("SELECT a, b from gen__tmp"),
+ sql("select key, value from mytable1").collect())
+ sql("DROP TABLE gen__tmp")
+
+ sql("create table gen__tmp(a double, b double) as select key, value from mytable1")
+ checkAnswer(
+ sql("SELECT a, b from gen__tmp"),
+ sql("select cast(key as double), cast(value as double) from mytable1").collect())
+ sql("DROP TABLE gen__tmp")
+
+ sql("drop table mytable1")
+ }
+
test("command substitution") {
sql("set tbl=src")
checkAnswer(
@@ -587,8 +603,8 @@ class SQLQuerySuite extends QueryTest {
// generates an invalid query plan.
val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}"""))
read.json(rdd).registerTempTable("data")
- val originalConf = getConf("spark.sql.hive.convertCTAS", "false")
- setConf("spark.sql.hive.convertCTAS", "false")
+ val originalConf = convertCTAS
+ setConf(HiveContext.CONVERT_CTAS, false)
sql("CREATE TABLE explodeTest (key bigInt)")
table("explodeTest").queryExecution.analyzed match {
@@ -605,7 +621,7 @@ class SQLQuerySuite extends QueryTest {
sql("DROP TABLE explodeTest")
dropTempTable("data")
- setConf("spark.sql.hive.convertCTAS", originalConf)
+ setConf(HiveContext.CONVERT_CTAS, originalConf)
}
test("sanity test for SPARK-6618") {
@@ -629,12 +645,20 @@ class SQLQuerySuite extends QueryTest {
.queryExecution.analyzed
}
- test("test script transform") {
+ test("test script transform for stdout") {
val data = (1 to 100000).map { i => (i, i, i) }
data.toDF("d1", "d2", "d3").registerTempTable("script_trans")
assert(100000 ===
sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans")
- .queryExecution.toRdd.count())
+ .queryExecution.toRdd.count())
+ }
+
+ ignore("test script transform for stderr") {
+ val data = (1 to 100000).map { i => (i, i, i) }
+ data.toDF("d1", "d2", "d3").registerTempTable("script_trans")
+ assert(0 ===
+ sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans")
+ .queryExecution.toRdd.count())
}
test("window function: udaf with aggregate expressin") {
@@ -851,6 +875,10 @@ class SQLQuerySuite extends QueryTest {
}
}
+ test("Cast STRING to BIGINT") {
+ checkAnswer(sql("SELECT CAST('775983671874188101' as BIGINT)"), Row(775983671874188101L))
+ }
+
// `Math.exp(1.0)` has different result for different jdk version, so not use createQueryTest
test("udf_java_method") {
checkAnswer(sql(
@@ -906,4 +934,32 @@ class SQLQuerySuite extends QueryTest {
sql("set hive.exec.dynamic.partition.mode=strict")
}
}
+
+ test("Call add jar in a different thread (SPARK-8306)") {
+ @volatile var error: Option[Throwable] = None
+ val thread = new Thread {
+ override def run() {
+ // To make sure this test works, this jar should not be loaded in another place.
+ TestHive.sql(
+ s"ADD JAR ${TestHive.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath()}")
+ try {
+ TestHive.sql(
+ """
+ |CREATE TEMPORARY FUNCTION example_max
+ |AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax'
+ """.stripMargin)
+ } catch {
+ case throwable: Throwable =>
+ error = Some(throwable)
+ }
+ }
+ }
+ thread.start()
+ thread.join()
+ error match {
+ case Some(throwable) =>
+ fail("CREATE TEMPORARY FUNCTION should not fail.", throwable)
+ case None => // OK
+ }
+ }
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
index 0e63d84e9824a..8707f9f936be6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
@@ -21,7 +21,7 @@ import java.io.File
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
index 57c23fe77f8b5..267d22c6b5f1e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
@@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterAll
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
@@ -52,9 +52,6 @@ case class Contact(name: String, phone: String)
case class Person(name: String, age: Int, contacts: Seq[Contact])
class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
- override val sqlContext = TestHive
-
- import TestHive.read
def getTempFilePath(prefix: String, suffix: String = ""): File = {
val tempFile = File.createTempFile(prefix, suffix)
@@ -69,7 +66,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
withOrcFile(data) { file =>
checkAnswer(
- read.format("orc").load(file),
+ sqlContext.read.format("orc").load(file),
data.toDF().collect())
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala
index 82e08caf46457..a0cdd0db42d65 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala
@@ -43,8 +43,14 @@ abstract class OrcSuite extends QueryTest with BeforeAndAfterAll {
orcTableDir.mkdir()
import org.apache.spark.sql.hive.test.TestHive.implicits._
+ // Originally we were using a 10-row RDD for testing. However, when default parallelism is
+ // greater than 10 (e.g., running on a node with 32 cores), this RDD contains empty partitions,
+ // which result in empty ORC files. Unfortunately, ORC doesn't handle empty files properly and
+ // causes build failure on Jenkins, which happens to have 32 cores. Please refer to SPARK-8501
+ // for more details. To workaround this issue before fixing SPARK-8501, we simply increase row
+ // number in this RDD to avoid empty partitions.
sparkContext
- .makeRDD(1 to 10)
+ .makeRDD(1 to 100)
.map(i => OrcData(i, s"part-$i"))
.toDF()
.registerTempTable(s"orc_temp_table")
@@ -70,35 +76,35 @@ abstract class OrcSuite extends QueryTest with BeforeAndAfterAll {
}
test("create temporary orc table") {
- checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10))
+ checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(100))
checkAnswer(
sql("SELECT * FROM normal_orc_source"),
- (1 to 10).map(i => Row(i, s"part-$i")))
+ (1 to 100).map(i => Row(i, s"part-$i")))
checkAnswer(
sql("SELECT * FROM normal_orc_source where intField > 5"),
- (6 to 10).map(i => Row(i, s"part-$i")))
+ (6 to 100).map(i => Row(i, s"part-$i")))
checkAnswer(
sql("SELECT COUNT(intField), stringField FROM normal_orc_source GROUP BY stringField"),
- (1 to 10).map(i => Row(1, s"part-$i")))
+ (1 to 100).map(i => Row(1, s"part-$i")))
}
test("create temporary orc table as") {
- checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_as_source"), Row(10))
+ checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_as_source"), Row(100))
checkAnswer(
sql("SELECT * FROM normal_orc_source"),
- (1 to 10).map(i => Row(i, s"part-$i")))
+ (1 to 100).map(i => Row(i, s"part-$i")))
checkAnswer(
sql("SELECT * FROM normal_orc_source WHERE intField > 5"),
- (6 to 10).map(i => Row(i, s"part-$i")))
+ (6 to 100).map(i => Row(i, s"part-$i")))
checkAnswer(
sql("SELECT COUNT(intField), stringField FROM normal_orc_source GROUP BY stringField"),
- (1 to 10).map(i => Row(1, s"part-$i")))
+ (1 to 100).map(i => Row(1, s"part-$i")))
}
test("appending insert") {
@@ -106,7 +112,7 @@ abstract class OrcSuite extends QueryTest with BeforeAndAfterAll {
checkAnswer(
sql("SELECT * FROM normal_orc_source"),
- (1 to 5).map(i => Row(i, s"part-$i")) ++ (6 to 10).flatMap { i =>
+ (1 to 5).map(i => Row(i, s"part-$i")) ++ (6 to 100).flatMap { i =>
Seq.fill(2)(Row(i, s"part-$i"))
})
}
@@ -119,7 +125,7 @@ abstract class OrcSuite extends QueryTest with BeforeAndAfterAll {
checkAnswer(
sql("SELECT * FROM normal_orc_as_source"),
- (6 to 10).map(i => Row(i, s"part-$i")))
+ (6 to 100).map(i => Row(i, s"part-$i")))
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
index 750f0b04aaa87..5daf691aa8c53 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala
@@ -22,13 +22,11 @@ import java.io.File
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
-import org.apache.spark.sql.hive.HiveContext
-import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql._
private[sql] trait OrcTest extends SQLTestUtils {
- protected def hiveContext = sqlContext.asInstanceOf[HiveContext]
+ lazy val sqlContext = org.apache.spark.sql.hive.test.TestHive
import sqlContext.sparkContext
import sqlContext.implicits._
@@ -53,7 +51,7 @@ private[sql] trait OrcTest extends SQLTestUtils {
protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag]
(data: Seq[T])
(f: DataFrame => Unit): Unit = {
- withOrcFile(data)(path => f(hiveContext.read.format("orc").load(path)))
+ withOrcFile(data)(path => f(sqlContext.read.format("orc").load(path)))
}
/**
@@ -65,7 +63,7 @@ private[sql] trait OrcTest extends SQLTestUtils {
(data: Seq[T], tableName: String)
(f: => Unit): Unit = {
withOrcDataFrame(data) { df =>
- hiveContext.registerDataFrameAsTable(df, tableName)
+ sqlContext.registerDataFrameAsTable(df, tableName)
withTempTable(tableName)(f)
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index e62ac909cbd0c..c2e09800933b5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -21,8 +21,6 @@ import java.io.File
import org.scalatest.BeforeAndAfterAll
-import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD}
import org.apache.spark.sql.hive.execution.HiveTableScan
import org.apache.spark.sql.hive.test.TestHive._
@@ -30,7 +28,7 @@ import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan}
import org.apache.spark.sql.sources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation}
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf, SaveMode}
+import org.apache.spark.sql.{DataFrame, QueryTest, Row, SQLConf, SaveMode}
import org.apache.spark.util.Utils
// The data where the partitioning key exists only in the directory structure.
@@ -155,7 +153,7 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest {
val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":[$i, null]}"""))
read.json(rdd2).registerTempTable("jt_array")
- setConf("spark.sql.hive.convertMetastoreParquet", "true")
+ setConf(HiveContext.CONVERT_METASTORE_PARQUET, true)
}
override def afterAll(): Unit = {
@@ -166,7 +164,7 @@ class ParquetMetastoreSuiteBase extends ParquetPartitioningTest {
sql("DROP TABLE normal_parquet")
sql("DROP TABLE IF EXISTS jt")
sql("DROP TABLE IF EXISTS jt_array")
- setConf("spark.sql.hive.convertMetastoreParquet", "false")
+ setConf(HiveContext.CONVERT_METASTORE_PARQUET, false)
}
test(s"conversion is working") {
@@ -201,14 +199,14 @@ class ParquetDataSourceOnMetastoreSuite extends ParquetMetastoreSuiteBase {
| OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
""".stripMargin)
- conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
+ conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, true)
}
override def afterAll(): Unit = {
super.afterAll()
sql("DROP TABLE IF EXISTS test_parquet")
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
}
test("scan an empty parquet table") {
@@ -548,12 +546,12 @@ class ParquetDataSourceOffMetastoreSuite extends ParquetMetastoreSuiteBase {
override def beforeAll(): Unit = {
super.beforeAll()
- conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
+ conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
}
override def afterAll(): Unit = {
super.afterAll()
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
}
test("MetastoreRelation in InsertIntoTable will not be converted") {
@@ -694,12 +692,12 @@ class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase {
override def beforeAll(): Unit = {
super.beforeAll()
- conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
+ conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, true)
}
override def afterAll(): Unit = {
super.afterAll()
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
}
test("values in arrays and maps stored in parquet are always nullable") {
@@ -752,12 +750,12 @@ class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase {
override def beforeAll(): Unit = {
super.beforeAll()
- conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
+ conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
}
override def afterAll(): Unit = {
super.afterAll()
- setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+ setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
index 0f959b3d0b86d..5d7cd16c129cd 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala
@@ -53,9 +53,10 @@ class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullW
numberFormat.setGroupingUsed(false)
override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
+ val uniqueWriteJobId = context.getConfiguration.get("spark.sql.sources.writeJobUUID")
val split = context.getTaskAttemptID.getTaskID.getId
val name = FileOutputFormat.getOutputName(context)
- new Path(outputFile, s"$name-${numberFormat.format(split)}-${UUID.randomUUID()}")
+ new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId")
}
}
@@ -156,6 +157,7 @@ class CommitFailureTestRelation(
context: TaskAttemptContext): OutputWriter = {
new SimpleTextOutputWriter(path, context) {
override def close(): Unit = {
+ super.close()
sys.error("Intentional task commitment failure for testing purpose.")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
index 74095426741e3..a16ab3a00ddb8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala
@@ -17,10 +17,16 @@
package org.apache.spark.sql.sources
+import scala.collection.JavaConversions._
+
import java.io.File
import com.google.common.io.Files
+import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
+import org.apache.parquet.hadoop.ParquetOutputCommitter
import org.apache.spark.{SparkException, SparkFunSuite}
import org.apache.spark.deploy.SparkHadoopUtil
@@ -30,12 +36,12 @@ import org.apache.spark.sql.test.SQLTestUtils
import org.apache.spark.sql.types._
abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
- override val sqlContext: SQLContext = TestHive
+ override lazy val sqlContext: SQLContext = TestHive
- import sqlContext._
+ import sqlContext.sql
import sqlContext.implicits._
- val dataSourceName = classOf[SimpleTextSource].getCanonicalName
+ val dataSourceName: String
val dataSchema =
StructType(
@@ -43,19 +49,19 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
StructField("a", IntegerType, nullable = false),
StructField("b", StringType, nullable = false)))
- val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b")
+ lazy val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b")
- val partitionedTestDF1 = (for {
+ lazy val partitionedTestDF1 = (for {
i <- 1 to 3
p2 <- Seq("foo", "bar")
} yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2")
- val partitionedTestDF2 = (for {
+ lazy val partitionedTestDF2 = (for {
i <- 1 to 3
p2 <- Seq("foo", "bar")
} yield (i, s"val_$i", 2, p2)).toDF("a", "b", "p1", "p2")
- val partitionedTestDF = partitionedTestDF1.unionAll(partitionedTestDF2)
+ lazy val partitionedTestDF = partitionedTestDF1.unionAll(partitionedTestDF2)
def checkQueries(df: DataFrame): Unit = {
// Selects everything
@@ -103,7 +109,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath)
checkAnswer(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("path", file.getCanonicalPath)
.option("dataSchema", dataSchema.json)
.load(),
@@ -117,7 +123,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
testDF.write.mode(SaveMode.Append).format(dataSourceName).save(file.getCanonicalPath)
checkAnswer(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(file.getCanonicalPath).orderBy("a"),
testDF.unionAll(testDF).orderBy("a").collect())
@@ -151,7 +157,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.save(file.getCanonicalPath)
checkQueries(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(file.getCanonicalPath))
}
@@ -172,7 +178,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.save(file.getCanonicalPath)
checkAnswer(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(file.getCanonicalPath),
partitionedTestDF.collect())
@@ -194,7 +200,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.save(file.getCanonicalPath)
checkAnswer(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(file.getCanonicalPath),
partitionedTestDF.unionAll(partitionedTestDF).collect())
@@ -216,7 +222,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.save(file.getCanonicalPath)
checkAnswer(
- read.format(dataSourceName)
+ sqlContext.read.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(file.getCanonicalPath),
partitionedTestDF.collect())
@@ -252,7 +258,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), testDF.collect())
+ checkAnswer(sqlContext.table("t"), testDF.collect())
}
}
@@ -261,7 +267,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
testDF.write.format(dataSourceName).mode(SaveMode.Append).saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), testDF.unionAll(testDF).orderBy("a").collect())
+ checkAnswer(sqlContext.table("t"), testDF.unionAll(testDF).orderBy("a").collect())
}
}
@@ -280,7 +286,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
withTempTable("t") {
testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t")
- assert(table("t").collect().isEmpty)
+ assert(sqlContext.table("t").collect().isEmpty)
}
}
@@ -291,7 +297,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkQueries(table("t"))
+ checkQueries(sqlContext.table("t"))
}
}
@@ -311,7 +317,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), partitionedTestDF.collect())
+ checkAnswer(sqlContext.table("t"), partitionedTestDF.collect())
}
}
@@ -331,7 +337,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), partitionedTestDF.unionAll(partitionedTestDF).collect())
+ checkAnswer(sqlContext.table("t"), partitionedTestDF.unionAll(partitionedTestDF).collect())
}
}
@@ -351,7 +357,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), partitionedTestDF.collect())
+ checkAnswer(sqlContext.table("t"), partitionedTestDF.collect())
}
}
@@ -400,7 +406,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.partitionBy("p1", "p2")
.saveAsTable("t")
- assert(table("t").collect().isEmpty)
+ assert(sqlContext.table("t").collect().isEmpty)
}
}
@@ -412,7 +418,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.partitionBy("p1", "p2")
.save(file.getCanonicalPath)
- val df = read
+ val df = sqlContext.read
.format(dataSourceName)
.option("dataSchema", dataSchema.json)
.load(s"${file.getCanonicalPath}/p1=*/p2=???")
@@ -452,7 +458,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTempTable("t") {
- checkAnswer(table("t"), input.collect())
+ checkAnswer(sqlContext.table("t"), input.collect())
}
}
}
@@ -467,9 +473,111 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils {
.saveAsTable("t")
withTable("t") {
- checkAnswer(table("t"), df.select('b, 'c, 'a).collect())
+ checkAnswer(sqlContext.table("t"), df.select('b, 'c, 'a).collect())
+ }
+ }
+
+ // NOTE: This test suite is not super deterministic. On nodes with only relatively few cores
+ // (4 or even 1), it's hard to reproduce the data loss issue. But on nodes with for example 8 or
+ // more cores, the issue can be reproduced steadily. Fortunately our Jenkins builder meets this
+ // requirement. We probably want to move this test case to spark-integration-tests or spark-perf
+ // later.
+ test("SPARK-8406: Avoids name collision while writing files") {
+ withTempPath { dir =>
+ val path = dir.getCanonicalPath
+ sqlContext
+ .range(10000)
+ .repartition(250)
+ .write
+ .mode(SaveMode.Overwrite)
+ .format(dataSourceName)
+ .save(path)
+
+ assertResult(10000) {
+ sqlContext
+ .read
+ .format(dataSourceName)
+ .option("dataSchema", StructType(StructField("id", LongType) :: Nil).json)
+ .load(path)
+ .count()
+ }
}
}
+
+ test("SPARK-8578 specified custom output committer will not be used to append data") {
+ val clonedConf = new Configuration(configuration)
+ try {
+ val df = sqlContext.range(1, 10).toDF("i")
+ withTempPath { dir =>
+ df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath)
+ configuration.set(
+ SQLConf.OUTPUT_COMMITTER_CLASS.key,
+ classOf[AlwaysFailOutputCommitter].getName)
+ // Since Parquet has its own output committer setting, also set it
+ // to AlwaysFailParquetOutputCommitter at here.
+ configuration.set("spark.sql.parquet.output.committer.class",
+ classOf[AlwaysFailParquetOutputCommitter].getName)
+ // Because there data already exists,
+ // this append should succeed because we will use the output committer associated
+ // with file format and AlwaysFailOutputCommitter will not be used.
+ df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath)
+ checkAnswer(
+ sqlContext.read
+ .format(dataSourceName)
+ .option("dataSchema", df.schema.json)
+ .load(dir.getCanonicalPath),
+ df.unionAll(df))
+
+ // This will fail because AlwaysFailOutputCommitter is used when we do append.
+ intercept[Exception] {
+ df.write.mode("overwrite").format(dataSourceName).save(dir.getCanonicalPath)
+ }
+ }
+ withTempPath { dir =>
+ configuration.set(
+ SQLConf.OUTPUT_COMMITTER_CLASS.key,
+ classOf[AlwaysFailOutputCommitter].getName)
+ // Since Parquet has its own output committer setting, also set it
+ // to AlwaysFailParquetOutputCommitter at here.
+ configuration.set("spark.sql.parquet.output.committer.class",
+ classOf[AlwaysFailParquetOutputCommitter].getName)
+ // Because there is no existing data,
+ // this append will fail because AlwaysFailOutputCommitter is used when we do append
+ // and there is no existing data.
+ intercept[Exception] {
+ df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath)
+ }
+ }
+ } finally {
+ // Hadoop 1 doesn't have `Configuration.unset`
+ configuration.clear()
+ clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue))
+ }
+ }
+}
+
+// This class is used to test SPARK-8578. We should not use any custom output committer when
+// we actually append data to an existing dir.
+class AlwaysFailOutputCommitter(
+ outputPath: Path,
+ context: TaskAttemptContext)
+ extends FileOutputCommitter(outputPath, context) {
+
+ override def commitJob(context: JobContext): Unit = {
+ sys.error("Intentional job commitment failure for testing purpose.")
+ }
+}
+
+// This class is used to test SPARK-8578. We should not use any custom output committer when
+// we actually append data to an existing dir.
+class AlwaysFailParquetOutputCommitter(
+ outputPath: Path,
+ context: TaskAttemptContext)
+ extends ParquetOutputCommitter(outputPath, context) {
+
+ override def commitJob(context: JobContext): Unit = {
+ sys.error("Intentional job commitment failure for testing purpose.")
+ }
}
class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest {
@@ -502,15 +610,17 @@ class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest {
}
class CommitFailureTestRelationSuite extends SparkFunSuite with SQLTestUtils {
- import TestHive.implicits._
-
override val sqlContext = TestHive
+ // When committing a task, `CommitFailureTestSource` throws an exception for testing purpose.
val dataSourceName: String = classOf[CommitFailureTestSource].getCanonicalName
test("SPARK-7684: commitTask() failure should fallback to abortTask()") {
withTempPath { file =>
- val df = (1 to 3).map(i => i -> s"val_$i").toDF("a", "b")
+ // Here we coalesce partition number to 1 to ensure that only a single task is issued. This
+ // prevents race condition happened when FileOutputCommitter tries to remove the `_temporary`
+ // directory while committing/aborting the job. See SPARK-8513 for more details.
+ val df = sqlContext.range(0, 10).coalesce(1)
intercept[SparkException] {
df.write.format(dataSourceName).save(file.getCanonicalPath)
}
@@ -594,4 +704,19 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest {
checkAnswer(read.format("parquet").load(path), df)
}
}
+
+ test("SPARK-8079: Avoid NPE thrown from BaseWriterContainer.abortJob") {
+ withTempPath { dir =>
+ intercept[AnalysisException] {
+ // Parquet doesn't allow field names with spaces. Here we are intentionally making an
+ // exception thrown from the `ParquetRelation2.prepareForWriteJob()` method to trigger
+ // the bug. Please refer to spark-8079 for more details.
+ range(1, 10)
+ .withColumnRenamed("id", "a b")
+ .write
+ .format("parquet")
+ .save(dir.getCanonicalPath)
+ }
+ }
+ }
}
diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
deleted file mode 100644
index dbc5e029e2047..0000000000000
--- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
+++ /dev/null
@@ -1,457 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.rmi.server.UID
-import java.util.{Properties, ArrayList => JArrayList}
-import java.io.{OutputStream, InputStream}
-
-import scala.collection.JavaConversions._
-import scala.language.implicitConversions
-import scala.reflect.ClassTag
-
-import com.esotericsoftware.kryo.Kryo
-import com.esotericsoftware.kryo.io.Input
-import com.esotericsoftware.kryo.io.Output
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.hive.common.StatsSetupConst
-import org.apache.hadoop.hive.common.`type`.HiveDecimal
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.ql.Context
-import org.apache.hadoop.hive.ql.exec.{UDF, Utilities}
-import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
-import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc}
-import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory
-import org.apache.hadoop.hive.serde.serdeConstants
-import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory}
-import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, PrimitiveObjectInspector}
-import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfo, TypeInfoFactory}
-import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo}
-import org.apache.hadoop.io.{NullWritable, Writable}
-import org.apache.hadoop.mapred.InputFormat
-import org.apache.hadoop.{io => hadoopIo}
-
-import org.apache.spark.Logging
-import org.apache.spark.sql.types.{Decimal, DecimalType, UTF8String}
-import org.apache.spark.util.Utils._
-
-/**
- * This class provides the UDF creation and also the UDF instance serialization and
- * de-serialization cross process boundary.
- *
- * Detail discussion can be found at https://github.com/apache/spark/pull/3640
- *
- * @param functionClassName UDF class name
- */
-private[hive] case class HiveFunctionWrapper(var functionClassName: String)
- extends java.io.Externalizable {
-
- // for Serialization
- def this() = this(null)
-
- @transient
- def deserializeObjectByKryo[T: ClassTag](
- kryo: Kryo,
- in: InputStream,
- clazz: Class[_]): T = {
- val inp = new Input(in)
- val t: T = kryo.readObject(inp,clazz).asInstanceOf[T]
- inp.close()
- t
- }
-
- @transient
- def serializeObjectByKryo(
- kryo: Kryo,
- plan: Object,
- out: OutputStream ) {
- val output: Output = new Output(out)
- kryo.writeObject(output, plan)
- output.close()
- }
-
- def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = {
- deserializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), is, clazz)
- .asInstanceOf[UDFType]
- }
-
- def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = {
- serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out)
- }
-
- private var instance: AnyRef = null
-
- def writeExternal(out: java.io.ObjectOutput) {
- // output the function name
- out.writeUTF(functionClassName)
-
- // Write a flag if instance is null or not
- out.writeBoolean(instance != null)
- if (instance != null) {
- // Some of the UDF are serializable, but some others are not
- // Hive Utilities can handle both cases
- val baos = new java.io.ByteArrayOutputStream()
- serializePlan(instance, baos)
- val functionInBytes = baos.toByteArray
-
- // output the function bytes
- out.writeInt(functionInBytes.length)
- out.write(functionInBytes, 0, functionInBytes.length)
- }
- }
-
- def readExternal(in: java.io.ObjectInput) {
- // read the function name
- functionClassName = in.readUTF()
-
- if (in.readBoolean()) {
- // if the instance is not null
- // read the function in bytes
- val functionInBytesLength = in.readInt()
- val functionInBytes = new Array[Byte](functionInBytesLength)
- in.read(functionInBytes, 0, functionInBytesLength)
-
- // deserialize the function object via Hive Utilities
- instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes),
- getContextOrSparkClassLoader.loadClass(functionClassName))
- }
- }
-
- def createFunction[UDFType <: AnyRef](): UDFType = {
- if (instance != null) {
- instance.asInstanceOf[UDFType]
- } else {
- val func = getContextOrSparkClassLoader
- .loadClass(functionClassName).newInstance.asInstanceOf[UDFType]
- if (!func.isInstanceOf[UDF]) {
- // We cache the function if it's no the Simple UDF,
- // as we always have to create new instance for Simple UDF
- instance = func
- }
- func
- }
- }
-}
-
-/**
- * A compatibility layer for interacting with Hive version 0.13.1.
- */
-private[hive] object HiveShim {
- val version = "0.13.1"
-
- def getTableDesc(
- serdeClass: Class[_ <: Deserializer],
- inputFormatClass: Class[_ <: InputFormat[_, _]],
- outputFormatClass: Class[_],
- properties: Properties) = {
- new TableDesc(inputFormatClass, outputFormatClass, properties)
- }
-
-
- def getStringWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.stringTypeInfo, getStringWritable(value))
-
- def getIntWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.intTypeInfo, getIntWritable(value))
-
- def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.doubleTypeInfo, getDoubleWritable(value))
-
- def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.booleanTypeInfo, getBooleanWritable(value))
-
- def getLongWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.longTypeInfo, getLongWritable(value))
-
- def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.floatTypeInfo, getFloatWritable(value))
-
- def getShortWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.shortTypeInfo, getShortWritable(value))
-
- def getByteWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.byteTypeInfo, getByteWritable(value))
-
- def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.binaryTypeInfo, getBinaryWritable(value))
-
- def getDateWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.dateTypeInfo, getDateWritable(value))
-
- def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value))
-
- def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.decimalTypeInfo, getDecimalWritable(value))
-
- def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector =
- PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
- TypeInfoFactory.voidTypeInfo, null)
-
- def getStringWritable(value: Any): hadoopIo.Text =
- if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString)
-
- def getIntWritable(value: Any): hadoopIo.IntWritable =
- if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])
-
- def getDoubleWritable(value: Any): hiveIo.DoubleWritable =
- if (value == null) {
- null
- } else {
- new hiveIo.DoubleWritable(value.asInstanceOf[Double])
- }
-
- def getBooleanWritable(value: Any): hadoopIo.BooleanWritable =
- if (value == null) {
- null
- } else {
- new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])
- }
-
- def getLongWritable(value: Any): hadoopIo.LongWritable =
- if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])
-
- def getFloatWritable(value: Any): hadoopIo.FloatWritable =
- if (value == null) {
- null
- } else {
- new hadoopIo.FloatWritable(value.asInstanceOf[Float])
- }
-
- def getShortWritable(value: Any): hiveIo.ShortWritable =
- if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])
-
- def getByteWritable(value: Any): hiveIo.ByteWritable =
- if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])
-
- def getBinaryWritable(value: Any): hadoopIo.BytesWritable =
- if (value == null) {
- null
- } else {
- new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])
- }
-
- def getDateWritable(value: Any): hiveIo.DateWritable =
- if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int])
-
- def getTimestampWritable(value: Any): hiveIo.TimestampWritable =
- if (value == null) {
- null
- } else {
- new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp])
- }
-
- def getDecimalWritable(value: Any): hiveIo.HiveDecimalWritable =
- if (value == null) {
- null
- } else {
- // TODO precise, scale?
- new hiveIo.HiveDecimalWritable(
- HiveShim.createDecimal(value.asInstanceOf[Decimal].toJavaBigDecimal))
- }
-
- def getPrimitiveNullWritable: NullWritable = NullWritable.get()
-
- def createDriverResultsArray = new JArrayList[Object]
-
- def processResults(results: JArrayList[Object]) = {
- results.map { r =>
- r match {
- case s: String => s
- case a: Array[Object] => a(0).asInstanceOf[String]
- }
- }
- }
-
- def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
-
- def getStatsSetupConstRawDataSize = StatsSetupConst.RAW_DATA_SIZE
-
- def createDefaultDBIfNeeded(context: HiveContext) = {
- context.runSqlHive("CREATE DATABASE default")
- context.runSqlHive("USE default")
- }
-
- def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
- CommandProcessorFactory.get(cmd, conf)
- }
-
- def createDecimal(bd: java.math.BigDecimal): HiveDecimal = {
- HiveDecimal.create(bd)
- }
-
- /*
- * This function in hive-0.13 become private, but we have to do this to walkaround hive bug
- */
- private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) {
- val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "")
- val result: StringBuilder = new StringBuilder(old)
- var first: Boolean = old.isEmpty
-
- for (col <- cols) {
- if (first) {
- first = false
- } else {
- result.append(',')
- }
- result.append(col)
- }
- conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString)
- }
-
- /*
- * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty
- */
- def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
- if (ids != null && ids.size > 0) {
- ColumnProjectionUtils.appendReadColumns(conf, ids)
- }
- if (names != null && names.size > 0) {
- appendReadColumnNames(conf, names)
- }
- }
-
- def getExternalTmpPath(context: Context, path: Path) = {
- context.getExternalTmpPath(path.toUri)
- }
-
- def getDataLocationPath(p: Partition) = p.getDataLocation
-
- def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl)
-
- def compatibilityBlackList = Seq()
-
- def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = {
- tbl.setDataLocation(new Path(crtTbl.getLocation()))
- }
-
- /*
- * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not.
- * Fix it through wrapper.
- * */
- implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = {
- var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed)
- f.setCompressCodec(w.compressCodec)
- f.setCompressType(w.compressType)
- f.setTableInfo(w.tableInfo)
- f.setDestTableId(w.destTableId)
- f
- }
-
- // Precision and scale to pass for unlimited decimals; these are the same as the precision and
- // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs)
- private val UNLIMITED_DECIMAL_PRECISION = 38
- private val UNLIMITED_DECIMAL_SCALE = 18
-
- def decimalMetastoreString(decimalType: DecimalType): String = decimalType match {
- case DecimalType.Fixed(precision, scale) => s"decimal($precision,$scale)"
- case _ => s"decimal($UNLIMITED_DECIMAL_PRECISION,$UNLIMITED_DECIMAL_SCALE)"
- }
-
- def decimalTypeInfo(decimalType: DecimalType): TypeInfo = decimalType match {
- case DecimalType.Fixed(precision, scale) => new DecimalTypeInfo(precision, scale)
- case _ => new DecimalTypeInfo(UNLIMITED_DECIMAL_PRECISION, UNLIMITED_DECIMAL_SCALE)
- }
-
- def decimalTypeInfoToCatalyst(inspector: PrimitiveObjectInspector): DecimalType = {
- val info = inspector.getTypeInfo.asInstanceOf[DecimalTypeInfo]
- DecimalType(info.precision(), info.scale())
- }
-
- def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = {
- if (hdoi.preferWritable()) {
- Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue,
- hdoi.precision(), hdoi.scale())
- } else {
- Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale())
- }
- }
-
- def getConvertedOI(inputOI: ObjectInspector, outputOI: ObjectInspector): ObjectInspector = {
- ObjectInspectorConverters.getConvertedOI(inputOI, outputOI)
- }
-
- /*
- * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that
- * is needed to initialize before serialization.
- */
- def prepareWritable(w: Writable): Writable = {
- w match {
- case w: AvroGenericRecordWritable =>
- w.setRecordReaderID(new UID())
- case _ =>
- }
- w
- }
-
- def setTblNullFormat(crtTbl: CreateTableDesc, tbl: Table) = {
- if (crtTbl != null && crtTbl.getNullFormat() != null) {
- tbl.setSerdeParam(serdeConstants.SERIALIZATION_NULL_FORMAT, crtTbl.getNullFormat())
- }
- }
-}
-
-/*
- * Bug introduced in hive-0.13. FileSinkDesc is serilizable, but its member path is not.
- * Fix it through wrapper.
- */
-private[hive] class ShimFileSinkDesc(
- var dir: String,
- var tableInfo: TableDesc,
- var compressed: Boolean)
- extends Serializable with Logging {
- var compressCodec: String = _
- var compressType: String = _
- var destTableId: Int = _
-
- def setCompressed(compressed: Boolean) {
- this.compressed = compressed
- }
-
- def getDirName = dir
-
- def setDestTableId(destTableId: Int) {
- this.destTableId = destTableId
- }
-
- def setTableInfo(tableInfo: TableDesc) {
- this.tableInfo = tableInfo
- }
-
- def setCompressCodec(intermediateCompressorCodec: String) {
- compressCodec = intermediateCompressorCodec
- }
-
- def setCompressType(intermediateCompressType: String) {
- compressType = intermediateCompressType
- }
-}
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 49d035a1e9696..697895e72fe5b 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -21,7 +21,7 @@
+ * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + *
+ * Note: This is not designed for general use cases, should not be used outside SQL.
+ */
+public final class UTF8String implements Comparable