From 6942b974adad396cba2799eac1fa90448cea4da7 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 11 Jan 2015 16:23:28 -0800 Subject: [PATCH 01/46] [SPARK-4951][Core] Fix the issue that a busy executor may be killed A few changes to fix this issue: 1. Handle the case that receiving `SparkListenerTaskStart` before `SparkListenerBlockManagerAdded`. 2. Don't add `executorId` to `removeTimes` when the executor is busy. 3. Use `HashMap.retain` to safely traverse the HashMap and remove items. 4. Use the same lock in ExecutorAllocationManager and ExecutorAllocationListener to fix the race condition in `totalPendingTasks`. 5. Move the blocking codes out of the message processing code in YarnSchedulerActor. Author: zsxwing Closes #3783 from zsxwing/SPARK-4951 and squashes the following commits: d51fa0d [zsxwing] Add comments 2e365ce [zsxwing] Remove expired executors from 'removeTimes' and add idle executors back when a new executor joins 49f61a9 [zsxwing] Eliminate duplicate executor registered warnings d4c4e9a [zsxwing] Minor fixes for the code style 05f6238 [zsxwing] Move the blocking codes out of the message processing code 105ba3a [zsxwing] Fix the race condition in totalPendingTasks d5c615d [zsxwing] Fix the issue that a busy executor may be killed --- .../spark/ExecutorAllocationManager.scala | 117 ++++++++++++------ .../cluster/YarnSchedulerBackend.scala | 23 +++- .../ExecutorAllocationManagerSuite.scala | 49 +++++++- 3 files changed, 144 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index e9e90e3f2f65a..a0ee2a7cbb2a2 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -65,6 +65,9 @@ private[spark] class ExecutorAllocationManager( listenerBus: LiveListenerBus, conf: SparkConf) extends Logging { + + allocationManager => + import ExecutorAllocationManager._ // Lower and upper bounds on the number of executors. These are required. @@ -121,7 +124,7 @@ private[spark] class ExecutorAllocationManager( private var clock: Clock = new RealClock // Listener for Spark events that impact the allocation policy - private val listener = new ExecutorAllocationListener(this) + private val listener = new ExecutorAllocationListener /** * Verify that the settings specified through the config are valid. @@ -209,11 +212,12 @@ private[spark] class ExecutorAllocationManager( addTime += sustainedSchedulerBacklogTimeout * 1000 } - removeTimes.foreach { case (executorId, expireTime) => - if (now >= expireTime) { + removeTimes.retain { case (executorId, expireTime) => + val expired = now >= expireTime + if (expired) { removeExecutor(executorId) - removeTimes.remove(executorId) } + !expired } } @@ -291,7 +295,7 @@ private[spark] class ExecutorAllocationManager( // Do not kill the executor if we have already reached the lower bound val numExistingExecutors = executorIds.size - executorsPendingToRemove.size if (numExistingExecutors - 1 < minNumExecutors) { - logInfo(s"Not removing idle executor $executorId because there are only " + + logDebug(s"Not removing idle executor $executorId because there are only " + s"$numExistingExecutors executor(s) left (limit $minNumExecutors)") return false } @@ -315,7 +319,11 @@ private[spark] class ExecutorAllocationManager( private def onExecutorAdded(executorId: String): Unit = synchronized { if (!executorIds.contains(executorId)) { executorIds.add(executorId) - executorIds.foreach(onExecutorIdle) + // If an executor (call this executor X) is not removed because the lower bound + // has been reached, it will no longer be marked as idle. When new executors join, + // however, we are no longer at the lower bound, and so we must mark executor X + // as idle again so as not to forget that it is a candidate for removal. (see SPARK-4951) + executorIds.filter(listener.isExecutorIdle).foreach(onExecutorIdle) logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") if (numExecutorsPending > 0) { numExecutorsPending -= 1 @@ -373,10 +381,14 @@ private[spark] class ExecutorAllocationManager( * the executor is not already marked as idle. */ private def onExecutorIdle(executorId: String): Unit = synchronized { - if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { - logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + if (executorIds.contains(executorId)) { + if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { + logDebug(s"Starting idle timer for $executorId because there are no more tasks " + + s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + } + } else { + logWarning(s"Attempted to mark unknown executor $executorId idle") } } @@ -396,25 +408,24 @@ private[spark] class ExecutorAllocationManager( * and consistency of events returned by the listener. For simplicity, it does not account * for speculated tasks. */ - private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager) - extends SparkListener { + private class ExecutorAllocationListener extends SparkListener { private val stageIdToNumTasks = new mutable.HashMap[Int, Int] private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { - synchronized { - val stageId = stageSubmitted.stageInfo.stageId - val numTasks = stageSubmitted.stageInfo.numTasks + val stageId = stageSubmitted.stageInfo.stageId + val numTasks = stageSubmitted.stageInfo.numTasks + allocationManager.synchronized { stageIdToNumTasks(stageId) = numTasks allocationManager.onSchedulerBacklogged() } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - synchronized { - val stageId = stageCompleted.stageInfo.stageId + val stageId = stageCompleted.stageInfo.stageId + allocationManager.synchronized { stageIdToNumTasks -= stageId stageIdToTaskIndices -= stageId @@ -426,39 +437,49 @@ private[spark] class ExecutorAllocationManager( } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { val stageId = taskStart.stageId val taskId = taskStart.taskInfo.taskId val taskIndex = taskStart.taskInfo.index val executorId = taskStart.taskInfo.executorId - // If this is the last pending task, mark the scheduler queue as empty - stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex - val numTasksScheduled = stageIdToTaskIndices(stageId).size - val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) - if (numTasksScheduled == numTasksTotal) { - // No more pending tasks for this stage - stageIdToNumTasks -= stageId - if (stageIdToNumTasks.isEmpty) { - allocationManager.onSchedulerQueueEmpty() + allocationManager.synchronized { + // This guards against the race condition in which the `SparkListenerTaskStart` + // event is posted before the `SparkListenerBlockManagerAdded` event, which is + // possible because these events are posted in different threads. (see SPARK-4951) + if (!allocationManager.executorIds.contains(executorId)) { + allocationManager.onExecutorAdded(executorId) + } + + // If this is the last pending task, mark the scheduler queue as empty + stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex + val numTasksScheduled = stageIdToTaskIndices(stageId).size + val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) + if (numTasksScheduled == numTasksTotal) { + // No more pending tasks for this stage + stageIdToNumTasks -= stageId + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } } - } - // Mark the executor on which this task is scheduled as busy - executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId - allocationManager.onExecutorBusy(executorId) + // Mark the executor on which this task is scheduled as busy + executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId + allocationManager.onExecutorBusy(executorId) + } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { val executorId = taskEnd.taskInfo.executorId val taskId = taskEnd.taskInfo.taskId - - // If the executor is no longer running scheduled any tasks, mark it as idle - if (executorIdToTaskIds.contains(executorId)) { - executorIdToTaskIds(executorId) -= taskId - if (executorIdToTaskIds(executorId).isEmpty) { - executorIdToTaskIds -= executorId - allocationManager.onExecutorIdle(executorId) + allocationManager.synchronized { + // If the executor is no longer running scheduled any tasks, mark it as idle + if (executorIdToTaskIds.contains(executorId)) { + executorIdToTaskIds(executorId) -= taskId + if (executorIdToTaskIds(executorId).isEmpty) { + executorIdToTaskIds -= executorId + allocationManager.onExecutorIdle(executorId) + } } } } @@ -466,7 +487,12 @@ private[spark] class ExecutorAllocationManager( override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { val executorId = blockManagerAdded.blockManagerId.executorId if (executorId != SparkContext.DRIVER_IDENTIFIER) { - allocationManager.onExecutorAdded(executorId) + // This guards against the race condition in which the `SparkListenerTaskStart` + // event is posted before the `SparkListenerBlockManagerAdded` event, which is + // possible because these events are posted in different threads. (see SPARK-4951) + if (!allocationManager.executorIds.contains(executorId)) { + allocationManager.onExecutorAdded(executorId) + } } } @@ -478,12 +504,23 @@ private[spark] class ExecutorAllocationManager( /** * An estimate of the total number of pending tasks remaining for currently running stages. Does * not account for tasks which may have failed and been resubmitted. + * + * Note: This is not thread-safe without the caller owning the `allocationManager` lock. */ def totalPendingTasks(): Int = { stageIdToNumTasks.map { case (stageId, numTasks) => numTasks - stageIdToTaskIndices.get(stageId).map(_.size).getOrElse(0) }.sum } + + /** + * Return true if an executor is not currently running a task, and false otherwise. + * + * Note: This is not thread-safe without the caller owning the `allocationManager` lock. + */ + def isExecutorIdle(executorId: String): Boolean = { + !executorIdToTaskIds.contains(executorId) + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 50721b9d6cd6c..f14aaeea0a25c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler.cluster +import scala.concurrent.{Future, ExecutionContext} + import akka.actor.{Actor, ActorRef, Props} import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} @@ -24,7 +26,9 @@ import org.apache.spark.SparkContext import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{AkkaUtils, Utils} + +import scala.util.control.NonFatal /** * Abstract Yarn scheduler backend that contains common logic @@ -97,6 +101,9 @@ private[spark] abstract class YarnSchedulerBackend( private class YarnSchedulerActor extends Actor { private var amActor: Option[ActorRef] = None + implicit val askAmActorExecutor = ExecutionContext.fromExecutor( + Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-executor")) + override def preStart(): Unit = { // Listen for disassociation events context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -110,7 +117,12 @@ private[spark] abstract class YarnSchedulerBackend( case r: RequestExecutors => amActor match { case Some(actor) => - sender ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + val driverActor = sender + Future { + driverActor ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + } onFailure { + case NonFatal(e) => logError(s"Sending $r to AM was unsuccessful", e) + } case None => logWarning("Attempted to request executors before the AM has registered!") sender ! false @@ -119,7 +131,12 @@ private[spark] abstract class YarnSchedulerBackend( case k: KillExecutors => amActor match { case Some(actor) => - sender ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + val driverActor = sender + Future { + driverActor ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + } onFailure { + case NonFatal(e) => logError(s"Sending $k to AM was unsuccessful", e) + } case None => logWarning("Attempted to kill executors before the AM has registered!") sender ! false diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index c817f6dcede75..0e4df17c1bf87 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark +import scala.collection.mutable + import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -143,11 +145,17 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Verify that running a task reduces the cap sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + assert(numExecutorsPending(manager) === 4) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsPending(manager) === 5) assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 7) + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 0) assert(numExecutorsPending(manager) === 7) assert(numExecutorsToAdd(manager) === 1) @@ -325,6 +333,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { val manager = sc.executorAllocationManager.get manager.setClock(clock) + executorIds(manager).asInstanceOf[mutable.Set[String]] ++= List("1", "2", "3") + // Starting remove timer is idempotent for each executor assert(removeTimes(manager).isEmpty) onExecutorIdle(manager, "1") @@ -597,6 +607,41 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).size === 1) } + test("SPARK-4951: call onTaskStart before onBlockManagerAdded") { + sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 0) + } + + test("SPARK-4951: onExecutorAdded should not add a busy executor to removeTimes") { + sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 0) + + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-2", "host1", 1), 100L)) + assert(executorIds(manager).size === 2) + assert(executorIds(manager).contains("executor-2")) + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("executor-2")) + assert(!removeTimes(manager).contains("executor-1")) + } } /** From f38ef6586c2980183c983b2aa14a5ddc1856b7b7 Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Sun, 11 Jan 2015 16:32:47 -0800 Subject: [PATCH 02/46] [SPARK-4033][Examples]Input of the SparkPi too big causes the emption exception If input of the SparkPi args is larger than the 25000, the integer 'n' inside the code will be overflow, and may be a negative number. And it causes the (0 until n) Seq as an empty seq, then doing the action 'reduce' will throw the UnsupportedOperationException("empty collection"). The max size of the input of sc.parallelize is Int.MaxValue - 1, not the Int.MaxValue. Author: huangzhaowei Closes #2874 from SaintBacchus/SparkPi and squashes the following commits: 62d7cd7 [huangzhaowei] Add a commit to explain the modify 4cdc388 [huangzhaowei] Update SparkPi.scala 9a2fb7b [huangzhaowei] Input of the SparkPi is too big --- .../src/main/scala/org/apache/spark/examples/SparkPi.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 9fbb0a800d735..35b8dd6c29b66 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -27,8 +27,8 @@ object SparkPi { val conf = new SparkConf().setAppName("Spark Pi") val spark = new SparkContext(conf) val slices = if (args.length > 0) args(0).toInt else 2 - val n = 100000 * slices - val count = spark.parallelize(1 to n, slices).map { i => + val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow + val count = spark.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 From 2130de9d8f50f52b9b2d296b377df81d840546b3 Mon Sep 17 00:00:00 2001 From: Travis Galoppo Date: Sun, 11 Jan 2015 21:31:16 -0800 Subject: [PATCH 03/46] SPARK-5018 [MLlib] [WIP] Make MultivariateGaussian public Moving MutlivariateGaussian from private[mllib] to public. The class uses Breeze vectors internally, so this involves creating a public interface using MLlib vectors and matrices. This initial commit provides public construction, accessors for mean/covariance, density and log-density. Other potential methods include entropy and sample generation. Author: Travis Galoppo Closes #3923 from tgaloppo/spark-5018 and squashes the following commits: 2b15587 [Travis Galoppo] Style correction b4121b4 [Travis Galoppo] Merge remote-tracking branch 'upstream/master' into spark-5018 e30a100 [Travis Galoppo] Made mu, sigma private[mllib] members of MultivariateGaussian Moved MultivariateGaussian (and test suite) from stat.impl to stat.distribution (required updates in GaussianMixture{EM,Model}.scala) Marked MultivariateGaussian as @DeveloperApi Fixed style error 9fa3bb7 [Travis Galoppo] Style improvements 91a5fae [Travis Galoppo] Rearranged equation for part of density function 8c35381 [Travis Galoppo] Fixed accessor methods to match member variable names. Modified calculations to avoid log(pow(x,y)) calculations 0943dc4 [Travis Galoppo] SPARK-5018 4dee9e1 [Travis Galoppo] SPARK-5018 --- .../mllib/clustering/GaussianMixtureEM.scala | 11 +-- .../clustering/GaussianMixtureModel.scala | 2 +- .../MultivariateGaussian.scala | 70 ++++++++++++++----- .../MultivariateGaussianSuite.scala | 33 +++++---- 4 files changed, 75 insertions(+), 41 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/stat/{impl => distribution}/MultivariateGaussian.scala (61%) rename mllib/src/test/scala/org/apache/spark/mllib/stat/{impl => distribution}/MultivariateGaussianSuite.scala (72%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala index b3c5631cc4cc6..d8e134619411b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala @@ -20,10 +20,11 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.IndexedSeq import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose} -import org.apache.spark.rdd.RDD + import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors, DenseVector, DenseMatrix, BLAS} -import org.apache.spark.mllib.stat.impl.MultivariateGaussian +import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils /** @@ -134,7 +135,7 @@ class GaussianMixtureEM private ( // derived from the samples val (weights, gaussians) = initialModel match { case Some(gmm) => (gmm.weight, gmm.mu.zip(gmm.sigma).map { case(mu, sigma) => - new MultivariateGaussian(mu.toBreeze.toDenseVector, sigma.toBreeze.toDenseMatrix) + new MultivariateGaussian(mu, sigma) }) case None => { @@ -176,8 +177,8 @@ class GaussianMixtureEM private ( } // Need to convert the breeze matrices to MLlib matrices - val means = Array.tabulate(k) { i => Vectors.fromBreeze(gaussians(i).mu) } - val sigmas = Array.tabulate(k) { i => Matrices.fromBreeze(gaussians(i).sigma) } + val means = Array.tabulate(k) { i => gaussians(i).mu } + val sigmas = Array.tabulate(k) { i => gaussians(i).sigma } new GaussianMixtureModel(weights, means, sigmas) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index b461ea4f0f06e..416cad080c408 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -21,7 +21,7 @@ import breeze.linalg.{DenseVector => BreezeVector} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Matrix, Vector} -import org.apache.spark.mllib.stat.impl.MultivariateGaussian +import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala similarity index 61% rename from mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala rename to mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index bc7f6c5197ac7..fd186b5ee6f72 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -15,13 +15,16 @@ * limitations under the License. */ -package org.apache.spark.mllib.stat.impl +package org.apache.spark.mllib.stat.distribution import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym} +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} import org.apache.spark.mllib.util.MLUtils /** + * :: DeveloperApi :: * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In * the event that the covariance matrix is singular, the density will be computed in a * reduced dimensional subspace under which the distribution is supported. @@ -30,33 +33,64 @@ import org.apache.spark.mllib.util.MLUtils * @param mu The mean vector of the distribution * @param sigma The covariance matrix of the distribution */ -private[mllib] class MultivariateGaussian( - val mu: DBV[Double], - val sigma: DBM[Double]) extends Serializable { +@DeveloperApi +class MultivariateGaussian ( + val mu: Vector, + val sigma: Matrix) extends Serializable { + require(sigma.numCols == sigma.numRows, "Covariance matrix must be square") + require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size") + + private val breezeMu = mu.toBreeze.toDenseVector + + /** + * private[mllib] constructor + * + * @param mu The mean vector of the distribution + * @param sigma The covariance matrix of the distribution + */ + private[mllib] def this(mu: DBV[Double], sigma: DBM[Double]) = { + this(Vectors.fromBreeze(mu), Matrices.fromBreeze(sigma)) + } + /** * Compute distribution dependent constants: - * rootSigmaInv = D^(-1/2) * U, where sigma = U * D * U.t - * u = (2*pi)^(-k/2) * det(sigma)^(-1/2) + * rootSigmaInv = D^(-1/2)^ * U, where sigma = U * D * U.t + * u = log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) */ private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants /** Returns density of this multivariate Gaussian at given point, x */ - def pdf(x: DBV[Double]): Double = { - val delta = x - mu + def pdf(x: Vector): Double = { + pdf(x.toBreeze.toDenseVector) + } + + /** Returns the log-density of this multivariate Gaussian at given point, x */ + def logpdf(x: Vector): Double = { + logpdf(x.toBreeze.toDenseVector) + } + + /** Returns density of this multivariate Gaussian at given point, x */ + private[mllib] def pdf(x: DBV[Double]): Double = { + math.exp(logpdf(x)) + } + + /** Returns the log-density of this multivariate Gaussian at given point, x */ + private[mllib] def logpdf(x: DBV[Double]): Double = { + val delta = x - breezeMu val v = rootSigmaInv * delta - u * math.exp(v.t * v * -0.5) + u + v.t * v * -0.5 } /** * Calculate distribution dependent components used for the density function: - * pdf(x) = (2*pi)^(-k/2) * det(sigma)^(-1/2) * exp( (-1/2) * (x-mu).t * inv(sigma) * (x-mu) ) + * pdf(x) = (2*pi)^(-k/2)^ * det(sigma)^(-1/2)^ * exp((-1/2) * (x-mu).t * inv(sigma) * (x-mu)) * where k is length of the mean vector. * * We here compute distribution-fixed parts - * (2*pi)^(-k/2) * det(sigma)^(-1/2) + * log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) * and - * D^(-1/2) * U, where sigma = U * D * U.t + * D^(-1/2)^ * U, where sigma = U * D * U.t * * Both the determinant and the inverse can be computed from the singular value decomposition * of sigma. Noting that covariance matrices are always symmetric and positive semi-definite, @@ -65,11 +99,11 @@ private[mllib] class MultivariateGaussian( * * sigma = U * D * U.t * inv(Sigma) = U * inv(D) * U.t - * = (D^{-1/2} * U).t * (D^{-1/2} * U) + * = (D^{-1/2}^ * U).t * (D^{-1/2}^ * U) * * and thus * - * -0.5 * (x-mu).t * inv(Sigma) * (x-mu) = -0.5 * norm(D^{-1/2} * U * (x-mu))^2 + * -0.5 * (x-mu).t * inv(Sigma) * (x-mu) = -0.5 * norm(D^{-1/2}^ * U * (x-mu))^2^ * * To guard against singular covariance matrices, this method computes both the * pseudo-determinant and the pseudo-inverse (Moore-Penrose). Singular values are considered @@ -77,21 +111,21 @@ private[mllib] class MultivariateGaussian( * relation to the maximum singular value (same tolerance used by, e.g., Octave). */ private def calculateCovarianceConstants: (DBM[Double], Double) = { - val eigSym.EigSym(d, u) = eigSym(sigma) // sigma = u * diag(d) * u.t + val eigSym.EigSym(d, u) = eigSym(sigma.toBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t // For numerical stability, values are considered to be non-zero only if they exceed tol. // This prevents any inverted value from exceeding (eps * n * max(d))^-1 val tol = MLUtils.EPSILON * max(d) * d.length try { - // pseudo-determinant is product of all non-zero singular values - val pdetSigma = d.activeValuesIterator.filter(_ > tol).reduce(_ * _) + // log(pseudo-determinant) is sum of the logs of all non-zero singular values + val logPseudoDetSigma = d.activeValuesIterator.filter(_ > tol).map(math.log).sum // calculate the root-pseudo-inverse of the diagonal matrix of singular values // by inverting the square root of all non-zero values val pinvS = diag(new DBV(d.map(v => if (v > tol) math.sqrt(1.0 / v) else 0.0).toArray)) - (pinvS * u, math.pow(2.0 * math.Pi, -mu.length / 2.0) * math.pow(pdetSigma, -0.5)) + (pinvS * u, -0.5 * (mu.size * math.log(2.0 * math.Pi) + logPseudoDetSigma)) } catch { case uex: UnsupportedOperationException => throw new IllegalArgumentException("Covariance matrix has no non-zero singular values") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala similarity index 72% rename from mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala index d58f2587e55aa..fac2498e4dcb3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/impl/MultivariateGaussianSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala @@ -15,54 +15,53 @@ * limitations under the License. */ -package org.apache.spark.mllib.stat.impl +package org.apache.spark.mllib.stat.distribution import org.scalatest.FunSuite -import breeze.linalg.{ DenseVector => BDV, DenseMatrix => BDM } - +import org.apache.spark.mllib.linalg.{ Vectors, Matrices } import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ class MultivariateGaussianSuite extends FunSuite with MLlibTestSparkContext { test("univariate") { - val x1 = new BDV(Array(0.0)) - val x2 = new BDV(Array(1.5)) + val x1 = Vectors.dense(0.0) + val x2 = Vectors.dense(1.5) - val mu = new BDV(Array(0.0)) - val sigma1 = new BDM(1, 1, Array(1.0)) + val mu = Vectors.dense(0.0) + val sigma1 = Matrices.dense(1, 1, Array(1.0)) val dist1 = new MultivariateGaussian(mu, sigma1) assert(dist1.pdf(x1) ~== 0.39894 absTol 1E-5) assert(dist1.pdf(x2) ~== 0.12952 absTol 1E-5) - val sigma2 = new BDM(1, 1, Array(4.0)) + val sigma2 = Matrices.dense(1, 1, Array(4.0)) val dist2 = new MultivariateGaussian(mu, sigma2) assert(dist2.pdf(x1) ~== 0.19947 absTol 1E-5) assert(dist2.pdf(x2) ~== 0.15057 absTol 1E-5) } test("multivariate") { - val x1 = new BDV(Array(0.0, 0.0)) - val x2 = new BDV(Array(1.0, 1.0)) + val x1 = Vectors.dense(0.0, 0.0) + val x2 = Vectors.dense(1.0, 1.0) - val mu = new BDV(Array(0.0, 0.0)) - val sigma1 = new BDM(2, 2, Array(1.0, 0.0, 0.0, 1.0)) + val mu = Vectors.dense(0.0, 0.0) + val sigma1 = Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0)) val dist1 = new MultivariateGaussian(mu, sigma1) assert(dist1.pdf(x1) ~== 0.15915 absTol 1E-5) assert(dist1.pdf(x2) ~== 0.05855 absTol 1E-5) - val sigma2 = new BDM(2, 2, Array(4.0, -1.0, -1.0, 2.0)) + val sigma2 = Matrices.dense(2, 2, Array(4.0, -1.0, -1.0, 2.0)) val dist2 = new MultivariateGaussian(mu, sigma2) assert(dist2.pdf(x1) ~== 0.060155 absTol 1E-5) assert(dist2.pdf(x2) ~== 0.033971 absTol 1E-5) } test("multivariate degenerate") { - val x1 = new BDV(Array(0.0, 0.0)) - val x2 = new BDV(Array(1.0, 1.0)) + val x1 = Vectors.dense(0.0, 0.0) + val x2 = Vectors.dense(1.0, 1.0) - val mu = new BDV(Array(0.0, 0.0)) - val sigma = new BDM(2, 2, Array(1.0, 1.0, 1.0, 1.0)) + val mu = Vectors.dense(0.0, 0.0) + val sigma = Matrices.dense(2, 2, Array(1.0, 1.0, 1.0, 1.0)) val dist = new MultivariateGaussian(mu, sigma) assert(dist.pdf(x1) ~== 0.11254 absTol 1E-5) assert(dist.pdf(x2) ~== 0.068259 absTol 1E-5) From 82fd38dcdcc9f7df18930c0e08cc8ec34eaee828 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Jan 2015 10:47:12 -0800 Subject: [PATCH 04/46] [SPARK-5200] Disable web UI in Hive ThriftServer tests Disables the Spark web UI in HiveThriftServer2Suite in order to prevent Jenkins test failures due to port contention. Author: Josh Rosen Closes #3998 from JoshRosen/SPARK-5200 and squashes the following commits: a384416 [Josh Rosen] [SPARK-5200] Disable web UI in Hive Thriftserver tests. --- .../spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 7814aa38f4146..b52a51d11e4ad 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -143,6 +143,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging { | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=http | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT}=$port | --driver-class-path ${sys.props("java.class.path")} + | --conf spark.ui.enabled=false """.stripMargin.split("\\s+").toSeq } else { s"""$startScript @@ -153,6 +154,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging { | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port | --driver-class-path ${sys.props("java.class.path")} + | --conf spark.ui.enabled=false """.stripMargin.split("\\s+").toSeq } From ef9224e08010420b570c21a0b9208d22792a24fe Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Mon, 12 Jan 2015 10:57:12 -0800 Subject: [PATCH 05/46] [SPARK-5102][Core]subclass of MapStatus needs to be registered with Kryo CompressedMapStatus and HighlyCompressedMapStatus needs to be registered with Kryo, because they are subclass of MapStatus. Author: lianhuiwang Closes #4007 from lianhuiwang/SPARK-5102 and squashes the following commits: 9d2238a [lianhuiwang] remove register of MapStatus 05a285d [lianhuiwang] subclass of MapStatus needs to be registered with Kryo --- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index d2947dcea4f7c..d56e23ce4478a 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -29,7 +29,7 @@ import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer @@ -207,7 +207,8 @@ private[serializer] object KryoSerializer { classOf[PutBlock], classOf[GotBlock], classOf[GetBlock], - classOf[MapStatus], + classOf[CompressedMapStatus], + classOf[HighlyCompressedMapStatus], classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Byte]], From 13e610b88eb83b11785737ab1d99927f676f81c6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 Jan 2015 11:00:56 -0800 Subject: [PATCH 06/46] SPARK-4159 [BUILD] Addendum: improve running of single test after enabling Java tests https://issues.apache.org/jira/browse/SPARK-4159 was resolved but as Sandy points out, the guidance in https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools under "Running Individual Tests" no longer quite works, not optimally. This minor change is not really the important change, which is an update to the wiki text. The correct way to run one Scala test suite in Maven is now: ``` mvn test -DwildcardSuites=org.apache.spark.io.CompressionCodecSuite -Dtests=none ``` The correct way to run one Java test is ``` mvn test -DwildcardSuites=none -Dtests=org.apache.spark.streaming.JavaAPISuite ``` Basically, you have to set two properties in order to suppress all of one type of test (with a non-existent test name like 'none') and all but one test of the other type. The change in the PR just prevents Surefire from barfing when it finds no "none" test. Author: Sean Owen Closes #3993 from srowen/SPARK-4159 and squashes the following commits: 83106d7 [Sean Owen] Default failIfNoTests to false to enable the -DwildcardSuites=... -Dtests=... syntax for running one test to work --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index aadcdfd1083c5..0403c008be4f4 100644 --- a/pom.xml +++ b/pom.xml @@ -1130,6 +1130,7 @@ ${test_classpath} true + false From a3978f3e156e0ca67e978f1795b238ddd69ff9a6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 12 Jan 2015 11:57:59 -0800 Subject: [PATCH 07/46] [SPARK-5078] Optionally read from SPARK_LOCAL_HOSTNAME Current spark lets you set the ip address using SPARK_LOCAL_IP, but then this is given to akka after doing a reverse DNS lookup. This makes it difficult to run spark in Docker. You can already change the hostname that is used programmatically, but it would be nice to be able to do this with an environment variable as well. Author: Michael Armbrust Closes #3893 from marmbrus/localHostnameEnv and squashes the following commits: 85045b6 [Michael Armbrust] Optionally read from SPARK_LOCAL_HOSTNAME --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c4f1898a2db15..578f1a9cf4868 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -701,7 +701,7 @@ private[spark] object Utils extends Logging { } } - private var customHostname: Option[String] = None + private var customHostname: Option[String] = sys.env.get("SPARK_LOCAL_HOSTNAME") /** * Allow setting a custom host name because when we run on Mesos we need to use the same From aff49a3ee1a91d730590f4a0b9ac485fd52dc8bf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 Jan 2015 12:15:34 -0800 Subject: [PATCH 08/46] SPARK-5172 [BUILD] spark-examples-***.jar shades a wrong Hadoop distribution In addition to the `hadoop-2.x` profiles in the parent POM, there is actually another set of profiles in `examples` that has to be activated differently to get the right Hadoop 1 vs 2 flavor of HBase. This wasn't actually used in making Hadoop 2 distributions, hence the problem. To reduce complexity, I suggest merging them with the parent POM profiles, which is possible now. You'll see this changes appears to update the HBase version, but actually, the default 0.94 version was not being used. HBase is only used in examples, and the examples POM always chose one profile or the other that updated the version to 0.98.x anyway. Author: Sean Owen Closes #3992 from srowen/SPARK-5172 and squashes the following commits: 17830d9 [Sean Owen] Control hbase hadoop1/2 flavor in the parent POM with existing hadoop-2.x profiles --- examples/pom.xml | 23 ----------------------- pom.xml | 5 ++++- 2 files changed, 4 insertions(+), 24 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 002d4458c4b3e..4b92147725f6b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -392,29 +392,6 @@ - - hbase-hadoop2 - - - hbase.profile - hadoop2 - - - - 0.98.7-hadoop2 - - - - hbase-hadoop1 - - - !hbase.profile - - - - 0.98.7-hadoop1 - - diff --git a/pom.xml b/pom.xml index 0403c008be4f4..f4466e56c2a53 100644 --- a/pom.xml +++ b/pom.xml @@ -122,7 +122,7 @@ 1.0.4 2.4.1 ${hadoop.version} - 0.94.6 + 0.98.7-hadoop1 hbase 1.4.0 3.4.5 @@ -1466,6 +1466,7 @@ 2.2.0 2.5.0 + 0.98.7-hadoop2 hadoop2 @@ -1476,6 +1477,7 @@ 2.3.0 2.5.0 0.9.0 + 0.98.7-hadoop2 3.1.1 hadoop2 @@ -1487,6 +1489,7 @@ 2.4.0 2.5.0 0.9.0 + 0.98.7-hadoop2 3.1.1 hadoop2 From 3aed3051c0b6cd5f38d7db7d20fa7a1680bfde6f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 12 Jan 2015 13:14:44 -0800 Subject: [PATCH 09/46] [SPARK-4999][Streaming] Change storeInBlockManager to false by default Currently WAL-backed block is read out from HDFS and put into BlockManger with storage level MEMORY_ONLY_SER by default, since WAL-backed block is already materialized in HDFS with fault-tolerance, no need to put into BlockManger again by default. Author: jerryshao Closes #3906 from jerryshao/SPARK-4999 and squashes the following commits: b95f95e [jerryshao] Change storeInBlockManager to false by default --- .../apache/spark/streaming/dstream/ReceiverInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index c834744631e02..afd3c4bc4c4fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -86,7 +86,7 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont }.toArray // Since storeInBlockManager = false, the storage level does not matter. new WriteAheadLogBackedBlockRDD[T](ssc.sparkContext, - blockIds, logSegments, storeInBlockManager = true, StorageLevel.MEMORY_ONLY_SER) + blockIds, logSegments, storeInBlockManager = false, StorageLevel.MEMORY_ONLY_SER) } else { new BlockRDD[T](ssc.sc, blockIds) } From 5d9fa550820543ee1b0ce82997917745973a5d65 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 12 Jan 2015 15:19:09 -0800 Subject: [PATCH 10/46] [SPARK-5049][SQL] Fix ordering of partition columns in ParquetTableScan Followup to #3870. Props to rahulaggarwalguavus for identifying the issue. Author: Michael Armbrust Closes #3990 from marmbrus/SPARK-5049 and squashes the following commits: dd03e4e [Michael Armbrust] Fill in the partition values of parquet scans instead of using JoinedRow --- .../spark/sql/parquet/ParquetRelation.scala | 4 +- .../sql/parquet/ParquetTableOperations.scala | 43 +++++++++++-------- .../spark/sql/parquet/parquetSuites.scala | 12 ++++++ 3 files changed, 41 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index b237a07c72d07..2835dc3408b96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -28,7 +28,7 @@ import parquet.schema.MessageType import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} /** @@ -67,6 +67,8 @@ private[sql] case class ParquetRelation( conf, sqlContext.isParquetBinaryAsString) + lazy val attributeMap = AttributeMap(output.map(o => o -> o)) + override def newInstance() = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 96bace1769f71..f5487740d3af9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -64,18 +64,17 @@ case class ParquetTableScan( // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes // by exprId. note: output cannot be transient, see // https://issues.apache.org/jira/browse/SPARK-1367 - val normalOutput = - attributes - .filterNot(a => relation.partitioningAttributes.map(_.exprId).contains(a.exprId)) - .flatMap(a => relation.output.find(o => o.exprId == a.exprId)) + val output = attributes.map(relation.attributeMap) - val partOutput = - attributes.flatMap(a => relation.partitioningAttributes.find(o => o.exprId == a.exprId)) + // A mapping of ordinals partitionRow -> finalOutput. + val requestedPartitionOrdinals = { + val partitionAttributeOrdinals = AttributeMap(relation.partitioningAttributes.zipWithIndex) - def output = partOutput ++ normalOutput - - assert(normalOutput.size + partOutput.size == attributes.size, - s"$normalOutput + $partOutput != $attributes, ${relation.output}") + attributes.zipWithIndex.flatMap { + case (attribute, finalOrdinal) => + partitionAttributeOrdinals.get(attribute).map(_ -> finalOrdinal) + } + }.toArray override def execute(): RDD[Row] = { import parquet.filter2.compat.FilterCompat.FilterPredicateCompat @@ -97,7 +96,7 @@ case class ParquetTableScan( // Store both requested and original schema in `Configuration` conf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(normalOutput)) + ParquetTypesConverter.convertToString(output)) conf.set( RowWriteSupport.SPARK_ROW_SCHEMA, ParquetTypesConverter.convertToString(relation.output)) @@ -125,7 +124,7 @@ case class ParquetTableScan( classOf[Row], conf) - if (partOutput.nonEmpty) { + if (requestedPartitionOrdinals.nonEmpty) { baseRDD.mapPartitionsWithInputSplit { case (split, iter) => val partValue = "([^=]+)=([^=]+)".r val partValues = @@ -138,15 +137,25 @@ case class ParquetTableScan( case _ => None }.toMap + // Convert the partitioning attributes into the correct types val partitionRowValues = - partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) + relation.partitioningAttributes + .map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) new Iterator[Row] { - private[this] val joinedRow = new JoinedRow5(Row(partitionRowValues:_*), null) - def hasNext = iter.hasNext - - def next() = joinedRow.withRight(iter.next()._2) + def next() = { + val row = iter.next()._2.asInstanceOf[SpecificMutableRow] + + // Parquet will leave partitioning columns empty, so we fill them in here. + var i = 0 + while (i < requestedPartitionOrdinals.size) { + row(requestedPartitionOrdinals(i)._2) = + partitionRowValues(requestedPartitionOrdinals(i)._1) + i += 1 + } + row + } } } } else { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index fc0e42c201d56..8bbb7f2fdbf48 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -174,6 +174,18 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll } Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table => + test(s"ordering of the partitioning columns $table") { + checkAnswer( + sql(s"SELECT p, stringField FROM $table WHERE p = 1"), + Seq.fill(10)((1, "part-1")) + ) + + checkAnswer( + sql(s"SELECT stringField, p FROM $table WHERE p = 1"), + Seq.fill(10)(("part-1", 1)) + ) + } + test(s"project the partitioning column $table") { checkAnswer( sql(s"SELECT p, count(*) FROM $table group by p"), From 1e42e96ece9e35ceed9ddebef66d589016878b56 Mon Sep 17 00:00:00 2001 From: Gabe Mulley Date: Mon, 12 Jan 2015 21:44:51 -0800 Subject: [PATCH 11/46] [SPARK-5138][SQL] Ensure schema can be inferred from a namedtuple When attempting to infer the schema of an RDD that contains namedtuples, pyspark fails to identify the records as namedtuples, resulting in it raising an error. Example: ```python from pyspark import SparkContext from pyspark.sql import SQLContext from collections import namedtuple import os sc = SparkContext() rdd = sc.textFile(os.path.join(os.getenv('SPARK_HOME'), 'README.md')) TextLine = namedtuple('TextLine', 'line length') tuple_rdd = rdd.map(lambda l: TextLine(line=l, length=len(l))) tuple_rdd.take(5) # This works sqlc = SQLContext(sc) # The following line raises an error schema_rdd = sqlc.inferSchema(tuple_rdd) ``` The error raised is: ``` File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 107, in main process() File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/worker.py", line 98, in process serializer.dump_stream(func(split_index, iterator), outfile) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/serializers.py", line 227, in dump_stream vs = list(itertools.islice(iterator, batch)) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/rdd.py", line 1107, in takeUpToNumLeft yield next(iterator) File "/opt/spark-1.2.0-bin-hadoop2.4/python/pyspark/sql.py", line 816, in convert_struct raise ValueError("unexpected tuple: %s" % obj) TypeError: not all arguments converted during string formatting ``` Author: Gabe Mulley Closes #3978 from mulby/inferschema-namedtuple and squashes the following commits: 98c61cc [Gabe Mulley] Ensure exception message is populated correctly 375d96b [Gabe Mulley] Ensure schema can be inferred from a namedtuple --- python/pyspark/sql.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 0e8b398fc6b97..014ac1791c849 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -807,14 +807,14 @@ def convert_struct(obj): return if isinstance(obj, tuple): - if hasattr(obj, "fields"): - d = dict(zip(obj.fields, obj)) - if hasattr(obj, "__FIELDS__"): + if hasattr(obj, "_fields"): + d = dict(zip(obj._fields, obj)) + elif hasattr(obj, "__FIELDS__"): d = dict(zip(obj.__FIELDS__, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): d = dict(obj) else: - raise ValueError("unexpected tuple: %s" % obj) + raise ValueError("unexpected tuple: %s" % str(obj)) elif isinstance(obj, dict): d = obj @@ -1327,6 +1327,16 @@ def inferSchema(self, rdd, samplingRatio=None): >>> srdd = sqlCtx.inferSchema(nestedRdd2) >>> srdd.collect() [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] + + >>> from collections import namedtuple + >>> CustomRow = namedtuple('CustomRow', 'field1 field2') + >>> rdd = sc.parallelize( + ... [CustomRow(field1=1, field2="row1"), + ... CustomRow(field1=2, field2="row2"), + ... CustomRow(field1=3, field2="row3")]) + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.collect()[0] + Row(field1=1, field2=u'row1') """ if isinstance(rdd, SchemaRDD): From f7741a9a72fef23c46f0ad9e1bd16b150967d816 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 13 Jan 2015 09:28:21 -0800 Subject: [PATCH 12/46] [SPARK-5006][Deploy]spark.port.maxRetries doesn't work https://issues.apache.org/jira/browse/SPARK-5006 I think the issue is produced in https://github.com/apache/spark/pull/1777. Not digging mesos's backend yet. Maybe should add same logic either. Author: WangTaoTheTonic Author: WangTao Closes #3841 from WangTaoTheTonic/SPARK-5006 and squashes the following commits: 8cdf96d [WangTao] indent thing 2d86d65 [WangTaoTheTonic] fix line length 7cdfd98 [WangTaoTheTonic] fit for new HttpServer constructor 61a370d [WangTaoTheTonic] some minor fixes bc6e1ec [WangTaoTheTonic] rebase 67bcb46 [WangTaoTheTonic] put conf at 3rd position, modify suite class, add comments f450cd1 [WangTaoTheTonic] startServiceOnPort will use a SparkConf arg 29b751b [WangTaoTheTonic] rebase as ExecutorRunnableUtil changed to ExecutorRunnable 396c226 [WangTaoTheTonic] make the grammar more like scala 191face [WangTaoTheTonic] invalid value name 62ec336 [WangTaoTheTonic] spark.port.maxRetries doesn't work --- .../org/apache/spark/HttpFileServer.scala | 3 ++- .../scala/org/apache/spark/HttpServer.scala | 3 ++- .../scala/org/apache/spark/SparkConf.scala | 6 +++-- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../spark/broadcast/HttpBroadcast.scala | 3 ++- .../spark/network/nio/ConnectionManager.scala | 2 +- .../org/apache/spark/ui/JettyUtils.scala | 2 +- .../org/apache/spark/util/AkkaUtils.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 23 +++++++++---------- .../streaming/flume/FlumeStreamSuite.scala | 2 +- .../streaming/mqtt/MQTTStreamSuite.scala | 3 ++- .../org/apache/spark/repl/SparkIMain.scala | 2 +- .../scala/org/apache/spark/repl/Main.scala | 2 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 9 +++----- 14 files changed, 33 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index edc3889c9ae51..677c5e0f89d72 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,6 +24,7 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer( + conf: SparkConf, securityManager: SecurityManager, requestedPort: Int = 0) extends Logging { @@ -41,7 +42,7 @@ private[spark] class HttpFileServer( fileDir.mkdir() jarDir.mkdir() logInfo("HTTP File server directory is " + baseDir) - httpServer = new HttpServer(baseDir, securityManager, requestedPort, "HTTP file server") + httpServer = new HttpServer(conf, baseDir, securityManager, requestedPort, "HTTP file server") httpServer.start() serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 912558d0cab7d..fa22787ce7ea3 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -42,6 +42,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes * around a Jetty server. */ private[spark] class HttpServer( + conf: SparkConf, resourceBase: File, securityManager: SecurityManager, requestedPort: Int = 0, @@ -57,7 +58,7 @@ private[spark] class HttpServer( } else { logInfo("Starting HTTP Server") val (actualServer, actualPort) = - Utils.startServiceOnPort[Server](requestedPort, doStart, serverName) + Utils.startServiceOnPort[Server](requestedPort, doStart, conf, serverName) server = actualServer port = actualPort } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index c14764f773982..a0ce107f43b16 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -370,7 +370,9 @@ private[spark] object SparkConf { } /** - * Return whether the given config is a Spark port config. + * Return true if the given config matches either `spark.*.port` or `spark.port.*`. */ - def isSparkPortConf(name: String): Boolean = name.startsWith("spark.") && name.endsWith(".port") + def isSparkPortConf(name: String): Boolean = { + (name.startsWith("spark.") && name.endsWith(".port")) || name.startsWith("spark.port.") + } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 43436a1697000..4d418037bd33f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -312,7 +312,7 @@ object SparkEnv extends Logging { val httpFileServer = if (isDriver) { val fileServerPort = conf.getInt("spark.fileserver.port", 0) - val server = new HttpFileServer(securityManager, fileServerPort) + val server = new HttpFileServer(conf, securityManager, fileServerPort) server.initialize() conf.set("spark.fileserver.uri", server.serverUri) server diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 31f0a462f84d8..31d6958c403b3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -153,7 +153,8 @@ private[broadcast] object HttpBroadcast extends Logging { private def createServer(conf: SparkConf) { broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) val broadcastPort = conf.getInt("spark.broadcast.port", 0) - server = new HttpServer(broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") + server = + new HttpServer(conf, broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") server.start() serverUri = server.uri logInfo("Broadcast server started at " + serverUri) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 3340fca08014e..03c4137ca0a81 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -174,7 +174,7 @@ private[nio] class ConnectionManager( serverChannel.socket.bind(new InetSocketAddress(port)) (serverChannel, serverChannel.socket.getLocalPort) } - Utils.startServiceOnPort[ServerSocketChannel](port, startService, name) + Utils.startServiceOnPort[ServerSocketChannel](port, startService, conf, name) serverChannel.register(selector, SelectionKey.OP_ACCEPT) val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 2a27d49d2de05..88fed833f922d 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -201,7 +201,7 @@ private[spark] object JettyUtils extends Logging { } } - val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, serverName) + val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, conf, serverName) ServerInfo(server, boundPort, collection) } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index db2531dc171f8..4c9b1e3c46f0f 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -53,7 +53,7 @@ private[spark] object AkkaUtils extends Logging { val startService: Int => (ActorSystem, Int) = { actualPort => doCreateActorSystem(name, host, actualPort, conf, securityManager) } - Utils.startServiceOnPort(port, startService, name) + Utils.startServiceOnPort(port, startService, conf, name) } private def doCreateActorSystem( diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 578f1a9cf4868..2c04e4ddfbcb7 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1690,17 +1690,15 @@ private[spark] object Utils extends Logging { } /** - * Default maximum number of retries when binding to a port before giving up. + * Maximum number of retries when binding to a port before giving up. */ - val portMaxRetries: Int = { - if (sys.props.contains("spark.testing")) { + def portMaxRetries(conf: SparkConf): Int = { + val maxRetries = conf.getOption("spark.port.maxRetries").map(_.toInt) + if (conf.contains("spark.testing")) { // Set a higher number of retries for tests... - sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100) + maxRetries.getOrElse(100) } else { - Option(SparkEnv.get) - .flatMap(_.conf.getOption("spark.port.maxRetries")) - .map(_.toInt) - .getOrElse(16) + maxRetries.getOrElse(16) } } @@ -1709,17 +1707,18 @@ private[spark] object Utils extends Logging { * Each subsequent attempt uses 1 + the port used in the previous attempt (unless the port is 0). * * @param startPort The initial port to start the service on. - * @param maxRetries Maximum number of retries to attempt. - * A value of 3 means attempting ports n, n+1, n+2, and n+3, for example. * @param startService Function to start service on a given port. * This is expected to throw java.net.BindException on port collision. + * @param conf A SparkConf used to get the maximum number of retries when binding to a port. + * @param serviceName Name of the service. */ def startServiceOnPort[T]( startPort: Int, startService: Int => (T, Int), - serviceName: String = "", - maxRetries: Int = portMaxRetries): (T, Int) = { + conf: SparkConf, + serviceName: String = ""): (T, Int) = { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" + val maxRetries = portMaxRetries(conf) for (offset <- 0 to maxRetries) { // Do not increment port if startPort is 0, which is treated as a special port val tryPort = if (startPort == 0) { diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 13943ed5442b9..f333e3891b5f0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -80,7 +80,7 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) - })._2 + }, conf)._2 } /** Setup and start the streaming context */ diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 30727dfa64437..fe53a29cba0c9 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -32,6 +32,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.SparkConf import org.apache.spark.util.Utils class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { @@ -106,7 +107,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) - })._2 + }, new SparkConf())._2 } def publishData(data: String): Unit = { diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 646c68e60c2e9..b646f0b6f0868 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -106,7 +106,7 @@ import org.apache.spark.util.Utils val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles /** Jetty server that will serve our classes to worker nodes */ val classServerPort = conf.getInt("spark.replClassServer.port", 0) - val classServer = new HttpServer(outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") + val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") private var currentSettings: Settings = initialSettings var printResults = true // whether to print result lines var totalSilence = false // whether to print anything diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 5e93a71995072..69e44d4f916e1 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -32,7 +32,7 @@ object Main extends Logging { val s = new Settings() s.processArguments(List("-Yrepl-class-based", "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-Yrepl-sync"), true) - val classServer = new HttpServer(outputDir, new SecurityManager(conf)) + val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) var sparkContext: SparkContext = _ var interp = new SparkILoop // this is a public var because tests reset it. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index ebf5616e8d303..c537da9f67552 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -148,12 +148,9 @@ class ExecutorRunnable( // registers with the Scheduler and transfers the spark configs. Since the Executor backend // uses Akka to connect to the scheduler, the akka settings are needed as well as the // authentication settings. - sparkConf.getAll. - filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - sparkConf.getAkkaConf. - foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + sparkConf.getAll + .filter { case (k, v) => SparkConf.isExecutorStartupConf(k) } + .foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. From 9dea64e53ad8df8a3160c0f4010811af1e73dd6f Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 13 Jan 2015 09:43:48 -0800 Subject: [PATCH 13/46] [SPARK-4697][YARN]System properties should override environment variables I found some arguments in yarn module take environment variables before system properties while the latter override the former in core module. Author: WangTaoTheTonic Author: WangTao Closes #3557 from WangTaoTheTonic/SPARK4697 and squashes the following commits: 836b9ef [WangTaoTheTonic] fix type mismatch e3e486a [WangTaoTheTonic] remove the comma 1262d57 [WangTaoTheTonic] handle spark.app.name and SPARK_YARN_APP_NAME in SparkSubmitArguments bee9447 [WangTaoTheTonic] wrong brace 81833bb [WangTaoTheTonic] rebase 40934b4 [WangTaoTheTonic] just switch blocks 5f43f45 [WangTao] System property can override environment variable --- .../spark/deploy/SparkSubmitArguments.scala | 5 +++++ .../spark/deploy/yarn/ClientArguments.scala | 4 ++-- .../cluster/YarnClientSchedulerBackend.scala | 19 +++++++++++-------- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f14ef4d299383..47059b08a397f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -149,6 +149,11 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St // Global defaults. These should be keep to minimum to avoid confusing behavior. master = Option(master).getOrElse("local[*]") + // In YARN mode, app name can be set via SPARK_YARN_APP_NAME (see SPARK-5222) + if (master.startsWith("yarn")) { + name = Option(name).orElse(env.get("SPARK_YARN_APP_NAME")).orNull + } + // Set name from main class if not given name = Option(name).orElse(Option(mainClass)).orNull if (name == null && primaryResource != null) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index fdbf9f8eed029..461a9ccd3c216 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -64,12 +64,12 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). files = Option(files) - .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) .orElse(sparkConf.getOption("spark.yarn.dist.files").map(p => Utils.resolveURIs(p))) + .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) .orNull archives = Option(archives) - .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) + .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orNull // If dynamic allocation is enabled, start at the max number of executors if (isDynamicAllocationEnabled) { diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index f99291553b7b8..690f927e938c3 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -74,8 +74,7 @@ private[spark] class YarnClientSchedulerBackend( ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), - ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), - ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue") ) // Warn against the following deprecated environment variables: env var -> suggestion val deprecatedEnvVars = Map( @@ -86,18 +85,22 @@ private[spark] class YarnClientSchedulerBackend( // Do the same for deprecated properties: property -> suggestion val deprecatedProps = Map("spark.master.memory" -> "--driver-memory through spark-submit") optionTuples.foreach { case (optionName, envVar, sparkProp) => - if (System.getenv(envVar) != null) { - extraArgs += (optionName, System.getenv(envVar)) - if (deprecatedEnvVars.contains(envVar)) { - logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") - } - } else if (sc.getConf.contains(sparkProp)) { + if (sc.getConf.contains(sparkProp)) { extraArgs += (optionName, sc.getConf.get(sparkProp)) if (deprecatedProps.contains(sparkProp)) { logWarning(s"NOTE: $sparkProp is deprecated. Use ${deprecatedProps(sparkProp)} instead.") } + } else if (System.getenv(envVar) != null) { + extraArgs += (optionName, System.getenv(envVar)) + if (deprecatedEnvVars.contains(envVar)) { + logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") + } } } + // The app name is a special case because "spark.app.name" is required of all applications. + // As a result, the corresponding "SPARK_YARN_APP_NAME" is already handled preemptively in + // SparkSubmitArguments if "spark.app.name" is not explicitly set by the user. (SPARK-5222) + sc.getConf.getOption("spark.app.name").foreach(v => extraArgs += ("--name", v)) extraArgs } From 39e333ec4350ddafe29ee0958c37eec07bec85df Mon Sep 17 00:00:00 2001 From: uncleGen Date: Tue, 13 Jan 2015 10:07:19 -0800 Subject: [PATCH 14/46] [SPARK-5131][Streaming][DOC]: There is a discrepancy in WAL implementation and configuration doc. There is a discrepancy in WAL implementation and configuration doc. Author: uncleGen Closes #3930 from uncleGen/master-clean-doc and squashes the following commits: 3a4245f [uncleGen] doc typo 8e407d3 [uncleGen] doc typo --- docs/configuration.md | 2 +- docs/streaming-programming-guide.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index f292bfbb7dcd6..673cdb371a512 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1228,7 +1228,7 @@ Apart from these, the following properties are also available, and may be useful - spark.streaming.receiver.writeAheadLogs.enable + spark.streaming.receiver.writeAheadLog.enable false Enable write ahead logs for receivers. All the input data received through receivers diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 01450efe35e55..e37a2bb37b9a4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1574,7 +1574,7 @@ To run a Spark Streaming applications, you need to have the following. recovery, thus ensuring zero data loss (discussed in detail in the [Fault-tolerance Semantics](#fault-tolerance-semantics) section). This can be enabled by setting the [configuration parameter](configuration.html#spark-streaming) - `spark.streaming.receiver.writeAheadLogs.enable` to `true`. However, these stronger semantics may + `spark.streaming.receiver.writeAheadLog.enable` to `true`. However, these stronger semantics may come at the cost of the receiving throughput of individual receivers. This can be corrected by running [more receivers in parallel](#level-of-parallelism-in-data-receiving) to increase aggregate throughput. Additionally, it is recommended that the replication of the From 8ead999fd627b12837fb2f082a0e76e9d121d269 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 13 Jan 2015 12:50:31 -0800 Subject: [PATCH 15/46] [SPARK-5223] [MLlib] [PySpark] fix MapConverter and ListConverter in MLlib It will introduce problems if the object in dict/list/tuple can not support by py4j, such as Vector. Also, pickle may have better performance for larger object (less RPC). In some cases that the object in dict/list can not be pickled (such as JavaObject), we should still use MapConvert/ListConvert. This PR should be ported into branch-1.2 Author: Davies Liu Closes #4023 from davies/listconvert and squashes the following commits: 55d4ab2 [Davies Liu] fix MapConverter and ListConverter in MLlib --- python/pyspark/mllib/common.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 33c49e2399908..3c5ee66cd8b64 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -18,7 +18,7 @@ import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject -from py4j.java_collections import MapConverter, ListConverter, JavaArray, JavaList +from py4j.java_collections import ListConverter, JavaArray, JavaList from pyspark import RDD, SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer @@ -70,9 +70,7 @@ def _py2java(sc, obj): obj = _to_java_object_rdd(obj) elif isinstance(obj, SparkContext): obj = obj._jsc - elif isinstance(obj, dict): - obj = MapConverter().convert(obj, sc._gateway._gateway_client) - elif isinstance(obj, (list, tuple)): + elif isinstance(obj, list) and (obj or isinstance(obj[0], JavaObject)): obj = ListConverter().convert(obj, sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass From 6463e0b9e8067cce70602c5c9006a2546856a9d6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 13 Jan 2015 13:01:27 -0800 Subject: [PATCH 16/46] [SPARK-4912][SQL] Persistent tables for the Spark SQL data sources api With changes in this PR, users can persist metadata of tables created based on the data source API in metastore through DDLs. Author: Yin Huai Author: Michael Armbrust Closes #3960 from yhuai/persistantTablesWithSchema2 and squashes the following commits: 069c235 [Yin Huai] Make exception messages user friendly. c07cbc6 [Yin Huai] Get the location of test file in a correct way. 4456e98 [Yin Huai] Test data. 5315dfc [Yin Huai] rxin's comments. 7fc4b56 [Yin Huai] Add DDLStrategy and HiveDDLStrategy to plan DDLs based on the data source API. aeaf4b3 [Yin Huai] Add comments. 06f9b0c [Yin Huai] Revert unnecessary changes. feb88aa [Yin Huai] Merge remote-tracking branch 'apache/master' into persistantTablesWithSchema2 172db80 [Yin Huai] Fix unit test. 49bf1ac [Yin Huai] Unit tests. 8f8f1a1 [Yin Huai] [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands. #3431 f47fda1 [Yin Huai] Unit tests. 2b59723 [Michael Armbrust] Set external when creating tables c00bb1b [Michael Armbrust] Don't use reflection to read options 1ea6e7b [Michael Armbrust] Don't fail when trying to uncache a table that doesn't exist 6edc710 [Michael Armbrust] Add tests. d7da491 [Michael Armbrust] First draft of persistent tables. --- .../org/apache/spark/sql/SQLContext.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 14 + .../apache/spark/sql/execution/commands.scala | 3 +- .../org/apache/spark/sql/sources/ddl.scala | 53 ++-- .../apache/spark/sql/sources/interfaces.scala | 17 +- .../spark/sql/sources/TableScanSuite.scala | 30 +++ .../apache/spark/sql/hive/HiveContext.scala | 12 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 79 +++++- .../spark/sql/hive/HiveStrategies.scala | 11 + .../org/apache/spark/sql/hive/TestHive.scala | 1 + .../spark/sql/hive/execution/commands.scala | 21 ++ sql/hive/src/test/resources/sample.json | 2 + .../sql/hive/MetastoreDataSourcesSuite.scala | 244 ++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 1 - 14 files changed, 461 insertions(+), 28 deletions(-) create mode 100644 sql/hive/src/test/resources/sample.json create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 6c575dd727b46..e7021cc3366ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -330,6 +330,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def strategies: Seq[Strategy] = extraStrategies ++ ( DataSourceStrategy :: + DDLStrategy :: TakeOrdered :: HashAggregation :: LeftSemiJoin :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 99b6611d3bbcf..d91b1fbc69834 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ @@ -310,4 +311,17 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => Nil } } + + object DDLStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case CreateTableUsing(tableName, userSpecifiedSchema, provider, true, options) => + ExecutedCommand( + CreateTempTableUsing(tableName, userSpecifiedSchema, provider, options)) :: Nil + + case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, options) => + sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") + + case _ => Nil + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 0d765c4c92f85..df8e61615104c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -177,7 +177,6 @@ case class DescribeCommand( override val output: Seq[Attribute]) extends RunnableCommand { override def run(sqlContext: SQLContext) = { - Row("# Registered as a temporary table", null, null) +: - child.output.map(field => Row(field.name, field.dataType.toString, null)) + child.output.map(field => Row(field.name, field.dataType.toString, null)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index fe2c4d8436b2b..f8741e0082098 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -92,21 +92,21 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi protected lazy val ddl: Parser[LogicalPlan] = createTable /** - * `CREATE TEMPORARY TABLE avroTable + * `CREATE [TEMPORARY] TABLE avroTable * USING org.apache.spark.sql.avro * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")` * or - * `CREATE TEMPORARY TABLE avroTable(intField int, stringField string...) + * `CREATE [TEMPORARY] TABLE avroTable(intField int, stringField string...) * USING org.apache.spark.sql.avro * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")` */ protected lazy val createTable: Parser[LogicalPlan] = ( - CREATE ~ TEMPORARY ~ TABLE ~> ident + (CREATE ~> TEMPORARY.? <~ TABLE) ~ ident ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ { - case tableName ~ columns ~ provider ~ opts => + case temp ~ tableName ~ columns ~ provider ~ opts => val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) - CreateTableUsing(tableName, userSpecifiedSchema, provider, opts) + CreateTableUsing(tableName, userSpecifiedSchema, provider, temp.isDefined, opts) } ) @@ -175,13 +175,12 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi primitiveType } -private[sql] case class CreateTableUsing( - tableName: String, - userSpecifiedSchema: Option[StructType], - provider: String, - options: Map[String, String]) extends RunnableCommand { - - def run(sqlContext: SQLContext) = { +object ResolvedDataSource { + def apply( + sqlContext: SQLContext, + userSpecifiedSchema: Option[StructType], + provider: String, + options: Map[String, String]): ResolvedDataSource = { val loader = Utils.getContextOrSparkClassLoader val clazz: Class[_] = try loader.loadClass(provider) catch { case cnf: java.lang.ClassNotFoundException => @@ -199,22 +198,44 @@ private[sql] case class CreateTableUsing( .asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider] .createRelation(sqlContext, new CaseInsensitiveMap(options), schema) case _ => - sys.error(s"${clazz.getCanonicalName} should extend SchemaRelationProvider.") + sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") } } case None => { clazz.newInstance match { - case dataSource: org.apache.spark.sql.sources.RelationProvider => + case dataSource: org.apache.spark.sql.sources.RelationProvider => dataSource .asInstanceOf[org.apache.spark.sql.sources.RelationProvider] .createRelation(sqlContext, new CaseInsensitiveMap(options)) case _ => - sys.error(s"${clazz.getCanonicalName} should extend RelationProvider.") + sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") } } } - sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName) + new ResolvedDataSource(clazz, relation) + } +} + +private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRelation) + +private[sql] case class CreateTableUsing( + tableName: String, + userSpecifiedSchema: Option[StructType], + provider: String, + temporary: Boolean, + options: Map[String, String]) extends Command + +private [sql] case class CreateTempTableUsing( + tableName: String, + userSpecifiedSchema: Option[StructType], + provider: String, + options: Map[String, String]) extends RunnableCommand { + + def run(sqlContext: SQLContext) = { + val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) + + sqlContext.baseRelationToSchemaRDD(resolved.relation).registerTempTable(tableName) Seq.empty } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 990f7e0e74bcf..2a7be23e37c74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} /** * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source. When - * Spark SQL is given a DDL operation with a USING clause specified, this interface is used to - * pass in the parameters specified by a user. + * Spark SQL is given a DDL operation with a USING clause specified (to specify the implemented + * RelationProvider), this interface is used to pass in the parameters specified by a user. * * Users may specify the fully qualified class name of a given data source. When that class is * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for @@ -46,10 +46,10 @@ trait RelationProvider { /** * ::DeveloperApi:: - * Implemented by objects that produce relations for a specific kind of data source. When - * Spark SQL is given a DDL operation with - * 1. USING clause: to specify the implemented SchemaRelationProvider - * 2. User defined schema: users can define schema optionally when create table + * Implemented by objects that produce relations for a specific kind of data source + * with a given schema. When Spark SQL is given a DDL operation with a USING clause specified ( + * to specify the implemented SchemaRelationProvider) and a user defined schema, this interface + * is used to pass in the parameters specified by a user. * * Users may specify the fully qualified class name of a given data source. When that class is * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for @@ -57,6 +57,11 @@ trait RelationProvider { * data source 'org.apache.spark.sql.json.DefaultSource' * * A new instance of this class with be instantiated each time a DDL call is made. + * + * The difference between a [[RelationProvider]] and a [[SchemaRelationProvider]] is that + * users need to provide a schema when using a SchemaRelationProvider. + * A relation provider can inherits both [[RelationProvider]] and [[SchemaRelationProvider]] + * if it can support both schema inference and user-specified schemas. */ @DeveloperApi trait SchemaRelationProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 605190f5ae6a2..a1d2468b2573c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -314,4 +314,34 @@ class TableScanSuite extends DataSourceTest { sql("SELECT * FROM oneToTenDef"), (1 to 10).map(Row(_)).toSeq) } + + test("exceptions") { + // Make sure we do throw correct exception when users use a relation provider that + // only implements the RelationProvier or the SchemaRelationProvider. + val schemaNotAllowed = intercept[Exception] { + sql( + """ + |CREATE TEMPORARY TABLE relationProvierWithSchema (i int) + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + } + assert(schemaNotAllowed.getMessage.contains("does not allow user-specified schemas")) + + val schemaNeeded = intercept[Exception] { + sql( + """ + |CREATE TEMPORARY TABLE schemaRelationProvierWithoutSchema + |USING org.apache.spark.sql.sources.AllDataTypesScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + } + assert(schemaNeeded.getMessage.contains("A schema needs to be specified when using")) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 02eac43b2103f..09ff4cc5ab437 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -115,6 +115,16 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) } + def refreshTable(tableName: String): Unit = { + // TODO: Database support... + catalog.refreshTable("default", tableName) + } + + protected[hive] def invalidateTable(tableName: String): Unit = { + // TODO: Database support... + catalog.invalidateTable("default", tableName) + } + /** * Analyzes the given table in the current database to generate statistics, which will be * used in query optimizations. @@ -340,6 +350,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def strategies: Seq[Strategy] = extraStrategies ++ Seq( DataSourceStrategy, HiveCommandStrategy(self), + HiveDDLStrategy, + DDLStrategy, TakeOrdered, ParquetOperations, InMemoryScans, 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 c25288e000122..daeabb6c8bab8 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 @@ -20,10 +20,11 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} +import com.google.common.cache.{LoadingCache, CacheLoader, CacheBuilder} + import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.hive.metastore.TableType -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition, FieldSchema} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table, HiveException} import org.apache.hadoop.hive.ql.metadata.InvalidTableException import org.apache.hadoop.hive.ql.plan.CreateTableDesc @@ -39,6 +40,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.sources.{LogicalRelation, ResolvedDataSource} import org.apache.spark.util.Utils /* Implicit conversions */ @@ -50,8 +52,76 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with /** Connection to hive metastore. Usages should lock on `this`. */ protected[hive] val client = Hive.get(hive.hiveconf) + // TODO: Use this everywhere instead of tuples or databaseName, tableName,. + /** A fully qualified identifier for a table (i.e., database.tableName) */ + case class QualifiedTableName(database: String, name: String) { + def toLowerCase = QualifiedTableName(database.toLowerCase, name.toLowerCase) + } + + /** A cache of Spark SQL data source tables that have been accessed. */ + protected[hive] val cachedDataSourceTables: LoadingCache[QualifiedTableName, LogicalPlan] = { + val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { + override def load(in: QualifiedTableName): LogicalPlan = { + logDebug(s"Creating new cached data source for $in") + val table = client.getTable(in.database, in.name) + val schemaString = table.getProperty("spark.sql.sources.schema") + val userSpecifiedSchema = + if (schemaString == null) { + None + } else { + Some(DataType.fromJson(schemaString).asInstanceOf[StructType]) + } + // It does not appear that the ql client for the metastore has a way to enumerate all the + // SerDe properties directly... + val options = table.getTTable.getSd.getSerdeInfo.getParameters.toMap + + val resolvedRelation = + ResolvedDataSource( + hive, + userSpecifiedSchema, + table.getProperty("spark.sql.sources.provider"), + options) + + LogicalRelation(resolvedRelation.relation) + } + } + + CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader) + } + + def refreshTable(databaseName: String, tableName: String): Unit = { + cachedDataSourceTables.refresh(QualifiedTableName(databaseName, tableName).toLowerCase) + } + + def invalidateTable(databaseName: String, tableName: String): Unit = { + cachedDataSourceTables.invalidate(QualifiedTableName(databaseName, tableName).toLowerCase) + } + val caseSensitive: Boolean = false + def createDataSourceTable( + tableName: String, + userSpecifiedSchema: Option[StructType], + provider: String, + options: Map[String, String]) = { + val (dbName, tblName) = processDatabaseAndTableName("default", tableName) + val tbl = new Table(dbName, tblName) + + tbl.setProperty("spark.sql.sources.provider", provider) + if (userSpecifiedSchema.isDefined) { + tbl.setProperty("spark.sql.sources.schema", userSpecifiedSchema.get.json) + } + options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } + + tbl.setProperty("EXTERNAL", "TRUE") + tbl.setTableType(TableType.EXTERNAL_TABLE) + + // create the table + synchronized { + client.createTable(tbl, false) + } + } + def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( @@ -72,7 +142,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last val table = client.getTable(databaseName, tblName) - if (table.isView) { + + if (table.getProperty("spark.sql.sources.provider") != null) { + cachedDataSourceTables(QualifiedTableName(databaseName, tblName).toLowerCase) + } else if (table.isView) { // if the unresolved relation is from hive view // parse the text into logic node. HiveQl.createPlanForView(table, alias) 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 c439b9ebfe104..cdff82e3d04d2 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 @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.sources.CreateTableUsing import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy} import scala.collection.JavaConversions._ @@ -208,6 +209,16 @@ private[hive] trait HiveStrategies { } } + object HiveDDLStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, options) => + ExecutedCommand( + CreateMetastoreDataSource(tableName, userSpecifiedSchema, provider, options)) :: Nil + + case _ => Nil + } + } + case class HiveCommandStrategy(context: HiveContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case describe: DescribeCommand => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 1358a0eccb353..31c7ce96398eb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -395,6 +395,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { clearCache() loadedTables.clear() + catalog.cachedDataSourceTables.invalidateAll() catalog.client.getAllTables("default").foreach { t => logDebug(s"Deleting table $t") val table = catalog.client.getTable("default", t) 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 6b733a280e6d5..e70cdeaad4c09 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.SQLContext @@ -52,6 +53,12 @@ case class DropTable( override def run(sqlContext: SQLContext) = { val hiveContext = sqlContext.asInstanceOf[HiveContext] val ifExistsClause = if (ifExists) "IF EXISTS " else "" + try { + hiveContext.tryUncacheQuery(hiveContext.table(tableName)) + } catch { + case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException => + } + hiveContext.invalidateTable(tableName) hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(Seq(tableName)) Seq.empty[Row] @@ -85,3 +92,17 @@ case class AddFile(path: String) extends RunnableCommand { Seq.empty[Row] } } + +case class CreateMetastoreDataSource( + tableName: String, + userSpecifiedSchema: Option[StructType], + provider: String, + options: Map[String, String]) extends RunnableCommand { + + override def run(sqlContext: SQLContext) = { + val hiveContext = sqlContext.asInstanceOf[HiveContext] + hiveContext.catalog.createDataSourceTable(tableName, userSpecifiedSchema, provider, options) + + Seq.empty[Row] + } +} diff --git a/sql/hive/src/test/resources/sample.json b/sql/hive/src/test/resources/sample.json new file mode 100644 index 0000000000000..a2c2ffd5e0330 --- /dev/null +++ b/sql/hive/src/test/resources/sample.json @@ -0,0 +1,2 @@ +{"a" : "2" ,"b" : "blah", "c_!@(3)":1} +{"" : {"d!" : [4, 5], "=" : [{"Dd2": null}, {"Dd2" : true}]}} 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 new file mode 100644 index 0000000000000..ec9ebb4a775a3 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -0,0 +1,244 @@ +/* + * 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 org.scalatest.BeforeAndAfterEach + +import org.apache.commons.io.FileUtils + +import org.apache.spark.sql._ +import org.apache.spark.util.Utils + +/* Implicits */ +import org.apache.spark.sql.hive.test.TestHive._ + +/** + * Tests for persisting tables created though the data sources API into the metastore. + */ +class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { + override def afterEach(): Unit = { + reset() + } + + val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile + + test ("persistent JSON table") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + jsonFile(filePath).collect().toSeq) + } + + test ("persistent JSON table with a user specified schema") { + sql( + s""" + |CREATE TABLE jsonTable ( + |a string, + |b String, + |`c_!@(3)` int, + |`` Struct<`d!`:array, `=`:array>>) + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + jsonFile(filePath).registerTempTable("expectedJsonTable") + + checkAnswer( + sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM jsonTable"), + sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM expectedJsonTable").collect().toSeq) + } + + test ("persistent JSON table with a user specified schema with a subset of fields") { + // This works because JSON objects are self-describing and JSONRelation can get needed + // field values based on field names. + sql( + s""" + |CREATE TABLE jsonTable (`` Struct<`=`:array>>, b String) + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + val innerStruct = StructType( + StructField("=", ArrayType(StructType(StructField("Dd2", BooleanType, true) :: Nil))) :: Nil) + val expectedSchema = StructType( + StructField("", innerStruct, true) :: + StructField("b", StringType, true) :: Nil) + + assert(expectedSchema == table("jsonTable").schema) + + jsonFile(filePath).registerTempTable("expectedJsonTable") + + checkAnswer( + sql("SELECT b, ``.`=` FROM jsonTable"), + sql("SELECT b, ``.`=` FROM expectedJsonTable").collect().toSeq) + } + + test("resolve shortened provider names") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + jsonFile(filePath).collect().toSeq) + } + + test("drop table") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + jsonFile(filePath).collect().toSeq) + + sql("DROP TABLE jsonTable") + + intercept[Exception] { + sql("SELECT * FROM jsonTable").collect() + } + } + + test("check change without refresh") { + val tempDir = File.createTempFile("sparksql", "json") + tempDir.delete() + sparkContext.parallelize(("a", "b") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${tempDir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + ("a", "b") :: Nil) + + FileUtils.deleteDirectory(tempDir) + sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + + // Schema is cached so the new column does not show. The updated values in existing columns + // will show. + checkAnswer( + sql("SELECT * FROM jsonTable"), + ("a1", "b1") :: Nil) + + refreshTable("jsonTable") + + // Check that the refresh worked + checkAnswer( + sql("SELECT * FROM jsonTable"), + ("a1", "b1", "c1") :: Nil) + FileUtils.deleteDirectory(tempDir) + } + + test("drop, change, recreate") { + val tempDir = File.createTempFile("sparksql", "json") + tempDir.delete() + sparkContext.parallelize(("a", "b") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${tempDir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + ("a", "b") :: Nil) + + FileUtils.deleteDirectory(tempDir) + sparkContext.parallelize(("a", "b", "c") :: Nil).toJSON.saveAsTextFile(tempDir.getCanonicalPath) + + sql("DROP TABLE jsonTable") + + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${tempDir.getCanonicalPath}' + |) + """.stripMargin) + + // New table should reflect new schema. + checkAnswer( + sql("SELECT * FROM jsonTable"), + ("a", "b", "c") :: Nil) + FileUtils.deleteDirectory(tempDir) + } + + test("invalidate cache and reload") { + sql( + s""" + |CREATE TABLE jsonTable (`c_!@(3)` int) + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + jsonFile(filePath).registerTempTable("expectedJsonTable") + + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) + + // Discard the cached relation. + invalidateTable("jsonTable") + + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) + + invalidateTable("jsonTable") + val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) + + assert(expectedSchema == table("jsonTable").schema) + } +} 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 700a45edb11d6..4decd1548534b 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 @@ -623,7 +623,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assertResult( Array( - Array("# Registered as a temporary table", null, null), Array("a", "IntegerType", null), Array("b", "StringType", null)) ) { From 14e3f114efb906937b2d7b7ac04484b2814a3b48 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 13 Jan 2015 13:30:35 -0800 Subject: [PATCH 17/46] [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext This change should be binary and source backward compatible since we didn't change any user facing APIs. Author: Reynold Xin Closes #3965 from rxin/SPARK-5168-sqlconf and squashes the following commits: 42eec09 [Reynold Xin] Fix default conf value. 0ef86cc [Reynold Xin] Fix constructor ordering. 4d7f910 [Reynold Xin] Properly override config. ccc8e6a [Reynold Xin] [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext --- .../org/apache/spark/sql/CacheManager.scala | 4 +- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 40 +++++++++++++++---- .../spark/sql/api/java/JavaSQLContext.scala | 6 +-- .../columnar/InMemoryColumnarTableScan.scala | 4 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../spark/sql/execution/SparkPlan.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 18 ++++----- .../apache/spark/sql/execution/commands.scala | 2 +- .../execution/joins/BroadcastHashJoin.scala | 2 +- .../apache/spark/sql/json/JSONRelation.scala | 4 +- .../spark/sql/parquet/ParquetRelation.scala | 7 ++-- .../spark/sql/parquet/ParquetTest.scala | 2 +- .../apache/spark/sql/parquet/newParquet.scala | 4 +- .../apache/spark/sql/sources/interfaces.scala | 2 +- .../spark/sql/test/TestSQLContext.scala | 5 ++- .../org/apache/spark/sql/JoinSuite.scala | 2 +- .../org/apache/spark/sql/SQLConfSuite.scala | 10 ++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 ++--- .../columnar/InMemoryColumnarQuerySuite.scala | 3 +- .../columnar/PartitionBatchPruningSuite.scala | 4 +- .../spark/sql/execution/PlannerSuite.scala | 4 +- .../org/apache/spark/sql/json/JsonSuite.scala | 2 +- .../spark/sql/parquet/ParquetIOSuite.scala | 4 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 14 +++---- .../execution/HiveCompatibilitySuite.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 11 ++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/TestHive.scala | 6 ++- .../sql/hive/api/java/JavaHiveContext.scala | 6 +-- .../spark/sql/hive/StatisticsSuite.scala | 20 +++++----- .../sql/hive/execution/HiveQuerySuite.scala | 4 +- 33 files changed, 124 insertions(+), 92 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index 3c9439b2e9a52..e715d9434a2ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -91,8 +91,8 @@ private[sql] trait CacheManager { CachedData( planToCache, InMemoryRelation( - useCompression, - columnBatchSize, + conf.useCompression, + conf.columnBatchSize, storageLevel, query.queryExecution.executedPlan, tableName)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index f5bf935522dad..206d16f5b34f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -61,7 +61,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -private[sql] trait SQLConf { +private[sql] class SQLConf { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index e7021cc3366ca..d8efce0cb43eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.util.Properties + +import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag @@ -49,7 +52,6 @@ import org.apache.spark.sql.sources.{DataSourceStrategy, BaseRelation, DDLParser @AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging - with SQLConf with CacheManager with ExpressionConversions with UDFRegistration @@ -57,6 +59,30 @@ class SQLContext(@transient val sparkContext: SparkContext) self => + // Note that this is a lazy val so we can override the default value in subclasses. + private[sql] lazy val conf: SQLConf = new SQLConf + + /** Set Spark SQL configuration properties. */ + def setConf(props: Properties): Unit = conf.setConf(props) + + /** Set the given Spark SQL configuration property. */ + def setConf(key: String, value: String): Unit = conf.setConf(key, value) + + /** Return the value of Spark SQL configuration property for the given key. */ + def getConf(key: String): String = conf.getConf(key) + + /** + * Return the value of Spark SQL configuration property for the given key. If the key is not set + * yet, return `defaultValue`. + */ + def getConf(key: String, defaultValue: String): String = conf.getConf(key, defaultValue) + + /** + * Return all the configuration properties that have been set (i.e. not the default). + * This creates a new copy of the config properties in the form of a Map. + */ + def getAllConfs: immutable.Map[String, String] = conf.getAllConfs + @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) @@ -212,7 +238,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = { - val columnNameOfCorruptJsonRecord = columnNameOfCorruptRecord + val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord val appliedSchema = Option(schema).getOrElse( JsonRDD.nullTypeToStringType( @@ -226,7 +252,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = { - val columnNameOfCorruptJsonRecord = columnNameOfCorruptRecord + val columnNameOfCorruptJsonRecord = conf.columnNameOfCorruptRecord val appliedSchema = JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) @@ -299,10 +325,10 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def sql(sqlText: String): SchemaRDD = { - if (dialect == "sql") { + if (conf.dialect == "sql") { new SchemaRDD(this, parseSql(sqlText)) } else { - sys.error(s"Unsupported SQL dialect: $dialect") + sys.error(s"Unsupported SQL dialect: ${conf.dialect}") } } @@ -323,9 +349,9 @@ class SQLContext(@transient val sparkContext: SparkContext) val sqlContext: SQLContext = self - def codegenEnabled = self.codegenEnabled + def codegenEnabled = self.conf.codegenEnabled - def numPartitions = self.numShufflePartitions + def numPartitions = self.conf.numShufflePartitions def strategies: Seq[Strategy] = extraStrategies ++ ( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 8884204e5079f..7f868cd4afca4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -52,7 +52,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { * @group userf */ def sql(sqlText: String): JavaSchemaRDD = { - if (sqlContext.dialect == "sql") { + if (sqlContext.conf.dialect == "sql") { new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlText)) } else { sys.error(s"Unsupported SQL dialect: $sqlContext.dialect") @@ -164,7 +164,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { * It goes through the entire dataset once to determine the schema. */ def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = { - val columnNameOfCorruptJsonRecord = sqlContext.columnNameOfCorruptRecord + val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord val appliedScalaSchema = JsonRDD.nullTypeToStringType( JsonRDD.inferSchema(json.rdd, 1.0, columnNameOfCorruptJsonRecord)) @@ -182,7 +182,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { */ @Experimental def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { - val columnNameOfCorruptJsonRecord = sqlContext.columnNameOfCorruptRecord + val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord val appliedScalaSchema = Option(asScalaDataType(schema)).getOrElse( JsonRDD.nullTypeToStringType( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 1e432485c4c29..065fae3c83df1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -82,7 +82,7 @@ private[sql] case class InMemoryRelation( if (batchStats.value.isEmpty) { // Underlying columnar RDD hasn't been materialized, no useful statistics information // available, return the default statistics. - Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) } else { // Underlying columnar RDD has been materialized, required information has also been collected // via the `batchStats` accumulator, compute the final statistics, and update `_statistics`. @@ -233,7 +233,7 @@ private[sql] case class InMemoryColumnarTableScan( val readPartitions = sparkContext.accumulator(0) val readBatches = sparkContext.accumulator(0) - private val inMemoryPartitionPruningEnabled = sqlContext.inMemoryPartitionPruning + private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning override def execute() = { readPartitions.setValue(0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index d7c811ca89022..7c0b72aab448e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -123,7 +123,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una */ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. - def numPartitions = sqlContext.numShufflePartitions + def numPartitions = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index d2d8cb1c62d40..069e950195302 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -69,7 +69,7 @@ case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLCont @transient override lazy val statistics = Statistics( // TODO: Instead of returning a default value here, find a way to return a meaningful size // estimate for RDDs. See PR 1238 for more discussions. - sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + sizeInBytes = BigInt(sqlContext.conf.defaultSizeInBytes) ) } @@ -106,6 +106,6 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ @transient override lazy val statistics = Statistics( // TODO: Instead of returning a default value here, find a way to return a meaningful size // estimate for RDDs. See PR 1238 for more discussions. - sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + sizeInBytes = BigInt(sqlContext.conf.defaultSizeInBytes) ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 017c78d2c66d4..6fecd1ff066c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -51,7 +51,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ // sqlContext will be null when we are being deserialized on the slaves. In this instance // the value of codegenEnabled will be set by the desserializer after the constructor has run. val codegenEnabled: Boolean = if (sqlContext != null) { - sqlContext.codegenEnabled + sqlContext.conf.codegenEnabled } else { false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index d91b1fbc69834..0652d2ff7c9ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -35,8 +35,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object LeftSemiJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) - if sqlContext.autoBroadcastJoinThreshold > 0 && - right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + if sqlContext.conf.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => val semiJoin = joins.BroadcastLeftSemiJoinHash( leftKeys, rightKeys, planLater(left), planLater(right)) condition.map(Filter(_, semiJoin)).getOrElse(semiJoin) :: Nil @@ -81,13 +81,13 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) - if sqlContext.autoBroadcastJoinThreshold > 0 && - right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + if sqlContext.conf.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildRight) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) - if sqlContext.autoBroadcastJoinThreshold > 0 && - left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + if sqlContext.conf.autoBroadcastJoinThreshold > 0 && + left.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => @@ -215,7 +215,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = - if (sqlContext.parquetFilterPushDown) { + if (sqlContext.conf.parquetFilterPushDown) { (predicates: Seq[Expression]) => { // Note: filters cannot be pushed down to Parquet if they contain more complex // expressions than simple "Attribute cmp Literal" comparisons. Here we remove all @@ -237,7 +237,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetTableScan( _, relation, - if (sqlContext.parquetFilterPushDown) filters else Nil)) :: Nil + if (sqlContext.conf.parquetFilterPushDown) filters else Nil)) :: Nil case _ => Nil } @@ -270,7 +270,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. execution.Sort(sortExprs, global = false, planLater(child)) :: Nil - case logical.Sort(sortExprs, global, child) if sqlContext.externalSortEnabled => + case logical.Sort(sortExprs, global, child) if sqlContext.conf.externalSortEnabled => execution.ExternalSort(sortExprs, global, planLater(child)):: Nil case logical.Sort(sortExprs, global, child) => execution.Sort(sortExprs, global, planLater(child)):: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index df8e61615104c..af6b07bd6c2f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -94,7 +94,7 @@ case class SetCommand( logWarning( s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${sqlContext.numShufflePartitions}")) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${sqlContext.conf.numShufflePartitions}")) // Queries a single property. case Some((key, None)) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index fbe1d06ed2e82..2dd22c020ef12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -43,7 +43,7 @@ case class BroadcastHashJoin( extends BinaryNode with HashJoin { val timeout = { - val timeoutValue = sqlContext.broadcastTimeout + val timeoutValue = sqlContext.conf.broadcastTimeout if (timeoutValue < 0) { Duration.Inf } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index a9a6696cb15e4..f5c02224c82a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -59,8 +59,8 @@ private[sql] case class JSONRelation( JsonRDD.inferSchema( baseRDD, samplingRatio, - sqlContext.columnNameOfCorruptRecord))) + sqlContext.conf.columnNameOfCorruptRecord))) override def buildScan() = - JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.columnNameOfCorruptRecord) + JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.conf.columnNameOfCorruptRecord) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 2835dc3408b96..cde5160149e9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -65,7 +65,7 @@ private[sql] case class ParquetRelation( ParquetTypesConverter.readSchemaFromFile( new Path(path.split(",").head), conf, - sqlContext.isParquetBinaryAsString) + sqlContext.conf.isParquetBinaryAsString) lazy val attributeMap = AttributeMap(output.map(o => o -> o)) @@ -80,7 +80,7 @@ private[sql] case class ParquetRelation( } // TODO: Use data from the footers. - override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) + override lazy val statistics = Statistics(sizeInBytes = sqlContext.conf.defaultSizeInBytes) } private[sql] object ParquetRelation { @@ -163,7 +163,8 @@ private[sql] object ParquetRelation { sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) conf.set(ParquetOutputFormat.COMPRESSION, shortParquetCompressionCodecNames.getOrElse( - sqlContext.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) + sqlContext.conf.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED) + .name()) ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala index b4d48902fd2c6..02ce1b3e6d811 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -54,7 +54,7 @@ trait ParquetTest { try f finally { keys.zip(currentValues).foreach { case (key, Some(value)) => setConf(key, value) - case (key, None) => unsetConf(key) + case (key, None) => conf.unsetConf(key) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 2e0c6c51c00e5..55a2728a85cc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -137,7 +137,7 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) ParquetTypesConverter.readSchemaFromFile( partitions.head.files.head.getPath, Some(sparkContext.hadoopConfiguration), - sqlContext.isParquetBinaryAsString)) + sqlContext.conf.isParquetBinaryAsString)) val dataIncludesKey = partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true) @@ -198,7 +198,7 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) predicates .reduceOption(And) .flatMap(ParquetFilters.createFilter) - .filter(_ => sqlContext.parquetFilterPushDown) + .filter(_ => sqlContext.conf.parquetFilterPushDown) .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) def percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 2a7be23e37c74..7f5564baa00f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -99,7 +99,7 @@ abstract class BaseRelation { * large to broadcast. This method will be called multiple times during query planning * and thus should not perform expensive operations for each invocation. */ - def sizeInBytes = sqlContext.defaultSizeInBytes + def sizeInBytes = sqlContext.conf.defaultSizeInBytes } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 6bb81c76ed8bd..8c80be106f3cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -29,6 +29,7 @@ object TestSQLContext new SparkConf().set("spark.sql.testkey", "true"))) { /** Fewer partitions to speed up testing. */ - override private[spark] def numShufflePartitions: Int = - getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + private[sql] override lazy val conf: SQLConf = new SQLConf { + override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index c7e136388fce8..e5ab16f9dd661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -387,7 +387,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("broadcasted left semi join operator selection") { clearCache() sql("CACHE TABLE testData") - val tmp = autoBroadcastJoinThreshold + val tmp = conf.autoBroadcastJoinThreshold sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=1000000000") Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 60701f0e154f8..bf73d0c7074a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -37,7 +37,7 @@ class SQLConfSuite extends QueryTest with FunSuiteLike { } test("programmatic ways of basic setting and getting") { - clear() + conf.clear() assert(getAllConfs.size === 0) setConf(testKey, testVal) @@ -51,11 +51,11 @@ class SQLConfSuite extends QueryTest with FunSuiteLike { assert(TestSQLContext.getConf(testKey, testVal + "_") == testVal) assert(TestSQLContext.getAllConfs.contains(testKey)) - clear() + conf.clear() } test("parse SQL set commands") { - clear() + conf.clear() sql(s"set $testKey=$testVal") assert(getConf(testKey, testVal + "_") == testVal) assert(TestSQLContext.getConf(testKey, testVal + "_") == testVal) @@ -73,11 +73,11 @@ class SQLConfSuite extends QueryTest with FunSuiteLike { sql(s"set $key=") assert(getConf(key, "0") == "") - clear() + conf.clear() } test("deprecated property") { - clear() + conf.clear() sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") assert(getConf(SQLConf.SHUFFLE_PARTITIONS) == "10") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index d9de5686dce48..bc72daf0880a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -79,7 +79,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("aggregation with codegen") { - val originalValue = codegenEnabled + val originalValue = conf.codegenEnabled setConf(SQLConf.CODEGEN_ENABLED, "true") sql("SELECT key FROM testData GROUP BY key").collect() setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) @@ -245,14 +245,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("sorting") { - val before = externalSortEnabled + val before = conf.externalSortEnabled setConf(SQLConf.EXTERNAL_SORT, "false") sortTest() setConf(SQLConf.EXTERNAL_SORT, before.toString) } test("external sorting") { - val before = externalSortEnabled + val before = conf.externalSortEnabled setConf(SQLConf.EXTERNAL_SORT, "true") sortTest() setConf(SQLConf.EXTERNAL_SORT, before.toString) @@ -600,7 +600,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SET commands semantics using sql()") { - clear() + conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" @@ -632,7 +632,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql(s"SET $nonexistentKey"), Seq(Seq(s"$nonexistentKey=")) ) - clear() + conf.clear() } test("apply schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index fc95dccc74e27..d94729ba92360 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -39,7 +39,8 @@ class InMemoryColumnarQuerySuite extends QueryTest { sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).registerTempTable("sizeTst") cacheTable("sizeTst") assert( - table("sizeTst").queryExecution.logical.statistics.sizeInBytes > autoBroadcastJoinThreshold) + table("sizeTst").queryExecution.logical.statistics.sizeInBytes > + conf.autoBroadcastJoinThreshold) } test("projection") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 1915c25392f1e..592cafbbdc203 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { - val originalColumnBatchSize = columnBatchSize - val originalInMemoryPartitionPruning = inMemoryPartitionPruning + val originalColumnBatchSize = conf.columnBatchSize + val originalInMemoryPartitionPruning = conf.inMemoryPartitionPruning override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index a5af71acfc79a..c5b6fce5fd297 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -60,7 +60,7 @@ class PlannerSuite extends FunSuite { } test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { - val origThreshold = autoBroadcastJoinThreshold + val origThreshold = conf.autoBroadcastJoinThreshold setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) // Using a threshold that is definitely larger than the small testing table (b) below @@ -78,7 +78,7 @@ class PlannerSuite extends FunSuite { } test("InMemoryRelation statistics propagation") { - val origThreshold = autoBroadcastJoinThreshold + val origThreshold = conf.autoBroadcastJoinThreshold setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) testData.limit(3).registerTempTable("tiny") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 8dce3372a8db3..b09f1ac49553b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -713,7 +713,7 @@ class JsonSuite extends QueryTest { test("Corrupt records") { // Test if we can query corrupt records. - val oldColumnNameOfCorruptRecord = TestSQLContext.columnNameOfCorruptRecord + val oldColumnNameOfCorruptRecord = TestSQLContext.conf.columnNameOfCorruptRecord TestSQLContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, "_unparsed") val jsonSchemaRDD = jsonRDD(corruptRecords) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 10a01474e95be..6ac67fcafe16b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -213,7 +213,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest { def checkCompressionCodec(codec: CompressionCodecName): Unit = { withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { withParquetFile(data) { path => - assertResult(parquetCompressionCodec.toUpperCase) { + assertResult(conf.parquetCompressionCodec.toUpperCase) { compressionCodecFor(path) } } @@ -221,7 +221,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest { } // Checks default compression codec - checkCompressionCodec(CompressionCodecName.fromConf(parquetCompressionCodec)) + checkCompressionCodec(CompressionCodecName.fromConf(conf.parquetCompressionCodec)) checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) checkCompressionCodec(CompressionCodecName.GZIP) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index a5fe2e8da2840..0a92336a3cb39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -88,7 +88,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestData // Load test data tables. private var testRDD: SchemaRDD = null - private val originalParquetFilterPushdownEnabled = TestSQLContext.parquetFilterPushDown + private val originalParquetFilterPushdownEnabled = TestSQLContext.conf.parquetFilterPushDown override def beforeAll() { ParquetTestData.writeFile() @@ -144,7 +144,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } ignore("Treat binary as string") { - val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString + val oldIsParquetBinaryAsString = TestSQLContext.conf.isParquetBinaryAsString // Create the test file. val file = getTempFilePath("parquet") @@ -174,7 +174,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Compression options for writing to a Parquetfile") { - val defaultParquetCompressionCodec = TestSQLContext.parquetCompressionCodec + val defaultParquetCompressionCodec = TestSQLContext.conf.parquetCompressionCodec import scala.collection.JavaConversions._ val file = getTempFilePath("parquet") @@ -186,7 +186,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(actualCodec === TestSQLContext.conf.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -202,7 +202,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(actualCodec === TestSQLContext.conf.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -234,7 +234,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(actualCodec === TestSQLContext.conf.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -250,7 +250,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(actualCodec === TestSQLContext.conf.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 23283fd3fe6b1..0d934620aca09 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -36,8 +36,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault - private val originalColumnBatchSize = TestHive.columnBatchSize - private val originalInMemoryPartitionPruning = TestHive.inMemoryPartitionPruning + private val originalColumnBatchSize = TestHive.conf.columnBatchSize + private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 09ff4cc5ab437..9aeebd7e54366 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -71,8 +71,9 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - // Change the default SQL dialect to HiveQL - override private[spark] def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + private[sql] override lazy val conf: SQLConf = new SQLConf { + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe @@ -87,12 +88,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def sql(sqlText: String): SchemaRDD = { // TODO: Create a framework for registering parsers instead of just hardcoding if statements. - if (dialect == "sql") { + if (conf.dialect == "sql") { super.sql(sqlText) - } else if (dialect == "hiveql") { + } else if (conf.dialect == "hiveql") { new SchemaRDD(this, ddlParser(sqlText).getOrElse(HiveQl.parseSql(sqlText))) } else { - sys.error(s"Unsupported SQL dialect: $dialect. Try 'sql' or 'hiveql'") + sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") } } 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 daeabb6c8bab8..785a6a14f49f4 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 @@ -515,7 +515,7 @@ private[hive] case class MetastoreRelation // if the size is still less than zero, we use default size Option(totalSize).map(_.toLong).filter(_ > 0) .getOrElse(Option(rawDataSize).map(_.toLong).filter(_ > 0) - .getOrElse(sqlContext.defaultSizeInBytes))) + .getOrElse(sqlContext.conf.defaultSizeInBytes))) } ) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 31c7ce96398eb..52e1f0d94fbd4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -102,8 +102,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { new this.QueryExecution { val logical = plan } /** Fewer partitions to speed up testing. */ - override private[spark] def numShufflePartitions: Int = - getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + private[sql] override lazy val conf: SQLConf = new SQLConf { + override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + } /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala index 1817c7832490e..038f63f6c7441 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -31,12 +31,12 @@ class JavaHiveContext(sqlContext: SQLContext) extends JavaSQLContext(sqlContext) override def sql(sqlText: String): JavaSchemaRDD = { // TODO: Create a framework for registering parsers instead of just hardcoding if statements. - if (sqlContext.dialect == "sql") { + if (sqlContext.conf.dialect == "sql") { super.sql(sqlText) - } else if (sqlContext.dialect == "hiveql") { + } else if (sqlContext.conf.dialect == "hiveql") { new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText)) } else { - sys.error(s"Unsupported SQL dialect: ${sqlContext.dialect}. Try 'sql' or 'hiveql'") + sys.error(s"Unsupported SQL dialect: ${sqlContext.conf.dialect}. Try 'sql' or 'hiveql'") } } 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 a758f921e0417..0b4e76c9d3d2f 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 @@ -81,7 +81,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // TODO: How does it works? needs to add it back for other hive version. if (HiveShim.version =="0.12.0") { - assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + assert(queryTotalSize("analyzeTable") === conf.defaultSizeInBytes) } sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") @@ -110,7 +110,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { |SELECT * FROM src """.stripMargin).collect() - assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) + assert(queryTotalSize("analyzeTable_part") === conf.defaultSizeInBytes) sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") @@ -151,8 +151,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { val sizes = rdd.queryExecution.analyzed.collect { case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(sizes.size === 2 && sizes(0) <= autoBroadcastJoinThreshold - && sizes(1) <= autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(0) <= conf.autoBroadcastJoinThreshold + && sizes(1) <= 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,8 +163,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { checkAnswer(rdd, expectedAnswer) // check correctness of output - TestHive.settings.synchronized { - val tmp = autoBroadcastJoinThreshold + TestHive.conf.settings.synchronized { + val tmp = conf.autoBroadcastJoinThreshold sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") rdd = sql(query) @@ -207,8 +207,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { .isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(sizes.size === 2 && sizes(1) <= autoBroadcastJoinThreshold - && sizes(0) <= autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(1) <= conf.autoBroadcastJoinThreshold + && sizes(0) <= 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,8 +221,8 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { checkAnswer(rdd, answer) // check correctness of output - TestHive.settings.synchronized { - val tmp = autoBroadcastJoinThreshold + TestHive.conf.settings.synchronized { + val tmp = conf.autoBroadcastJoinThreshold sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") rdd = sql(leftSemiJoinQuery) 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 4decd1548534b..c14f0d24e0dc3 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 @@ -847,7 +847,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { case Row(key: String, value: String) => key -> value case Row(KV(key, value)) => key -> value }.toSet - clear() + conf.clear() // "SET" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... @@ -879,7 +879,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { collectResults(sql(s"SET $nonexistentKey")) } - clear() + conf.clear() } createQueryTest("select from thrift based table", From f9969098c8cb15e36c718b80c6cf5b534a6cf7c3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 13 Jan 2015 17:16:41 -0800 Subject: [PATCH 18/46] [SPARK-5123][SQL] Reconcile Java/Scala API for data types. Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box. As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code. This subsumes https://github.com/apache/spark/pull/3925 Author: Reynold Xin Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits: 66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API). --- .../scala/org/apache/spark/ml/Pipeline.scala | 5 +- .../org/apache/spark/ml/Transformer.scala | 2 +- .../classification/LogisticRegression.scala | 1 + .../BinaryClassificationEvaluator.scala | 3 +- .../apache/spark/ml/feature/HashingTF.scala | 2 +- .../spark/ml/feature/StandardScaler.scala | 1 + .../apache/spark/ml/feature/Tokenizer.scala | 2 +- .../spark/ml/tuning/CrossValidator.scala | 3 +- .../apache/spark/mllib/linalg/Vectors.scala | 3 +- project/MimaExcludes.scala | 12 + project/SparkBuild.scala | 4 +- sql/README.md | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 5 +- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 4 +- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 3 +- .../spark/sql/catalyst/expressions/Rand.scala | 2 +- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../sql/catalyst/expressions/ScalaUdf.scala | 3 +- .../expressions/SpecificMutableRow.scala | 2 +- .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 2 +- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../expressions/codegen/CodeGenerator.scala | 10 +- .../codegen/GenerateOrdering.scala | 2 +- .../codegen/GenerateProjection.scala | 2 +- .../catalyst/expressions/complexTypes.scala | 2 +- .../expressions/decimalFunctions.scala | 4 +- .../sql/catalyst/expressions/generators.scala | 2 +- .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 3 +- .../sql/catalyst/expressions/predicates.scala | 2 +- .../spark/sql/catalyst/expressions/sets.scala | 2 +- .../expressions/stringOperations.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 4 +- .../apache/spark/sql/catalyst/package.scala | 2 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../catalyst/plans/logical/LogicalPlan.scala | 17 +- .../plans/logical/basicOperators.scala | 2 +- .../plans/physical/partitioning.scala | 2 +- .../spark/sql/types/DataTypeConversions.scala | 68 ++++ .../apache/spark/sql/types/DataTypes.java} | 50 +-- .../{catalyst/util => types}/Metadata.scala | 16 +- .../SQLUserDefinedType.java | 3 +- .../sql/{catalyst => }/types/dataTypes.scala | 294 +++++++++++++-- .../types/decimal/Decimal.scala | 2 +- .../sql/{catalyst => }/types/package.scala | 3 +- .../sql/catalyst/ScalaReflectionSuite.scala | 2 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../analysis/DecimalPrecisionSuite.scala | 2 +- .../analysis/HiveTypeCoercionSuite.scala | 2 +- .../ExpressionEvaluationSuite.scala | 10 +- .../optimizer/ConstantFoldingSuite.scala | 2 +- .../catalyst/optimizer/OptimizeInSuite.scala | 2 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 2 +- .../sql/catalyst/util/MetadataSuite.scala | 2 + .../spark/sql/types}/DataTypeSuite.scala | 2 +- .../types/decimal/DecimalSuite.scala | 2 +- .../apache/spark/sql/api/java/ArrayType.java | 68 ---- .../apache/spark/sql/api/java/BinaryType.java | 27 -- .../spark/sql/api/java/BooleanType.java | 27 -- .../apache/spark/sql/api/java/ByteType.java | 27 -- .../apache/spark/sql/api/java/DateType.java | 27 -- .../spark/sql/api/java/DecimalType.java | 79 ---- .../apache/spark/sql/api/java/DoubleType.java | 27 -- .../apache/spark/sql/api/java/FloatType.java | 27 -- .../spark/sql/api/java/IntegerType.java | 27 -- .../apache/spark/sql/api/java/LongType.java | 27 -- .../apache/spark/sql/api/java/MapType.java | 78 ---- .../apache/spark/sql/api/java/Metadata.java | 31 -- .../spark/sql/api/java/MetadataBuilder.java | 28 -- .../apache/spark/sql/api/java/NullType.java | 27 -- .../apache/spark/sql/api/java/ShortType.java | 27 -- .../apache/spark/sql/api/java/StringType.java | 27 -- .../spark/sql/api/java/StructField.java | 91 ----- .../apache/spark/sql/api/java/StructType.java | 58 --- .../spark/sql/api/java/TimestampType.java | 27 -- .../spark/sql/api/java/UserDefinedType.java | 54 --- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../org/apache/spark/sql/SchemaRDD.scala | 6 +- .../org/apache/spark/sql/SparkSQLParser.scala | 4 +- .../spark/sql/api/java/JavaSQLContext.scala | 69 ++-- .../spark/sql/api/java/JavaSchemaRDD.scala | 10 +- .../org/apache/spark/sql/api/java/Row.scala | 2 - .../spark/sql/api/java/UDFRegistration.scala | 139 ++++--- .../spark/sql/api/java/UDTWrappers.scala | 75 ---- .../spark/sql/columnar/ColumnAccessor.scala | 4 +- .../spark/sql/columnar/ColumnBuilder.scala | 2 +- .../spark/sql/columnar/ColumnStats.scala | 2 +- .../spark/sql/columnar/ColumnType.scala | 2 +- .../CompressibleColumnAccessor.scala | 2 +- .../CompressibleColumnBuilder.scala | 2 +- .../compression/CompressionScheme.scala | 2 +- .../compression/compressionSchemes.scala | 3 +- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../sql/execution/GeneratedAggregate.scala | 2 +- .../sql/execution/SparkSqlSerializer.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 6 +- .../spark/sql/execution/debug/package.scala | 2 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../apache/spark/sql/json/JSONRelation.scala | 3 +- .../org/apache/spark/sql/json/JsonRDD.scala | 8 +- .../scala/org/apache/spark/sql/package.scala | 349 ------------------ .../spark/sql/parquet/ParquetConverter.scala | 17 +- .../spark/sql/parquet/ParquetFilters.scala | 2 +- .../sql/parquet/ParquetTableSupport.scala | 4 +- .../spark/sql/parquet/ParquetTypes.scala | 16 +- .../apache/spark/sql/parquet/newParquet.scala | 7 +- .../spark/sql/sources/LogicalRelation.scala | 4 +- .../org/apache/spark/sql/sources/ddl.scala | 10 +- .../apache/spark/sql/sources/interfaces.scala | 3 +- .../spark/sql/test/ExamplePointUDT.scala | 4 +- .../sql/types/util/DataTypeConversions.scala | 175 --------- .../spark/sql/api/java/JavaAPISuite.java | 9 +- .../sql/api/java/JavaApplySchemaSuite.java | 23 +- .../java/JavaSideDataTypeConversionSuite.java | 150 -------- .../org/apache/spark/sql/DslQuerySuite.scala | 1 + .../scala/org/apache/spark/sql/RowSuite.scala | 1 + .../org/apache/spark/sql/SQLQuerySuite.scala | 3 +- .../sql/ScalaReflectionRelationSuite.scala | 1 - .../spark/sql/UserDefinedTypeSuite.scala | 3 +- .../spark/sql/api/java/JavaSQLSuite.scala | 9 +- .../ScalaSideDataTypeConversionSuite.scala | 89 ----- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 2 +- .../sql/columnar/ColumnarTestUtils.scala | 2 +- .../NullableColumnAccessorSuite.scala | 2 +- .../columnar/NullableColumnBuilderSuite.scala | 2 +- .../compression/DictionaryEncodingSuite.scala | 2 +- .../compression/IntegralDeltaSuite.scala | 2 +- .../compression/RunLengthEncodingSuite.scala | 2 +- .../TestCompressibleColumnBuilder.scala | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 16 +- .../spark/sql/parquet/ParquetIOSuite.scala | 6 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +- .../sql/parquet/ParquetSchemaSuite.scala | 21 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 + .../spark/sql/sources/PrunedScanSuite.scala | 1 + .../spark/sql/sources/TableScanSuite.scala | 2 +- .../spark/sql/hive/thriftserver/Shim12.scala | 4 +- .../spark/sql/hive/thriftserver/Shim13.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveInspectors.scala | 15 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../spark/sql/hive/HiveStrategies.scala | 7 +- .../hive/execution/HiveNativeCommand.scala | 2 +- .../sql/hive/execution/HiveTableScan.scala | 4 +- .../spark/sql/hive/execution/commands.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 19 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 2 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 1 + .../sql/hive/MetastoreDataSourcesSuite.scala | 1 + .../org/apache/spark/sql/hive/Shim12.scala | 20 +- .../org/apache/spark/sql/hive/Shim13.scala | 11 +- 160 files changed, 756 insertions(+), 2096 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala rename sql/{core/src/main/java/org/apache/spark/sql/api/java/DataType.java => catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java} (81%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalyst/util => types}/Metadata.scala (96%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalyst/annotation => types}/SQLUserDefinedType.java (93%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalyst => }/types/dataTypes.scala (76%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalyst => }/types/decimal/Decimal.scala (99%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalyst => }/types/package.scala (96%) rename sql/{core/src/test/scala/org/apache/spark/sql => catalyst/src/test/scala/org/apache/spark/sql/types}/DataTypeSuite.scala (98%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/{catalyst => }/types/decimal/DecimalSuite.scala (99%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala delete mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index 081a574beea5d..ad6fed178fae9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -21,8 +21,9 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.{Params, Param, ParamMap} -import org.apache.spark.sql.{SchemaRDD, StructType} +import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.types.StructType /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 23fbd228d01cb..1331b9124045c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.api.java.JavaSchemaRDD import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions.ScalaUdf -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 85b8899636ca5..8c570812f8316 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} import org.apache.spark.storage.StorageLevel /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 0b0504e036ec9..12473cb2b5719 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -21,7 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.sql.{DoubleType, Row, SchemaRDD} +import org.apache.spark.sql.{Row, SchemaRDD} +import org.apache.spark.sql.types.DoubleType /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index e0bfb1e484a2e..0956062643f23 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -22,7 +22,7 @@ import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{VectorUDT, Vector} -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.types.DataType /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 896a6b83b67bf..72825f6e02182 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -25,6 +25,7 @@ import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.types.{StructField, StructType} /** * Params for [[StandardScaler]] and [[StandardScalerModel]]. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 9352f40f372d3..e622a5cf9e6f3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.ParamMap -import org.apache.spark.sql.{DataType, StringType, ArrayType} +import org.apache.spark.sql.types.{DataType, StringType, ArrayType} /** * :: AlphaComponent :: diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 194b9bfd9a9e6..08fe99176424a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -24,7 +24,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params} import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.{SchemaRDD, StructType} +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.types.StructType /** * Params for [[CrossValidator]] and [[CrossValidatorModel]]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index bf1faa25ef0e0..adbd8266ed6fa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -27,9 +27,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.apache.spark.SparkException import org.apache.spark.mllib.util.NumericParser -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Row} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * Represents a numeric vector, whose index type is Int and value type is Double. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 51e8bd4cf6419..f6f9f491f4ceb 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,6 +60,18 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." + "removeAndGetProcessor") + ) ++ Seq( + // SPARK-5123 (SparkSQL data type change) - alpha component only + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.ml.feature.HashingTF.outputDataType"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.ml.feature.Tokenizer.outputDataType"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.feature.Tokenizer.validateInputType"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.classification.LogisticRegressionModel.validateAndTransformSchema"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.ml.classification.LogisticRegression.validateAndTransformSchema") ) case v if v.startsWith("1.2") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 46a54c6818409..b2c546da21c70 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -254,10 +254,10 @@ object SQL { |import org.apache.spark.sql.catalyst.expressions._ |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ - |import org.apache.spark.sql.catalyst.types._ |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.test.TestSQLContext._ + |import org.apache.spark.sql.types._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, cleanupCommands in console := "sparkContext.stop()" ) @@ -284,11 +284,11 @@ object Hive { |import org.apache.spark.sql.catalyst.expressions._ |import org.apache.spark.sql.catalyst.plans.logical._ |import org.apache.spark.sql.catalyst.rules._ - |import org.apache.spark.sql.catalyst.types._ |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ + |import org.apache.spark.sql.types._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, cleanupCommands in console := "sparkContext.stop()", // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce diff --git a/sql/README.md b/sql/README.md index 8d2f3cf4283e0..d058a6b011d37 100644 --- a/sql/README.md +++ b/sql/README.md @@ -34,11 +34,11 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.types._ Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 2cf241de61f7a..d169da691d797 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -20,11 +20,10 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} import org.apache.spark.util.Utils -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 5d974df98b699..d19563e95cc73 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * A very simple SQL parser. Based loosely on: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c009cc1e1e85c..bd00ff22ba80a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -22,8 +22,8 @@ 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._ -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.IntegerType /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 242f28f670298..15353361d97cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index b2262e5e6efb6..bdac7504ed027 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index fa80b07f8e6be..76a9f08dea85f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.trees /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 4ede0b4821fe3..00961f09916b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -22,8 +22,8 @@ import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ac5b02c2e6aed..cf14992ef835c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} -import org.apache.spark.sql.catalyst.util.Metadata +import org.apache.spark.sql.types._ abstract class Expression extends TreeNode[Expression] { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 851db95b9177e..b2c6d3029031d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Random -import org.apache.spark.sql.catalyst.types.DoubleType +import org.apache.spark.sql.types.DoubleType case object Rand extends LeafExpression { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 463f3667fc445..dcda53bb717a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.types.NativeType object Row { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 18c96da2f87fb..8a36c6810790d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.types.DataType -import org.apache.spark.util.ClosureCleaner +import org.apache.spark.sql.types.DataType /** * User-defined function. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 570379c533e1f..37d9f0ed5c79e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * A parent class for mutable container objects that are reused when the values are changed, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 1a4ac06c7a79d..8328278544a1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.language.dynamics -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.types.DataType /** * The data type representing [[DynamicRow]] values. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 5ea9868e9e846..735b7488fdcbd 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import com.clearspring.analytics.stream.cardinality.HyperLogLog -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.util.collection.OpenHashSet diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 168a963e29c90..574907f566c0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 90c81b2631e59..a5d6423391295 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -18,14 +18,14 @@ package org.apache.spark.sql.catalyst.expressions.codegen import com.google.common.cache.{CacheLoader, CacheBuilder} -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types.decimal.Decimal import scala.language.existentials import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ // These classes are here to avoid issues with serialization and integration with quasiquotes. class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] @@ -541,11 +541,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin childEval.code ++ q""" var $nullTerm = ${childEval.nullTerm} - var $primitiveTerm: org.apache.spark.sql.catalyst.types.decimal.Decimal = + var $primitiveTerm: org.apache.spark.sql.types.decimal.Decimal = ${defaultPrimitive(DecimalType())} if (!$nullTerm) { - $primitiveTerm = new org.apache.spark.sql.catalyst.types.decimal.Decimal() + $primitiveTerm = new org.apache.spark.sql.types.decimal.Decimal() $primitiveTerm = $primitiveTerm.setOrNull(${childEval.primitiveTerm}, $precision, $scale) $nullTerm = $primitiveTerm == null } @@ -627,7 +627,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case LongType => ru.Literal(Constant(1L)) case ByteType => ru.Literal(Constant(-1.toByte)) case DoubleType => ru.Literal(Constant(-1.toDouble)) - case DecimalType() => q"org.apache.spark.sql.catalyst.types.decimal.Decimal(-1)" + case DecimalType() => q"org.apache.spark.sql.types.decimal.Decimal(-1)" case IntegerType => ru.Literal(Constant(-1)) case _ => ru.Literal(Constant(null)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 094ff14552283..0db29eb404bd1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{StringType, NumericType} +import org.apache.spark.sql.types.{StringType, NumericType} /** * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 2ff61169a17db..cc97cb4f50b69 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 9aec601886efc..1bc34f71441fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index d1eab2eb4ed56..e54cfa144a173 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.types.{DecimalType, LongType, DoubleType, DataType} +import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.types.{DecimalType, LongType, DoubleType, DataType} /** Return the unscaled Long value of a Decimal, assuming it fits in a Long */ case class UnscaledValue(child: Expression) extends UnaryExpression { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index ab0701fd9a80b..43b6482c0171c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * An expression that produces zero or more rows given a single input row. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 94e1d37c1c3ab..8ee4bbd8caa62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal object Literal { def apply(v: Any): Literal = v match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index a3c300b5d90e9..3035d934ff9f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.util.Metadata +import org.apache.spark.sql.types._ object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index cb5ff67959868..c84cc95520a19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.types.BooleanType object InterpretedPredicate { def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index 3d4c4a8853c12..3a5bdca1f07c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index f6349767764a3..f85ee0a9bb6d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -23,7 +23,7 @@ import scala.collection.IndexedSeqOptimized import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types.{BinaryType, BooleanType, DataType, StringType} +import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType} trait StringRegexExpression { self: BinaryExpression => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index cd3137980ca43..17b4f9c23a978 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.plans.RightOuter import org.apache.spark.sql.catalyst.plans.LeftSemi import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal abstract class Optimizer extends RuleExecutor[LogicalPlan] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index a38079ced34b2..105cdf52500c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -27,6 +27,6 @@ package object catalyst { * scala.reflect.*. Note that Scala Reflection API is made thread-safe in 2.11, but not yet for * 2.10.* builds. See SI-6240 for more details. */ - protected[catalyst] object ScalaReflectionLock + protected[sql] object ScalaReflectionLock } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index dcbbb62c0aca4..619f42859cbb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType} abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ed578e081be73..65ae066e4b4b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.catalyst.trees /** @@ -191,14 +191,13 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { case (Nil, _) => expression case (requestedField :: rest, StructType(fields)) => val actualField = fields.filter(f => resolver(f.name, requestedField)) - actualField match { - case Seq() => - sys.error( - s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") - case Seq(singleMatch) => - resolveNesting(rest, GetField(expression, singleMatch.name), resolver) - case multipleMatches => - sys.error(s"Ambiguous reference to fields ${multipleMatches.mkString(", ")}") + if (actualField.length == 0) { + sys.error( + s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") + } else if (actualField.length == 1) { + resolveNesting(rest, GetField(expression, actualField(0).name), resolver) + } else { + sys.error(s"Ambiguous reference to fields ${actualField.mkString(", ")}") } case (_, dt) => sys.error(s"Can't access nested field in type $dt") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 0b9f01cbae9ea..1483beacc9088 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index ccb0df113c063..3c3d7a3119064 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder} -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.types.IntegerType /** * Specifies how tuples that share common expressions will be distributed when a query is executed diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala new file mode 100644 index 0000000000000..2a8914cde248c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala @@ -0,0 +1,68 @@ +/* + * 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.types + +import java.text.SimpleDateFormat + +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.types.decimal.Decimal + + +protected[sql] object DataTypeConversions { + + def stringToTime(s: String): java.util.Date = { + if (!s.contains('T')) { + // JDBC escape string + if (s.contains(' ')) { + java.sql.Timestamp.valueOf(s) + } else { + java.sql.Date.valueOf(s) + } + } else if (s.endsWith("Z")) { + // this is zero timezone of ISO8601 + stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") + } else if (s.indexOf("GMT") == -1) { + // timezone with ISO8601 + val inset = "+00.00".length + val s0 = s.substring(0, s.length - inset) + val s1 = s.substring(s.length - inset, s.length) + if (s0.substring(s0.lastIndexOf(':')).contains('.')) { + stringToTime(s0 + "GMT" + s1) + } else { + stringToTime(s0 + ".0GMT" + s1) + } + } else { + // ISO8601 with GMT insert + val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) + ISO8601GMT.parse(s) + } + } + + /** Converts Java objects to catalyst rows / types */ + def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type + case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d)) + case (other, _) => other + } + + /** Converts Java objects to catalyst rows / types */ + def convertCatalystToJava(a: Any): Any = a match { + case d: scala.math.BigDecimal => d.underlying() + case other => other + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java similarity index 81% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java index c69bbd5736a5b..e457542c647e7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypes.java @@ -15,77 +15,74 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package org.apache.spark.sql.types; import java.util.*; /** - * The base type of all Spark SQL data types. - * * To get/create specific data type, users should use singleton objects and factory methods * provided by this class. */ -public abstract class DataType { - +public class DataTypes { /** * Gets the StringType object. */ - public static final StringType StringType = new StringType(); + public static final DataType StringType = StringType$.MODULE$; /** * Gets the BinaryType object. */ - public static final BinaryType BinaryType = new BinaryType(); + public static final DataType BinaryType = BinaryType$.MODULE$; /** * Gets the BooleanType object. */ - public static final BooleanType BooleanType = new BooleanType(); + public static final DataType BooleanType = BooleanType$.MODULE$; /** * Gets the DateType object. */ - public static final DateType DateType = new DateType(); + public static final DataType DateType = DateType$.MODULE$; /** * Gets the TimestampType object. */ - public static final TimestampType TimestampType = new TimestampType(); + public static final DataType TimestampType = TimestampType$.MODULE$; /** * Gets the DoubleType object. */ - public static final DoubleType DoubleType = new DoubleType(); + public static final DataType DoubleType = DoubleType$.MODULE$; /** * Gets the FloatType object. */ - public static final FloatType FloatType = new FloatType(); + public static final DataType FloatType = FloatType$.MODULE$; /** * Gets the ByteType object. */ - public static final ByteType ByteType = new ByteType(); + public static final DataType ByteType = ByteType$.MODULE$; /** * Gets the IntegerType object. */ - public static final IntegerType IntegerType = new IntegerType(); + public static final DataType IntegerType = IntegerType$.MODULE$; /** * Gets the LongType object. */ - public static final LongType LongType = new LongType(); + public static final DataType LongType = LongType$.MODULE$; /** * Gets the ShortType object. */ - public static final ShortType ShortType = new ShortType(); + public static final DataType ShortType = ShortType$.MODULE$; /** * Gets the NullType object. */ - public static final NullType NullType = new NullType(); + public static final DataType NullType = NullType$.MODULE$; /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). @@ -95,7 +92,6 @@ public static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, true); } @@ -107,10 +103,17 @@ public static ArrayType createArrayType(DataType elementType, boolean containsNu if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, containsNull); } + public static DecimalType createDecimalType(int precision, int scale) { + return DecimalType$.MODULE$.apply(precision, scale); + } + + public static DecimalType createDecimalType() { + return DecimalType$.MODULE$.Unlimited(); + } + /** * Creates a MapType by specifying the data type of keys ({@code keyType}) and values * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}. @@ -122,7 +125,6 @@ public static MapType createMapType(DataType keyType, DataType valueType) { if (valueType == null) { throw new IllegalArgumentException("valueType should not be null."); } - return new MapType(keyType, valueType, true); } @@ -141,7 +143,6 @@ public static MapType createMapType( if (valueType == null) { throw new IllegalArgumentException("valueType should not be null."); } - return new MapType(keyType, valueType, valueContainsNull); } @@ -163,7 +164,6 @@ public static StructField createStructField( if (metadata == null) { throw new IllegalArgumentException("metadata should not be null."); } - return new StructField(name, dataType, nullable, metadata); } @@ -191,18 +191,18 @@ public static StructType createStructType(StructField[] fields) { throw new IllegalArgumentException("fields should not be null."); } Set distinctNames = new HashSet(); - for (StructField field: fields) { + for (StructField field : fields) { if (field == null) { throw new IllegalArgumentException( "fields should not contain any null."); } - distinctNames.add(field.getName()); + distinctNames.add(field.name()); } if (distinctNames.size() != fields.length) { throw new IllegalArgumentException("fields should have distinct names."); } - return new StructType(fields); + return StructType$.MODULE$.apply(fields); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala similarity index 96% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 8172733e94dd5..e50e9761431f5 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -15,24 +15,31 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.util +package org.apache.spark.sql.types import scala.collection.mutable import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.annotation.DeveloperApi + + /** + * :: DeveloperApi :: + * * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and * Array[Metadata]. JSON is used for serialization. * * The default constructor is private. User should use either [[MetadataBuilder]] or - * [[Metadata$#fromJson]] to create Metadata instances. + * [[Metadata.fromJson()]] to create Metadata instances. * * @param map an immutable map that stores the data */ -sealed class Metadata private[util] (private[util] val map: Map[String, Any]) extends Serializable { +@DeveloperApi +sealed class Metadata private[types] (private[types] val map: Map[String, Any]) + extends Serializable { /** Tests whether this Metadata contains a binding for a key. */ def contains(key: String): Boolean = map.contains(key) @@ -201,8 +208,11 @@ object Metadata { } /** + * :: DeveloperApi :: + * * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. */ +@DeveloperApi class MetadataBuilder { private val map: mutable.Map[String, Any] = mutable.Map.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java similarity index 93% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java index e966aeea1cb23..a64d2bb7cde37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/SQLUserDefinedType.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.annotation; +package org.apache.spark.sql.types; import java.lang.annotation.*; import org.apache.spark.annotation.DeveloperApi; -import org.apache.spark.sql.catalyst.types.UserDefinedType; /** * ::DeveloperApi:: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala similarity index 76% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 892b7e1a97c8b..fa0a355ebc9b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.types +package org.apache.spark.sql.types import java.sql.{Date, Timestamp} -import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral} +import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} import scala.util.parsing.combinator.RegexParsers @@ -31,11 +31,11 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.ScalaReflectionLock -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Row} -import org.apache.spark.sql.catalyst.types.decimal._ -import org.apache.spark.sql.catalyst.util.Metadata +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.types.decimal._ import org.apache.spark.util.Utils + object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) @@ -140,7 +140,7 @@ object DataType { protected lazy val structType: Parser[DataType] = "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { - case fields => new StructType(fields) + case fields => StructType(fields) } protected lazy val dataType: Parser[DataType] = @@ -181,7 +181,7 @@ object DataType { /** * Compares two types, ignoring nullability of ArrayType, MapType, StructType. */ - def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { + private[sql] def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = { (left, right) match { case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) => equalsIgnoreNullability(leftElementType, rightElementType) @@ -200,6 +200,15 @@ object DataType { } } + +/** + * :: DeveloperApi :: + * + * The base type of all Spark SQL data types. + * + * @group dataType + */ +@DeveloperApi abstract class DataType { /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { @@ -218,8 +227,18 @@ abstract class DataType { def prettyJson: String = pretty(render(jsonValue)) } + +/** + * :: DeveloperApi :: + * + * The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]]. + * + * @group dataType + */ +@DeveloperApi case object NullType extends DataType + object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -237,10 +256,12 @@ object NativeType { StringType -> 4096) } + trait PrimitiveType extends DataType { override def isPrimitive = true } + object PrimitiveType { private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap @@ -267,12 +288,31 @@ abstract class NativeType extends DataType { } } + +/** + * :: DeveloperApi :: + * + * The data type representing `String` values. Please use the singleton [[DataTypes.StringType]]. + * + * @group dataType + */ +@DeveloperApi case object StringType extends NativeType with PrimitiveType { private[sql] type JvmType = String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] } + +/** + * :: DeveloperApi :: + * + * The data type representing `Array[Byte]` values. + * Please use the singleton [[DataTypes.BinaryType]]. + * + * @group dataType + */ +@DeveloperApi case object BinaryType extends NativeType with PrimitiveType { private[sql] type JvmType = Array[Byte] @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -287,12 +327,31 @@ case object BinaryType extends NativeType with PrimitiveType { } } + +/** + * :: DeveloperApi :: + * + * The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]]. + * + *@group dataType + */ +@DeveloperApi case object BooleanType extends NativeType with PrimitiveType { private[sql] type JvmType = Boolean @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] } + +/** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Timestamp` values. + * Please use the singleton [[DataTypes.TimestampType]]. + * + * @group dataType + */ +@DeveloperApi case object TimestampType extends NativeType { private[sql] type JvmType = Timestamp @@ -303,6 +362,16 @@ case object TimestampType extends NativeType { } } + +/** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Date` values. + * Please use the singleton [[DataTypes.DateType]]. + * + * @group dataType + */ +@DeveloperApi case object DateType extends NativeType { private[sql] type JvmType = Date @@ -313,6 +382,7 @@ case object DateType extends NativeType { } } + abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a @@ -322,10 +392,12 @@ abstract class NumericType extends NativeType with PrimitiveType { private[sql] val numeric: Numeric[JvmType] } + object NumericType { def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] } + /** Matcher for any expressions that evaluate to [[IntegralType]]s */ object IntegralType { def unapply(a: Expression): Boolean = a match { @@ -334,10 +406,20 @@ object IntegralType { } } -abstract class IntegralType extends NumericType { + +sealed abstract class IntegralType extends NumericType { private[sql] val integral: Integral[JvmType] } + +/** + * :: DeveloperApi :: + * + * The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]]. + * + * @group dataType + */ +@DeveloperApi case object LongType extends IntegralType { private[sql] type JvmType = Long @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -346,6 +428,15 @@ case object LongType extends IntegralType { private[sql] val ordering = implicitly[Ordering[JvmType]] } + +/** + * :: DeveloperApi :: + * + * The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]]. + * + * @group dataType + */ +@DeveloperApi case object IntegerType extends IntegralType { private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -354,6 +445,15 @@ case object IntegerType extends IntegralType { private[sql] val ordering = implicitly[Ordering[JvmType]] } + +/** + * :: DeveloperApi :: + * + * The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]]. + * + * @group dataType + */ +@DeveloperApi case object ShortType extends IntegralType { private[sql] type JvmType = Short @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -362,6 +462,15 @@ case object ShortType extends IntegralType { private[sql] val ordering = implicitly[Ordering[JvmType]] } + +/** + * :: DeveloperApi :: + * + * The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]]. + * + * @group dataType + */ +@DeveloperApi case object ByteType extends IntegralType { private[sql] type JvmType = Byte @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -370,6 +479,7 @@ case object ByteType extends IntegralType { private[sql] val ordering = implicitly[Ordering[JvmType]] } + /** Matcher for any expressions that evaluate to [[FractionalType]]s */ object FractionalType { def unapply(a: Expression): Boolean = a match { @@ -378,15 +488,28 @@ object FractionalType { } } -abstract class FractionalType extends NumericType { + +sealed abstract class FractionalType extends NumericType { private[sql] val fractional: Fractional[JvmType] private[sql] val asIntegral: Integral[JvmType] } + /** Precision parameters for a Decimal */ case class PrecisionInfo(precision: Int, scale: Int) -/** A Decimal that might have fixed precision and scale, or unlimited values for these */ + +/** + * :: DeveloperApi :: + * + * The data type representing `scala.math.BigDecimal` values. + * A Decimal that might have fixed precision and scale, or unlimited values for these. + * + * Please use [[DataTypes.createDecimalType()]] to create a specific instance. + * + * @group dataType + */ +@DeveloperApi case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { private[sql] type JvmType = Decimal @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -395,6 +518,10 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT private[sql] val ordering = Decimal.DecimalIsFractional private[sql] val asIntegral = Decimal.DecimalAsIfIntegral + def precision: Int = precisionInfo.map(_.precision).getOrElse(-1) + + def scale: Int = precisionInfo.map(_.scale).getOrElse(-1) + override def typeName: String = precisionInfo match { case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" case None => "decimal" @@ -406,6 +533,7 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT } } + /** Extra factory methods and pattern matchers for Decimals */ object DecimalType { val Unlimited: DecimalType = DecimalType(None) @@ -437,6 +565,15 @@ object DecimalType { } } + +/** + * :: DeveloperApi :: + * + * The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]]. + * + * @group dataType + */ +@DeveloperApi case object DoubleType extends FractionalType { private[sql] type JvmType = Double @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -446,6 +583,15 @@ case object DoubleType extends FractionalType { private[sql] val asIntegral = DoubleAsIfIntegral } + +/** + * :: DeveloperApi :: + * + * The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]]. + * + * @group dataType + */ +@DeveloperApi case object FloatType extends FractionalType { private[sql] type JvmType = Float @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } @@ -455,18 +601,31 @@ case object FloatType extends FractionalType { private[sql] val asIntegral = FloatAsIfIntegral } + object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } + /** + * :: DeveloperApi :: + * * The data type for collections of multiple values. * Internally these are represented as columns that contain a ``scala.collection.Seq``. * + * Please use [[DataTypes.createArrayType()]] to create a specific instance. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * * @param elementType The data type of values. * @param containsNull Indicates if values have `null` values + * + * @group dataType */ +@DeveloperApi case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append( @@ -480,8 +639,10 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT ("containsNull" -> containsNull) } + /** * A field inside a StructType. + * * @param name The name of this field. * @param dataType The data type of this field. * @param nullable Indicates if values of this field can be `null` values. @@ -510,19 +671,92 @@ case class StructField( } } + object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) + + def apply(fields: java.util.List[StructField]): StructType = { + StructType(fields.toArray.asInstanceOf[Array[StructField]]) + } } -case class StructType(fields: Seq[StructField]) extends DataType { - /** - * Returns all field names in a [[Seq]]. - */ - lazy val fieldNames: Seq[String] = fields.map(_.name) +/** + * :: DeveloperApi :: + * + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Any names without matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[org.apache.spark.sql.catalyst.expressions.Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ +@DeveloperApi +case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { + + /** Returns all field names in an array. */ + def fieldNames: Array[String] = fields.map(_.name) + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + /** * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not * have a name matching the given name, `null` will be returned. @@ -532,8 +766,8 @@ case class StructType(fields: Seq[StructField]) extends DataType { } /** - * Returns a [[StructType]] containing [[StructField]]s of the given names. - * Those names which do not have matching fields will be ignored. + * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the + * original order of fields. Those names which do not have matching fields will be ignored. */ def apply(names: Set[String]): StructType = { val nonExistFields = names -- fieldNamesSet @@ -545,8 +779,8 @@ case class StructType(fields: Seq[StructField]) extends DataType { StructType(fields.filter(f => names.contains(f.name))) } - protected[sql] def toAttributes = - fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + protected[sql] def toAttributes: Seq[AttributeReference] = + map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) def treeString: String = { val builder = new StringBuilder @@ -565,23 +799,38 @@ case class StructType(fields: Seq[StructField]) extends DataType { override private[sql] def jsonValue = ("type" -> typeName) ~ - ("fields" -> fields.map(_.jsonValue)) + ("fields" -> map(_.jsonValue)) + + override def apply(fieldIndex: Int): StructField = fields(fieldIndex) + + override def length: Int = fields.length + + override def iterator: Iterator[StructField] = fields.iterator } + object MapType { /** * Construct a [[MapType]] object with the given key type and value type. * The `valueContainsNull` is true. */ def apply(keyType: DataType, valueType: DataType): MapType = - MapType(keyType: DataType, valueType: DataType, true) + MapType(keyType: DataType, valueType: DataType, valueContainsNull = true) } + /** + * :: DeveloperApi :: + * * The data type for Maps. Keys in a map are not allowed to have `null` values. + * + * Please use [[DataTypes.createMapType()]] to create a specific instance. + * * @param keyType The data type of map keys. * @param valueType The data type of map values. * @param valueContainsNull Indicates if map values have `null` values. + * + * @group dataType */ case class MapType( keyType: DataType, @@ -602,6 +851,7 @@ case class MapType( ("valueContainsNull" -> valueContainsNull) } + /** * ::DeveloperApi:: * The data type for User Defined Types (UDTs). @@ -611,7 +861,7 @@ case class MapType( * a SchemaRDD which has class X in the schema. * * For SparkSQL to recognize UDTs, the UDT must be annotated with - * [[org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType]]. + * [[SQLUserDefinedType]]. * * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD. * The conversion via `deserialize` occurs when reading from a `SchemaRDD`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala similarity index 99% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala index 708362acf32dc..c7864d1ae9e74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.types.decimal +package org.apache.spark.sql.types.decimal import org.apache.spark.annotation.DeveloperApi diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala similarity index 96% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala index de24449590f9a..346a51ea10c82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/package.scala @@ -15,7 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst +package org.apache.spark.sql + /** * Contains a type system for attributes produced by relations, including complex types like * structs, arrays and maps. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 7be24bea7d5a6..117725df32e19 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp} import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ case class PrimitiveData( intField: Int, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f430057ef7191..3aea337460d42 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index bbbeb4f2e4fe3..bc2ec754d5865 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfter, FunSuite} class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index dfa2d958c0faf..f5a502b43f80b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ class HiveTypeCoercionSuite extends FunSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 4ba7d87ba8c5a..8552448b8d108 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -21,16 +21,14 @@ import java.sql.{Date, Timestamp} import scala.collection.immutable.HashSet -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.scalactic.TripleEqualsSupport.Spread - -import org.apache.spark.sql.catalyst.types._ - -/* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal + class ExpressionEvaluationSuite extends FunSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 0a27cce337482..9fdf3efa02bb6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ // For implicit conversions import org.apache.spark.sql.catalyst.dsl.plans._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 017b180c574b4..da912ab382179 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ // For implicit conversions import org.apache.spark.sql.catalyst.dsl.plans._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 036fd3fa1d6a1..cdb843f959704 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{StringType, NullType} +import org.apache.spark.sql.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala index f005b7df21043..d7d60efee50fa 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.util import org.json4s.jackson.JsonMethods.parse import org.scalatest.FunSuite +import org.apache.spark.sql.types.{MetadataBuilder, Metadata} + class MetadataSuite extends FunSuite { val baseMetadata = new MetadataBuilder() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index e9740d913cf57..892195f46ea24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.types import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala similarity index 99% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala index e32f1ac382130..813377df00132 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.types.decimal +package org.apache.spark.sql.types.decimal import org.scalatest.{PrivateMethodTester, FunSuite} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java deleted file mode 100644 index b73a371e93001..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java +++ /dev/null @@ -1,68 +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.api.java; - -/** - * The data type representing Lists. - * An ArrayType object comprises two fields, {@code DataType elementType} and - * {@code boolean containsNull}. The field of {@code elementType} is used to specify the type of - * array elements. The field of {@code containsNull} is used to specify if the array has - * {@code null} values. - * - * To create an {@link ArrayType}, - * {@link DataType#createArrayType(DataType)} or - * {@link DataType#createArrayType(DataType, boolean)} - * should be used. - */ -public class ArrayType extends DataType { - private DataType elementType; - private boolean containsNull; - - protected ArrayType(DataType elementType, boolean containsNull) { - this.elementType = elementType; - this.containsNull = containsNull; - } - - public DataType getElementType() { - return elementType; - } - - public boolean isContainsNull() { - return containsNull; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ArrayType arrayType = (ArrayType) o; - - if (containsNull != arrayType.containsNull) return false; - if (!elementType.equals(arrayType.elementType)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = elementType.hashCode(); - result = 31 * result + (containsNull ? 1 : 0); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java deleted file mode 100644 index 7daad60f62a0b..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing byte[] values. - * - * {@code BinaryType} is represented by the singleton object {@link DataType#BinaryType}. - */ -public class BinaryType extends DataType { - protected BinaryType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java deleted file mode 100644 index 5a1f52725631b..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing boolean and Boolean values. - * - * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}. - */ -public class BooleanType extends DataType { - protected BooleanType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java deleted file mode 100644 index e5cdf06b21bbe..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing byte and Byte values. - * - * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}. - */ -public class ByteType extends DataType { - protected ByteType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java deleted file mode 100644 index 6677793baa365..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing java.sql.Date values. - * - * {@code DateType} is represented by the singleton object {@link DataType#DateType}. - */ -public class DateType extends DataType { - protected DateType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java deleted file mode 100644 index 60752451ecfc7..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java +++ /dev/null @@ -1,79 +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.api.java; - -/** - * The data type representing java.math.BigDecimal values. - */ -public class DecimalType extends DataType { - private boolean hasPrecisionInfo; - private int precision; - private int scale; - - public DecimalType(int precision, int scale) { - this.hasPrecisionInfo = true; - this.precision = precision; - this.scale = scale; - } - - public DecimalType() { - this.hasPrecisionInfo = false; - this.precision = -1; - this.scale = -1; - } - - public boolean isUnlimited() { - return !hasPrecisionInfo; - } - - public boolean isFixed() { - return hasPrecisionInfo; - } - - /** Return the precision, or -1 if no precision is set */ - public int getPrecision() { - return precision; - } - - /** Return the scale, or -1 if no precision is set */ - public int getScale() { - return scale; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - DecimalType that = (DecimalType) o; - - if (hasPrecisionInfo != that.hasPrecisionInfo) return false; - if (precision != that.precision) return false; - if (scale != that.scale) return false; - - return true; - } - - @Override - public int hashCode() { - int result = (hasPrecisionInfo ? 1 : 0); - result = 31 * result + precision; - result = 31 * result + scale; - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java deleted file mode 100644 index f0060d0bcf9f5..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing double and Double values. - * - * {@code DoubleType} is represented by the singleton object {@link DataType#DoubleType}. - */ -public class DoubleType extends DataType { - protected DoubleType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java deleted file mode 100644 index 4a6a37f69176a..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing float and Float values. - * - * {@code FloatType} is represented by the singleton object {@link DataType#FloatType}. - */ -public class FloatType extends DataType { - protected FloatType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java deleted file mode 100644 index bfd70490bbbbb..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing int and Integer values. - * - * {@code IntegerType} is represented by the singleton object {@link DataType#IntegerType}. - */ -public class IntegerType extends DataType { - protected IntegerType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java deleted file mode 100644 index af13a46eb165c..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing long and Long values. - * - * {@code LongType} is represented by the singleton object {@link DataType#LongType}. - */ -public class LongType extends DataType { - protected LongType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java deleted file mode 100644 index 063e6b34abc48..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java +++ /dev/null @@ -1,78 +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.api.java; - -/** - * The data type representing Maps. A MapType object comprises two fields, - * {@code DataType keyType}, {@code DataType valueType}, and {@code boolean valueContainsNull}. - * The field of {@code keyType} is used to specify the type of keys in the map. - * The field of {@code valueType} is used to specify the type of values in the map. - * The field of {@code valueContainsNull} is used to specify if map values have - * {@code null} values. - * For values of a MapType column, keys are not allowed to have {@code null} values. - * - * To create a {@link MapType}, - * {@link DataType#createMapType(DataType, DataType)} or - * {@link DataType#createMapType(DataType, DataType, boolean)} - * should be used. - */ -public class MapType extends DataType { - private DataType keyType; - private DataType valueType; - private boolean valueContainsNull; - - protected MapType(DataType keyType, DataType valueType, boolean valueContainsNull) { - this.keyType = keyType; - this.valueType = valueType; - this.valueContainsNull = valueContainsNull; - } - - public DataType getKeyType() { - return keyType; - } - - public DataType getValueType() { - return valueType; - } - - public boolean isValueContainsNull() { - return valueContainsNull; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - MapType mapType = (MapType) o; - - if (valueContainsNull != mapType.valueContainsNull) return false; - if (!keyType.equals(mapType.keyType)) return false; - if (!valueType.equals(mapType.valueType)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = keyType.hashCode(); - result = 31 * result + valueType.hashCode(); - result = 31 * result + (valueContainsNull ? 1 : 0); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java deleted file mode 100644 index 0f819fb01a76a..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java +++ /dev/null @@ -1,31 +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.api.java; - -/** - * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, - * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and - * Array[Metadata]. JSON is used for serialization. - * - * The default constructor is private. User should use [[MetadataBuilder]]. - */ -class Metadata extends org.apache.spark.sql.catalyst.util.Metadata { - Metadata(scala.collection.immutable.Map map) { - super(map); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java deleted file mode 100644 index 6e6b12f0722c5..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java +++ /dev/null @@ -1,28 +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.api.java; - -/** - * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. - */ -public class MetadataBuilder extends org.apache.spark.sql.catalyst.util.MetadataBuilder { - @Override - public Metadata build() { - return new Metadata(getMap()); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java deleted file mode 100644 index 6d5ecdf46e551..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing null and NULL values. - * - * {@code NullType} is represented by the singleton object {@link DataType#NullType}. - */ -public class NullType extends DataType { - protected NullType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java deleted file mode 100644 index 7d7604b4e3d2d..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing short and Short values. - * - * {@code ShortType} is represented by the singleton object {@link DataType#ShortType}. - */ -public class ShortType extends DataType { - protected ShortType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java deleted file mode 100644 index f4ba0c07c9c6e..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing String values. - * - * {@code StringType} is represented by the singleton object {@link DataType#StringType}. - */ -public class StringType extends DataType { - protected StringType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java deleted file mode 100644 index 7c60d492bcdf0..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java +++ /dev/null @@ -1,91 +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.api.java; - -import java.util.Map; - -/** - * A StructField object represents a field in a StructType object. - * A StructField object comprises three fields, {@code String name}, {@code DataType dataType}, - * and {@code boolean nullable}. The field of {@code name} is the name of a StructField. - * The field of {@code dataType} specifies the data type of a StructField. - * The field of {@code nullable} specifies if values of a StructField can contain {@code null} - * values. - * The field of {@code metadata} provides extra information of the StructField. - * - * To create a {@link StructField}, - * {@link DataType#createStructField(String, DataType, boolean, Metadata)} - * should be used. - */ -public class StructField { - private String name; - private DataType dataType; - private boolean nullable; - private Metadata metadata; - - protected StructField( - String name, - DataType dataType, - boolean nullable, - Metadata metadata) { - this.name = name; - this.dataType = dataType; - this.nullable = nullable; - this.metadata = metadata; - } - - public String getName() { - return name; - } - - public DataType getDataType() { - return dataType; - } - - public boolean isNullable() { - return nullable; - } - - public Metadata getMetadata() { - return metadata; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StructField that = (StructField) o; - - if (nullable != that.nullable) return false; - if (!dataType.equals(that.dataType)) return false; - if (!name.equals(that.name)) return false; - if (!metadata.equals(that.metadata)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = name.hashCode(); - result = 31 * result + dataType.hashCode(); - result = 31 * result + (nullable ? 1 : 0); - result = 31 * result + metadata.hashCode(); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java deleted file mode 100644 index a4b501efd9a10..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java +++ /dev/null @@ -1,58 +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.api.java; - -import java.util.Arrays; - -/** - * The data type representing Rows. - * A StructType object comprises an array of StructFields. - * - * To create an {@link StructType}, - * {@link DataType#createStructType(java.util.List)} or - * {@link DataType#createStructType(StructField[])} - * should be used. - */ -public class StructType extends DataType { - private StructField[] fields; - - protected StructType(StructField[] fields) { - this.fields = fields; - } - - public StructField[] getFields() { - return fields; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StructType that = (StructType) o; - - if (!Arrays.equals(fields, that.fields)) return false; - - return true; - } - - @Override - public int hashCode() { - return Arrays.hashCode(fields); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java deleted file mode 100644 index 06d44c731cdfe..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java +++ /dev/null @@ -1,27 +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.api.java; - -/** - * The data type representing java.sql.Timestamp values. - * - * {@code TimestampType} is represented by the singleton object {@link DataType#TimestampType}. - */ -public class TimestampType extends DataType { - protected TimestampType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java deleted file mode 100644 index f0d079d25b5d4..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java +++ /dev/null @@ -1,54 +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.api.java; - -import java.io.Serializable; - -import org.apache.spark.annotation.DeveloperApi; - -/** - * ::DeveloperApi:: - * The data type representing User-Defined Types (UDTs). - * UDTs may use any other DataType for an underlying representation. - */ -@DeveloperApi -public abstract class UserDefinedType extends DataType implements Serializable { - - protected UserDefinedType() { } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - @SuppressWarnings("unchecked") - UserDefinedType that = (UserDefinedType) o; - return this.sqlType().equals(that.sqlType()); - } - - /** Underlying storage type for this UDT */ - public abstract DataType sqlType(); - - /** Convert the user type to a SQL datum */ - public abstract Object serialize(Object obj); - - /** Convert a SQL datum to the user type */ - public abstract UserType deserialize(Object datum); - - /** Class object for the UserType */ - public abstract Class userClass(); -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d8efce0cb43eb..d9f3b3a53f582 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,7 +24,6 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration - import org.apache.spark.SparkContext import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD @@ -32,14 +31,14 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} +import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types.UserDefinedType -import org.apache.spark.sql.execution.{SparkStrategies, _} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{DataSourceStrategy, BaseRelation, DDLParser, LogicalRelation} +import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation} +import org.apache.spark.sql.types._ /** * :: AlphaComponent :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 80787b61ce1bf..686bcdfbb4ff1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql -import java.util.{Map => JMap, List => JList} - +import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -37,8 +36,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.{BooleanType, StructType} import org.apache.spark.storage.StorageLevel /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala index 65358b7d4ea8e..f10ee7b66feb7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql +import scala.util.parsing.combinator.RegexParsers + import org.apache.spark.sql.catalyst.{SqlLexical, AbstractSparkSQLParser} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, SetCommand} +import org.apache.spark.sql.types.StringType -import scala.util.parsing.combinator.RegexParsers /** * The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 7f868cd4afca4..a75f55992826a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -23,15 +23,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.sql.{SQLContext, StructType => SStructType} -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation} -import org.apache.spark.sql.types.util.DataTypeConversions -import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils /** @@ -126,9 +124,8 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { @DeveloperApi def applySchema(rowRDD: JavaRDD[Row], schema: StructType): JavaSchemaRDD = { val scalaRowRDD = rowRDD.rdd.map(r => r.row) - val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] val logicalPlan = - LogicalRDD(scalaSchema.toAttributes, scalaRowRDD)(sqlContext) + LogicalRDD(schema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -184,10 +181,10 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord val appliedScalaSchema = - Option(asScalaDataType(schema)).getOrElse( + Option(schema).getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( - json.rdd, 1.0, columnNameOfCorruptJsonRecord))).asInstanceOf[SStructType] + json.rdd, 1.0, columnNameOfCorruptJsonRecord))) val scalaRowRDD = JsonRDD.jsonStringToRow( json.rdd, appliedScalaSchema, columnNameOfCorruptJsonRecord) val logicalPlan = @@ -218,43 +215,25 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val (dataType, nullable) = property.getPropertyType match { case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) - case c: Class[_] if c == classOf[java.lang.String] => - (org.apache.spark.sql.StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => - (org.apache.spark.sql.ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => - (org.apache.spark.sql.IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => - (org.apache.spark.sql.LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => - (org.apache.spark.sql.DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => - (org.apache.spark.sql.ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => - (org.apache.spark.sql.FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => - (org.apache.spark.sql.BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => - (org.apache.spark.sql.ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => - (org.apache.spark.sql.IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => - (org.apache.spark.sql.LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => - (org.apache.spark.sql.DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => - (org.apache.spark.sql.ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => - (org.apache.spark.sql.FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => - (org.apache.spark.sql.BooleanType, true) - case c: Class[_] if c == classOf[java.math.BigDecimal] => - (org.apache.spark.sql.DecimalType(), true) - case c: Class[_] if c == classOf[java.sql.Date] => - (org.apache.spark.sql.DateType, true) - case c: Class[_] if c == classOf[java.sql.Timestamp] => - (org.apache.spark.sql.TimestampType, true) + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) } AttributeReference(property.getName, dataType, nullable)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index 5b9c612487ace..9e10e532fb011 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.api.java import java.util.{List => JList} import org.apache.spark.Partitioner -import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.{JavaRDD, JavaRDDLike} import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.sql.types.util.DataTypeConversions +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import DataTypeConversions._ -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel /** @@ -59,8 +58,7 @@ class JavaSchemaRDD( override def toString: String = baseSchemaRDD.toString /** Returns the schema of this JavaSchemaRDD (represented by a StructType). */ - def schema: StructType = - asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType] + def schema: StructType = baseSchemaRDD.schema.asInstanceOf[StructType] // ======================================================================= // Base RDD functions that do NOT change schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 401798e317e96..207e2805fffe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.api.java -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.annotation.varargs import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} import scala.collection.JavaConversions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala index 158f26e3d445f..4186c274515ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.api.java import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} -import org.apache.spark.sql.types.util.DataTypeConversions._ +import org.apache.spark.sql.types.DataType /** * A collection of functions that allow Java users to register UDFs. In order to handle functions @@ -38,10 +38,9 @@ private[java] trait UDFRegistration { println(s""" |def registerFunction( | name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { - | val scalaType = asScalaDataType(dataType) | sqlContext.functionRegistry.registerFunction( | name, - | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e)) + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), dataType, e)) |} """.stripMargin) } @@ -94,159 +93,159 @@ private[java] trait UDFRegistration { */ // scalastyle:off - def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF1[_, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), dataType, e)) } - def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF2[_, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF3[_, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF4[_, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF5[_, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF6[_, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF7[_, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF8[_, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + + def registerFunction( + name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - // scalastyle:on } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala deleted file mode 100644 index a7d0f4f127ecc..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala +++ /dev/null @@ -1,75 +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.api.java - -import org.apache.spark.sql.catalyst.types.{UserDefinedType => ScalaUserDefinedType} -import org.apache.spark.sql.{DataType => ScalaDataType} -import org.apache.spark.sql.types.util.DataTypeConversions - -/** - * Scala wrapper for a Java UserDefinedType - */ -private[sql] class JavaToScalaUDTWrapper[UserType](val javaUDT: UserDefinedType[UserType]) - extends ScalaUserDefinedType[UserType] with Serializable { - - /** Underlying storage type for this UDT */ - val sqlType: ScalaDataType = DataTypeConversions.asScalaDataType(javaUDT.sqlType()) - - /** Convert the user type to a SQL datum */ - def serialize(obj: Any): Any = javaUDT.serialize(obj) - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType = javaUDT.deserialize(datum) - - val userClass: java.lang.Class[UserType] = javaUDT.userClass() -} - -/** - * Java wrapper for a Scala UserDefinedType - */ -private[sql] class ScalaToJavaUDTWrapper[UserType](val scalaUDT: ScalaUserDefinedType[UserType]) - extends UserDefinedType[UserType] with Serializable { - - /** Underlying storage type for this UDT */ - val sqlType: DataType = DataTypeConversions.asJavaDataType(scalaUDT.sqlType) - - /** Convert the user type to a SQL datum */ - def serialize(obj: Any): java.lang.Object = scalaUDT.serialize(obj).asInstanceOf[java.lang.Object] - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType = scalaUDT.deserialize(datum) - - val userClass: java.lang.Class[UserType] = scalaUDT.userClass -} - -private[sql] object UDTWrappers { - - def wrapAsScala(udtType: UserDefinedType[_]): ScalaUserDefinedType[_] = { - udtType match { - case t: ScalaToJavaUDTWrapper[_] => t.scalaUDT - case _ => new JavaToScalaUDTWrapper(udtType) - } - } - - def wrapAsJava(udtType: ScalaUserDefinedType[_]): UserDefinedType[_] = { - udtType match { - case t: JavaToScalaUDTWrapper[_] => t.javaUDT - case _ => new ScalaToJavaUDTWrapper(udtType) - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 538dd5b734664..91c4c105b14e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor +import org.apache.spark.sql.types.{BinaryType, DataType, NativeType} /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c68dceef3b142..3a4977b836af7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnBuilder._ import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder} +import org.apache.spark.sql.types._ private[sql] trait ColumnBuilder { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 668efe4a3b2a8..391b3dae5c8ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -21,7 +21,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index ab66c85c4f242..fcf2faa0914c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -24,8 +24,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ /** * An abstract class that represents type of a column. Used to append/extract Java objects into/from diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index 27ac5f4dbdbbc..7dff9deac8dc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} +import org.apache.spark.sql.types.NativeType private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { this: NativeColumnAccessor[T] => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 628d9cec41d6b..aead768ecdf0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.Logging import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} +import org.apache.spark.sql.types.NativeType /** * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index acb06cb5376b4..879d29bcfa6f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} +import org.apache.spark.sql.types.NativeType private[sql] trait Encoder[T <: NativeType] { def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 29edcf17242c5..64673248394c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -25,10 +25,11 @@ import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils + private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 069e950195302..20b14834bb0d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{StructType, Row, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} +import org.apache.spark.sql.types.StructType /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 7c3bf947e743d..4abe26fe4afc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ case class AggregateEvaluation( schema: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 84d96e612f0dc..131146012ecac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -29,7 +29,7 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0652d2ff7c9ab..0cc9d049c9640 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,16 +17,16 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.parquet._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 61be5ed2db65c..46245cd5a1869 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext._ import org.apache.spark.sql.{SchemaRDD, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 5a41399971dda..741ccb8fb8fa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -33,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.{Accumulator, Logging => SparkLogging} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f5c02224c82a0..1af96c28d5fdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.json import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType + private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 00449c200704a..c92ec543e2936 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.types.util.DataTypeConversions - import java.io.StringWriter import scala.collection.Map @@ -34,8 +31,9 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { @@ -246,7 +244,7 @@ private[sql] object JsonRDD extends Logging { // The value associated with the key is an array. // Handle inner structs of an array. def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { - case ArrayType(StructType(Nil), containsNull) => { + case ArrayType(e: StructType, containsNull) => { // The elements of this arrays are structs. v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap { element => allKeysWithValueTypes(element) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 1fd8e6220f83b..b75266d5aa409 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -116,358 +116,9 @@ package object sql { @DeveloperApi val Row = catalyst.expressions.Row - /** - * :: DeveloperApi :: - * - * The base type of all Spark SQL data types. - * - * @group dataType - */ - @DeveloperApi - type DataType = catalyst.types.DataType - - @DeveloperApi - val DataType = catalyst.types.DataType - - /** - * :: DeveloperApi :: - * - * The data type representing `String` values - * - * @group dataType - */ - @DeveloperApi - val StringType = catalyst.types.StringType - - /** - * :: DeveloperApi :: - * - * The data type representing `Array[Byte]` values. - * - * @group dataType - */ - @DeveloperApi - val BinaryType = catalyst.types.BinaryType - - /** - * :: DeveloperApi :: - * - * The data type representing `Boolean` values. - * - *@group dataType - */ - @DeveloperApi - val BooleanType = catalyst.types.BooleanType - - /** - * :: DeveloperApi :: - * - * The data type representing `java.sql.Timestamp` values. - * - * @group dataType - */ - @DeveloperApi - val TimestampType = catalyst.types.TimestampType - - /** - * :: DeveloperApi :: - * - * The data type representing `java.sql.Date` values. - * - * @group dataType - */ - @DeveloperApi - val DateType = catalyst.types.DateType - - /** - * :: DeveloperApi :: - * - * The data type representing `scala.math.BigDecimal` values. - * - * TODO(matei): explain precision and scale - * - * @group dataType - */ - @DeveloperApi - type DecimalType = catalyst.types.DecimalType - - /** - * :: DeveloperApi :: - * - * The data type representing `scala.math.BigDecimal` values. - * - * TODO(matei): explain precision and scale - * - * @group dataType - */ - @DeveloperApi - val DecimalType = catalyst.types.DecimalType - - /** - * :: DeveloperApi :: - * - * The data type representing `Double` values. - * - * @group dataType - */ - @DeveloperApi - val DoubleType = catalyst.types.DoubleType - - /** - * :: DeveloperApi :: - * - * The data type representing `Float` values. - * - * @group dataType - */ - @DeveloperApi - val FloatType = catalyst.types.FloatType - - /** - * :: DeveloperApi :: - * - * The data type representing `Byte` values. - * - * @group dataType - */ - @DeveloperApi - val ByteType = catalyst.types.ByteType - - /** - * :: DeveloperApi :: - * - * The data type representing `Int` values. - * - * @group dataType - */ - @DeveloperApi - val IntegerType = catalyst.types.IntegerType - - /** - * :: DeveloperApi :: - * - * The data type representing `Long` values. - * - * @group dataType - */ - @DeveloperApi - val LongType = catalyst.types.LongType - - /** - * :: DeveloperApi :: - * - * The data type representing `Short` values. - * - * @group dataType - */ - @DeveloperApi - val ShortType = catalyst.types.ShortType - - /** - * :: DeveloperApi :: - * - * The data type representing `NULL` values. - * - * @group dataType - */ - @DeveloperApi - val NullType = catalyst.types.NullType - - /** - * :: DeveloperApi :: - * - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. - * - * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and - * `containsNull: Boolean`. The field of `elementType` is used to specify the type of - * array elements. The field of `containsNull` is used to specify if the array has `null` values. - * - * @group dataType - */ - @DeveloperApi - type ArrayType = catalyst.types.ArrayType - - /** - * :: DeveloperApi :: - * - * An [[ArrayType]] object can be constructed with two ways, - * {{{ - * ArrayType(elementType: DataType, containsNull: Boolean) - * }}} and - * {{{ - * ArrayType(elementType: DataType) - * }}} - * For `ArrayType(elementType)`, the field of `containsNull` is set to `false`. - * - * @group dataType - */ - @DeveloperApi - val ArrayType = catalyst.types.ArrayType - - /** - * :: DeveloperApi :: - * - * The data type representing `Map`s. A [[MapType]] object comprises three fields, - * `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`. - * The field of `keyType` is used to specify the type of keys in the map. - * The field of `valueType` is used to specify the type of values in the map. - * The field of `valueContainsNull` is used to specify if values of this map has `null` values. - * For values of a MapType column, keys are not allowed to have `null` values. - * - * @group dataType - */ - @DeveloperApi - type MapType = catalyst.types.MapType - - /** - * :: DeveloperApi :: - * - * A [[MapType]] object can be constructed with two ways, - * {{{ - * MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) - * }}} and - * {{{ - * MapType(keyType: DataType, valueType: DataType) - * }}} - * For `MapType(keyType: DataType, valueType: DataType)`, - * the field of `valueContainsNull` is set to `true`. - * - * @group dataType - */ - @DeveloperApi - val MapType = catalyst.types.MapType - - /** - * :: DeveloperApi :: - * - * The data type representing [[Row]]s. - * A [[StructType]] object comprises a [[Seq]] of [[StructField]]s. - * - * @group dataType - */ - @DeveloperApi - type StructType = catalyst.types.StructType - - /** - * :: DeveloperApi :: - * - * A [[StructType]] object can be constructed by - * {{{ - * StructType(fields: Seq[StructField]) - * }}} - * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. - * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. - * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single StructField, a `null` will be returned. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val struct = - * StructType( - * StructField("a", IntegerType, true) :: - * StructField("b", LongType, false) :: - * StructField("c", BooleanType, false) :: Nil) - * - * // Extract a single StructField. - * val singleField = struct("b") - * // singleField: StructField = StructField(b,LongType,false) - * - * // This struct does not have a field called "d". null will be returned. - * val nonExisting = struct("d") - * // nonExisting: StructField = null - * - * // Extract multiple StructFields. Field names are provided in a set. - * // A StructType object will be returned. - * val twoFields = struct(Set("b", "c")) - * // twoFields: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * - * // Those names do not have matching fields will be ignored. - * // For the case shown below, "d" will be ignored and - * // it is treated as struct(Set("b", "c")). - * val ignoreNonExisting = struct(Set("b", "c", "d")) - * // ignoreNonExisting: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * }}} - * - * A [[Row]] object is used as a value of the StructType. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val innerStruct = - * StructType( - * StructField("f1", IntegerType, true) :: - * StructField("f2", LongType, false) :: - * StructField("f3", BooleanType, false) :: Nil) - * - * val struct = StructType( - * StructField("a", innerStruct, true) :: Nil) - * - * // Create a Row with the schema defined by struct - * val row = Row(Row(1, 2, true)) - * // row: Row = [[1,2,true]] - * }}} - * - * @group dataType - */ - @DeveloperApi - val StructType = catalyst.types.StructType - - /** - * :: DeveloperApi :: - * - * A [[StructField]] object represents a field in a [[StructType]] object. - * A [[StructField]] object comprises three fields, `name: [[String]]`, `dataType: [[DataType]]`, - * and `nullable: Boolean`. The field of `name` is the name of a `StructField`. The field of - * `dataType` specifies the data type of a `StructField`. - * The field of `nullable` specifies if values of a `StructField` can contain `null` values. - * - * @group field - */ - @DeveloperApi - type StructField = catalyst.types.StructField - - /** - * :: DeveloperApi :: - * - * A [[StructField]] object can be constructed by - * {{{ - * StructField(name: String, dataType: DataType, nullable: Boolean) - * }}} - * - * @group dataType - */ - @DeveloperApi - val StructField = catalyst.types.StructField - /** * Converts a logical plan into zero or more SparkPlans. */ @DeveloperApi type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] - - /** - * :: DeveloperApi :: - * - * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, - * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and - * Array[Metadata]. JSON is used for serialization. - * - * The default constructor is private. User should use either [[MetadataBuilder]] or - * [[Metadata$#fromJson]] to create Metadata instances. - * - * @param map an immutable map that stores the data - */ - @DeveloperApi - type Metadata = catalyst.util.Metadata - - /** - * :: DeveloperApi :: - * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. - */ - @DeveloperApi - type MetadataBuilder = catalyst.util.MetadataBuilder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 1bbb66aaa19a3..7f437c40777f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql.parquet -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** * Collection of converters of Parquet types (group and primitive types) that @@ -91,8 +90,8 @@ private[sql] object CatalystConverter { case ArrayType(elementType: DataType, true) => { new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) } - case StructType(fields: Seq[StructField]) => { - new CatalystStructConverter(fields.toArray, fieldIndex, parent) + case StructType(fields: Array[StructField]) => { + new CatalystStructConverter(fields, fieldIndex, parent) } case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { new CatalystMapConverter( @@ -436,7 +435,7 @@ private[parquet] object CatalystArrayConverter { * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (complex or primitive) * @param index The position of this (array) field inside its parent converter @@ -500,7 +499,7 @@ private[parquet] class CatalystArrayConverter( * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (native) * @param index The position of this (array) field inside its parent converter @@ -621,7 +620,7 @@ private[parquet] class CatalystNativeArrayConverter( * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array contains null (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (complex or primitive) * @param index The position of this (array) field inside its parent converter @@ -727,7 +726,7 @@ private[parquet] class CatalystStructConverter( * A `parquet.io.api.GroupConverter` that converts two-element groups that * match the characteristics of a map (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.MapType]]. + * [[org.apache.spark.sql.types.MapType]]. * * @param schema * @param index diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 56e7d11b2feea..f08350878f239 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -29,7 +29,7 @@ import parquet.io.api.Binary import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 9049eb5932b79..af7248fdf451d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -29,8 +29,8 @@ import parquet.schema.MessageType import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** * A `parquet.io.api.RecordMaterializer` for Rows. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 97447871a11ee..6d8c682ccced8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -36,7 +36,7 @@ import parquet.schema.Type.Repetition import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ // Implicits import scala.collection.JavaConversions._ @@ -80,7 +80,7 @@ private[parquet] object ParquetTypesConverter extends Logging { /** * Converts a given Parquet `Type` into the corresponding - * [[org.apache.spark.sql.catalyst.types.DataType]]. + * [[org.apache.spark.sql.types.DataType]]. * * We apply the following conversion rules: *
    @@ -191,7 +191,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } /** - * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return + * For a given Catalyst [[org.apache.spark.sql.types.DataType]] return * the name of the corresponding Parquet primitive type or None if the given type * is not primitive. * @@ -231,21 +231,21 @@ private[parquet] object ParquetTypesConverter extends Logging { } /** - * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into + * Converts a given Catalyst [[org.apache.spark.sql.types.DataType]] into * the corresponding Parquet `Type`. * * The conversion follows the rules below: *
      *
    • Primitive types are converted into Parquet's primitive types.
    • - *
    • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted + *
    • [[org.apache.spark.sql.types.StructType]]s are converted * into Parquet's `GroupType` with the corresponding field types.
    • - *
    • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converted + *
    • [[org.apache.spark.sql.types.ArrayType]]s are converted * into a 2-level nested group, where the outer group has the inner * group as sole field. The inner group has name `values` and * repetition level `REPEATED` and has the element type of * the array as schema. We use Parquet's `ConversionPatterns` for this * purpose.
    • - *
    • [[org.apache.spark.sql.catalyst.types.MapType]]s are converted + *
    • [[org.apache.spark.sql.types.MapType]]s are converted * into a nested (2-level) Parquet `GroupType` with two fields: a key * type and a value type. The nested group has repetition level * `REPEATED` and name `map`. We use Parquet's `ConversionPatterns` @@ -319,7 +319,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } - new ParquetGroupType(repetition, name, fields) + new ParquetGroupType(repetition, name, fields.toSeq) } case MapType(keyType, valueType, valueContainsNull) => { val parquetKeyType = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 55a2728a85cc7..1b50afbbabcb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.parquet import java.util.{List => JList} +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job} -import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import parquet.hadoop.ParquetInputFormat import parquet.hadoop.util.ContextUtil @@ -30,13 +31,11 @@ import parquet.hadoop.util.ContextUtil import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{Partition => SparkPartition, Logging} import org.apache.spark.rdd.{NewHadoopPartition, RDD} - import org.apache.spark.sql.{SQLConf, Row, SQLContext} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, StructField, StructType} import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -import scala.collection.JavaConversions._ /** * Allows creation of parquet based tables using the syntax diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala index 4d87f6817dcb9..12b59ba20bb10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.AttributeMap +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LeafNode, LogicalPlan} /** @@ -27,7 +27,7 @@ private[sql] case class LogicalRelation(relation: BaseRelation) extends LeafNode with MultiInstanceRelation { - override val output = relation.schema.toAttributes + override val output: Seq[AttributeReference] = relation.schema.toAttributes // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any) = other match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index f8741e0082098..4cc9641c4d9e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -23,11 +23,11 @@ import scala.util.parsing.combinator.PackratParsers import org.apache.spark.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.SqlLexical +import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * A parser for foreign DDL commands. @@ -162,10 +162,10 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi protected lazy val structType: Parser[DataType] = (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { - case fields => new StructType(fields) + case fields => StructType(fields) }) | (STRUCT ~> "<>" ^^ { - case fields => new StructType(Nil) + case fields => StructType(Nil) }) private[sql] lazy val dataType: Parser[DataType] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 7f5564baa00f4..cd82cc6ecb61b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.sources import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext, StructType} +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} +import org.apache.spark.sql.types.StructType /** * ::DeveloperApi:: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index b9569e96c0312..006b16fbe07bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -20,9 +20,7 @@ package org.apache.spark.sql.test import java.util import scala.collection.JavaConverters._ - -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * An example class to demonstrate UDT in Scala, Java, and Python. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala deleted file mode 100644 index d4ef517981699..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ /dev/null @@ -1,175 +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.types.util - -import java.text.SimpleDateFormat - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql._ -import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, - MetadataBuilder => JMetaDataBuilder, UDTWrappers} -import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.types.UserDefinedType - -protected[sql] object DataTypeConversions { - - /** - * Returns the equivalent StructField in Scala for the given StructField in Java. - */ - def asJavaStructField(scalaStructField: StructField): JStructField = { - JDataType.createStructField( - scalaStructField.name, - asJavaDataType(scalaStructField.dataType), - scalaStructField.nullable, - (new JMetaDataBuilder).withMetadata(scalaStructField.metadata).build()) - } - - /** - * Returns the equivalent DataType in Java for the given DataType in Scala. - */ - def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match { - case udtType: UserDefinedType[_] => - UDTWrappers.wrapAsJava(udtType) - - case StringType => JDataType.StringType - case BinaryType => JDataType.BinaryType - case BooleanType => JDataType.BooleanType - case DateType => JDataType.DateType - case TimestampType => JDataType.TimestampType - case DecimalType.Fixed(precision, scale) => new JDecimalType(precision, scale) - case DecimalType.Unlimited => new JDecimalType() - case DoubleType => JDataType.DoubleType - case FloatType => JDataType.FloatType - case ByteType => JDataType.ByteType - case IntegerType => JDataType.IntegerType - case LongType => JDataType.LongType - case ShortType => JDataType.ShortType - case NullType => JDataType.NullType - - case arrayType: ArrayType => JDataType.createArrayType( - asJavaDataType(arrayType.elementType), arrayType.containsNull) - case mapType: MapType => JDataType.createMapType( - asJavaDataType(mapType.keyType), - asJavaDataType(mapType.valueType), - mapType.valueContainsNull) - case structType: StructType => JDataType.createStructType( - structType.fields.map(asJavaStructField).asJava) - } - - /** - * Returns the equivalent StructField in Scala for the given StructField in Java. - */ - def asScalaStructField(javaStructField: JStructField): StructField = { - StructField( - javaStructField.getName, - asScalaDataType(javaStructField.getDataType), - javaStructField.isNullable, - javaStructField.getMetadata) - } - - /** - * Returns the equivalent DataType in Scala for the given DataType in Java. - */ - def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { - case udtType: org.apache.spark.sql.api.java.UserDefinedType[_] => - UDTWrappers.wrapAsScala(udtType) - - case stringType: org.apache.spark.sql.api.java.StringType => - StringType - case binaryType: org.apache.spark.sql.api.java.BinaryType => - BinaryType - case booleanType: org.apache.spark.sql.api.java.BooleanType => - BooleanType - case dateType: org.apache.spark.sql.api.java.DateType => - DateType - case timestampType: org.apache.spark.sql.api.java.TimestampType => - TimestampType - case decimalType: org.apache.spark.sql.api.java.DecimalType => - if (decimalType.isFixed) { - DecimalType(decimalType.getPrecision, decimalType.getScale) - } else { - DecimalType.Unlimited - } - case doubleType: org.apache.spark.sql.api.java.DoubleType => - DoubleType - case floatType: org.apache.spark.sql.api.java.FloatType => - FloatType - case byteType: org.apache.spark.sql.api.java.ByteType => - ByteType - case integerType: org.apache.spark.sql.api.java.IntegerType => - IntegerType - case longType: org.apache.spark.sql.api.java.LongType => - LongType - case shortType: org.apache.spark.sql.api.java.ShortType => - ShortType - - case arrayType: org.apache.spark.sql.api.java.ArrayType => - ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull) - case mapType: org.apache.spark.sql.api.java.MapType => - MapType( - asScalaDataType(mapType.getKeyType), - asScalaDataType(mapType.getValueType), - mapType.isValueContainsNull) - case structType: org.apache.spark.sql.api.java.StructType => - StructType(structType.getFields.map(asScalaStructField)) - } - - def stringToTime(s: String): java.util.Date = { - if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - java.sql.Timestamp.valueOf(s) - } else { - java.sql.Date.valueOf(s) - } - } else if (s.endsWith("Z")) { - // this is zero timezone of ISO8601 - stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") - } else if (s.indexOf("GMT") == -1) { - // timezone with ISO8601 - val inset = "+00.00".length - val s0 = s.substring(0, s.length - inset) - val s1 = s.substring(s.length - inset, s.length) - if (s0.substring(s0.lastIndexOf(':')).contains('.')) { - stringToTime(s0 + "GMT" + s1) - } else { - stringToTime(s0 + ".0GMT" + s1) - } - } else { - // ISO8601 with GMT insert - val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) - ISO8601GMT.parse(s) - } - } - - /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type - case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d)) - case (other, _) => other - } - - /** Converts Java objects to catalyst rows / types */ - def convertCatalystToJava(a: Any): Any = a match { - case d: scala.math.BigDecimal => d.underlying() - case other => other - } -} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index a9a11285def54..88017eb47d908 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -19,15 +19,12 @@ import java.io.Serializable; -import org.apache.spark.sql.api.java.UDF1; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runners.Suite; -import org.junit.runner.RunWith; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.types.DataTypes; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -60,7 +57,7 @@ public void udf1Test() { public Integer call(String str) throws Exception { return str.length(); } - }, DataType.IntegerType); + }, DataTypes.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); @@ -81,7 +78,7 @@ public void udf2Test() { public Integer call(String str1, String str2) throws Exception { return str1.length() + str2.length(); } - }, DataType.IntegerType); + }, DataTypes.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index a04b8060cd658..de586ba635913 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -31,6 +31,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; +import org.apache.spark.sql.types.*; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -93,9 +94,9 @@ public Row call(Person person) throws Exception { }); List fields = new ArrayList(2); - fields.add(DataType.createStructField("name", DataType.StringType, false)); - fields.add(DataType.createStructField("age", DataType.IntegerType, false)); - StructType schema = DataType.createStructType(fields); + fields.add(DataTypes.createStructField("name", DataTypes.StringType, false)); + fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); + StructType schema = DataTypes.createStructType(fields); JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema); schemaRDD.registerTempTable("people"); @@ -118,14 +119,14 @@ public void applySchemaToJSON() { "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + "\"boolean\":false, \"null\":null}")); List fields = new ArrayList(7); - fields.add(DataType.createStructField("bigInteger", new DecimalType(), true)); - fields.add(DataType.createStructField("boolean", DataType.BooleanType, true)); - fields.add(DataType.createStructField("double", DataType.DoubleType, true)); - fields.add(DataType.createStructField("integer", DataType.IntegerType, true)); - fields.add(DataType.createStructField("long", DataType.LongType, true)); - fields.add(DataType.createStructField("null", DataType.StringType, true)); - fields.add(DataType.createStructField("string", DataType.StringType, true)); - StructType expectedSchema = DataType.createStructType(fields); + fields.add(DataTypes.createStructField("bigInteger", DataTypes.createDecimalType(), true)); + fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, true)); + fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, true)); + fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, true)); + fields.add(DataTypes.createStructField("long", DataTypes.LongType, true)); + fields.add(DataTypes.createStructField("null", DataTypes.StringType, true)); + fields.add(DataTypes.createStructField("string", DataTypes.StringType, true)); + StructType expectedSchema = DataTypes.createStructType(fields); List expectedResult = new ArrayList(2); expectedResult.add( Row.create( diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java deleted file mode 100644 index 8396a29c61c4c..0000000000000 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ /dev/null @@ -1,150 +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.api.java; - -import java.util.List; -import java.util.ArrayList; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.spark.sql.types.util.DataTypeConversions; - -public class JavaSideDataTypeConversionSuite { - public void checkDataType(DataType javaDataType) { - org.apache.spark.sql.catalyst.types.DataType scalaDataType = - DataTypeConversions.asScalaDataType(javaDataType); - DataType actual = DataTypeConversions.asJavaDataType(scalaDataType); - Assert.assertEquals(javaDataType, actual); - } - - @Test - public void createDataTypes() { - // Simple DataTypes. - checkDataType(DataType.StringType); - checkDataType(DataType.BinaryType); - checkDataType(DataType.BooleanType); - checkDataType(DataType.DateType); - checkDataType(DataType.TimestampType); - checkDataType(new DecimalType()); - checkDataType(new DecimalType(10, 4)); - checkDataType(DataType.DoubleType); - checkDataType(DataType.FloatType); - checkDataType(DataType.ByteType); - checkDataType(DataType.IntegerType); - checkDataType(DataType.LongType); - checkDataType(DataType.ShortType); - - // Simple ArrayType. - DataType simpleJavaArrayType = DataType.createArrayType(DataType.StringType, true); - checkDataType(simpleJavaArrayType); - - // Simple MapType. - DataType simpleJavaMapType = DataType.createMapType(DataType.StringType, DataType.LongType); - checkDataType(simpleJavaMapType); - - // Simple StructType. - List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false)); - DataType simpleJavaStructType = DataType.createStructType(simpleFields); - checkDataType(simpleJavaStructType); - - // Complex StructType. - List complexFields = new ArrayList(); - complexFields.add(DataType.createStructField("simpleArray", simpleJavaArrayType, true)); - complexFields.add(DataType.createStructField("simpleMap", simpleJavaMapType, true)); - complexFields.add(DataType.createStructField("simpleStruct", simpleJavaStructType, true)); - complexFields.add(DataType.createStructField("boolean", DataType.BooleanType, false)); - DataType complexJavaStructType = DataType.createStructType(complexFields); - checkDataType(complexJavaStructType); - - // Complex ArrayType. - DataType complexJavaArrayType = DataType.createArrayType(complexJavaStructType, true); - checkDataType(complexJavaArrayType); - - // Complex MapType. - DataType complexJavaMapType = - DataType.createMapType(complexJavaStructType, complexJavaArrayType, false); - checkDataType(complexJavaMapType); - } - - @Test - public void illegalArgument() { - // ArrayType - try { - DataType.createArrayType(null, true); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - - // MapType - try { - DataType.createMapType(null, DataType.StringType); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createMapType(DataType.StringType, null); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createMapType(null, null); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - - // StructField - try { - DataType.createStructField(null, DataType.StringType, true); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createStructField("name", null, true); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createStructField(null, null, true); - } catch (IllegalArgumentException expectedException) { - } - - // StructType - try { - List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - simpleFields.add(null); - DataType.createStructType(simpleFields); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - DataType.createStructType(simpleFields); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index ab88f3ad10d66..efe622f8bcb27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ /* Implicits */ import org.apache.spark.sql.catalyst.dsl._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 811319e0a6601..f5b945f468dad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.types._ class RowSuite extends FunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bc72daf0880a1..cbdb3e64bb66b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ /* Implicits */ import org.apache.spark.sql.TestData._ @@ -748,7 +749,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { val metadata = new MetadataBuilder() .putString(docKey, docValue) .build() - val schemaWithMeta = new StructType(Seq( + val schemaWithMeta = new StructType(Array( schema("id"), schema("name").copy(metadata = metadata), schema("age"))) val personWithMeta = applySchema(person, schemaWithMeta) def validateMetadata(rdd: SchemaRDD): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index cf3a59e545905..40fb8d5779fdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 1806a1dd82023..a0d54d17f5f13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType -import org.apache.spark.sql.catalyst.types.UserDefinedType import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types._ @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 8afc3a9fb2187..fdbb4282baf48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.api.java -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.beans.BeanProperty import org.scalatest.FunSuite @@ -26,6 +24,7 @@ import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.types.NullType // Implicits import scala.collection.JavaConversions._ @@ -78,10 +77,10 @@ class JavaSQLSuite extends FunSuite { schemaRDD.registerTempTable("people") val nullRDD = javaSqlCtx.sql("SELECT null FROM people") - val structFields = nullRDD.schema.getFields() + val structFields = nullRDD.schema.fields assert(structFields.size == 1) - assert(structFields(0).getDataType().isInstanceOf[NullType]) - assert(nullRDD.collect.head.row === Seq(null)) + assert(structFields(0).dataType === NullType) + assert(nullRDD.collect().head.row === Seq(null)) } test("all types in JavaBeans") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala deleted file mode 100644 index 62fe59dd345d7..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ /dev/null @@ -1,89 +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.api.java - -import org.scalatest.FunSuite - -import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField, StructType => SStructType} -import org.apache.spark.sql.types.util.DataTypeConversions._ - -class ScalaSideDataTypeConversionSuite extends FunSuite { - - def checkDataType(scalaDataType: SDataType) { - val javaDataType = asJavaDataType(scalaDataType) - val actual = asScalaDataType(javaDataType) - assert(scalaDataType === actual, s"Converted data type ${actual} " + - s"does not equal the expected data type ${scalaDataType}") - } - - test("convert data types") { - // Simple DataTypes. - checkDataType(org.apache.spark.sql.StringType) - checkDataType(org.apache.spark.sql.BinaryType) - checkDataType(org.apache.spark.sql.BooleanType) - checkDataType(org.apache.spark.sql.DateType) - checkDataType(org.apache.spark.sql.TimestampType) - checkDataType(org.apache.spark.sql.DecimalType.Unlimited) - checkDataType(org.apache.spark.sql.DoubleType) - checkDataType(org.apache.spark.sql.FloatType) - checkDataType(org.apache.spark.sql.ByteType) - checkDataType(org.apache.spark.sql.IntegerType) - checkDataType(org.apache.spark.sql.LongType) - checkDataType(org.apache.spark.sql.ShortType) - - // Simple ArrayType. - val simpleScalaArrayType = - org.apache.spark.sql.ArrayType(org.apache.spark.sql.StringType, true) - checkDataType(simpleScalaArrayType) - - // Simple MapType. - val simpleScalaMapType = - org.apache.spark.sql.MapType(org.apache.spark.sql.StringType, org.apache.spark.sql.LongType) - checkDataType(simpleScalaMapType) - - // Simple StructType. - val simpleScalaStructType = SStructType( - SStructField("a", org.apache.spark.sql.DecimalType.Unlimited, false) :: - SStructField("b", org.apache.spark.sql.BooleanType, true) :: - SStructField("c", org.apache.spark.sql.LongType, true) :: - SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil) - checkDataType(simpleScalaStructType) - - // Complex StructType. - val metadata = new MetadataBuilder() - .putString("name", "age") - .build() - val complexScalaStructType = SStructType( - SStructField("simpleArray", simpleScalaArrayType, true) :: - SStructField("simpleMap", simpleScalaMapType, true) :: - SStructField("simpleStruct", simpleScalaStructType, true) :: - SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: - SStructField("withMeta", org.apache.spark.sql.DoubleType, false, metadata) :: Nil) - checkDataType(complexScalaStructType) - - // Complex ArrayType. - val complexScalaArrayType = - org.apache.spark.sql.ArrayType(complexScalaStructType, true) - checkDataType(complexScalaArrayType) - - // Complex MapType. - val complexScalaMapType = - org.apache.spark.sql.MapType(complexScalaStructType, complexScalaArrayType, false) - checkDataType(complexScalaMapType) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index a9f0851f8826c..9be0b38e689ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 3f3f35d50188b..87e608a8853dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -24,9 +24,9 @@ import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ class ColumnTypeSuite extends FunSuite with Logging { val DEFAULT_BUFFER_SIZE = 512 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index a1f21219eaf2f..f941465fa3e35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.{DataType, NativeType} +import org.apache.spark.sql.types.{DataType, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int) = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 21906e3fdcc6f..f95c895587f3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.types.DataType class TestNullableColumnAccessor[T <: DataType, JvmType]( buffer: ByteBuffer, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index cb73f3da81e24..80bd5c94570cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 1cdb909146d57..c82d9799359c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -22,9 +22,9 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.NativeType class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 73f31c0233343..88011631ee4e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.IntegralType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new IntColumnStats, INT, IntDelta) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 4ce2552112c92..08df1db375097 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.NativeType class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new NoopColumnStats, BOOLEAN) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 7db723d648d80..0b18b4119268f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.columnar.compression -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.types.NativeType class TestCompressibleColumnBuilder[T <: NativeType]( override val columnStats: ColumnStats, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index b09f1ac49553b..01c1ce2a61023 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,19 +17,19 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} -import org.apache.spark.sql.{Row, SQLConf, QueryTest} +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ - -import java.sql.{Date, Timestamp} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.{QueryTest, Row, SQLConf} class JsonSuite extends QueryTest { - import TestJsonData._ + import org.apache.spark.sql.json.TestJsonData._ TestJsonData test("Type promotion") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 6ac67fcafe16b..973819aaa4d77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -21,8 +21,6 @@ import scala.collection.JavaConversions._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} import parquet.example.data.simple.SimpleGroup import parquet.example.data.{Group, GroupWriter} import parquet.hadoop.api.WriteSupport @@ -32,11 +30,13 @@ import parquet.hadoop.{ParquetFileWriter, ParquetWriter} import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types.DecimalType import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types.DecimalType import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 0a92336a3cb39..fe781ec05fb6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -29,10 +29,10 @@ import parquet.io.api.Binary import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils case class TestRDDEntry(key: Int, value: String) @@ -911,20 +911,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) } - test("DataType string parser compatibility") { - val schema = StructType(List( - StructField("c1", IntegerType, false), - StructField("c2", BinaryType, false))) - - val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) - val fromJson = ParquetTypesConverter.convertFromString(schema.json) - - (fromCaseClassString, fromJson).zipped.foreach { (a, b) => - assert(a.name == b.name) - assert(a.dataType === b.dataType) - } - } - test("read/write fixed-length decimals") { for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { val tempDir = getTempFilePath("parquetTest").getCanonicalPath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 34d61bf908483..64274950b868e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.FunSuite import parquet.schema.MessageTypeParser import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.types.{BinaryType, IntegerType, StructField, StructType} import org.apache.spark.sql.test.TestSQLContext class ParquetSchemaSuite extends FunSuite with ParquetTest { @@ -148,12 +147,20 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { """.stripMargin) test("DataType string parser compatibility") { - val schema = StructType(List( - StructField("c1", IntegerType, false), - StructField("c2", BinaryType, true))) - - val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) - val fromJson = ParquetTypesConverter.convertFromString(schema.json) + // This is the generated string from previous versions of the Spark SQL, using the following: + // val schema = StructType(List( + // StructField("c1", IntegerType, false), + // StructField("c2", BinaryType, true))) + val caseClassString = + "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))" + + val jsonString = + """ + |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]} + """.stripMargin + + val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString) + val fromJson = ParquetTypesConverter.convertFromString(jsonString) (fromCaseClassString, fromJson).zipped.foreach { (a, b) => assert(a.name == b.name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 939b3c0c66de7..390538d35a348 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.sources import scala.language.existentials import org.apache.spark.sql._ +import org.apache.spark.sql.types._ + class FilteredScanSource extends RelationProvider { override def createRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index fee2e22611cdc..7900b3e8948d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql._ +import org.apache.spark.sql.types._ class PrunedScanSource extends RelationProvider { override def createRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index a1d2468b2573c..382dddcdeac38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.sources import java.sql.{Timestamp, Date} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.types._ class DefaultSource extends SimpleScanSource diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 742acba58d776..171d707b138b6 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -32,11 +32,11 @@ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow} import org.apache.spark.sql.execution.SetCommand import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.types._ /** * A compatibility layer for interacting with Hive version 0.12.0. diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index b82156427a88c..bec9d9aca30cf 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -30,11 +30,11 @@ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} import org.apache.spark.sql.execution.SetCommand -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} +import org.apache.spark.sql.types._ /** * A compatibility layer for interacting with Hive version 0.13.1. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 9aeebd7e54366..bf56e60cf995f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -37,10 +37,10 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types.DecimalType import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, SetCommand, QueryExecutionException} import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTableCommand} import org.apache.spark.sql.sources.DataSourceStrategy +import org.apache.spark.sql.types._ /** * DEPRECATED: Use HiveContext instead. 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 a156d6f7e285d..245b847cf4cd4 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 @@ -24,9 +24,9 @@ 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.types -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -43,7 +43,7 @@ import scala.collection.JavaConversions._ * long / scala.Long * short / scala.Short * byte / scala.Byte - * org.apache.spark.sql.catalyst.types.decimal.Decimal + * org.apache.spark.sql.types.decimal.Decimal * Array[Byte] * java.sql.Date * java.sql.Timestamp @@ -504,7 +504,8 @@ private[hive] trait HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) + java.util.Arrays.asList(fields.map(f => f.name) :_*), + java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)) :_*)) } /** @@ -618,7 +619,9 @@ private[hive] trait HiveInspectors { case ArrayType(elemType, _) => getListTypeInfo(elemType.toTypeInfo) case StructType(fields) => - getStructTypeInfo(fields.map(_.name), fields.map(_.dataType.toTypeInfo)) + getStructTypeInfo( + java.util.Arrays.asList(fields.map(_.name) :_*), + java.util.Arrays.asList(fields.map(_.dataType.toTypeInfo) :_*)) case MapType(keyType, valueType, _) => getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo) case BinaryType => binaryTypeInfo 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 785a6a14f49f4..d40f9936fd3b5 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 @@ -39,8 +39,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.sources.{LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils /* Implicit conversions */ 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 34622b5f57873..b13ef7276bf3c 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.sql.Date + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.lib.Node @@ -31,10 +32,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ 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 cdff82e3d04d2..6952b126cf894 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 @@ -17,23 +17,24 @@ package org.apache.spark.sql.hive +import scala.collection.JavaConversions._ + import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.sources.CreateTableUsing -import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy} +import org.apache.spark.sql.types.StringType -import scala.collection.JavaConversions._ private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. 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 8ba818af5f9d0..781a2e9164c82 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 @@ -22,7 +22,7 @@ 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.catalyst.types.StringType +import org.apache.spark.sql.types.StringType /** * :: DeveloperApi :: 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 8bbcd6fec1f3b..b56175fe76376 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 @@ -19,20 +19,18 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ -import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ +import org.apache.spark.sql.types.{BooleanType, DataType} /** * :: DeveloperApi :: 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 e70cdeaad4c09..cf72345efa63f 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 @@ -18,11 +18,11 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types.StructType 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.StructType /** * :: DeveloperApi :: 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 7d863f9d89dae..d898b876c39f8 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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils.getContextOrSparkClassLoader /* Implicit conversions */ 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 f90d3607915ae..dc23d9a101d17 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 @@ -17,22 +17,21 @@ package org.apache.spark.sql.hive -import java.sql.Date import java.util +import java.sql.Date import java.util.{Locale, TimeZone} -import org.apache.hadoop.hive.serde2.io.DoubleWritable -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.scalatest.FunSuite - import org.apache.hadoop.hive.ql.udf.UDAFPercentile -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StructObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.io.DoubleWritable +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.io.LongWritable +import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{Literal, Row} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal class HiveInspectorSuite extends FunSuite with HiveInspectors { test("Test wrap SettableStructObjectInspector") { @@ -93,7 +92,6 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val row = data.map(_.eval(null)) val dataTypes = data.map(_.dataType) - import scala.collection.JavaConversions._ def toWritableInspector(dataType: DataType): ObjectInspector = dataType match { case ArrayType(tpe, _) => ObjectInspectorFactory.getStandardListObjectInspector(toWritableInspector(tpe)) @@ -115,7 +113,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { case DecimalType() => PrimitiveObjectInspectorFactory.writableHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( - fields.map(f => f.name), fields.map(f => toWritableInspector(f.dataType))) + java.util.Arrays.asList(fields.map(f => f.name) :_*), + java.util.Arrays.asList(fields.map(f => toWritableInspector(f.dataType)) :_*)) } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 041a36f1295ef..fa6905f31f81e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.hive import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.sources.DDLParser import org.apache.spark.sql.test.ExamplePointUDT +import org.apache.spark.sql.types.StructType class HiveMetastoreCatalogSuite extends FunSuite { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index fb481edc853b7..7cfb875e05db3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -22,6 +22,7 @@ import java.io.File 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.types._ /* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ 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 ec9ebb4a775a3..8ff833e0d60d1 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 @@ -25,6 +25,7 @@ import org.apache.commons.io.FileUtils import org.apache.spark.sql._ import org.apache.spark.util.Utils +import org.apache.spark.sql.types._ /* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 25fdf5c5f3da6..a5587460fd696 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -18,8 +18,12 @@ package org.apache.spark.sql.hive import java.net.URI -import java.util.{ArrayList => JArrayList} -import java.util.Properties +import java.util.{ArrayList => JArrayList, Properties} + +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.`type`.HiveDecimal @@ -29,20 +33,16 @@ 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._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, PrimitiveObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import scala.collection.JavaConversions._ -import scala.language.implicitConversions -import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.decimal.Decimal case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { // for Serialization 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 index e47002cb0b8c8..a7121360dd350 100644 --- 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 @@ -20,6 +20,9 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.NullWritable @@ -37,12 +40,10 @@ import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} -import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.types.DecimalType -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import scala.collection.JavaConversions._ -import scala.language.implicitConversions +import org.apache.spark.Logging +import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.decimal.Decimal /** From d5eeb35167e1ab72fab7778757163ff0aacaef2c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Jan 2015 00:38:55 -0800 Subject: [PATCH 19/46] [SPARK-5167][SQL] Move Row into sql package and make it usable for Java. Mostly just moving stuff around. This should still be source compatible since we type aliased Row previously in org.apache.spark.sql.Row. Added the following APIs to Row: ```scala def getMap[K, V](i: Int): scala.collection.Map[K, V] def getJavaMap[K, V](i: Int): java.util.Map[K, V] def getSeq[T](i: Int): Seq[T] def getList[T](i: Int): java.util.List[T] def getStruct(i: Int): StructType ``` Author: Reynold Xin Closes #4030 from rxin/sql-row and squashes the following commits: 6c85c29 [Reynold Xin] Fixed style violation by adding a new line to Row.scala. 82b064a [Reynold Xin] [SPARK-5167][SQL] Move Row into sql package and make it usable for Java. --- .../java/org/apache/spark/sql/RowFactory.java | 34 +++ .../main/scala/org/apache/spark/sql/Row.scala | 240 ++++++++++++++++++ .../sql/catalyst/expressions/package.scala | 4 + .../expressions/{Row.scala => rows.scala} | 113 ++------- .../apache/spark/sql/types/dataTypes.scala | 2 +- .../org/apache/spark/sql/api/java/Row.scala | 2 +- .../scala/org/apache/spark/sql/package.scala | 83 ------ 7 files changed, 304 insertions(+), 174 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{Row.scala => rows.scala} (68%) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java new file mode 100644 index 0000000000000..62fcec824d09a --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java @@ -0,0 +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; + +import org.apache.spark.sql.catalyst.expressions.GenericRow; + +/** + * A factory class used to construct {@link Row} objects. + */ +public class RowFactory { + + /** + * Create a {@link Row} from an array of values. Position i in the array becomes position i + * in the created {@link Row} object. + */ + public static Row create(Object[] values) { + return new GenericRow(values); + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala new file mode 100644 index 0000000000000..d7a4e014ce6a6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -0,0 +1,240 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.GenericRow + + +object Row { + /** + * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + */ + def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) + + /** + * This method can be used to construct a [[Row]] with the given values. + */ + def apply(values: Any*): Row = new GenericRow(values.toArray) + + /** + * This method can be used to construct a [[Row]] from a [[Seq]] of values. + */ + def fromSeq(values: Seq[Any]): Row = new GenericRow(values.toArray) +} + + +/** + * Represents one row of output from a relational operator. Allows both generic access by ordinal, + * which will incur boxing overhead for primitives, as well as native primitive access. + * + * It is invalid to use the native primitive interface to retrieve a value that is null, instead a + * user must check `isNullAt` before attempting to retrieve a value that might be null. + * + * To create a new Row, use [[RowFactory.create()]] in Java or [[Row.apply()]] in Scala. + * + * A [[Row]] object can be constructed by providing field values. Example: + * {{{ + * import org.apache.spark.sql._ + * + * // Create a Row from values. + * Row(value1, value2, value3, ...) + * // Create a Row from a Seq of values. + * Row.fromSeq(Seq(value1, value2, ...)) + * }}} + * + * A value of a row can be accessed through both generic access by ordinal, + * which will incur boxing overhead for primitives, as well as native primitive access. + * An example of generic access by ordinal: + * {{{ + * import org.apache.spark.sql._ + * + * val row = Row(1, true, "a string", null) + * // row: Row = [1,true,a string,null] + * val firstValue = row(0) + * // firstValue: Any = 1 + * val fourthValue = row(3) + * // fourthValue: Any = null + * }}} + * + * For native primitive access, it is invalid to use the native primitive interface to retrieve + * a value that is null, instead a user must check `isNullAt` before attempting to retrieve a + * value that might be null. + * An example of native primitive access: + * {{{ + * // using the row from the previous example. + * val firstValue = row.getInt(0) + * // firstValue: Int = 1 + * val isNull = row.isNullAt(3) + * // isNull: Boolean = true + * }}} + * + * Interfaces related to native primitive access are: + * + * `isNullAt(i: Int): Boolean` + * + * `getInt(i: Int): Int` + * + * `getLong(i: Int): Long` + * + * `getDouble(i: Int): Double` + * + * `getFloat(i: Int): Float` + * + * `getBoolean(i: Int): Boolean` + * + * `getShort(i: Int): Short` + * + * `getByte(i: Int): Byte` + * + * `getString(i: Int): String` + * + * In Scala, fields in a [[Row]] object can be extracted in a pattern match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + * + * @group row + */ +trait Row extends Seq[Any] with Serializable { + def apply(i: Int): Any + + /** Returns the value at position i. If the value is null, null is returned. */ + def get(i: Int): Any = apply(i) + + /** Checks whether the value at position i is null. */ + def isNullAt(i: Int): Boolean + + /** + * Returns the value at position i as a primitive int. + * Throws an exception if the type mismatches or if the value is null. + */ + def getInt(i: Int): Int + + /** + * Returns the value at position i as a primitive long. + * Throws an exception if the type mismatches or if the value is null. + */ + def getLong(i: Int): Long + + /** + * Returns the value at position i as a primitive double. + * Throws an exception if the type mismatches or if the value is null. + */ + def getDouble(i: Int): Double + + /** + * Returns the value at position i as a primitive float. + * Throws an exception if the type mismatches or if the value is null. + */ + def getFloat(i: Int): Float + + /** + * Returns the value at position i as a primitive boolean. + * Throws an exception if the type mismatches or if the value is null. + */ + def getBoolean(i: Int): Boolean + + /** + * Returns the value at position i as a primitive short. + * Throws an exception if the type mismatches or if the value is null. + */ + def getShort(i: Int): Short + + /** + * Returns the value at position i as a primitive byte. + * Throws an exception if the type mismatches or if the value is null. + */ + def getByte(i: Int): Byte + + /** + * Returns the value at position i as a String object. + * Throws an exception if the type mismatches or if the value is null. + */ + def getString(i: Int): String + + /** + * Return the value at position i of array type as a Scala Seq. + * Throws an exception if the type mismatches. + */ + def getSeq[T](i: Int): Seq[T] = apply(i).asInstanceOf[Seq[T]] + + /** + * Return the value at position i of array type as [[java.util.List]]. + * Throws an exception if the type mismatches. + */ + def getList[T](i: Int): java.util.List[T] = { + scala.collection.JavaConversions.seqAsJavaList(getSeq[T](i)) + } + + /** + * Return the value at position i of map type as a Scala Map. + * Throws an exception if the type mismatches. + */ + def getMap[K, V](i: Int): scala.collection.Map[K, V] = apply(i).asInstanceOf[Map[K, V]] + + /** + * Return the value at position i of array type as a [[java.util.Map]]. + * Throws an exception if the type mismatches. + */ + def getJavaMap[K, V](i: Int): java.util.Map[K, V] = { + scala.collection.JavaConversions.mapAsJavaMap(getMap[K, V](i)) + } + + /** + * Return the value at position i of struct type as an [[Row]] object. + * Throws an exception if the type mismatches. + */ + def getStruct(i: Int): Row = getAs[Row](i) + + /** + * Returns the value at position i. + * Throws an exception if the type mismatches. + */ + def getAs[T](i: Int): T = apply(i).asInstanceOf[T] + + override def toString(): String = s"[${this.mkString(",")}]" + + /** + * Make a copy of the current [[Row]] object. + */ + def copy(): Row + + /** Returns true if there are any NULL values in this row. */ + def anyNull: Boolean = { + val l = length + var i = 0 + while (i < l) { + if (isNullAt(i)) { return true } + i += 1 + } + false + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 55d95991c5f11..fbc97b2e75312 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -49,6 +49,10 @@ package org.apache.spark.sql.catalyst */ package object expressions { + type Row = org.apache.spark.sql.Row + + val Row = org.apache.spark.sql.Row + /** * Converts a [[Row]] to another Row given a sequence of expression that define each column of the * new row. If the schema of the input row is specified, then the given expression will be bound diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala similarity index 68% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index dcda53bb717a9..c22b8426841da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -19,68 +19,6 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types.NativeType -object Row { - /** - * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: - * {{{ - * import org.apache.spark.sql._ - * - * val pairs = sql("SELECT key, value FROM src").rdd.map { - * case Row(key: Int, value: String) => - * key -> value - * } - * }}} - */ - def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) - - /** - * This method can be used to construct a [[Row]] with the given values. - */ - def apply(values: Any*): Row = new GenericRow(values.toArray) - - /** - * This method can be used to construct a [[Row]] from a [[Seq]] of values. - */ - def fromSeq(values: Seq[Any]): Row = new GenericRow(values.toArray) -} - -/** - * Represents one row of output from a relational operator. Allows both generic access by ordinal, - * which will incur boxing overhead for primitives, as well as native primitive access. - * - * It is invalid to use the native primitive interface to retrieve a value that is null, instead a - * user must check [[isNullAt]] before attempting to retrieve a value that might be null. - */ -trait Row extends Seq[Any] with Serializable { - def apply(i: Int): Any - - def isNullAt(i: Int): Boolean - - def getInt(i: Int): Int - def getLong(i: Int): Long - def getDouble(i: Int): Double - def getFloat(i: Int): Float - def getBoolean(i: Int): Boolean - def getShort(i: Int): Short - def getByte(i: Int): Byte - def getString(i: Int): String - def getAs[T](i: Int): T = apply(i).asInstanceOf[T] - - override def toString() = - s"[${this.mkString(",")}]" - - def copy(): Row - - /** Returns true if there are any NULL values in this row. */ - def anyNull: Boolean = { - var i = 0 - while (i < length) { - if (isNullAt(i)) { return true } - i += 1 - } - false - } -} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -105,22 +43,19 @@ trait MutableRow extends Row { * A row with no data. Calling any methods will result in an error. Can be used as a placeholder. */ object EmptyRow extends Row { - def apply(i: Int): Any = throw new UnsupportedOperationException - - def iterator = Iterator.empty - def length = 0 - def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException - - def getInt(i: Int): Int = throw new UnsupportedOperationException - def getLong(i: Int): Long = throw new UnsupportedOperationException - def getDouble(i: Int): Double = throw new UnsupportedOperationException - def getFloat(i: Int): Float = throw new UnsupportedOperationException - def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException - def getShort(i: Int): Short = throw new UnsupportedOperationException - def getByte(i: Int): Byte = throw new UnsupportedOperationException - def getString(i: Int): String = throw new UnsupportedOperationException + override def apply(i: Int): Any = throw new UnsupportedOperationException + override def iterator = Iterator.empty + override def length = 0 + override def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException + override def getInt(i: Int): Int = throw new UnsupportedOperationException + override def getLong(i: Int): Long = throw new UnsupportedOperationException + override def getDouble(i: Int): Double = throw new UnsupportedOperationException + override def getFloat(i: Int): Float = throw new UnsupportedOperationException + override def getBoolean(i: Int): Boolean = throw new UnsupportedOperationException + override def getShort(i: Int): Short = throw new UnsupportedOperationException + override def getByte(i: Int): Byte = throw new UnsupportedOperationException + override def getString(i: Int): String = throw new UnsupportedOperationException override def getAs[T](i: Int): T = throw new UnsupportedOperationException - def copy() = this } @@ -135,50 +70,50 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { def this(size: Int) = this(new Array[Any](size)) - def iterator = values.iterator + override def iterator = values.iterator - def length = values.length + override def length = values.length - def apply(i: Int) = values(i) + override def apply(i: Int) = values(i) - def isNullAt(i: Int) = values(i) == null + override def isNullAt(i: Int) = values(i) == null - def getInt(i: Int): Int = { + override def getInt(i: Int): Int = { if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") values(i).asInstanceOf[Int] } - def getLong(i: Int): Long = { + override def getLong(i: Int): Long = { if (values(i) == null) sys.error("Failed to check null bit for primitive long value.") values(i).asInstanceOf[Long] } - def getDouble(i: Int): Double = { + override def getDouble(i: Int): Double = { if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") values(i).asInstanceOf[Double] } - def getFloat(i: Int): Float = { + override def getFloat(i: Int): Float = { if (values(i) == null) sys.error("Failed to check null bit for primitive float value.") values(i).asInstanceOf[Float] } - def getBoolean(i: Int): Boolean = { + override def getBoolean(i: Int): Boolean = { if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") values(i).asInstanceOf[Boolean] } - def getShort(i: Int): Short = { + override def getShort(i: Int): Short = { if (values(i) == null) sys.error("Failed to check null bit for primitive short value.") values(i).asInstanceOf[Short] } - def getByte(i: Int): Byte = { + override def getByte(i: Int): Byte = { if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") values(i).asInstanceOf[Byte] } - def getString(i: Int): String = { + override def getString(i: Int): String = { if (values(i) == null) sys.error("Failed to check null bit for primitive String value.") values(i).asInstanceOf[String] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index fa0a355ebc9b0..e38ad63f2e2c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -727,7 +727,7 @@ object StructType { * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) * }}} * - * A [[org.apache.spark.sql.catalyst.expressions.Row]] object is used as a value of the StructType. + * A [[org.apache.spark.sql.Row]] object is used as a value of the StructType. * Example: * {{{ * import org.apache.spark.sql._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 207e2805fffe3..4faa79af2568a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions import scala.math.BigDecimal import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap -import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} +import org.apache.spark.sql.{Row => ScalaRow} /** * A result row from a Spark SQL query. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index b75266d5aa409..6dd39be807037 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -33,89 +33,6 @@ import org.apache.spark.sql.execution.SparkPlan */ package object sql { - /** - * :: DeveloperApi :: - * - * Represents one row of output from a relational operator. - * @group row - */ - @DeveloperApi - type Row = catalyst.expressions.Row - - /** - * :: DeveloperApi :: - * - * A [[Row]] object can be constructed by providing field values. Example: - * {{{ - * import org.apache.spark.sql._ - * - * // Create a Row from values. - * Row(value1, value2, value3, ...) - * // Create a Row from a Seq of values. - * Row.fromSeq(Seq(value1, value2, ...)) - * }}} - * - * A value of a row can be accessed through both generic access by ordinal, - * which will incur boxing overhead for primitives, as well as native primitive access. - * An example of generic access by ordinal: - * {{{ - * import org.apache.spark.sql._ - * - * val row = Row(1, true, "a string", null) - * // row: Row = [1,true,a string,null] - * val firstValue = row(0) - * // firstValue: Any = 1 - * val fourthValue = row(3) - * // fourthValue: Any = null - * }}} - * - * For native primitive access, it is invalid to use the native primitive interface to retrieve - * a value that is null, instead a user must check `isNullAt` before attempting to retrieve a - * value that might be null. - * An example of native primitive access: - * {{{ - * // using the row from the previous example. - * val firstValue = row.getInt(0) - * // firstValue: Int = 1 - * val isNull = row.isNullAt(3) - * // isNull: Boolean = true - * }}} - * - * Interfaces related to native primitive access are: - * - * `isNullAt(i: Int): Boolean` - * - * `getInt(i: Int): Int` - * - * `getLong(i: Int): Long` - * - * `getDouble(i: Int): Double` - * - * `getFloat(i: Int): Float` - * - * `getBoolean(i: Int): Boolean` - * - * `getShort(i: Int): Short` - * - * `getByte(i: Int): Byte` - * - * `getString(i: Int): String` - * - * Fields in a [[Row]] object can be extracted in a pattern match. Example: - * {{{ - * import org.apache.spark.sql._ - * - * val pairs = sql("SELECT key, value FROM src").rdd.map { - * case Row(key: Int, value: String) => - * key -> value - * } - * }}} - * - * @group row - */ - @DeveloperApi - val Row = catalyst.expressions.Row - /** * Converts a logical plan into zero or more SparkPlans. */ From a3f7421b42f45e39f3e53679188e4eae2ed1f208 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 14 Jan 2015 09:36:59 -0800 Subject: [PATCH 20/46] [SPARK-5248] [SQL] move sql.types.decimal.Decimal to sql.types.Decimal rxin follow up of #3732 Author: Daoyuan Wang Closes #4041 from adrian-wang/decimal and squashes the following commits: aa3d738 [Daoyuan Wang] fix auto refactor 7777a58 [Daoyuan Wang] move sql.types.decimal.Decimal to sql.types.Decimal --- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 1 - .../scala/org/apache/spark/sql/catalyst/dsl/package.scala | 1 - .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 1 - .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 7 +++---- .../spark/sql/catalyst/expressions/decimalFunctions.scala | 3 +-- .../apache/spark/sql/catalyst/expressions/literals.scala | 1 - .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 1 - .../org/apache/spark/sql/types/DataTypeConversions.scala | 2 +- .../org/apache/spark/sql/types/{decimal => }/Decimal.scala | 4 ++-- .../main/scala/org/apache/spark/sql/types/dataTypes.scala | 1 - .../catalyst/expressions/ExpressionEvaluationSuite.scala | 1 - .../org/apache/spark/sql/types/decimal/DecimalSuite.scala | 1 + .../apache/spark/sql/execution/SparkSqlSerializer.scala | 3 ++- .../src/main/scala/org/apache/spark/sql/json/JsonRDD.scala | 1 - .../org/apache/spark/sql/parquet/ParquetConverter.scala | 1 - .../org/apache/spark/sql/parquet/ParquetTableSupport.scala | 1 - .../test/scala/org/apache/spark/sql/json/JsonSuite.scala | 1 - .../scala/org/apache/spark/sql/hive/HiveInspectors.scala | 3 +-- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 1 - .../org/apache/spark/sql/hive/HiveInspectorSuite.scala | 1 - .../src/main/scala/org/apache/spark/sql/hive/Shim12.scala | 3 +-- .../src/main/scala/org/apache/spark/sql/hive/Shim13.scala | 3 +-- 22 files changed, 13 insertions(+), 29 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/types/{decimal => }/Decimal.scala (98%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d169da691d797..697bacfedc624 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -23,7 +23,6 @@ import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index bdac7504ed027..8bc36a238dbbb 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 00961f09916b3..1a2133bbbcec7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -23,7 +23,6 @@ import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index a5d6423391295..4cae5c4718683 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen import com.google.common.cache.{CacheLoader, CacheBuilder} -import org.apache.spark.sql.types.decimal.Decimal import scala.language.existentials @@ -541,11 +540,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin childEval.code ++ q""" var $nullTerm = ${childEval.nullTerm} - var $primitiveTerm: org.apache.spark.sql.types.decimal.Decimal = + var $primitiveTerm: org.apache.spark.sql.types.Decimal = ${defaultPrimitive(DecimalType())} if (!$nullTerm) { - $primitiveTerm = new org.apache.spark.sql.types.decimal.Decimal() + $primitiveTerm = new org.apache.spark.sql.types.Decimal() $primitiveTerm = $primitiveTerm.setOrNull(${childEval.primitiveTerm}, $precision, $scale) $nullTerm = $primitiveTerm == null } @@ -627,7 +626,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case LongType => ru.Literal(Constant(1L)) case ByteType => ru.Literal(Constant(-1.toByte)) case DoubleType => ru.Literal(Constant(-1.toDouble)) - case DecimalType() => q"org.apache.spark.sql.types.decimal.Decimal(-1)" + case DecimalType() => q"org.apache.spark.sql.types.Decimal(-1)" case IntegerType => ru.Literal(Constant(-1)) case _ => ru.Literal(Constant(null)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index e54cfa144a173..83d8c1d42bca4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.decimal.Decimal -import org.apache.spark.sql.types.{DecimalType, LongType, DoubleType, DataType} +import org.apache.spark.sql.types._ /** Return the unscaled Long value of a Decimal, assuming it fits in a Long */ case class UnscaledValue(child: Expression) extends UnaryExpression { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 8ee4bbd8caa62..c94a947fb275c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal object Literal { def apply(v: Any): Literal = v match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 17b4f9c23a978..d4a4c35691bcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.LeftSemi import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal abstract class Optimizer extends RuleExecutor[LogicalPlan] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala index 2a8914cde248c..08bb933a2b33e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import java.text.SimpleDateFormat import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.types.Decimal protected[sql] object DataTypeConversions { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala similarity index 98% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index c7864d1ae9e74..3744d77c0736e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/decimal/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.types.decimal +package org.apache.spark.sql.types import org.apache.spark.annotation.DeveloperApi @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi * - Otherwise, the decimal value is longVal / (10 ** _scale) */ final class Decimal extends Ordered[Decimal] with Serializable { - import Decimal.{MAX_LONG_DIGITS, POW_10, ROUNDING_MODE, BIG_DEC_ZERO} + import org.apache.spark.sql.types.Decimal.{BIG_DEC_ZERO, MAX_LONG_DIGITS, POW_10, ROUNDING_MODE} private var decimalVal: BigDecimal = null private var longVal: Long = 0L diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index e38ad63f2e2c3..e1cbe6650aaaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -32,7 +32,6 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} -import org.apache.spark.sql.types.decimal._ import org.apache.spark.util.Utils diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 8552448b8d108..37e64adeea853 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal class ExpressionEvaluationSuite extends FunSuite { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala index 813377df00132..de6a2cd448c47 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.types.decimal +import org.apache.spark.sql.types.Decimal import org.scalatest.{PrivateMethodTester, FunSuite} import scala.language.postfixOps diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 131146012ecac..7a0249137a207 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import org.apache.spark.sql.types.Decimal + import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog @@ -29,7 +31,6 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index c92ec543e2936..453b560ff8716 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 7f437c40777f2..b4aed04199129 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -25,7 +25,6 @@ import parquet.schema.MessageType import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /** * Collection of converters of Parquet types (group and primitive types) that diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index af7248fdf451d..fd63ad8144064 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -30,7 +30,6 @@ import parquet.schema.MessageType import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /** * A `parquet.io.api.RecordMaterializer` for Rows. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 01c1ce2a61023..1dd85a3bb43a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.sql.{QueryTest, Row, SQLConf} class JsonSuite extends QueryTest { 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 245b847cf4cd4..5140d2064c5fb 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 @@ -26,7 +26,6 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -43,7 +42,7 @@ import scala.collection.JavaConversions._ * long / scala.Long * short / scala.Short * byte / scala.Byte - * org.apache.spark.sql.types.decimal.Decimal + * org.apache.spark.sql.types.Decimal * Array[Byte] * java.sql.Date * java.sql.Timestamp 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 b13ef7276bf3c..5e29e57d93585 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 @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ 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 dc23d9a101d17..4864607252034 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 @@ -31,7 +31,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{Literal, Row} import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.decimal.Decimal class HiveInspectorSuite extends FunSuite with HiveInspectors { test("Test wrap SettableStructObjectInspector") { diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index a5587460fd696..58417a15bbedc 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -41,8 +41,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.types.DecimalType -import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.types.{Decimal, DecimalType} case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { // for Serialization 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 index a7121360dd350..1f768ca971249 100644 --- 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 @@ -42,8 +42,7 @@ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging -import org.apache.spark.sql.types.DecimalType -import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.types.{Decimal, DecimalType} /** From 81f72a0df2250debe8a6a0773d809d8c42eeabb9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 14 Jan 2015 09:47:30 -0800 Subject: [PATCH 21/46] [SPARK-5211][SQL]Restore HiveMetastoreTypes.toDataType jira: https://issues.apache.org/jira/browse/SPARK-5211 Author: Yin Huai Closes #4026 from yhuai/SPARK-5211 and squashes the following commits: 15ee32b [Yin Huai] Remove extra line. c6c1651 [Yin Huai] Get back HiveMetastoreTypes.toDataType. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++++++- .../apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala | 5 +---- 2 files changed, 8 insertions(+), 5 deletions(-) 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 d40f9936fd3b5..1a49f09bd9988 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 @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.sources.{LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.sources.{DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -558,6 +558,12 @@ private[hive] case class MetastoreRelation } object HiveMetastoreTypes { + protected val ddlParser = new DDLParser + + def toDataType(metastoreType: String): DataType = synchronized { + ddlParser.parseType(metastoreType) + } + def toMetastoreType(dt: DataType): String = dt match { case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" case StructType(fields) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index fa6905f31f81e..aad48ada52642 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive import org.scalatest.FunSuite -import org.apache.spark.sql.sources.DDLParser import org.apache.spark.sql.test.ExamplePointUDT import org.apache.spark.sql.types.StructType @@ -28,9 +27,7 @@ class HiveMetastoreCatalogSuite extends FunSuite { test("struct field should accept underscore in sub-column name") { val metastr = "struct" - val ddlParser = new DDLParser - - val datatype = ddlParser.parseType(metastr) + val datatype = HiveMetastoreTypes.toDataType(metastr) assert(datatype.isInstanceOf[StructType]) } From 38bdc992a1a0485ac630af500da54f0a77e133bf Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 14 Jan 2015 09:50:01 -0800 Subject: [PATCH 22/46] [SQL] some comments fix for GROUPING SETS Author: Daoyuan Wang Closes #4000 from adrian-wang/comment and squashes the following commits: 9c24fc4 [Daoyuan Wang] some comments --- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index bd00ff22ba80a..7f4cc234dc9cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -126,10 +126,10 @@ class Analyzer(catalog: Catalog, } /* - * GROUP BY a, b, c, WITH ROLLUP + * GROUP BY a, b, c WITH ROLLUP * is equivalent to - * GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (a), ( )). - * Group Count: N + 1 (N is the number of group expression) + * GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (a), ( ) ). + * Group Count: N + 1 (N is the number of group expressions) * * We need to get all of its subsets for the rule described above, the subset is * represented as the bit masks. @@ -139,12 +139,12 @@ class Analyzer(catalog: Catalog, } /* - * GROUP BY a, b, c, WITH CUBE + * GROUP BY a, b, c WITH CUBE * is equivalent to * GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (b, c), (a, c), (a), (b), (c), ( ) ). - * Group Count: 2^N (N is the number of group expression) + * Group Count: 2 ^ N (N is the number of group expressions) * - * We need to get all of its sub sets for a given GROUPBY expressions, the subset is + * We need to get all of its subsets for a given GROUPBY expression, the subsets are * represented as the bit masks. */ def bitmasks(c: Cube): Seq[Int] = { From 5840f5464bad8431810d459c97d6e4635eea175c Mon Sep 17 00:00:00 2001 From: MechCoder Date: Wed, 14 Jan 2015 11:03:11 -0800 Subject: [PATCH 23/46] [SPARK-2909] [MLlib] [PySpark] SparseVector in pyspark now supports indexing Slightly different than the scala code which converts the sparsevector into a densevector and then checks the index. I also hope I've added tests in the right place. Author: MechCoder Closes #4025 from MechCoder/spark-2909 and squashes the following commits: 07d0f26 [MechCoder] STY: Rename item to index f02148b [MechCoder] [SPARK-2909] [Mlib] SparseVector in pyspark now supports indexing --- python/pyspark/mllib/linalg.py | 17 +++++++++++++++++ python/pyspark/mllib/tests.py | 12 ++++++++++++ 2 files changed, 29 insertions(+) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 4f8491f43e457..7f21190ed8c25 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -510,6 +510,23 @@ def __eq__(self, other): and np.array_equal(other.indices, self.indices) and np.array_equal(other.values, self.values)) + def __getitem__(self, index): + inds = self.indices + vals = self.values + if not isinstance(index, int): + raise ValueError( + "Indices must be of type integer, got type %s" % type(index)) + if index < 0: + index += self.size + if index >= self.size or index < 0: + raise ValueError("Index %d out of bounds." % index) + + insert_index = np.searchsorted(inds, index) + row_ind = inds[insert_index] + if row_ind == index: + return vals[insert_index] + return 0. + def __ne__(self, other): return not self.__eq__(other) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 1f48bc1219dba..140c22b5fd4e8 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -120,6 +120,18 @@ def test_conversion(self): dv = DenseVector(v) self.assertTrue(dv.array.dtype == 'float64') + def test_sparse_vector_indexing(self): + sv = SparseVector(4, {1: 1, 3: 2}) + self.assertEquals(sv[0], 0.) + self.assertEquals(sv[3], 2.) + self.assertEquals(sv[1], 1.) + self.assertEquals(sv[2], 0.) + self.assertEquals(sv[-1], 2) + self.assertEquals(sv[-2], 0) + self.assertEquals(sv[-4], 0) + for ind in [4, -5, 7.8]: + self.assertRaises(ValueError, sv.__getitem__, ind) + class ListTests(PySparkTestCase): From 9d4449c4b3d0f5e60e71fef3a9b2e1c58a80b9e8 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 14 Jan 2015 11:10:29 -0800 Subject: [PATCH 24/46] [SPARK-5228][WebUI] Hide tables for "Active Jobs/Completed Jobs/Failed Jobs" when they are empty In current WebUI, tables for Active Stages, Completed Stages, Skipped Stages and Failed Stages are hidden when they are empty while tables for Active Jobs, Completed Jobs and Failed Jobs are not hidden though they are empty. This is before my patch is applied. ![2015-01-13 14 13 03](https://cloud.githubusercontent.com/assets/4736016/5730793/2b73d6f4-9b32-11e4-9a24-1784d758c644.png) And this is after my patch is applied. ![2015-01-13 14 38 13](https://cloud.githubusercontent.com/assets/4736016/5730797/359ea2da-9b32-11e4-97b0-544739ddbf4c.png) Author: Kousuke Saruta Closes #4028 from sarutak/SPARK-5228 and squashes the following commits: b1e6e8b [Kousuke Saruta] Fixed a small typo daab563 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5228 9493a1d [Kousuke Saruta] Modified AllJobPage.scala so that hide Active Jobs/Completed Jobs/Failed Jobs when they are empty --- .../apache/spark/ui/jobs/AllJobsPage.scala | 62 +++++++++++++------ 1 file changed, 43 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index ea2d187a0e8e4..1d1c701878447 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -47,7 +47,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } - val isComplete = job.status == JobExecutionStatus.SUCCEEDED + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { @@ -107,6 +107,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val failedJobsTable = jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + val shouldShowActiveJobs = activeJobs.nonEmpty + val shouldShowCompletedJobs = completedJobs.nonEmpty + val shouldShowFailedJobs = failedJobs.nonEmpty + val summary: NodeSeq =
        @@ -121,27 +125,47 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { Scheduling Mode: {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} -
      • - Active Jobs: - {activeJobs.size} -
      • -
      • - Completed Jobs: - {completedJobs.size} -
      • -
      • - Failed Jobs: - {failedJobs.size} -
      • + { + if (shouldShowActiveJobs) { +
      • + Active Jobs: + {activeJobs.size} +
      • + } + } + { + if (shouldShowCompletedJobs) { +
      • + Completed Jobs: + {completedJobs.size} +
      • + } + } + { + if (shouldShowFailedJobs) { +
      • + Failed Jobs: + {failedJobs.size} +
      • + } + }
      - val content = summary ++ -

      Active Jobs ({activeJobs.size})

      ++ activeJobsTable ++ -

      Completed Jobs ({completedJobs.size})

      ++ completedJobsTable ++ -

      Failed Jobs ({failedJobs.size})

      ++ failedJobsTable - - val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + var content = summary + if (shouldShowActiveJobs) { + content ++=

      Active Jobs ({activeJobs.size})

      ++ + activeJobsTable + } + if (shouldShowCompletedJobs) { + content ++=

      Completed Jobs ({completedJobs.size})

      ++ + completedJobsTable + } + if (shouldShowFailedJobs) { + content ++=

      Failed Jobs ({failedJobs.size})

      ++ + failedJobsTable + } + val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" + " Click on a job's title to see information about the stages of tasks associated with" + " the job." From 259936be710f367e1d1e019ee7b93fb68dfc33d0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 14 Jan 2015 11:45:40 -0800 Subject: [PATCH 25/46] [SPARK-4014] Add TaskContext.attemptNumber and deprecate TaskContext.attemptId `TaskContext.attemptId` is misleadingly-named, since it currently returns a taskId, which uniquely identifies a particular task attempt within a particular SparkContext, instead of an attempt number, which conveys how many times a task has been attempted. This patch deprecates `TaskContext.attemptId` and add `TaskContext.taskId` and `TaskContext.attemptNumber` fields. Prior to this change, it was impossible to determine whether a task was being re-attempted (or was a speculative copy), which made it difficult to write unit tests for tasks that fail on early attempts or speculative tasks that complete faster than original tasks. Earlier versions of the TaskContext docs suggest that `attemptId` behaves like `attemptNumber`, so there's an argument to be made in favor of changing this method's implementation. Since we've decided against making that change in maintenance branches, I think it's simpler to add better-named methods and retain the old behavior for `attemptId`; if `attemptId` behaved differently in different branches, then this would cause confusing build-breaks when backporting regression tests that rely on the new `attemptId` behavior. Most of this patch is fairly straightforward, but there is a bit of trickiness related to Mesos tasks: since there's no field in MesosTaskInfo to encode the attemptId, I packed it into the `data` field alongside the task binary. Author: Josh Rosen Closes #3849 from JoshRosen/SPARK-4014 and squashes the following commits: 89d03e0 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 5cfff05 [Josh Rosen] Introduce wrapper for serializing Mesos task launch data. 38574d4 [Josh Rosen] attemptId -> taskAttemptId in PairRDDFunctions a180b88 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 1d43aa6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 eee6a45 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 0b10526 [Josh Rosen] Use putInt instead of putLong (silly mistake) 8c387ce [Josh Rosen] Use local with maxRetries instead of local-cluster. cbe4d76 [Josh Rosen] Preserve attemptId behavior and deprecate it: b2dffa3 [Josh Rosen] Address some of Reynold's minor comments 9d8d4d1 [Josh Rosen] Doc typo 1e7a933 [Josh Rosen] [SPARK-4014] Change TaskContext.attemptId to return attempt number instead of task ID. fd515a5 [Josh Rosen] Add failing test for SPARK-4014 --- .../java/org/apache/spark/TaskContext.java | 24 +++++++++- .../org/apache/spark/TaskContextImpl.scala | 9 +++- .../CoarseGrainedExecutorBackend.scala | 3 +- .../org/apache/spark/executor/Executor.scala | 17 +++++-- .../spark/executor/MesosExecutorBackend.scala | 5 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 5 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 9 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../org/apache/spark/scheduler/Task.scala | 12 ++++- .../spark/scheduler/TaskDescription.scala | 1 + .../spark/scheduler/TaskSetManager.scala | 3 +- .../cluster/mesos/MesosSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosTaskLaunchData.scala | 46 +++++++++++++++++++ .../spark/scheduler/local/LocalBackend.scala | 3 +- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../org/apache/spark/CacheManagerSuite.scala | 8 ++-- .../org/apache/spark/rdd/PipedRDDSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 31 ++++++++++++- .../mesos/MesosSchedulerBackendSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 6 +-- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- project/MimaExcludes.scala | 6 +++ .../sql/parquet/ParquetTableOperations.scala | 5 +- .../hive/execution/InsertIntoHiveTable.scala | 5 +- 25 files changed, 168 insertions(+), 48 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 0d6973203eba1..095f9fb94fdf0 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -62,7 +62,7 @@ static void unset() { */ public abstract boolean isInterrupted(); - /** @deprecated: use isRunningLocally() */ + /** @deprecated use {@link #isRunningLocally()} */ @Deprecated public abstract boolean runningLocally(); @@ -87,19 +87,39 @@ static void unset() { * is for HadoopRDD to register a callback to close the input stream. * Will be called in any situation - success, failure, or cancellation. * - * @deprecated: use addTaskCompletionListener + * @deprecated use {@link #addTaskCompletionListener(scala.Function1)} * * @param f Callback function. */ @Deprecated public abstract void addOnCompleteCallback(final Function0 f); + /** + * The ID of the stage that this task belong to. + */ public abstract int stageId(); + /** + * The ID of the RDD partition that is computed by this task. + */ public abstract int partitionId(); + /** + * How many times this task has been attempted. The first task attempt will be assigned + * attemptNumber = 0, and subsequent attempts will have increasing attempt numbers. + */ + public abstract int attemptNumber(); + + /** @deprecated use {@link #taskAttemptId()}; it was renamed to avoid ambiguity. */ + @Deprecated public abstract long attemptId(); + /** + * An ID that is unique to this task attempt (within the same SparkContext, no two task attempts + * will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID. + */ + public abstract long taskAttemptId(); + /** ::DeveloperApi:: */ @DeveloperApi public abstract TaskMetrics taskMetrics(); diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index afd2b85d33a77..9bb0c61e441f8 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -22,14 +22,19 @@ import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerExce import scala.collection.mutable.ArrayBuffer -private[spark] class TaskContextImpl(val stageId: Int, +private[spark] class TaskContextImpl( + val stageId: Int, val partitionId: Int, - val attemptId: Long, + override val taskAttemptId: Long, + override val attemptNumber: Int, val runningLocally: Boolean = false, val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext with Logging { + // For backwards-compatibility; this method is now deprecated as of 1.3.0. + override def attemptId: Long = taskAttemptId + // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c794a7bc3599e..9a4adfbbb3d71 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -71,7 +71,8 @@ private[spark] class CoarseGrainedExecutorBackend( val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) - executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask) + executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber, + taskDesc.name, taskDesc.serializedTask) } case KillTask(taskId, _, interruptThread) => diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0f99cd9f3b08a..b75c77b5b4457 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -108,8 +108,13 @@ private[spark] class Executor( startDriverHeartbeater() def launchTask( - context: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) { - val tr = new TaskRunner(context, taskId, taskName, serializedTask) + context: ExecutorBackend, + taskId: Long, + attemptNumber: Int, + taskName: String, + serializedTask: ByteBuffer) { + val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName, + serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } @@ -134,7 +139,11 @@ private[spark] class Executor( private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum class TaskRunner( - execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) + execBackend: ExecutorBackend, + val taskId: Long, + val attemptNumber: Int, + taskName: String, + serializedTask: ByteBuffer) extends Runnable { @volatile private var killed = false @@ -180,7 +189,7 @@ private[spark] class Executor( // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() - val value = task.run(taskId.toInt) + val value = task.run(taskAttemptId = taskId, attemptNumber = attemptNumber) val taskFinish = System.currentTimeMillis() // If the task has been killed, let's fail it. diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 2e23ae0a4f831..cfd672e1d8a97 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -28,6 +28,7 @@ import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.cluster.mesos.{MesosTaskLaunchData} import org.apache.spark.util.{SignalLogger, Utils} private[spark] class MesosExecutorBackend @@ -77,11 +78,13 @@ private[spark] class MesosExecutorBackend override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { val taskId = taskInfo.getTaskId.getValue.toLong + val taskData = MesosTaskLaunchData.fromByteString(taskInfo.getData) if (executor == null) { logError("Received launchTask but executor was null") } else { SparkHadoopUtil.get.runAsSparkUser { () => - executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer) + executor.launchTask(this, taskId = taskId, attemptNumber = taskData.attemptNumber, + taskInfo.getName, taskData.serializedTask) } } } 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 7ba1182f0ed27..1c13e2c372845 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -95,7 +95,8 @@ private[spark] object CheckpointRDD extends Logging { val finalOutputName = splitIdToFile(ctx.partitionId) val finalOutputPath = new Path(outputDir, finalOutputName) - val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId) + val tempOutputPath = + new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptNumber) if (fs.exists(tempOutputPath)) { throw new IOException("Checkpoint failed: temporary path " + @@ -119,7 +120,7 @@ private[spark] object CheckpointRDD extends Logging { logInfo("Deleting tempOutputPath " + tempOutputPath) fs.delete(tempOutputPath, false) throw new IOException("Checkpoint failed: failed to save output of task: " - + ctx.attemptId + " and final output path does not exist") + + ctx.attemptNumber + " and final output path does not exist") } else { // Some other copy of this task must've finished before us and renamed it logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it") 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 0001c2329c83a..37e0c13029d8b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -229,7 +229,7 @@ class HadoopRDD[K, V]( var reader: RecordReader[K, V] = null val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.stageId, theSplit.index, context.attemptId.toInt, jobConf) + context.stageId, theSplit.index, context.attemptNumber, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. 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 38f8f36a4a4db..e43e5066655b9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -978,12 +978,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { val config = wrappedConf.value - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, - attemptNumber) + context.attemptNumber) val hadoopContext = newTaskAttemptContext(config, attemptId) val format = outfmt.newInstance format match { @@ -1062,11 +1059,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val config = wrappedConf.value // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context, config) - writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() try { var recordsWritten = 0L 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 61d09d73e17cb..8cb15918baa8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -635,8 +635,8 @@ class DAGScheduler( try { val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) - val taskContext = - new TaskContextImpl(job.finalStage.id, job.partitions(0), 0, true) + val taskContext = new TaskContextImpl(job.finalStage.id, job.partitions(0), taskAttemptId = 0, + attemptNumber = 0, runningLocally = true) TaskContextHelper.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index d7dde4fe38436..2367f7e2cf67e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -44,8 +44,16 @@ import org.apache.spark.util.Utils */ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { - final def run(attemptId: Long): T = { - context = new TaskContextImpl(stageId, partitionId, attemptId, runningLocally = false) + /** + * Called by Executor to run this task. + * + * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. + * @param attemptNumber how many times this task has been attempted (0 for the first attempt) + * @return the result of the task + */ + final def run(taskAttemptId: Long, attemptNumber: Int): T = { + context = new TaskContextImpl(stageId = stageId, partitionId = partitionId, + taskAttemptId = taskAttemptId, attemptNumber = attemptNumber, runningLocally = false) TaskContextHelper.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 4c96b9e5fef60..1c7c81c488c3a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -27,6 +27,7 @@ import org.apache.spark.util.SerializableBuffer */ private[spark] class TaskDescription( val taskId: Long, + val attemptNumber: Int, val executorId: String, val name: String, val index: Int, // Index within this task's TaskSet 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 4667850917151..5c94c6bbcb37b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -487,7 +487,8 @@ private[spark] class TaskSetManager( taskName, taskId, host, taskLocality, serializedTask.limit)) sched.dagScheduler.taskStarted(task, info) - return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask)) + return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, + taskName, index, serializedTask)) } case _ => } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 10e6886c16a4f..75d8ddf375e27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -22,7 +22,7 @@ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} @@ -296,7 +296,7 @@ private[spark] class MesosSchedulerBackend( .setExecutor(createExecutorInfo(slaveId)) .setName(task.name) .addResources(cpuResource) - .setData(ByteString.copyFrom(task.serializedTask)) + .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString) .build() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala new file mode 100644 index 0000000000000..4416ce92ade25 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala @@ -0,0 +1,46 @@ +/* + * 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.scheduler.cluster.mesos + +import java.nio.ByteBuffer + +import org.apache.mesos.protobuf.ByteString + +/** + * Wrapper for serializing the data sent when launching Mesos tasks. + */ +private[spark] case class MesosTaskLaunchData( + serializedTask: ByteBuffer, + attemptNumber: Int) { + + def toByteString: ByteString = { + val dataBuffer = ByteBuffer.allocate(4 + serializedTask.limit) + dataBuffer.putInt(attemptNumber) + dataBuffer.put(serializedTask) + ByteString.copyFrom(dataBuffer) + } +} + +private[spark] object MesosTaskLaunchData { + def fromByteString(byteString: ByteString): MesosTaskLaunchData = { + val byteBuffer = byteString.asReadOnlyByteBuffer() + val attemptNumber = byteBuffer.getInt // updates the position by 4 bytes + val serializedTask = byteBuffer.slice() // subsequence starting at the current position + MesosTaskLaunchData(serializedTask, attemptNumber) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index b3bd3110ac809..05b6fa54564b7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -76,7 +76,8 @@ private[spark] class LocalActor( val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= scheduler.CPUS_PER_TASK - executor.launchTask(executorBackend, task.taskId, task.name, task.serializedTask) + executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber, + task.name, task.serializedTask) } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 5ce299d05824b..07b1e44d04be6 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -820,7 +820,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContextImpl(0, 0, 0L, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, 0, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index c0735f448d193..d7d9dc7b50f30 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -66,7 +66,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) @@ -81,7 +81,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, true) + val context = new TaskContextImpl(0, 0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -102,7 +102,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 271a90c6646bb..1a9a0e857e546 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -174,7 +174,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContextImpl(0, 0, 0) + val tContext = new TaskContextImpl(0, 0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 561a5e9cd90c4..057e226916027 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -45,13 +45,13 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val task = new ResultTask[String, String]( 0, sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) intercept[RuntimeException] { - task.run(0) + task.run(0, 0) } assert(TaskContextSuite.completed === true) } test("all TaskCompletionListeners should be called even if some fail") { - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val listener = mock(classOf[TaskCompletionListener]) context.addTaskCompletionListener(_ => throw new Exception("blah")) context.addTaskCompletionListener(listener) @@ -63,6 +63,33 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte verify(listener, times(1)).onTaskCompletion(any()) } + + test("TaskContext.attemptNumber should return attempt number, not task id (SPARK-4014)") { + sc = new SparkContext("local[1,2]", "test") // use maxRetries = 2 because we test failed tasks + // Check that attemptIds are 0 for all tasks' initial attempts + val attemptIds = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter => + Seq(TaskContext.get().attemptNumber).iterator + }.collect() + assert(attemptIds.toSet === Set(0)) + + // Test a job with failed tasks + val attemptIdsWithFailedTask = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter => + val attemptId = TaskContext.get().attemptNumber + if (iter.next() == 1 && attemptId == 0) { + throw new Exception("First execution of task failed") + } + Seq(attemptId).iterator + }.collect() + assert(attemptIdsWithFailedTask.toSet === Set(0, 1)) + } + + test("TaskContext.attemptId returns taskAttemptId for backwards-compatibility (SPARK-4014)") { + sc = new SparkContext("local", "test") + val attemptIds = sc.parallelize(Seq(1, 2, 3, 4), 4).mapPartitions { iter => + Seq(TaskContext.get().attemptId).iterator + }.collect() + assert(attemptIds.toSet === Set(0, 1, 2, 3)) + } } private object TaskContextSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index e60e70afd3218..48f5e40f506d9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -80,7 +80,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea mesosOffers.get(2).getHostname, 2 )) - val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) + val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 1eaabb93adbed..37b593b2c5f79 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -89,7 +89,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + new TaskContextImpl(0, 0, 0, 0), transfer, blockManager, blocksByAddress, @@ -154,7 +154,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, @@ -217,7 +217,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 787f4c2b5a8b2..e85a436cdba17 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -173,7 +173,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { // Simulate fetch failures: val mappedData = data.map { x => val taskContext = TaskContext.get - if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + if (taskContext.attemptNumber == 0) { // Cause this stage to fail on its first attempt. val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index f6f9f491f4ceb..d3ea594245722 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -72,6 +72,12 @@ object MimaExcludes { "org.apache.spark.ml.classification.LogisticRegressionModel.validateAndTransformSchema"), ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.ml.classification.LogisticRegression.validateAndTransformSchema") + ) ++ Seq( + // SPARK-4014 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.taskAttemptId"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.TaskContext.attemptNumber") ) case v if v.startsWith("1.2") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index f5487740d3af9..28cd17fde46ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -301,12 +301,9 @@ case class InsertIntoParquetTable( } def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, - attemptNumber) + context.attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = new AppendingParquetOutputFormat(taskIdOffset) val committer = format.getOutputCommitter(hadoopContext) 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 ca0ec1513917f..42bc8a0b67933 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 @@ -100,10 +100,7 @@ case class InsertIntoHiveTable( val wrappers = fieldOIs.map(wrapperFor) val outputData = new Array[Any](fieldOIs.length) - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt - writerContainer.executorSideSetup(context.stageId, context.partitionId, attemptNumber) + writerContainer.executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) iterator.foreach { row => var i = 0 From 2fd7f72b6b0b24bec12331c7bbbcf6bfc265d2ec Mon Sep 17 00:00:00 2001 From: Alex Baretta Date: Wed, 14 Jan 2015 11:51:55 -0800 Subject: [PATCH 26/46] [SPARK-5235] Make SQLConf Serializable Declare SQLConf to be serializable to fix "Task not serializable" exceptions in SparkSQL Author: Alex Baretta Closes #4031 from alexbaretta/SPARK-5235-SQLConf and squashes the following commits: c2103f5 [Alex Baretta] [SPARK-5235] Make SQLConf Serializable --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 206d16f5b34f5..3bc201a2425b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -61,7 +61,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -private[sql] class SQLConf { +private[sql] class SQLConf extends Serializable { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ From 76389c5b99183e456ff85fd92ea68d95c4c13e82 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 14 Jan 2015 11:53:43 -0800 Subject: [PATCH 27/46] [SPARK-5234][ml]examples for ml don't have sparkContext.stop JIRA issue: https://issues.apache.org/jira/browse/SPARK-5234 simply add the call. Author: Yuhao Yang Closes #4044 from hhbyyh/addscStop and squashes the following commits: c1f75ac [Yuhao Yang] add SparkContext.stop to 3 ml examples --- .../org/apache/spark/examples/ml/CrossValidatorExample.scala | 2 ++ .../org/apache/spark/examples/ml/SimpleParamsExample.scala | 2 ++ .../spark/examples/ml/SimpleTextClassificationPipeline.scala | 2 ++ 3 files changed, 6 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index ce6bc066bd70d..d8c7ef38ee46d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -106,5 +106,7 @@ object CrossValidatorExample { .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) } + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 44d5b084c269a..e8a2adff929cb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -97,5 +97,7 @@ object SimpleParamsExample { .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) } + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 92895a05e479a..b9a6ef0229def 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -85,5 +85,7 @@ object SimpleTextClassificationPipeline { .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) } + + sc.stop() } } From 13d2406781714daea2bbf3bfb7fec0dead10760c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 Jan 2015 17:50:33 -0800 Subject: [PATCH 28/46] [SPARK-5254][MLLIB] Update the user guide to position spark.ml better The current statement in the user guide may deliver confusing messages to users. spark.ml contains high-level APIs for building ML pipelines. But it doesn't mean that spark.mllib is being deprecated. First of all, the pipeline API is in its alpha stage and we need to see more use cases from the community to stabilizes it, which may take several releases. Secondly, the components in spark.ml are simple wrappers over spark.mllib implementations. Neither the APIs or the implementations from spark.mllib are being deprecated. We expect users use spark.ml pipeline APIs to build their ML pipelines, but we will keep supporting and adding features to spark.mllib. For example, there are many features in review at https://spark-prs.appspot.com/#mllib. So users should be comfortable with using spark.mllib features and expect more coming. The user guide needs to be updated to make the message clear. Author: Xiangrui Meng Closes #4052 from mengxr/SPARK-5254 and squashes the following commits: 6d5f1d3 [Xiangrui Meng] typo 0cc935b [Xiangrui Meng] update user guide to position spark.ml better --- docs/ml-guide.md | 17 ++++++++++------- docs/mllib-guide.md | 18 +++++++++++------- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 1c2e27341473b..88158fd77edae 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -3,13 +3,16 @@ layout: global title: Spark ML Programming Guide --- -Spark ML is Spark's new machine learning package. It is currently an alpha component but is potentially a successor to [MLlib](mllib-guide.html). The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. - -MLlib vs. Spark ML: - -* Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. Since Spark ML is an alpha component, its API may change in future releases. -* Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. See below for more details. -* Spark ML only has Scala and Java APIs, whereas MLlib also has a Python API. +`spark.ml` is a new package introduced in Spark 1.2, which aims to provide a uniform set of +high-level APIs that help users create and tune practical machine learning pipelines. +It is currently an alpha component, and we would like to hear back from the community about +how it fits real-world use cases and how it could be improved. + +Note that we will keep supporting and adding features to `spark.mllib` along with the +development of `spark.ml`. +Users should be comfortable using `spark.mllib` features and expect more features coming. +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute +to `spark.ml`. **Table of Contents** diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index efd7dda310712..39c64d06926bf 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,16 +35,20 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. -# spark.ml: The New ML Package +# spark.ml: high-level APIs for ML pipelines -Spark 1.2 includes a new machine learning package called `spark.ml`, currently an alpha component but potentially a successor to `spark.mllib`. The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. +Spark 1.2 includes a new package called `spark.ml`, which aims to provide a uniform set of +high-level APIs that help users create and tune practical machine learning pipelines. +It is currently an alpha component, and we would like to hear back from the community about +how it fits real-world use cases and how it could be improved. -See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. - -Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. +Note that we will keep supporting and adding features to `spark.mllib` along with the +development of `spark.ml`. +Users should be comfortable using `spark.mllib` features and expect more features coming. +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute +to `spark.ml`. -Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. -See the `spark.ml` programming guide linked above for more details. +See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. # Dependencies From cfa397c126c857bfc9843d9e598a14b7c1e0457f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Jan 2015 18:36:15 -0800 Subject: [PATCH 29/46] [SPARK-5193][SQL] Tighten up SQLContext API 1. Removed 2 implicits (logicalPlanToSparkQuery and baseRelationToSchemaRDD) 2. Moved extraStrategies into ExperimentalMethods. 3. Made private methods protected[sql] so they don't show up in javadocs. 4. Removed createParquetFile. 5. Added Java version of applySchema to SQLContext. Author: Reynold Xin Closes #4049 from rxin/sqlContext-refactor and squashes the following commits: a326a1a [Reynold Xin] Remove createParquetFile and add applySchema for Java to SQLContext. ecd6685 [Reynold Xin] Added baseRelationToSchemaRDD back. 4a38c9b [Reynold Xin] [SPARK-5193][SQL] Tighten up SQLContext API --- .../spark/sql/ExperimentalMethods.scala | 36 ++++ .../org/apache/spark/sql/SQLContext.scala | 152 ++++++++++------- .../apache/spark/sql/execution/commands.scala | 10 +- .../org/apache/spark/sql/sources/ddl.scala | 5 +- .../spark/sql/test/TestSQLContext.scala | 16 +- .../apache/spark/sql/InsertIntoSuite.scala | 160 ------------------ .../spark/sql/parquet/ParquetQuerySuite.scala | 26 --- .../sql/parquet/ParquetQuerySuite2.scala | 22 --- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../org/apache/spark/sql/hive/TestHive.scala | 2 +- 10 files changed, 152 insertions(+), 281 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala new file mode 100644 index 0000000000000..f0e6a8f332188 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -0,0 +1,36 @@ +/* + * 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 + +import org.apache.spark.annotation.Experimental + +/** + * Holder for experimental methods for the bravest. We make NO guarantee about the stability + * regarding binary compatibility and source compatibility of methods here. + */ +@Experimental +class ExperimentalMethods protected[sql](sqlContext: SQLContext) { + + /** + * Allows extra strategies to be injected into the query planner at runtime. Note this API + * should be consider experimental and is not intended to be stable across releases. + */ + @Experimental + var extraStrategies: Seq[Strategy] = Nil + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d9f3b3a53f582..279671ced0a17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -17,15 +17,16 @@ package org.apache.spark.sql +import java.beans.Introspector import java.util.Properties import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkContext import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ @@ -36,9 +37,9 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ -import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation} +import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation, DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * :: AlphaComponent :: @@ -59,7 +60,7 @@ class SQLContext(@transient val sparkContext: SparkContext) self => // Note that this is a lazy val so we can override the default value in subclasses. - private[sql] lazy val conf: SQLConf = new SQLConf + protected[sql] lazy val conf: SQLConf = new SQLConf /** Set Spark SQL configuration properties. */ def setConf(props: Properties): Unit = conf.setConf(props) @@ -117,15 +118,6 @@ class SQLContext(@transient val sparkContext: SparkContext) case _ => } - /** - * :: DeveloperApi :: - * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan - * interface is considered internal, and thus not guaranteed to be stable. As a result, using - * them directly is not recommended. - */ - @DeveloperApi - implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) - /** * Creates a SchemaRDD from an RDD of case classes. * @@ -139,8 +131,11 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, LogicalRDD(attributeSeq, rowRDD)(self)) } - implicit def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = { - logicalPlanToSparkQuery(LogicalRelation(baseRelation)) + /** + * Convert a [[BaseRelation]] created for external data sources into a [[SchemaRDD]]. + */ + def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = { + new SchemaRDD(this, LogicalRelation(baseRelation)) } /** @@ -181,6 +176,43 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, logicalPlan) } + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, + * SELECT * queries will return the columns in an undefined order. + */ + def applySchema(rdd: RDD[_], beanClass: Class[_]): SchemaRDD = { + val attributeSeq = getSchema(beanClass) + val className = beanClass.getName + val rowRdd = rdd.mapPartitions { iter => + // BeanInfo is not serializable so we must rediscover it remotely for each partition. + val localBeanInfo = Introspector.getBeanInfo( + Class.forName(className, true, Utils.getContextOrSparkClassLoader)) + val extractors = + localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) + + iter.map { row => + new GenericRow( + extractors.zip(attributeSeq).map { case (e, attr) => + DataTypeConversions.convertJavaToCatalyst(e.invoke(row), attr.dataType) + }.toArray[Any] + ) : Row + } + } + new SchemaRDD(this, LogicalRDD(attributeSeq, rowRdd)(this)) + } + + /** + * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, + * SELECT * queries will return the columns in an undefined order. + */ + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): SchemaRDD = { + applySchema(rdd.rdd, beanClass) + } + /** * Loads a Parquet file, returning the result as a [[SchemaRDD]]. * @@ -259,41 +291,6 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } - /** - * :: Experimental :: - * Creates an empty parquet file with the schema of class `A`, which can be registered as a table. - * This registered table can be used as the target of future `insertInto` operations. - * - * {{{ - * val sqlContext = new SQLContext(...) - * import sqlContext._ - * - * case class Person(name: String, age: Int) - * createParquetFile[Person]("path/to/file.parquet").registerTempTable("people") - * sql("INSERT INTO people SELECT 'michael', 29") - * }}} - * - * @tparam A A case class type that describes the desired schema of the parquet file to be - * created. - * @param path The path where the directory containing parquet metadata should be created. - * Data inserted into this table will also be stored at this location. - * @param allowExisting When false, an exception will be thrown if this directory already exists. - * @param conf A Hadoop configuration object that can be used to specify options to the parquet - * output format. - * - * @group userf - */ - @Experimental - def createParquetFile[A <: Product : TypeTag]( - path: String, - allowExisting: Boolean = true, - conf: Configuration = new Configuration()): SchemaRDD = { - new SchemaRDD( - this, - ParquetRelation.createEmpty( - path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) - } - /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. @@ -336,12 +333,10 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, catalog.lookupRelation(Seq(tableName))) /** - * :: DeveloperApi :: - * Allows extra strategies to be injected into the query planner at runtime. Note this API - * should be consider experimental and is not intended to be stable across releases. + * A collection of methods that are considered experimental, but can be used to hook into + * the query planner for advanced functionalities. */ - @DeveloperApi - var extraStrategies: Seq[Strategy] = Nil + val experimental: ExperimentalMethods = new ExperimentalMethods(this) protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -353,7 +348,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def numPartitions = self.conf.numShufflePartitions def strategies: Seq[Strategy] = - extraStrategies ++ ( + experimental.extraStrategies ++ ( DataSourceStrategy :: DDLStrategy :: TakeOrdered :: @@ -479,14 +474,14 @@ class SQLContext(@transient val sparkContext: SparkContext) * have the same format as the one generated by `toString` in scala. * It is only used by PySpark. */ - private[sql] def parseDataType(dataTypeString: String): DataType = { + protected[sql] def parseDataType(dataTypeString: String): DataType = { DataType.fromJson(dataTypeString) } /** * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. */ - private[sql] def applySchemaToPythonRDD( + protected[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schemaString: String): SchemaRDD = { val schema = parseDataType(schemaString).asInstanceOf[StructType] @@ -496,7 +491,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Apply a schema defined by the schema to an RDD. It is only used by PySpark. */ - private[sql] def applySchemaToPythonRDD( + protected[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { @@ -527,4 +522,43 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } + + /** + * Returns a Catalyst Schema for the given java bean class. + */ + protected def getSchema(beanClass: Class[_]): Seq[AttributeReference] = { + // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. + val beanInfo = Introspector.getBeanInfo(beanClass) + + // Note: The ordering of elements may differ from when the schema is inferred in Scala. + // This is because beanInfo.getPropertyDescriptors gives no guarantees about + // element ordering. + val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + fields.map { property => + val (dataType, nullable) = property.getPropertyType match { + case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) + } + AttributeReference(property.getName, dataType, nullable)() + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index af6b07bd6c2f4..52a31f01a4358 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{SQLConf, SQLContext} /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -137,14 +137,12 @@ case class CacheTableCommand( isLazy: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext) = { - import sqlContext._ - - plan.foreach(_.registerTempTable(tableName)) - cacheTable(tableName) + plan.foreach(p => new SchemaRDD(sqlContext, p).registerTempTable(tableName)) + sqlContext.cacheTable(tableName) if (!isLazy) { // Performs eager caching - table(tableName).count() + sqlContext.table(tableName).count() } Seq.empty[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 4cc9641c4d9e0..381298caba6f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -22,7 +22,7 @@ import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.combinator.PackratParsers import org.apache.spark.Logging -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SchemaRDD, SQLContext} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.SqlLexical import org.apache.spark.sql.execution.RunnableCommand @@ -234,8 +234,7 @@ private [sql] case class CreateTempTableUsing( def run(sqlContext: SQLContext) = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) - - sqlContext.baseRelationToSchemaRDD(resolved.relation).registerTempTable(tableName) + new SchemaRDD(sqlContext, LogicalRelation(resolved.relation)).registerTempTable(tableName) Seq.empty } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 8c80be106f3cb..f9c082216085d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.test +import scala.language.implicitConversions + import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{SQLConf, SQLContext} +import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** A SQLContext that can be used for local testing. */ object TestSQLContext @@ -29,7 +32,16 @@ object TestSQLContext new SparkConf().set("spark.sql.testkey", "true"))) { /** Fewer partitions to speed up testing. */ - private[sql] override lazy val conf: SQLConf = new SQLConf { + protected[sql] override lazy val conf: SQLConf = new SQLConf { override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt } + + /** + * Turn a logical plan into a SchemaRDD. This should be removed once we have an easier way to + * construct SchemaRDD directly out of local data without relying on implicits. + */ + protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = { + new SchemaRDD(this, plan) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala deleted file mode 100644 index c87d762751e6d..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ /dev/null @@ -1,160 +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 - -import _root_.java.io.File - -/* Implicits */ -import org.apache.spark.sql.test.TestSQLContext._ - -class InsertIntoSuite extends QueryTest { - TestData // Initialize TestData - import TestData._ - - test("insertInto() created parquet file") { - val testFilePath = File.createTempFile("sparkSql", "pqt") - testFilePath.delete() - testFilePath.deleteOnExit() - val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerTempTable("createAndInsertTest") - - // Add some data. - testData.insertInto("createAndInsertTest") - - // Make sure its there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq - ) - - // Add more data. - testData.insertInto("createAndInsertTest") - - // Make sure all data is there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq ++ testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq ++ testData.collect().toSeq - ) - - // Now overwrite. - testData.insertInto("createAndInsertTest", overwrite = true) - - // Make sure its there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq - ) - - testFilePath.delete() - } - - test("INSERT INTO parquet table") { - val testFilePath = File.createTempFile("sparkSql", "pqt") - testFilePath.delete() - testFilePath.deleteOnExit() - val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerTempTable("createAndInsertSQLTest") - - sql("INSERT INTO createAndInsertSQLTest SELECT * FROM testData") - - // Make sure its there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertSQLTest"), - testData.collect().toSeq - ) - - // Append more data. - sql("INSERT INTO createAndInsertSQLTest SELECT * FROM testData") - - // Make sure all data is there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq ++ testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertSQLTest"), - testData.collect().toSeq ++ testData.collect().toSeq - ) - - sql("INSERT OVERWRITE INTO createAndInsertSQLTest SELECT * FROM testData") - - // Make sure its there for a new instance of parquet file. - checkAnswer( - parquetFile(testFilePath.getCanonicalPath), - testData.collect().toSeq - ) - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertSQLTest"), - testData.collect().toSeq - ) - - testFilePath.delete() - } - - test("Double create fails when allowExisting = false") { - val testFilePath = File.createTempFile("sparkSql", "pqt") - testFilePath.delete() - testFilePath.deleteOnExit() - val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - - intercept[RuntimeException] { - createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = false) - } - - testFilePath.delete() - } - - test("Double create does not fail when allowExisting = true") { - val testFilePath = File.createTempFile("sparkSql", "pqt") - testFilePath.delete() - testFilePath.deleteOnExit() - val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - - createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = true) - - testFilePath.delete() - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index fe781ec05fb6f..3a073a6b7057e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -402,23 +402,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(file) } - test("Insert (overwrite) via Scala API") { - val dirname = Utils.createTempDir() - val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) - .map(i => TestRDDEntry(i, s"val_$i")) - source_rdd.registerTempTable("source") - val dest_rdd = createParquetFile[TestRDDEntry](dirname.toString) - dest_rdd.registerTempTable("dest") - sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() - val rdd_copy1 = sql("SELECT * FROM dest").collect() - assert(rdd_copy1.size === 100) - - sql("INSERT INTO dest SELECT * FROM source") - val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) - assert(rdd_copy2.size === 200) - Utils.deleteRecursively(dirname) - } - test("Insert (appending) to same table via Scala API") { sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() @@ -902,15 +885,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) } - test("Querying on empty parquet throws exception (SPARK-3536)") { - val tmpdir = Utils.createTempDir() - Utils.deleteRecursively(tmpdir) - createParquetFile[TestRDDEntry](tmpdir.toString()).registerTempTable("tmpemptytable") - val result1 = sql("SELECT * FROM tmpemptytable").collect() - assert(result1.size === 0) - Utils.deleteRecursively(tmpdir) - } - test("read/write fixed-length decimals") { for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { val tempDir = getTempFilePath("parquetTest").getCanonicalPath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala index daa7ca65cd993..4c081fb4510b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite2.scala @@ -34,19 +34,6 @@ class ParquetQuerySuite2 extends QueryTest with ParquetTest { } } - test("insertion") { - withTempDir { dir => - val data = (0 until 10).map(i => (i, i.toString)) - withParquetTable(data, "t") { - createParquetFile[(Int, String)](dir.toString).registerTempTable("dest") - withTempTable("dest") { - sql("INSERT OVERWRITE INTO dest SELECT * FROM t") - checkAnswer(table("dest"), data) - } - } - } - } - test("appending") { val data = (0 until 10).map(i => (i, i.toString)) withParquetTable(data, "t") { @@ -98,13 +85,4 @@ class ParquetQuerySuite2 extends QueryTest with ParquetTest { checkAnswer(sql(s"SELECT _1 FROM t WHERE _1 < 10"), (1 to 9).map(Row.apply(_))) } } - - test("SPARK-3536 regression: query empty Parquet file shouldn't throw") { - withTempDir { dir => - createParquetFile[(Int, String)](dir.toString).registerTempTable("t") - withTempTable("t") { - checkAnswer(sql("SELECT * FROM t"), Seq.empty[Row]) - } - } - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index bf56e60cf995f..a9a20a54bebe8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -71,7 +71,7 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - private[sql] override lazy val conf: SQLConf = new SQLConf { + protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } @@ -348,7 +348,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val hivePlanner = new SparkPlanner with HiveStrategies { val hiveContext = self - override def strategies: Seq[Strategy] = extraStrategies ++ Seq( + override def strategies: Seq[Strategy] = experimental.extraStrategies ++ Seq( DataSourceStrategy, HiveCommandStrategy(self), HiveDDLStrategy, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 52e1f0d94fbd4..47431cef03e13 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -102,7 +102,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { new this.QueryExecution { val logical = plan } /** Fewer partitions to speed up testing. */ - private[sql] override lazy val conf: SQLConf = new SQLConf { + protected[sql] override lazy val conf: SQLConf = new SQLConf { override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } From 6abc45e340d3be5f07236adc104db5f8dda0d514 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 Jan 2015 18:54:17 -0800 Subject: [PATCH 30/46] [SPARK-5254][MLLIB] remove developers section from spark.ml guide Forgot to remove this section in #4052. Author: Xiangrui Meng Closes #4053 from mengxr/SPARK-5254-update and squashes the following commits: f295bde [Xiangrui Meng] remove developers section from spark.ml guide --- docs/ml-guide.md | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 88158fd77edae..be178d7689fdd 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -689,17 +689,3 @@ Spark ML currently depends on MLlib and has the same dependencies. Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info. Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. - -# Developers - -**Development plan** - -If all goes well, `spark.ml` will become the primary ML package at the time of the Spark 1.3 release. Initially, simple wrappers will be used to port algorithms to `spark.ml`, but eventually, code will be moved to `spark.ml` and `spark.mllib` will be deprecated. - -**Advice to developers** - -During the next development cycle, new algorithms should be contributed to `spark.mllib`, but we welcome patches sent to either package. If an algorithm is best expressed using the new API (e.g., feature transformers), we may ask for developers to use the new `spark.ml` API. -Wrappers for old and new algorithms can be contributed to `spark.ml`. - -Users will be able to use algorithms from either of the two packages. The main difficulty will be the differences in APIs between the two packages. - From 4b325c77a270ec32d6858d204313d4f161774fae Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Jan 2015 20:31:02 -0800 Subject: [PATCH 31/46] [SPARK-5193][SQL] Tighten up HiveContext API 1. Removed the deprecated LocalHiveContext 2. Made private[sql] fields protected[sql] so they don't show up in javadoc. 3. Added javadoc to refreshTable. 4. Added Experimental tag to analyze command. Author: Reynold Xin Closes #4054 from rxin/hivecontext-api and squashes the following commits: 25cc00a [Reynold Xin] Add implicit conversion back. cbca886 [Reynold Xin] [SPARK-5193][SQL] Tighten up HiveContext API --- .../apache/spark/sql/hive/HiveContext.scala | 48 +++++-------------- 1 file changed, 13 insertions(+), 35 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index a9a20a54bebe8..4246b8b0918fd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import java.io.{BufferedReader, File, InputStreamReader, PrintStream} +import java.io.{BufferedReader, InputStreamReader, PrintStream} import java.sql.{Date, Timestamp} import scala.collection.JavaConversions._ @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry} @@ -42,28 +43,6 @@ import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DescribeHiveTable import org.apache.spark.sql.sources.DataSourceStrategy import org.apache.spark.sql.types._ -/** - * DEPRECATED: Use HiveContext instead. - */ -@deprecated(""" - Use HiveContext instead. It will still create a local metastore if one is not specified. - However, note that the default directory is ./metastore_db, not ./metastore - """, "1.1") -class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { - - lazy val metastorePath = new File("metastore").getCanonicalPath - lazy val warehousePath: String = new File("warehouse").getCanonicalPath - - /** Sets up the system initially or after a RESET command */ - protected def configure() { - setConf("javax.jdo.option.ConnectionURL", - s"jdbc:derby:;databaseName=$metastorePath;create=true") - setConf("hive.metastore.warehouse.dir", warehousePath) - } - - configure() // Must be called before initializing the catalog below. -} - /** * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. @@ -80,7 +59,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive * SerDe. */ - private[spark] def convertMetastoreParquet: Boolean = + protected[sql] def convertMetastoreParquet: Boolean = getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true" override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = @@ -97,14 +76,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - @deprecated("hiveql() is deprecated as the sql function now parses using HiveQL by default. " + - s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") - def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) - - @deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " + - s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") - def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) - /** * Creates a table using the schema of the given class. * @@ -116,6 +87,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) } + /** + * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, + * Spark SQL or the external data source library it uses might cache certain metadata about a + * table, such as the location of blocks. When those change outside of Spark SQL, users should + * call this function to invalidate the cache. + */ def refreshTable(tableName: String): Unit = { // TODO: Database support... catalog.refreshTable("default", tableName) @@ -133,6 +110,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ + @Experimental def analyze(tableName: String) { val relation = EliminateAnalysisOperators(catalog.lookupRelation(Seq(tableName))) @@ -289,7 +267,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { results } - /** * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. @@ -345,7 +322,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } @transient - val hivePlanner = new SparkPlanner with HiveStrategies { + private val hivePlanner = new SparkPlanner with HiveStrategies { val hiveContext = self override def strategies: Seq[Strategy] = experimental.extraStrategies ++ Seq( @@ -410,7 +387,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } -object HiveContext { + +private object HiveContext { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) From 3c8650c12ad7a97852e7bd76153210493fd83e92 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 15 Jan 2015 11:40:41 -0800 Subject: [PATCH 32/46] [SPARK-5224] [PySpark] improve performance of parallelize list/ndarray After the default batchSize changed to 0 (batched based on the size of object), but parallelize() still use BatchedSerializer with batchSize=1, this PR will use batchSize=1024 for parallelize by default. Also, BatchedSerializer did not work well with list and numpy.ndarray, this improve BatchedSerializer by using __len__ and __getslice__. Here is the benchmark for parallelize 1 millions int with list or ndarray: | before | after | improvements ------- | ------------ | ------------- | ------- list | 11.7 s | 0.8 s | 14x numpy.ndarray | 32 s | 0.7 s | 40x Author: Davies Liu Closes #4024 from davies/opt_numpy and squashes the following commits: 7618c7c [Davies Liu] improve performance of parallelize list/ndarray --- python/pyspark/context.py | 2 +- python/pyspark/serializers.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 593d74bca5fff..64f6a3ca6bf4c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -319,7 +319,7 @@ def f(split, iterator): # Make sure we distribute data evenly if it's smaller than self.batchSize if "__len__" not in dir(c): c = list(c) # Make it a list so we can compute its length - batchSize = max(1, min(len(c) // numSlices, self._batchSize)) + batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024)) serializer = BatchedSerializer(self._unbatched_serializer, batchSize) serializer.dump_stream(c, tempFile) tempFile.close() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index bd08c9a6d20d6..b8bda835174b2 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -181,6 +181,10 @@ def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE): def _batched(self, iterator): if self.batchSize == self.UNLIMITED_BATCH_SIZE: yield list(iterator) + elif hasattr(iterator, "__len__") and hasattr(iterator, "__getslice__"): + n = len(iterator) + for i in xrange(0, n, self.batchSize): + yield iterator[i: i + self.batchSize] else: items = [] count = 0 From 1881431dd50e93a6948e4966d33742727f27e917 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 15 Jan 2015 16:15:12 -0800 Subject: [PATCH 33/46] [SPARK-5274][SQL] Reconcile Java and Scala UDFRegistration. As part of SPARK-5193: 1. Removed UDFRegistration as a mixin in SQLContext and made it a field ("udf"). 2. For Java UDFs, renamed dataType to returnType. 3. For Scala UDFs, added type tags. 4. Added all Java UDF registration methods to Scala's UDFRegistration. 5. Documentation Author: Reynold Xin Closes #4056 from rxin/udf-registration and squashes the following commits: ae9c556 [Reynold Xin] Updated example. 675a3c9 [Reynold Xin] Style fix 47c24ff [Reynold Xin] Python fix. 5f00c45 [Reynold Xin] Restore data type position in java udf and added typetags. 032f006 [Reynold Xin] [SPARK-5193][SQL] Reconcile Java and Scala UDFRegistration. --- python/pyspark/sql.py | 16 +- .../org/apache/spark/sql/SQLContext.scala | 29 +- .../apache/spark/sql/UdfRegistration.scala | 692 ++++++++++++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 9 +- .../spark/sql/UserDefinedTypeSuite.scala | 2 +- .../sql/hive/execution/HiveUdfSuite.scala | 2 +- 7 files changed, 674 insertions(+), 78 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 014ac1791c849..dcd3b60a6062b 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1281,14 +1281,14 @@ def registerFunction(self, name, f, returnType=StringType()): self._sc._gateway._gateway_client) includes = ListConverter().convert(self._sc._python_includes, self._sc._gateway._gateway_client) - self._ssql_ctx.registerPython(name, - bytearray(pickled_command), - env, - includes, - self._sc.pythonExec, - broadcast_vars, - self._sc._javaAccumulator, - returnType.json()) + self._ssql_ctx.udf().registerPython(name, + bytearray(pickled_command), + env, + includes, + self._sc.pythonExec, + broadcast_vars, + self._sc._javaAccumulator, + returnType.json()) def inferSchema(self, rdd, samplingRatio=None): """Infer and apply a schema to an RDD of L{Row}. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 279671ced0a17..8ad1753dab757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -54,7 +54,6 @@ class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging with CacheManager with ExpressionConversions - with UDFRegistration with Serializable { self => @@ -338,6 +337,34 @@ class SQLContext(@transient val sparkContext: SparkContext) */ val experimental: ExperimentalMethods = new ExperimentalMethods(this) + /** + * A collection of methods for registering user-defined functions (UDF). + * + * The following example registers a Scala closure as UDF: + * {{{ + * sqlContext.udf.register("myUdf", (arg1: Int, arg2: String) => arg2 + arg1) + * }}} + * + * The following example registers a UDF in Java: + * {{{ + * sqlContext.udf().register("myUDF", + * new UDF2() { + * @Override + * public String call(Integer arg1, String arg2) { + * return arg2 + arg1; + * } + * }, DataTypes.StringType); + * }}} + * + * Or, to use Java 8 lambda syntax: + * {{{ + * sqlContext.udf().register("myUDF", + * (Integer arg1, String arg2) -> arg2 + arg1), + * DataTypes.StringType); + * }}} + */ + val udf: UDFRegistration = new UDFRegistration(this) + protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 5fb472686c9e1..2e9d037f93c03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -19,22 +19,26 @@ package org.apache.spark.sql import java.util.{List => JList, Map => JMap} +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.Accumulator import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} import org.apache.spark.sql.execution.PythonUDF +import org.apache.spark.sql.types.DataType -import scala.reflect.runtime.universe.{TypeTag, typeTag} /** - * Functions for registering scala lambda functions as UDFs in a SQLContext. + * Functions for registering user-defined functions. */ -private[sql] trait UDFRegistration { - self: SQLContext => +class UDFRegistration (sqlContext: SQLContext) extends org.apache.spark.Logging { + + private val functionRegistry = sqlContext.functionRegistry - private[spark] def registerPython( + protected[sql] def registerPython( name: String, command: Array[Byte], envVars: JMap[String, String], @@ -55,7 +59,7 @@ private[sql] trait UDFRegistration { """.stripMargin) - val dataType = parseDataType(stringDataType) + val dataType = sqlContext.parseDataType(stringDataType) def builder(e: Seq[Expression]) = PythonUDF( @@ -72,133 +76,699 @@ private[sql] trait UDFRegistration { functionRegistry.registerFunction(name, builder) } - /** registerFunction 0-22 were generated by this script + // scalastyle:off + + /* registerFunction 0-22 were generated by this script (0 to 22).map { x => - val types = (1 to x).foldRight("T")((_, s) => {s"_, $s"}) - s""" - def registerFunction[T: TypeTag](name: String, func: Function$x[$types]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) + val typeTags = (1 to x).map(i => s"A${i}: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) + val argDocs = (1 to x).map(i => s" * @tparam A$i type of the UDF argument at position $i.").foldLeft("")(_ + "\n" + _) + println(s""" + /** + * Register a Scala closure of ${x} arguments as user-defined function (UDF). + * @tparam RT return type of UDF.$argDocs + */ + def register[$typeTags](name: String, func: Function$x[$types]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) - } - """ + }""") } - */ - // scalastyle:off - def registerFunction[T: TypeTag](name: String, func: Function0[T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + (1 to 22).foreach { i => + val extTypeArgs = (1 to i).map(_ => "_").mkString(", ") + val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ") + val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" + val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") + println(s""" + |/** + | * Register a user-defined function with ${i} arguments. + | */ + |def register(name: String, f: UDF$i[$extTypeArgs, _], returnType: DataType) = { + | functionRegistry.registerFunction( + | name, + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), returnType, e)) + |}""".stripMargin) + } + */ + + /** + * Register a Scala closure of 0 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + */ + def register[RT: TypeTag](name: String, func: Function0[RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 1 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + */ + def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 2 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 3 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 4 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 5 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 6 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 7 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 8 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 9 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 10 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 11 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 12 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 13 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 14 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 15 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 16 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 17 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 18 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + * @tparam A18 type of the UDF argument at position 18. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 19 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + * @tparam A18 type of the UDF argument at position 18. + * @tparam A19 type of the UDF argument at position 19. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 20 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + * @tparam A18 type of the UDF argument at position 18. + * @tparam A19 type of the UDF argument at position 19. + * @tparam A20 type of the UDF argument at position 20. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 21 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + * @tparam A18 type of the UDF argument at position 18. + * @tparam A19 type of the UDF argument at position 19. + * @tparam A20 type of the UDF argument at position 20. + * @tparam A21 type of the UDF argument at position 21. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } - def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) + /** + * Register a Scala closure of 22 arguments as user-defined function (UDF). + * @tparam RT return type of UDF. + * @tparam A1 type of the UDF argument at position 1. + * @tparam A2 type of the UDF argument at position 2. + * @tparam A3 type of the UDF argument at position 3. + * @tparam A4 type of the UDF argument at position 4. + * @tparam A5 type of the UDF argument at position 5. + * @tparam A6 type of the UDF argument at position 6. + * @tparam A7 type of the UDF argument at position 7. + * @tparam A8 type of the UDF argument at position 8. + * @tparam A9 type of the UDF argument at position 9. + * @tparam A10 type of the UDF argument at position 10. + * @tparam A11 type of the UDF argument at position 11. + * @tparam A12 type of the UDF argument at position 12. + * @tparam A13 type of the UDF argument at position 13. + * @tparam A14 type of the UDF argument at position 14. + * @tparam A15 type of the UDF argument at position 15. + * @tparam A16 type of the UDF argument at position 16. + * @tparam A17 type of the UDF argument at position 17. + * @tparam A18 type of the UDF argument at position 18. + * @tparam A19 type of the UDF argument at position 19. + * @tparam A20 type of the UDF argument at position 20. + * @tparam A21 type of the UDF argument at position 21. + * @tparam A22 type of the UDF argument at position 22. + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) functionRegistry.registerFunction(name, builder) } + + /** + * Register a user-defined function with 1 arguments. + */ + def register(name: String, f: UDF1[_, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), returnType, e)) + } + + /** + * Register a user-defined function with 2 arguments. + */ + def register(name: String, f: UDF2[_, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 3 arguments. + */ + def register(name: String, f: UDF3[_, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 4 arguments. + */ + def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 5 arguments. + */ + def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 6 arguments. + */ + def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 7 arguments. + */ + def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 8 arguments. + */ + def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 9 arguments. + */ + def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 10 arguments. + */ + def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 11 arguments. + */ + def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 12 arguments. + */ + def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 13 arguments. + */ + def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 14 arguments. + */ + def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 15 arguments. + */ + def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 16 arguments. + */ + def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 17 arguments. + */ + def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 18 arguments. + */ + def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 19 arguments. + */ + def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 20 arguments. + */ + def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 21 arguments. + */ + def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + + /** + * Register a user-defined function with 22 arguments. + */ + def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType) = { + functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), returnType, e)) + } + // scalastyle:on } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index cbdb3e64bb66b..6c95bad6974d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -766,7 +766,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3371 Renaming a function expression with group by gives error") { - registerFunction("len", (s: String) => s.length) + udf.register("len", (s: String) => s.length) checkAnswer( sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 720953ae3765a..0c98120031242 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -27,23 +27,22 @@ case class FunctionResult(f1: String, f2: String) class UDFSuite extends QueryTest { test("Simple UDF") { - registerFunction("strLenScala", (_: String).length) + udf.register("strLenScala", (_: String).length) assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) } test("ZeroArgument UDF") { - registerFunction("random0", () => { Math.random()}) + udf.register("random0", () => { Math.random()}) assert(sql("SELECT random0()").first().getDouble(0) >= 0.0) } test("TwoArgument UDF") { - registerFunction("strLenScala", (_: String).length + (_:Int)) + udf.register("strLenScala", (_: String).length + (_:Int)) assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) } - test("struct UDF") { - registerFunction("returnStruct", (f1: String, f2: String) => FunctionResult(f1, f2)) + udf.register("returnStruct", (f1: String, f2: String) => FunctionResult(f1, f2)) val result= sql("SELECT returnStruct('test', 'test2') as ret") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index a0d54d17f5f13..fbc8704f7837b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -81,7 +81,7 @@ class UserDefinedTypeSuite extends QueryTest { } test("UDTs and UDFs") { - registerFunction("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) + udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) pointsRDD.registerTempTable("points") checkAnswer( sql("SELECT testType(features) from points"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 5fc8d8dbe3a9f..5dafcd6c0a76a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -50,7 +50,7 @@ class HiveUdfSuite extends QueryTest { import TestHive._ test("spark sql udf test that returns a struct") { - registerFunction("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) + udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) assert(sql( """ |SELECT getStruct(1).f1, From 65858ba555c4c0aea988b8f4b1c8476c42377eb9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 15 Jan 2015 17:07:44 -0800 Subject: [PATCH 34/46] [Minor] Fix tiny typo in BlockManager In BlockManager, there is a word `BlockTranserService` but I think it's typo for `BlockTransferService`. Author: Kousuke Saruta Closes #4046 from sarutak/fix-tiny-typo and squashes the following commits: a3e2a2f [Kousuke Saruta] Fixed tiny typo in BlockManager --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 d7b184f8a10e9..1427305d91cff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -34,10 +34,9 @@ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.netty.{SparkTransportConf, NettyBlockTransferService} +import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo -import org.apache.spark.network.util.{ConfigProvider, TransportConf} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager @@ -120,7 +119,7 @@ private[spark] class BlockManager( private[spark] var shuffleServerId: BlockManagerId = _ // Client to read other executors' shuffle files. This is either an external service, or just the - // standard BlockTranserService to directly connect to other Executors. + // standard BlockTransferService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { val transConf = SparkTransportConf.fromSparkConf(conf, numUsableCores) new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled()) From 96c2c714f4f9abe20d4c42d99ffaafcb269714a1 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Thu, 15 Jan 2015 17:53:42 -0800 Subject: [PATCH 35/46] [SPARK-4857] [CORE] Adds Executor membership events to SparkListener Adds onExecutorAdded and onExecutorRemoved events to the SparkListener. This will allow a client to get notified when an executor has been added/removed and provide additional information such as how many vcores it is consuming. In addition, this commit adds a SparkListenerAdapter to the Java API that provides default implementations to the SparkListener. This is to get around the fact that default implementations for traits don't work in Java. Having Java clients extend SparkListenerAdapter moving forward will prevent breakage in java when we add new events to SparkListener. Author: Kostas Sakellis Closes #3711 from ksakellis/kostas-spark-4857 and squashes the following commits: 946d2c5 [Kostas Sakellis] Added executorAdded/Removed events to MesosSchedulerBackend b1d054a [Kostas Sakellis] Remove executorInfo from ExecutorRemoved event 1727b38 [Kostas Sakellis] Renamed ExecutorDetails back to ExecutorInfo and other CR feedback 14fe78d [Kostas Sakellis] Added executor added/removed events to json protocol 93d087b [Kostas Sakellis] [SPARK-4857] [CORE] Adds Executor membership events to SparkListener --- .../org/apache/spark/JavaSparkListener.java | 97 +++++++++++++++++++ .../spark/deploy/master/ApplicationInfo.scala | 14 +-- ...{ExecutorInfo.scala => ExecutorDesc.scala} | 4 +- .../apache/spark/deploy/master/Master.scala | 2 +- .../spark/deploy/master/WorkerInfo.scala | 6 +- .../deploy/master/ui/ApplicationPage.scala | 4 +- .../scheduler/EventLoggingListener.scala | 4 + .../spark/scheduler/SparkListener.scala | 22 ++++- .../spark/scheduler/SparkListenerBus.scala | 4 + .../CoarseGrainedSchedulerBackend.scala | 6 +- .../scheduler/cluster/ExecutorData.scala | 6 +- .../scheduler/cluster/ExecutorInfo.scala | 45 +++++++++ .../cluster/mesos/MesosSchedulerBackend.scala | 32 ++++-- .../org/apache/spark/util/JsonProtocol.scala | 40 +++++++- .../scheduler/EventLoggingListenerSuite.scala | 3 +- .../SparkListenerWithClusterSuite.scala | 62 ++++++++++++ .../mesos/MesosSchedulerBackendSuite.scala | 16 ++- .../apache/spark/util/JsonProtocolSuite.scala | 41 ++++++++ 18 files changed, 375 insertions(+), 33 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/JavaSparkListener.java rename core/src/main/scala/org/apache/spark/deploy/master/{ExecutorInfo.scala => ExecutorDesc.scala} (95%) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java new file mode 100644 index 0000000000000..646496f313507 --- /dev/null +++ b/core/src/main/java/org/apache/spark/JavaSparkListener.java @@ -0,0 +1,97 @@ +/* + * 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; + +import org.apache.spark.scheduler.SparkListener; +import org.apache.spark.scheduler.SparkListenerApplicationEnd; +import org.apache.spark.scheduler.SparkListenerApplicationStart; +import org.apache.spark.scheduler.SparkListenerBlockManagerAdded; +import org.apache.spark.scheduler.SparkListenerBlockManagerRemoved; +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate; +import org.apache.spark.scheduler.SparkListenerExecutorAdded; +import org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate; +import org.apache.spark.scheduler.SparkListenerExecutorRemoved; +import org.apache.spark.scheduler.SparkListenerJobEnd; +import org.apache.spark.scheduler.SparkListenerJobStart; +import org.apache.spark.scheduler.SparkListenerStageCompleted; +import org.apache.spark.scheduler.SparkListenerStageSubmitted; +import org.apache.spark.scheduler.SparkListenerTaskEnd; +import org.apache.spark.scheduler.SparkListenerTaskGettingResult; +import org.apache.spark.scheduler.SparkListenerTaskStart; +import org.apache.spark.scheduler.SparkListenerUnpersistRDD; + +/** + * Java clients should extend this class instead of implementing + * SparkListener directly. This is to prevent java clients + * from breaking when new events are added to the SparkListener + * trait. + * + * This is a concrete class instead of abstract to enforce + * new events get added to both the SparkListener and this adapter + * in lockstep. + */ +public class JavaSparkListener implements SparkListener { + + @Override + public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { } + + @Override + public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { } + + @Override + public void onTaskStart(SparkListenerTaskStart taskStart) { } + + @Override + public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { } + + @Override + public void onTaskEnd(SparkListenerTaskEnd taskEnd) { } + + @Override + public void onJobStart(SparkListenerJobStart jobStart) { } + + @Override + public void onJobEnd(SparkListenerJobEnd jobEnd) { } + + @Override + public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { } + + @Override + public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { } + + @Override + public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { } + + @Override + public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { } + + @Override + public void onApplicationStart(SparkListenerApplicationStart applicationStart) { } + + @Override + public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { } + + @Override + public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { } + + @Override + public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { } + + @Override + public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index ad7d81747c377..ede0a9dbefb8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -38,8 +38,8 @@ private[spark] class ApplicationInfo( extends Serializable { @transient var state: ApplicationState.Value = _ - @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _ - @transient var removedExecutors: ArrayBuffer[ExecutorInfo] = _ + @transient var executors: mutable.HashMap[Int, ExecutorDesc] = _ + @transient var removedExecutors: ArrayBuffer[ExecutorDesc] = _ @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ @@ -55,12 +55,12 @@ private[spark] class ApplicationInfo( private def init() { state = ApplicationState.WAITING - executors = new mutable.HashMap[Int, ExecutorInfo] + executors = new mutable.HashMap[Int, ExecutorDesc] coresGranted = 0 endTime = -1L appSource = new ApplicationSource(this) nextExecutorId = 0 - removedExecutors = new ArrayBuffer[ExecutorInfo] + removedExecutors = new ArrayBuffer[ExecutorDesc] } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -75,14 +75,14 @@ private[spark] class ApplicationInfo( } } - def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = { - val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) executors(exec.id) = exec coresGranted += cores exec } - def removeExecutor(exec: ExecutorInfo) { + def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.id)) { removedExecutors += executors(exec.id) executors -= exec.id diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index d417070c51016..5d620dfcabad5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} -private[spark] class ExecutorInfo( +private[spark] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, @@ -37,7 +37,7 @@ private[spark] class ExecutorInfo( override def equals(other: Any): Boolean = { other match { - case info: ExecutorInfo => + case info: ExecutorDesc => fullId == info.fullId && worker.id == info.worker.id && cores == info.cores && diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 4b631ec639071..d92d99310a583 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -581,7 +581,7 @@ private[spark] class Master( } } - def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo) { + def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 473ddc23ff0f3..e94aae93e4495 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -38,7 +38,7 @@ private[spark] class WorkerInfo( Utils.checkHost(host, "Expected hostname") assert (port > 0) - @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // executorId => info + @transient var executors: mutable.HashMap[String, ExecutorDesc] = _ // executorId => info @transient var drivers: mutable.HashMap[String, DriverInfo] = _ // driverId => info @transient var state: WorkerState.Value = _ @transient var coresUsed: Int = _ @@ -70,13 +70,13 @@ private[spark] class WorkerInfo( host + ":" + port } - def addExecutor(exec: ExecutorInfo) { + def addExecutor(exec: ExecutorDesc) { executors(exec.fullId) = exec coresUsed += exec.cores memoryUsed += exec.memory } - def removeExecutor(exec: ExecutorInfo) { + def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.fullId)) { executors -= exec.fullId coresUsed -= exec.cores diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 4588c130ef439..3aae2b95d7396 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -27,7 +27,7 @@ import org.json4s.JValue import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import org.apache.spark.deploy.master.ExecutorInfo +import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -109,7 +109,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - private def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorDesc): Seq[Node] = { {executor.id} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 27bf4f1599076..30075c172bdb1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -168,6 +168,10 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + override def onExecutorAdded(event: SparkListenerExecutorAdded) = + logEvent(event, flushLogger = true) + override def onExecutorRemoved(event: SparkListenerExecutorRemoved) = + logEvent(event, flushLogger = true) // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index b62b0c1312693..4840d8bd2d2f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -25,6 +25,7 @@ import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} @@ -84,6 +85,14 @@ case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockMan @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorAdded(executorId: String, executorInfo: ExecutorInfo) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorRemoved(executorId: String) + extends SparkListenerEvent + /** * Periodic updates from executors. * @param execId executor id @@ -109,7 +118,8 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** * :: DeveloperApi :: * Interface for listening to events from the Spark scheduler. Note that this is an internal - * interface which might change in different Spark releases. + * interface which might change in different Spark releases. Java clients should extend + * {@link JavaSparkListener} */ @DeveloperApi trait SparkListener { @@ -183,6 +193,16 @@ trait SparkListener { * Called when the driver receives task metrics from an executor in a heartbeat. */ def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } + + /** + * Called when the driver registers a new executor. + */ + def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) { } + + /** + * Called when the driver removes an executor. + */ + def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index e79ffd7a3587d..e700c6af542f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -70,6 +70,10 @@ private[spark] trait SparkListenerBus extends Logging { foreachListener(_.onApplicationEnd(applicationEnd)) case metricsUpdate: SparkListenerExecutorMetricsUpdate => foreachListener(_.onExecutorMetricsUpdate(metricsUpdate)) + case executorAdded: SparkListenerExecutorAdded => + foreachListener(_.onExecutorAdded(executorAdded)) + case executorRemoved: SparkListenerExecutorRemoved => + foreachListener(_.onExecutorRemoved(executorRemoved)) case SparkListenerShutdown => } } 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 fe9914b50bc54..5786d367464f4 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 @@ -28,7 +28,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} -import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} @@ -66,6 +66,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Number of executors requested from the cluster manager that have not registered yet private var numPendingExecutors = 0 + private val listenerBus = scheduler.sc.listenerBus + // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] @@ -106,6 +108,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } + listenerBus.post(SparkListenerExecutorAdded(executorId, data)) makeOffers() } @@ -213,6 +216,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste totalCoreCount.addAndGet(-executorInfo.totalCores) totalRegisteredExecutors.addAndGet(-1) scheduler.executorLost(executorId, SlaveLost(reason)) + listenerBus.post(SparkListenerExecutorRemoved(executorId)) case None => logError(s"Asked to remove non-existent executor $executorId") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index b71bd5783d6df..eb52ddfb1eab1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -31,7 +31,7 @@ import akka.actor.{Address, ActorRef} private[cluster] class ExecutorData( val executorActor: ActorRef, val executorAddress: Address, - val executorHost: String , + override val executorHost: String, var freeCores: Int, - val totalCores: Int -) + override val totalCores: Int +) extends ExecutorInfo(executorHost, totalCores) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala new file mode 100644 index 0000000000000..b4738e64c9391 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -0,0 +1,45 @@ +/* + * 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.scheduler.cluster + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Stores information about an executor to pass from the scheduler to SparkListeners. + */ +@DeveloperApi +class ExecutorInfo( + val executorHost: String, + val totalCores: Int +) { + + def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] + + override def equals(other: Any): Boolean = other match { + case that: ExecutorInfo => + (that canEqual this) && + executorHost == that.executorHost && + totalCores == that.totalCores + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(executorHost, totalCores) + state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 75d8ddf375e27..d252fe8595fb8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -27,9 +27,11 @@ import scala.collection.mutable.{HashMap, HashSet} import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, + ExecutorInfo => MesosExecutorInfo, _} import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -62,6 +64,9 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null + // The listener bus to publish executor added/removed events. + val listenerBus = sc.listenerBus + @volatile var appId: String = _ override def start() { @@ -87,7 +92,7 @@ private[spark] class MesosSchedulerBackend( } } - def createExecutorInfo(execId: String): ExecutorInfo = { + def createExecutorInfo(execId: String): MesosExecutorInfo = { val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility .getOrElse { @@ -141,7 +146,7 @@ private[spark] class MesosSchedulerBackend( Value.Scalar.newBuilder() .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) .build() - ExecutorInfo.newBuilder() + MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) @@ -237,6 +242,7 @@ private[spark] class MesosSchedulerBackend( } val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] @@ -260,6 +266,10 @@ private[spark] class MesosSchedulerBackend( val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? mesosTasks.foreach { case (slaveId, tasks) => + slaveIdToWorkerOffer.get(slaveId).foreach(o => + listenerBus.post(SparkListenerExecutorAdded(slaveId, + new ExecutorInfo(o.host, o.cores))) + ) d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } @@ -315,7 +325,7 @@ private[spark] class MesosSchedulerBackend( synchronized { if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { // We lost the executor on this slave, so remember that it's gone - slaveIdsWithExecutors -= taskIdToSlaveId(tid) + removeExecutor(taskIdToSlaveId(tid)) } if (isFinished(status.getState)) { taskIdToSlaveId.remove(tid) @@ -344,12 +354,20 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + /** + * Remove executor associated with slaveId in a thread safe manner. + */ + private def removeExecutor(slaveId: String) = { + synchronized { + listenerBus.post(SparkListenerExecutorRemoved(slaveId)) + slaveIdsWithExecutors -= slaveId + } + } + private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - slaveIdsWithExecutors -= slaveId.getValue - } + removeExecutor(slaveId.getValue) scheduler.executorLost(slaveId.getValue, reason) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d94e8252650d2..a025011006156 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.util.{Properties, UUID} +import org.apache.spark.scheduler.cluster.ExecutorInfo + import scala.collection.JavaConverters._ import scala.collection.Map @@ -83,7 +85,10 @@ private[spark] object JsonProtocol { applicationStartToJson(applicationStart) case applicationEnd: SparkListenerApplicationEnd => applicationEndToJson(applicationEnd) - + case executorAdded: SparkListenerExecutorAdded => + executorAddedToJson(executorAdded) + case executorRemoved: SparkListenerExecutorRemoved => + executorRemovedToJson(executorRemoved) // These aren't used, but keeps compiler happy case SparkListenerShutdown => JNothing case SparkListenerExecutorMetricsUpdate(_, _) => JNothing @@ -194,6 +199,16 @@ private[spark] object JsonProtocol { ("Timestamp" -> applicationEnd.time) } + def executorAddedToJson(executorAdded: SparkListenerExecutorAdded): JValue = { + ("Event" -> Utils.getFormattedClassName(executorAdded)) ~ + ("Executor ID" -> executorAdded.executorId) ~ + ("Executor Info" -> executorInfoToJson(executorAdded.executorInfo)) + } + + def executorRemovedToJson(executorRemoved: SparkListenerExecutorRemoved): JValue = { + ("Event" -> Utils.getFormattedClassName(executorRemoved)) ~ + ("Executor ID" -> executorRemoved.executorId) + } /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -362,6 +377,10 @@ private[spark] object JsonProtocol { ("Disk Size" -> blockStatus.diskSize) } + def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { + ("Host" -> executorInfo.executorHost) ~ + ("Total Cores" -> executorInfo.totalCores) + } /** ------------------------------ * * Util JSON serialization methods | @@ -416,6 +435,8 @@ private[spark] object JsonProtocol { val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) + val executorAdded = Utils.getFormattedClassName(SparkListenerExecutorAdded) + val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -431,6 +452,8 @@ private[spark] object JsonProtocol { case `unpersistRDD` => unpersistRDDFromJson(json) case `applicationStart` => applicationStartFromJson(json) case `applicationEnd` => applicationEndFromJson(json) + case `executorAdded` => executorAddedFromJson(json) + case `executorRemoved` => executorRemovedFromJson(json) } } @@ -523,6 +546,16 @@ private[spark] object JsonProtocol { SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) } + def executorAddedFromJson(json: JValue): SparkListenerExecutorAdded = { + val executorId = (json \ "Executor ID").extract[String] + val executorInfo = executorInfoFromJson(json \ "Executor Info") + SparkListenerExecutorAdded(executorId, executorInfo) + } + + def executorRemovedFromJson(json: JValue): SparkListenerExecutorRemoved = { + val executorId = (json \ "Executor ID").extract[String] + SparkListenerExecutorRemoved(executorId) + } /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | @@ -745,6 +778,11 @@ private[spark] object JsonProtocol { BlockStatus(storageLevel, memorySize, diskSize, tachyonSize) } + def executorInfoFromJson(json: JValue): ExecutorInfo = { + val executorHost = (json \ "Host").extract[String] + val totalCores = (json \ "Total Cores").extract[Int] + new ExecutorInfo(executorHost, totalCores) + } /** -------------------------------- * * Util JSON deserialization methods | diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 1de7e130039a5..437d8693c0b1f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -160,7 +160,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { val conf = getLoggingConf(testDirPath, compressionCodec) - val sc = new SparkContext("local", "test", conf) + val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val expectedLogDir = testDir.toURI().toString() @@ -184,6 +184,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val eventSet = mutable.Set( SparkListenerApplicationStart, SparkListenerBlockManagerAdded, + SparkListenerExecutorAdded, SparkListenerEnvironmentUpdate, SparkListenerJobStart, SparkListenerJobEnd, diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala new file mode 100644 index 0000000000000..623a687c359a2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala @@ -0,0 +1,62 @@ +/* + * 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.scheduler + +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.{SparkContext, LocalSparkContext} + +import org.scalatest.{FunSuite, BeforeAndAfter, BeforeAndAfterAll} + +import scala.collection.mutable + +/** + * Unit tests for SparkListener that require a local cluster. + */ +class SparkListenerWithClusterSuite extends FunSuite with LocalSparkContext + with BeforeAndAfter with BeforeAndAfterAll { + + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + + before { + sc = new SparkContext("local-cluster[2,1,512]", "SparkListenerSuite") + } + + test("SparkListener sends executor added message") { + val listener = new SaveExecutorInfo + sc.addSparkListener(listener) + + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.map(_.toString) + rdd2.setName("Target RDD") + rdd2.count() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(listener.addedExecutorInfo.size == 2) + assert(listener.addedExecutorInfo("0").totalCores == 1) + assert(listener.addedExecutorInfo("1").totalCores == 1) + } + + private class SaveExecutorInfo extends SparkListener { + val addedExecutorInfo = mutable.Map[String, ExecutorInfo]() + + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfo(executor.executorId) = executor.executorInfo + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index 48f5e40f506d9..78a30a40bf19a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -18,17 +18,20 @@ package org.apache.spark.scheduler.mesos import org.scalatest.FunSuite -import org.apache.spark.{scheduler, SparkConf, SparkContext, LocalSparkContext} -import org.apache.spark.scheduler.{TaskDescription, WorkerOffer, TaskSchedulerImpl} +import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} +import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus, + TaskDescription, WorkerOffer, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend} import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos._ -import org.scalatest.mock.EasyMockSugar +import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, _} import org.apache.mesos.Protos.Value.Scalar import org.easymock.{Capture, EasyMock} import java.nio.ByteBuffer import java.util.Collections import java.util +import org.scalatest.mock.EasyMockSugar + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -52,11 +55,16 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) + listenerBus.post(SparkListenerExecutorAdded("s1", new ExecutorInfo("host1", 2))) + EasyMock.replay(listenerBus) + val sc = EasyMock.createMock(classOf[SparkContext]) EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() + EasyMock.expect(sc.listenerBus).andReturn(listenerBus) EasyMock.replay(sc) val minMem = MemoryUtils.calculateTotalMemory(sc).toInt diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 63c2559c5c5f5..5ba94ff67d395 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException import scala.collection.Map @@ -69,6 +70,9 @@ class JsonProtocolSuite extends FunSuite { val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) + val executorAdded = SparkListenerExecutorAdded("exec1", + new ExecutorInfo("Hostee.awesome.com", 11)) + val executorRemoved = SparkListenerExecutorRemoved("exec2") testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -85,6 +89,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(unpersistRdd, unpersistRDDJsonString) testEvent(applicationStart, applicationStartJsonString) testEvent(applicationEnd, applicationEndJsonString) + testEvent(executorAdded, executorAddedJsonString) + testEvent(executorRemoved, executorRemovedJsonString) } test("Dependent Classes") { @@ -94,6 +100,7 @@ class JsonProtocolSuite extends FunSuite { testTaskMetrics(makeTaskMetrics( 33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false, hasOutput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500)) + testExecutorInfo(new ExecutorInfo("host", 43)) // StorageLevel testStorageLevel(StorageLevel.NONE) @@ -303,6 +310,10 @@ class JsonProtocolSuite extends FunSuite { assert(blockId === newBlockId) } + private def testExecutorInfo(info: ExecutorInfo) { + val newInfo = JsonProtocol.executorInfoFromJson(JsonProtocol.executorInfoToJson(info)) + assertEquals(info, newInfo) + } /** -------------------------------- * | Util methods for comparing events | @@ -335,6 +346,11 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) + case (e1: SparkListenerExecutorAdded, e2: SparkListenerExecutorAdded) => + assert(e1.executorId == e1.executorId) + assertEquals(e1.executorInfo, e2.executorInfo) + case (e1: SparkListenerExecutorRemoved, e2: SparkListenerExecutorRemoved) => + assert(e1.executorId == e1.executorId) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") @@ -387,6 +403,11 @@ class JsonProtocolSuite extends FunSuite { assert(info1.accumulables === info2.accumulables) } + private def assertEquals(info1: ExecutorInfo, info2: ExecutorInfo) { + assert(info1.executorHost == info2.executorHost) + assert(info1.totalCores == info2.totalCores) + } + private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { assert(metrics1.hostname === metrics2.hostname) assert(metrics1.executorDeserializeTime === metrics2.executorDeserializeTime) @@ -1407,4 +1428,24 @@ class JsonProtocolSuite extends FunSuite { | "Timestamp": 42 |} """ + + private val executorAddedJsonString = + """ + |{ + | "Event": "SparkListenerExecutorAdded", + | "Executor ID": "exec1", + | "Executor Info": { + | "Host": "Hostee.awesome.com", + | "Total Cores": 11 + | } + |} + """ + + private val executorRemovedJsonString = + """ + |{ + | "Event": "SparkListenerExecutorRemoved", + | "Executor ID": "exec2" + |} + """ } From a79a9f923c47f2ce7da93cf0ecfe2b66fcb9fdd4 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Thu, 15 Jan 2015 18:48:39 -0800 Subject: [PATCH 36/46] [SPARK-4092] [CORE] Fix InputMetrics for coalesce'd Rdds When calculating the input metrics there was an assumption that one task only reads from one block - this is not true for some operations including coalesce. This patch simply increments the task's input metrics if previous ones existed of the same read method. A limitation to this patch is that if a task reads from two different blocks of different read methods, one will override the other. Author: Kostas Sakellis Closes #3120 from ksakellis/kostas-spark-4092 and squashes the following commits: 54e6658 [Kostas Sakellis] Drops metrics if conflicting read methods exist f0e0cc5 [Kostas Sakellis] Add bytesReadCallback to InputMetrics a2a36d4 [Kostas Sakellis] CR feedback 5a0c770 [Kostas Sakellis] [SPARK-4092] [CORE] Fix InputMetrics for coalesce'd Rdds --- .../scala/org/apache/spark/CacheManager.scala | 6 +- .../org/apache/spark/executor/Executor.scala | 1 + .../apache/spark/executor/TaskMetrics.scala | 75 ++++++- .../org/apache/spark/rdd/HadoopRDD.scala | 39 ++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 40 ++-- .../apache/spark/storage/BlockManager.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 6 +- .../metrics/InputOutputMetricsSuite.scala | 195 ++++++++++++++---- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- 10 files changed, 270 insertions(+), 102 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 80da62c44edc5..a0c0372b7f0ef 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -44,7 +44,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { blockManager.get(key) match { case Some(blockResult) => // Partition is already materialized, so just return its values - context.taskMetrics.inputMetrics = Some(blockResult.inputMetrics) + val inputMetrics = blockResult.inputMetrics + val existingMetrics = context.taskMetrics + .getInputMetricsForReadMethod(inputMetrics.readMethod) + existingMetrics.addBytesRead(inputMetrics.bytesRead) + new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) case None => diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b75c77b5b4457..6660b98eb8ce9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -379,6 +379,7 @@ private[spark] class Executor( if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => metrics.updateShuffleReadMetrics + metrics.updateInputMetrics() metrics.jvmGCTime = curGCTime - taskRunner.startGCTime if (isLocal) { // JobProgressListener will hold an reference of it during 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 51b5328cb4c8f..7eb10f95e023b 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,11 @@ package org.apache.spark.executor +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark.executor.DataReadMethod +import org.apache.spark.executor.DataReadMethod.DataReadMethod + import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi @@ -80,7 +85,17 @@ class TaskMetrics extends Serializable { * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read * are stored here. */ - var inputMetrics: Option[InputMetrics] = None + private var _inputMetrics: Option[InputMetrics] = None + + def inputMetrics = _inputMetrics + + /** + * This should only be used when recreating TaskMetrics, not when updating input metrics in + * executors + */ + private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { + _inputMetrics = inputMetrics + } /** * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much @@ -133,6 +148,30 @@ class TaskMetrics extends Serializable { readMetrics } + /** + * Returns the input metrics object that the task should use. Currently, if + * there exists an input metric with the same readMethod, we return that one + * so the caller can accumulate bytes read. If the readMethod is different + * than previously seen by this task, we return a new InputMetric but don't + * record it. + * + * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, + * we can store all the different inputMetrics (one per readMethod). + */ + private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): + InputMetrics =synchronized { + _inputMetrics match { + case None => + val metrics = new InputMetrics(readMethod) + _inputMetrics = Some(metrics) + metrics + case Some(metrics @ InputMetrics(method)) if method == readMethod => + metrics + case Some(InputMetrics(method)) => + new InputMetrics(readMethod) + } + } + /** * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. */ @@ -146,6 +185,10 @@ class TaskMetrics extends Serializable { } _shuffleReadMetrics = Some(merged) } + + private[spark] def updateInputMetrics() = synchronized { + inputMetrics.foreach(_.updateBytesRead()) + } } private[spark] object TaskMetrics { @@ -179,10 +222,38 @@ object DataWriteMethod extends Enumeration with Serializable { */ @DeveloperApi case class InputMetrics(readMethod: DataReadMethod.Value) { + + private val _bytesRead: AtomicLong = new AtomicLong() + /** * Total bytes read. */ - var bytesRead: Long = 0L + def bytesRead: Long = _bytesRead.get() + @volatile @transient var bytesReadCallback: Option[() => Long] = None + + /** + * Adds additional bytes read for this read method. + */ + def addBytesRead(bytes: Long) = { + _bytesRead.addAndGet(bytes) + } + + /** + * Invoke the bytesReadCallback and mutate bytesRead. + */ + def updateBytesRead() { + bytesReadCallback.foreach { c => + _bytesRead.set(c()) + } + } + + /** + * Register a function that can be called to get up-to-date information on how many bytes the task + * has read from an input source. + */ + def setBytesReadCallback(f: Option[() => Long]) { + bytesReadCallback = f + } } /** 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 37e0c13029d8b..3b99d3a6cafd1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -213,18 +213,19 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics + .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( - split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) - } else { - None - } - if (bytesReadCallback.isDefined) { - context.taskMetrics.inputMetrics = Some(inputMetrics) - } + val bytesReadCallback = inputMetrics.bytesReadCallback.orElse( + split.inputSplit.value match { + case split: FileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, jobConf) + case _ => None + } + ) + inputMetrics.setBytesReadCallback(bytesReadCallback) var reader: RecordReader[K, V] = null val inputFormat = getInputFormat(jobConf) @@ -237,8 +238,6 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - var recordsSinceMetricsUpdate = 0 - override def getNext() = { try { finished = !reader.next(key, value) @@ -246,16 +245,6 @@ class HadoopRDD[K, V]( case eof: EOFException => finished = true } - - // Update bytes read metric every few records - if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES - && bytesReadCallback.isDefined) { - recordsSinceMetricsUpdate = 0 - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() - } else { - recordsSinceMetricsUpdate += 1 - } (key, value) } @@ -263,14 +252,12 @@ class HadoopRDD[K, V]( try { reader.close() if (bytesReadCallback.isDefined) { - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() + inputMetrics.updateBytesRead() } else if (split.inputSplit.value.isInstanceOf[FileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.bytesRead = split.inputSplit.value.getLength - context.taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(split.inputSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 e55d03d391e03..890ec677c2690 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -109,18 +109,19 @@ class NewHadoopRDD[K, V]( logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics + .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( - split.serializableHadoopSplit.value.asInstanceOf[FileSplit].getPath, conf) - } else { - None - } - if (bytesReadCallback.isDefined) { - context.taskMetrics.inputMetrics = Some(inputMetrics) - } + val bytesReadCallback = inputMetrics.bytesReadCallback.orElse( + split.serializableHadoopSplit.value match { + case split: FileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, conf) + case _ => None + } + ) + inputMetrics.setBytesReadCallback(bytesReadCallback) val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) @@ -153,34 +154,19 @@ class NewHadoopRDD[K, V]( throw new java.util.NoSuchElementException("End of stream") } havePair = false - - // Update bytes read metric every few records - if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES - && bytesReadCallback.isDefined) { - recordsSinceMetricsUpdate = 0 - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() - } else { - recordsSinceMetricsUpdate += 1 - } - (reader.getCurrentKey, reader.getCurrentValue) } private def close() { try { reader.close() - - // Update metrics with final amount if (bytesReadCallback.isDefined) { - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() + inputMetrics.updateBytesRead() } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength - context.taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 1427305d91cff..8bc5a1cd18b64 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -53,7 +53,7 @@ private[spark] class BlockResult( readMethod: DataReadMethod.Value, bytes: Long) { val inputMetrics = new InputMetrics(readMethod) - inputMetrics.bytesRead = bytes + inputMetrics.addBytesRead(bytes) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a025011006156..ee3756c226fe3 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -637,8 +637,8 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson)) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.inputMetrics = - Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson) + metrics.setInputMetrics( + Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) metrics.outputMetrics = Utils.jsonOption(json \ "Output Metrics").map(outputMetricsFromJson) metrics.updatedBlocks = @@ -671,7 +671,7 @@ private[spark] object JsonProtocol { def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.bytesRead = (json \ "Bytes Read").extract[Long] + metrics.addBytesRead((json \ "Bytes Read").extract[Long]) metrics } diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index f8bcde12a371a..10a39990f80ce 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -17,66 +17,185 @@ package org.apache.spark.metrics -import java.io.{FileWriter, PrintWriter, File} +import java.io.{File, FileWriter, PrintWriter} -import org.apache.spark.SharedSparkContext -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} +import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite -import org.scalatest.Matchers import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.SharedSparkContext +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.util.Utils + +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers { - test("input metrics when reading text file with single split") { - val file = new File(getClass.getSimpleName + ".txt") - val pw = new PrintWriter(new FileWriter(file)) - pw.println("some stuff") - pw.println("some other stuff") - pw.println("yet more stuff") - pw.println("too much stuff") + @transient var tmpDir: File = _ + @transient var tmpFile: File = _ + @transient var tmpFilePath: String = _ + + override def beforeAll() { + super.beforeAll() + + tmpDir = Utils.createTempDir() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") + val pw = new PrintWriter(new FileWriter(tmpFile)) + for (x <- 1 to 1000000) { + pw.println("s") + } pw.close() - file.deleteOnExit() - val taskBytesRead = new ArrayBuffer[Long]() - sc.addSparkListener(new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead - } - }) - sc.textFile("file://" + file.getAbsolutePath, 2).count() + // Path to tmpFile + tmpFilePath = "file://" + tmpFile.getAbsolutePath + } - // Wait for task end events to come in - sc.listenerBus.waitUntilEmpty(500) - assert(taskBytesRead.length == 2) - assert(taskBytesRead.sum >= file.length()) + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) } - test("input metrics when reading text file with multiple splits") { - val file = new File(getClass.getSimpleName + ".txt") - val pw = new PrintWriter(new FileWriter(file)) - for (i <- 0 until 10000) { - pw.println("some stuff") + test("input metrics for old hadoop with coalesce") { + val bytesRead = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 4).count() + } + val bytesRead2 = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 4).coalesce(2).count() + } + assert(bytesRead != 0) + assert(bytesRead == bytesRead2) + assert(bytesRead2 >= tmpFile.length()) + } + + test("input metrics with cache and coalesce") { + // prime the cache manager + val rdd = sc.textFile(tmpFilePath, 4).cache() + rdd.collect() + + val bytesRead = runAndReturnBytesRead { + rdd.count() + } + val bytesRead2 = runAndReturnBytesRead { + rdd.coalesce(4).count() } - pw.close() - file.deleteOnExit() + // for count and coelesce, the same bytes should be read. + assert(bytesRead != 0) + assert(bytesRead2 == bytesRead) + } + + /** + * This checks the situation where we have interleaved reads from + * different sources. Currently, we only accumulate fron the first + * read method we find in the task. This test uses cartesian to create + * the interleaved reads. + * + * Once https://issues.apache.org/jira/browse/SPARK-5225 is fixed + * this test should break. + */ + test("input metrics with mixed read method") { + // prime the cache manager + val numPartitions = 2 + val rdd = sc.parallelize(1 to 100, numPartitions).cache() + rdd.collect() + + val rdd2 = sc.textFile(tmpFilePath, numPartitions) + + val bytesRead = runAndReturnBytesRead { + rdd.count() + } + val bytesRead2 = runAndReturnBytesRead { + rdd2.count() + } + + val cartRead = runAndReturnBytesRead { + rdd.cartesian(rdd2).count() + } + + assert(cartRead != 0) + assert(bytesRead != 0) + // We read from the first rdd of the cartesian once per partition. + assert(cartRead == bytesRead * numPartitions) + } + + test("input metrics for new Hadoop API with coalesce") { + val bytesRead = runAndReturnBytesRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).count() + } + val bytesRead2 = runAndReturnBytesRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).coalesce(5).count() + } + assert(bytesRead != 0) + assert(bytesRead2 == bytesRead) + assert(bytesRead >= tmpFile.length()) + } + + test("input metrics when reading text file") { + val bytesRead = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 2).count() + } + assert(bytesRead >= tmpFile.length()) + } + + test("input metrics with interleaved reads") { + val numPartitions = 2 + val cartVector = 0 to 9 + val cartFile = new File(tmpDir, getClass.getSimpleName + "_cart.txt") + val cartFilePath = "file://" + cartFile.getAbsolutePath + + // write files to disk so we can read them later. + sc.parallelize(cartVector).saveAsTextFile(cartFilePath) + val aRdd = sc.textFile(cartFilePath, numPartitions) + + val tmpRdd = sc.textFile(tmpFilePath, numPartitions) + + val firstSize= runAndReturnBytesRead { + aRdd.count() + } + val secondSize = runAndReturnBytesRead { + tmpRdd.count() + } + + val cartesianBytes = runAndReturnBytesRead { + aRdd.cartesian(tmpRdd).count() + } + + // Computing the amount of bytes read for a cartesian operation is a little involved. + // Cartesian interleaves reads between two partitions eg. p1 and p2. + // Here are the steps: + // 1) First it creates an iterator for p1 + // 2) Creates an iterator for p2 + // 3) Reads the first element of p1 and then all the elements of p2 + // 4) proceeds to the next element of p1 + // 5) Creates a new iterator for p2 + // 6) rinse and repeat. + // As a result we read from the second partition n times where n is the number of keys in + // p1. Thus the math below for the test. + assert(cartesianBytes != 0) + assert(cartesianBytes == firstSize * numPartitions + (cartVector.length * secondSize)) + } + + private def runAndReturnBytesRead(job : => Unit): Long = { val taskBytesRead = new ArrayBuffer[Long]() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead } }) - sc.textFile("file://" + file.getAbsolutePath, 2).count() - // Wait for task end events to come in + job + sc.listenerBus.waitUntilEmpty(500) - assert(taskBytesRead.length == 2) - assert(taskBytesRead.sum >= file.length()) + taskBytesRead.sum } test("output metrics when writing text file") { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 12af60caf7d54..f865d8ca04d1b 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -231,8 +231,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.diskBytesSpilled = base + 5 taskMetrics.memoryBytesSpilled = base + 6 val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - taskMetrics.inputMetrics = Some(inputMetrics) - inputMetrics.bytesRead = base + 7 + taskMetrics.setInputMetrics(Some(inputMetrics)) + inputMetrics.addBytesRead(base + 7) val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) taskMetrics.outputMetrics = Some(outputMetrics) outputMetrics.bytesWritten = base + 8 diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 5ba94ff67d395..71dfed1289850 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -630,8 +630,8 @@ class JsonProtocolSuite extends FunSuite { if (hasHadoopInput) { val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - inputMetrics.bytesRead = d + e + f - t.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(d + e + f) + t.setInputMetrics(Some(inputMetrics)) } else { val sr = new ShuffleReadMetrics sr.remoteBytesRead = b + d From 2be82b1e66cd188456bbf1e5abb13af04d1629d5 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Fri, 16 Jan 2015 09:16:56 -0800 Subject: [PATCH 37/46] [SPARK-1507][YARN]specify # cores for ApplicationMaster Based on top of changes in https://github.com/apache/spark/pull/3806. https://issues.apache.org/jira/browse/SPARK-1507 `--driver-cores` and `spark.driver.cores` for all cluster modes and `spark.yarn.am.cores` for yarn client mode. Author: WangTaoTheTonic Author: WangTao Closes #4018 from WangTaoTheTonic/SPARK-1507 and squashes the following commits: 01419d3 [WangTaoTheTonic] amend the args name b255795 [WangTaoTheTonic] indet thing d86557c [WangTaoTheTonic] some comments amend 43c9392 [WangTao] fix compile error b39a100 [WangTao] specify # cores for ApplicationMaster --- .../apache/spark/deploy/ClientArguments.scala | 6 ++--- .../org/apache/spark/deploy/SparkSubmit.scala | 1 + .../spark/deploy/SparkSubmitArguments.scala | 5 ++++ docs/configuration.md | 15 ++++++++---- docs/running-on-yarn.md | 17 +++++++++++++ .../org/apache/spark/deploy/yarn/Client.scala | 1 + .../spark/deploy/yarn/ClientArguments.scala | 24 +++++++++++++++---- 7 files changed, 58 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 2e1e52906ceeb..e5873ce724b9f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ListBuffer import org.apache.log4j.Level -import org.apache.spark.util.MemoryParam +import org.apache.spark.util.{IntParam, MemoryParam} /** * Command-line parser for the driver client. @@ -51,8 +51,8 @@ private[spark] class ClientArguments(args: Array[String]) { parse(args.toList) def parse(args: List[String]): Unit = args match { - case ("--cores" | "-c") :: value :: tail => - cores = value.toInt + case ("--cores" | "-c") :: IntParam(value) :: tail => + cores = value parse(tail) case ("--memory" | "-m") :: MemoryParam(value) :: tail => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 955cbd6dab96d..050ba91eb2bc3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -200,6 +200,7 @@ object SparkSubmit { // Yarn cluster only OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"), OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"), + OptionAssigner(args.driverCores, YARN, CLUSTER, clOption = "--driver-cores"), OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"), OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"), OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"), diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 47059b08a397f..81ec08cb6d501 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -108,6 +108,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St .orElse(sparkProperties.get("spark.driver.memory")) .orElse(env.get("SPARK_DRIVER_MEMORY")) .orNull + driverCores = Option(driverCores) + .orElse(sparkProperties.get("spark.driver.cores")) + .orNull executorMemory = Option(executorMemory) .orElse(sparkProperties.get("spark.executor.memory")) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) @@ -406,6 +409,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | --total-executor-cores NUM Total cores for all executors. | | YARN-only: + | --driver-cores NUM Number of cores used by the driver, only in cluster mode + | (Default: 1). | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). diff --git a/docs/configuration.md b/docs/configuration.md index 673cdb371a512..efbab4085317a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -102,11 +102,10 @@ of the most common options to set are: - spark.executor.memory - 512m + spark.driver.cores + 1 - Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. 512m, 2g). + Number of cores to use for the driver process, only in cluster mode. @@ -117,6 +116,14 @@ of the most common options to set are: (e.g. 512m, 2g). + + spark.executor.memory + 512m + + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m, 2g). + + spark.driver.maxResultSize 1g diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 4f273098c5db3..68ab127bcf087 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -29,6 +29,23 @@ Most of the configs are the same for Spark on YARN as for other deployment modes In cluster mode, use spark.driver.memory instead. + + spark.driver.cores + 1 + + Number of cores used by the driver in YARN cluster mode. + Since the driver is run in the same JVM as the YARN Application Master in cluster mode, this also controls the cores used by the YARN AM. + In client mode, use spark.yarn.am.cores to control the number of cores used by the YARN AM instead. + + + + spark.yarn.am.cores + 1 + + Number of cores to use for the YARN Application Master in client mode. + In cluster mode, use spark.driver.cores instead. + + spark.yarn.am.waitTime 100000 diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 032106371cd60..d4eeccf64275f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -127,6 +127,7 @@ private[spark] class Client( } val capability = Records.newRecord(classOf[Resource]) capability.setMemory(args.amMemory + amMemoryOverhead) + capability.setVirtualCores(args.amCores) appContext.setResource(capability) appContext } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 461a9ccd3c216..79bead77ba6e4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -36,14 +36,18 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var numExecutors = DEFAULT_NUMBER_EXECUTORS var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB + var amCores: Int = 1 var appName: String = "Spark" var priority = 0 def isClusterMode: Boolean = userClass != null private var driverMemory: Int = 512 // MB + private var driverCores: Int = 1 private val driverMemOverheadKey = "spark.yarn.driver.memoryOverhead" private val amMemKey = "spark.yarn.am.memory" private val amMemOverheadKey = "spark.yarn.am.memoryOverhead" + private val driverCoresKey = "spark.driver.cores" + private val amCoresKey = "spark.yarn.am.cores" private val isDynamicAllocationEnabled = sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) @@ -92,19 +96,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) "You must specify at least 1 executor!\n" + getUsageMessage()) } if (isClusterMode) { - for (key <- Seq(amMemKey, amMemOverheadKey)) { + for (key <- Seq(amMemKey, amMemOverheadKey, amCoresKey)) { if (sparkConf.contains(key)) { println(s"$key is set but does not apply in cluster mode.") } } amMemory = driverMemory + amCores = driverCores } else { - if (sparkConf.contains(driverMemOverheadKey)) { - println(s"$driverMemOverheadKey is set but does not apply in client mode.") + for (key <- Seq(driverMemOverheadKey, driverCoresKey)) { + if (sparkConf.contains(key)) { + println(s"$key is set but does not apply in client mode.") + } } sparkConf.getOption(amMemKey) .map(Utils.memoryStringToMb) .foreach { mem => amMemory = mem } + sparkConf.getOption(amCoresKey) + .map(_.toInt) + .foreach { cores => amCores = cores } } } @@ -140,6 +150,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) driverMemory = value args = tail + case ("--driver-cores") :: IntParam(value) :: tail => + driverCores = value + args = tail + case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => if (args(0) == "--num-workers") { println("--num-workers is deprecated. Use --num-executors instead.") @@ -198,7 +212,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + """ + message + + """ |Usage: org.apache.spark.deploy.yarn.Client [options] |Options: | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster @@ -209,6 +224,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | --num-executors NUM Number of executors to start (Default: 2) | --executor-cores NUM Number of cores for the executors (Default: 1). | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) + | --driver-cores NUM Number of cores used by the driver (Default: 1). | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) | --name NAME The name of your application (Default: Spark) | --queue QUEUE The hadoop queue to use for allocation requests (Default: From e200ac8e53a533d64a79c18561b557ea445f1cc9 Mon Sep 17 00:00:00 2001 From: Ye Xianjin Date: Fri, 16 Jan 2015 09:20:53 -0800 Subject: [PATCH 38/46] [SPARK-5201][CORE] deal with int overflow in the ParallelCollectionRDD.slice method There is an int overflow in the ParallelCollectionRDD.slice method. That's originally reported by SaintBacchus. ``` sc.makeRDD(1 to (Int.MaxValue)).count // result = 0 sc.makeRDD(1 to (Int.MaxValue - 1)).count // result = 2147483646 = Int.MaxValue - 1 sc.makeRDD(1 until (Int.MaxValue)).count // result = 2147483646 = Int.MaxValue - 1 ``` see https://github.com/apache/spark/pull/2874 for more details. This pr try to fix the overflow. However, There's another issue I don't address. ``` val largeRange = Int.MinValue to Int.MaxValue largeRange.length // throws java.lang.IllegalArgumentException: -2147483648 to 2147483647 by 1: seqs cannot contain more than Int.MaxValue elements. ``` So, the range we feed to sc.makeRDD cannot contain more than Int.MaxValue elements. This is the limitation of Scala. However I think we may want to support that kind of range. But the fix is beyond this pr. srowen andrewor14 would you mind take a look at this pr? Author: Ye Xianjin Closes #4002 from advancedxy/SPARk-5201 and squashes the following commits: 96265a1 [Ye Xianjin] Update slice method comment and some responding docs. e143d7a [Ye Xianjin] Update inclusive range check for splitting inclusive range. b3f5577 [Ye Xianjin] We can include the last element in the last slice in general for inclusive range, hence eliminate the need to check Int.MaxValue or Int.MinValue. 7d39b9e [Ye Xianjin] Convert the two cases pattern matching to one case. 651c959 [Ye Xianjin] rename sign to needsInclusiveRange. add some comments 196f8a8 [Ye Xianjin] Add test cases for ranges end with Int.MaxValue or Int.MinValue e66e60a [Ye Xianjin] Deal with inclusive and exclusive ranges in one case. If the range is inclusive and the end of the range is (Int.MaxValue or Int.MinValue), we should use inclusive range instead of exclusive --- .../scala/org/apache/spark/SparkContext.scala | 7 +++--- .../spark/rdd/ParallelCollectionRDD.scala | 21 +++++++--------- .../rdd/ParallelCollectionSplitSuite.scala | 25 +++++++++++++++++++ 3 files changed, 37 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ff5d796ee2766..6a354ed4d1486 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -520,10 +520,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Distribute a local Scala collection to form an RDD. * - * @note Parallelize acts lazily. If `seq` is a mutable collection and is - * altered after the call to parallelize and before the first action on the - * RDD, the resultant RDD will reflect the modified collection. Pass a copy of - * the argument to avoid this. + * @note Parallelize acts lazily. If `seq` is a mutable collection and is altered after the call + * to parallelize and before the first action on the RDD, the resultant RDD will reflect the + * modified collection. Pass a copy of the argument to avoid this. */ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 87b22de6ae697..f12d0cffaba34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -111,7 +111,8 @@ private object ParallelCollectionRDD { /** * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes - * it efficient to run Spark over RDDs representing large sets of numbers. + * it efficient to run Spark over RDDs representing large sets of numbers. And if the collection + * is an inclusive Range, we use inclusive range for the last slice. */ def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { if (numSlices < 1) { @@ -127,19 +128,15 @@ private object ParallelCollectionRDD { }) } seq match { - case r: Range.Inclusive => { - val sign = if (r.step < 0) { - -1 - } else { - 1 - } - slice(new Range( - r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) - } case r: Range => { - positions(r.length, numSlices).map({ - case (start, end) => + positions(r.length, numSlices).zipWithIndex.map({ case ((start, end), index) => + // If the range is inclusive, use inclusive range for the last slice + if (r.isInclusive && index == numSlices - 1) { + new Range.Inclusive(r.start + start * r.step, r.end, r.step) + } + else { new Range(r.start + start * r.step, r.start + end * r.step, r.step) + } }).toSeq.asInstanceOf[Seq[Seq[T]]] } case nr: NumericRange[_] => { diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index 1b112f1a41ca9..cd193ae4f5238 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -76,6 +76,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(0).mkString(",") === (0 to 32).mkString(",")) assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) + assert(slices(2).isInstanceOf[Range.Inclusive]) } test("empty data") { @@ -227,4 +228,28 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } + + test("inclusive ranges with Int.MaxValue and Int.MinValue") { + val data1 = 1 to Int.MaxValue + val slices1 = ParallelCollectionRDD.slice(data1, 3) + assert(slices1.size === 3) + assert(slices1.map(_.size).sum === Int.MaxValue) + assert(slices1(2).isInstanceOf[Range.Inclusive]) + val data2 = -2 to Int.MinValue by -1 + val slices2 = ParallelCollectionRDD.slice(data2, 3) + assert(slices2.size == 3) + assert(slices2.map(_.size).sum === Int.MaxValue) + assert(slices2(2).isInstanceOf[Range.Inclusive]) + } + + test("empty ranges with Int.MaxValue and Int.MinValue") { + val data1 = Int.MaxValue until Int.MaxValue + val slices1 = ParallelCollectionRDD.slice(data1, 5) + assert(slices1.size === 5) + for (i <- 0 until 5) assert(slices1(i).size === 0) + val data2 = Int.MaxValue until Int.MaxValue + val slices2 = ParallelCollectionRDD.slice(data2, 5) + assert(slices2.size === 5) + for (i <- 0 until 5) assert(slices2(i).size === 0) + } } From f6b852aade7668c99f37c69f606c64763cb265d2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 16 Jan 2015 09:28:44 -0800 Subject: [PATCH 39/46] [DOCS] Fix typo in return type of cogroup This fixes a simple typo in the cogroup docs noted in http://mail-archives.apache.org/mod_mbox/spark-user/201501.mbox/%3CCAMAsSdJ8_24evMAMg7fOZCQjwimisbYWa9v8BN6Rc3JCauja6wmail.gmail.com%3E I didn't bother with a JIRA Author: Sean Owen Closes #4072 from srowen/CogroupDocFix and squashes the following commits: 43c850b [Sean Owen] Fix typo in return type of cogroup --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 5e0d5c15d7069..0211bbabc1132 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -913,7 +913,7 @@ for details. cogroup(otherDataset, [numTasks]) - When called on datasets of type (K, V) and (K, W), returns a dataset of (K, Iterable<V>, Iterable<W>) tuples. This operation is also called groupWith. + When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith. cartesian(otherDataset) From e8422c521bc76bc4b03c337605f136403ea9f64a Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 16 Jan 2015 10:05:11 -0800 Subject: [PATCH 40/46] [SPARK-5231][WebUI] History Server shows wrong job submission time. History Server doesn't show collect job submission time. It's because `JobProgressListener` updates job submission time every time `onJobStart` method is invoked from `ReplayListenerBus`. Author: Kousuke Saruta Closes #4029 from sarutak/SPARK-5231 and squashes the following commits: 0af9e22 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5231 da8bd14 [Kousuke Saruta] Made submissionTime in SparkListenerJobStartas and completionTime in SparkListenerJobEnd as regular Long 0412a6a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-5231 26b9b99 [Kousuke Saruta] Fixed the test cases 2d47bd3 [Kousuke Saruta] Fixed to record job submission time and completion time collectly --- .../apache/spark/scheduler/DAGScheduler.scala | 16 ++++++---- .../spark/scheduler/SparkListener.scala | 7 ++++- .../apache/spark/ui/jobs/AllJobsPage.scala | 15 +++++---- .../spark/ui/jobs/JobProgressListener.scala | 10 +++--- .../org/apache/spark/ui/jobs/UIData.scala | 6 ++-- .../org/apache/spark/util/JsonProtocol.scala | 11 +++++-- .../spark/scheduler/SparkListenerSuite.scala | 10 +++--- .../ui/jobs/JobProgressListenerSuite.scala | 6 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 31 ++++++++++++++++--- 9 files changed, 77 insertions(+), 35 deletions(-) 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 8cb15918baa8c..3bca59e0646d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -661,7 +661,7 @@ class DAGScheduler( // completion events or stage abort stageIdToStage -= s.id jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), jobResult)) } } @@ -710,7 +710,7 @@ class DAGScheduler( stage.latestInfo.stageFailed(stageFailedMessage) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) } } @@ -749,9 +749,11 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) val shouldRunLocally = localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 + val jobSubmissionTime = clock.getTime() if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobSubmissionTime, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job @@ -759,7 +761,8 @@ class DAGScheduler( finalStage.resultOfJob = Some(job) val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) - listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) submitStage(finalStage) } } @@ -965,7 +968,8 @@ class DAGScheduler( if (job.numFinished == job.numPartitions) { markStageAsFinished(stage) cleanupStateForJobAndIndependentStages(job) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) + listenerBus.post( + SparkListenerJobEnd(job.jobId, clock.getTime(), JobSucceeded)) } // taskSucceeded runs some user code that might throw an exception. Make sure @@ -1234,7 +1238,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) cleanupStateForJobAndIndependentStages(job) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 4840d8bd2d2f0..e5d1eb767e109 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -59,6 +59,7 @@ case class SparkListenerTaskEnd( @DeveloperApi case class SparkListenerJobStart( jobId: Int, + time: Long, stageInfos: Seq[StageInfo], properties: Properties = null) extends SparkListenerEvent { @@ -68,7 +69,11 @@ case class SparkListenerJobStart( } @DeveloperApi -case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +case class SparkListenerJobEnd( + jobId: Int, + time: Long, + jobResult: JobResult) + extends SparkListenerEvent @DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 1d1c701878447..81212708ba524 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -21,7 +21,6 @@ import scala.xml.{Node, NodeSeq} import javax.servlet.http.HttpServletRequest -import org.apache.spark.JobExecutionStatus import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData.JobUIData @@ -51,13 +50,13 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { - job.startTime.map { start => - val end = job.endTime.getOrElse(System.currentTimeMillis()) + job.submissionTime.map { start => + val end = job.completionTime.getOrElse(System.currentTimeMillis()) end - start } } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") - val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") val detailUrl = "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) @@ -68,7 +67,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
      {lastStageDescription}
      {lastStageName} - + {formattedSubmissionTime} {formattedDuration} @@ -101,11 +100,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val now = System.currentTimeMillis val activeJobsTable = - jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) + jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse) val completedJobsTable = - jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + jobsTable(completedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) val failedJobsTable = - jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + jobsTable(failedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) val shouldShowActiveJobs = activeJobs.nonEmpty val shouldShowCompletedJobs = completedJobs.nonEmpty diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 72935beb3a34a..b0d3bed1300b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -153,14 +153,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val jobData: JobUIData = new JobUIData( jobId = jobStart.jobId, - startTime = Some(System.currentTimeMillis), - endTime = None, + submissionTime = Option(jobStart.time).filter(_ >= 0), stageIds = jobStart.stageIds, jobGroup = jobGroup, status = JobExecutionStatus.RUNNING) // Compute (a potential underestimate of) the number of tasks that will be run by this job. // This may be an underestimate because the job start event references all of the result - // stages's transitive stage dependencies, but some of these stages might be skipped if their + // stages' transitive stage dependencies, but some of these stages might be skipped if their // output is available from earlier runs. // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. jobData.numTasks = { @@ -186,7 +185,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } - jobData.endTime = Some(System.currentTimeMillis()) + jobData.completionTime = Option(jobEnd.time).filter(_ >= 0) + jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData @@ -309,7 +309,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task - // compeletion event is for. Let's just drop it here. This means we might have some speculation + // completion event is for. Let's just drop it here. This means we might have some speculation // tasks on the web ui that's never marked as complete. if (info != null && taskEnd.stageAttemptId != -1) { val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 48fd7caa1a1ed..01f7e23212c3d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -40,15 +40,15 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, - var startTime: Option[Long] = None, - var endTime: Option[Long] = None, + var submissionTime: Option[Long] = None, + var completionTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, /* Tasks */ // `numTasks` is a potential underestimate of the true number of tasks that this job will run. // This may be an underestimate because the job start event references all of the result - // stages's transitive stage dependencies, but some of these stages might be skipped if their + // stages' transitive stage dependencies, but some of these stages might be skipped if their // output is available from earlier runs. // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. var numTasks: Int = 0, diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index ee3756c226fe3..76709a230f836 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,6 +32,7 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +import org.apache.hadoop.hdfs.web.JsonUtil /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- @@ -141,6 +142,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Submission Time" -> jobStart.time) ~ ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) @@ -150,6 +152,7 @@ private[spark] object JsonProtocol { val jobResult = jobResultToJson(jobEnd.jobResult) ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ ("Job ID" -> jobEnd.jobId) ~ + ("Completion Time" -> jobEnd.time) ~ ("Job Result" -> jobResult) } @@ -492,6 +495,8 @@ private[spark] object JsonProtocol { def jobStartFromJson(json: JValue): SparkListenerJobStart = { val jobId = (json \ "Job ID").extract[Int] + val submissionTime = + Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]).getOrElse(-1L) val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") // The "Stage Infos" field was added in Spark 1.2.0 @@ -499,13 +504,15 @@ private[spark] object JsonProtocol { .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) } - SparkListenerJobStart(jobId, stageInfos, properties) + SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { val jobId = (json \ "Job ID").extract[Int] + val completionTime = + Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]).getOrElse(-1L) val jobResult = jobResultFromJson(json \ "Job Result") - SparkListenerJobEnd(jobId, jobResult) + SparkListenerJobEnd(jobId, completionTime, jobResult) } def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 24f41bf8cccda..0fb1bdd30d975 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -34,6 +34,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 + val jobCompletionTime = 1421191296660L + before { sc = new SparkContext("local", "SparkListenerSuite") } @@ -44,7 +46,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(counter) // Listener bus hasn't started yet, so posting events should not increment counter - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(counter.count === 0) // Starting listener bus should flush all buffered events @@ -54,7 +56,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers // After listener bus has stopped, posting events should not increment counter bus.stop() - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(counter.count === 5) // Listener bus must not be started twice @@ -99,7 +101,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(blockingListener) bus.start() - bus.post(SparkListenerJobEnd(0, JobSucceeded)) + bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() // Listener should be blocked after start @@ -345,7 +347,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.start() // Post events to all listeners, and wait until the queue is drained - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) // The exception should be caught, and the event should be propagated to other listeners diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index f865d8ca04d1b..c9417ea1ed8f6 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -28,6 +28,8 @@ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers { + val jobSubmissionTime = 1421191042750L + val jobCompletionTime = 1421191296660L private def createStageStartEvent(stageId: Int) = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") @@ -46,12 +48,12 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val stageInfos = stageIds.map { stageId => new StageInfo(stageId, 0, stageId.toString, 0, null, "") } - SparkListenerJobStart(jobId, stageInfos) + SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { val result = if (failed) JobFailed(new Exception("dummy failure")) else JobSucceeded - SparkListenerJobEnd(jobId, result) + SparkListenerJobEnd(jobId, jobCompletionTime, result) } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 71dfed1289850..db400b4162910 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -34,6 +34,9 @@ import org.apache.spark.storage._ class JsonProtocolSuite extends FunSuite { + val jobSubmissionTime = 1421191042750L + val jobCompletionTime = 1421191296660L + test("SparkListenerEvent") { val stageSubmitted = SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) @@ -54,9 +57,9 @@ class JsonProtocolSuite extends FunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) - SparkListenerJobStart(10, stageInfos, properties) + SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) } - val jobEnd = SparkListenerJobEnd(20, JobSucceeded) + val jobEnd = SparkListenerJobEnd(20, jobCompletionTime, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), @@ -247,13 +250,31 @@ class JsonProtocolSuite extends FunSuite { val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) val dummyStageInfos = stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) - val jobStart = SparkListenerJobStart(10, stageInfos, properties) + val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = - SparkListenerJobStart(10, dummyStageInfos, properties) + SparkListenerJobStart(10, jobSubmissionTime, dummyStageInfos, properties) assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) } + test("SparkListenerJobStart and SparkListenerJobEnd backward compatibility") { + // Prior to Spark 1.3.0, SparkListenerJobStart did not have a "Submission Time" property. + // Also, SparkListenerJobEnd did not have a "Completion Time" property. + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => makeStageInfo(x * 10, x * 20, x * 30, x * 40, x * 50)) + val jobStart = SparkListenerJobStart(11, jobSubmissionTime, stageInfos, properties) + val oldStartEvent = JsonProtocol.jobStartToJson(jobStart) + .removeField({ _._1 == "Submission Time"}) + val expectedJobStart = SparkListenerJobStart(11, -1, stageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldStartEvent)) + + val jobEnd = SparkListenerJobEnd(11, jobCompletionTime, JobSucceeded) + val oldEndEvent = JsonProtocol.jobEndToJson(jobEnd) + .removeField({ _._1 == "Completion Time"}) + val expectedJobEnd = SparkListenerJobEnd(11, -1, JobSucceeded) + assertEquals(expectedJobEnd, JsonProtocol.jobEndFromJson(oldEndEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -1075,6 +1096,7 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Submission Time": 1421191042750, | "Stage Infos": [ | { | "Stage ID": 1, @@ -1349,6 +1371,7 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobEnd", | "Job ID": 20, + | "Completion Time": 1421191296660, | "Job Result": { | "Result": "JobSucceeded" | } From ecf943d35342191a1362d80bb26f2a098c152f27 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 16 Jan 2015 12:19:08 -0800 Subject: [PATCH 41/46] [WebUI] Fix collapse of WebUI layout When we decrease the width of browsers, the header of WebUI wraps and collapses like as following image. ![2015-01-11 19 49 37](https://cloud.githubusercontent.com/assets/4736016/5698887/b0b9aeee-99cd-11e4-9020-08f3f0014de0.png) Author: Kousuke Saruta Closes #3995 from sarutak/fixed-collapse-webui-layout and squashes the following commits: 3e60b5b [Kousuke Saruta] Modified line-height property in webui.css 7bfb5fb [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into fixed-collapse-webui-layout 5d83e18 [Kousuke Saruta] Fixed collapse of WebUI layout --- .../main/resources/org/apache/spark/ui/static/webui.css | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 5751964b792ce..f02b035a980b1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -19,6 +19,7 @@ height: 50px; font-size: 15px; margin-bottom: 15px; + min-width: 1200px } .navbar .navbar-inner { @@ -39,12 +40,12 @@ .navbar .nav > li a { height: 30px; - line-height: 30px; + line-height: 2; } .navbar-text { height: 50px; - line-height: 50px; + line-height: 3.3; } table.sortable thead { @@ -170,7 +171,7 @@ span.additional-metric-title { } .version { - line-height: 30px; + line-height: 2.5; vertical-align: bottom; font-size: 12px; padding: 0; From d05c9ee6e8441e54732e40de45d1d2311307908f Mon Sep 17 00:00:00 2001 From: Chip Senkbeil Date: Fri, 16 Jan 2015 12:56:40 -0800 Subject: [PATCH 42/46] [SPARK-4923][REPL] Add Developer API to REPL to allow re-publishing the REPL jar As requested in [SPARK-4923](https://issues.apache.org/jira/browse/SPARK-4923), I've provided a rough DeveloperApi for the repl. I've only done this for Scala 2.10 because it does not appear that Scala 2.11 is implemented. The Scala 2.11 repl still has the old `scala.tools.nsc` package and the SparkIMain does not appear to have the class server needed for shipping code over (unless this functionality has been moved elsewhere?). I also left alone the `ExecutorClassLoader` and `ConstructorCleaner` as I have no experience working with those classes. This marks the majority of methods in `SparkIMain` as _private_ with a few special cases being _private[repl]_ as other classes within the same package access them. Any public method has been marked with `DeveloperApi` as suggested by pwendell and I took the liberty of writing up a Scaladoc for each one to further elaborate their usage. As the Scala 2.11 REPL [conforms]((https://github.com/scala/scala/pull/2206)) to [JSR-223](http://docs.oracle.com/javase/8/docs/technotes/guides/scripting/), the [Spark Kernel](https://github.com/ibm-et/spark-kernel) uses the SparkIMain of Scala 2.10 in the same manner. So, I've taken care to expose methods predominately related to necessary functionality towards a JSR-223 scripting engine implementation. 1. The ability to _get_ variables from the interpreter (and other information like class/symbol/type) 2. The ability to _put_ variables into the interpreter 3. The ability to _compile_ code 4. The ability to _execute_ code 5. The ability to get contextual information regarding the scripting environment Additional functionality that I marked as exposed included the following: 1. The blocking initialization method (needed to actually start SparkIMain instance) 2. The class server uri (needed to set the _spark.repl.class.uri_ property after initialization), reduced from the entire class server 3. The class output directory (beneficial for tools like ours that need to inspect and use the directory where class files are served) 4. Suppression (quiet/silence) mechanics for output 5. Ability to add a jar to the compile/runtime classpath 6. The reset/close functionality 7. Metric information (last variable assignment, "needed" for extracting results from last execution, real variable name for better debugging) 8. Execution wrapper (useful to have, but debatable) Aside from `SparkIMain`, I updated other classes/traits and their methods in the _repl_ package to be private/package protected where possible. A few odd cases (like the SparkHelper being in the scala.tools.nsc package to expose a private variable) still exist, but I did my best at labelling them. `SparkCommandLine` has proven useful to extract settings and `SparkJLineCompletion` has proven to be useful in implementing auto-completion in the [Spark Kernel](https://github.com/ibm-et/spark-kernel) project. Other than those - and `SparkIMain` - my experience has yielded that other classes/methods are not necessary for interactive applications taking advantage of the REPL API. Tested via the following: $ export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" $ mvn -Phadoop-2.3 -DskipTests clean package && mvn -Phadoop-2.3 test Also did a quick verification that I could start the shell and execute some code: $ ./bin/spark-shell ... scala> val x = 3 x: Int = 3 scala> sc.parallelize(1 to 10).reduce(_+_) ... res1: Int = 55 Author: Chip Senkbeil Author: Chip Senkbeil Closes #4034 from rcsenkbeil/AddDeveloperApiToRepl and squashes the following commits: 053ca75 [Chip Senkbeil] Fixed failed build by adding missing DeveloperApi import c1b88aa [Chip Senkbeil] Added DeveloperApi to public classes in repl 6dc1ee2 [Chip Senkbeil] Added missing method to expose error reporting flag 26fd286 [Chip Senkbeil] Refactored other Scala 2.10 classes and methods to be private/package protected where possible 925c112 [Chip Senkbeil] Added DeveloperApi and Scaladocs to SparkIMain for Scala 2.10 --- .../apache/spark/repl/SparkCommandLine.scala | 9 +- .../apache/spark/repl/SparkExprTyper.scala | 2 +- .../org/apache/spark/repl/SparkHelper.scala | 17 + .../org/apache/spark/repl/SparkILoop.scala | 150 +++-- .../apache/spark/repl/SparkILoopInit.scala | 2 +- .../org/apache/spark/repl/SparkIMain.scala | 592 ++++++++++++++---- .../org/apache/spark/repl/SparkImports.scala | 2 +- .../spark/repl/SparkJLineCompletion.scala | 56 +- .../apache/spark/repl/SparkJLineReader.scala | 4 +- .../spark/repl/SparkMemberHandlers.scala | 2 +- .../spark/repl/SparkRunnerSettings.scala | 3 +- 11 files changed, 644 insertions(+), 195 deletions(-) diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala index 05816941b54b3..6480e2d24e044 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala @@ -19,14 +19,21 @@ package org.apache.spark.repl import scala.tools.nsc.{Settings, CompilerCommand} import scala.Predef._ +import org.apache.spark.annotation.DeveloperApi /** * Command class enabling Spark-specific command line options (provided by * org.apache.spark.repl.SparkRunnerSettings). + * + * @example new SparkCommandLine(Nil).settings + * + * @param args The list of command line arguments + * @param settings The underlying settings to associate with this set of + * command-line options */ +@DeveloperApi class SparkCommandLine(args: List[String], override val settings: Settings) extends CompilerCommand(args, settings) { - def this(args: List[String], error: String => Unit) { this(args, new SparkRunnerSettings(error)) } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index f8432c8af6ed2..5fb378112ef92 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -15,7 +15,7 @@ import scala.tools.nsc.ast.parser.Tokens.EOF import org.apache.spark.Logging -trait SparkExprTyper extends Logging { +private[repl] trait SparkExprTyper extends Logging { val repl: SparkIMain import repl._ diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala index 5340951d91331..955be17a73b85 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala @@ -17,6 +17,23 @@ package scala.tools.nsc +import org.apache.spark.annotation.DeveloperApi + +// NOTE: Forced to be public (and in scala.tools.nsc package) to access the +// settings "explicitParentLoader" method + +/** + * Provides exposure for the explicitParentLoader method on settings instances. + */ +@DeveloperApi object SparkHelper { + /** + * Retrieves the explicit parent loader for the provided settings. + * + * @param settings The settings whose explicit parent loader to retrieve + * + * @return The Optional classloader representing the explicit parent loader + */ + @DeveloperApi def explicitParentLoader(settings: Settings) = settings.explicitParentLoader } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index e56b74edba88c..72c1a989999b4 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -10,6 +10,8 @@ package org.apache.spark.repl import java.net.URL +import org.apache.spark.annotation.DeveloperApi + import scala.reflect.io.AbstractFile import scala.tools.nsc._ import scala.tools.nsc.backend.JavaPlatform @@ -57,20 +59,22 @@ import org.apache.spark.util.Utils * @author Lex Spoon * @version 1.2 */ -class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, - val master: Option[String]) - extends AnyRef - with LoopCommands - with SparkILoopInit - with Logging -{ +@DeveloperApi +class SparkILoop( + private val in0: Option[BufferedReader], + protected val out: JPrintWriter, + val master: Option[String] +) extends AnyRef with LoopCommands with SparkILoopInit with Logging { def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) def this() = this(None, new JPrintWriter(Console.out, true), None) - var in: InteractiveReader = _ // the input stream from which commands come - var settings: Settings = _ - var intp: SparkIMain = _ + private var in: InteractiveReader = _ // the input stream from which commands come + + // NOTE: Exposed in package for testing + private[repl] var settings: Settings = _ + + private[repl] var intp: SparkIMain = _ @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i @@ -123,6 +127,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } + // NOTE: Must be public for visibility + @DeveloperApi var sparkContext: SparkContext = _ override def echoCommandMessage(msg: String) { @@ -130,45 +136,45 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } // def isAsync = !settings.Yreplsync.value - def isAsync = false + private[repl] def isAsync = false // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) - def history = in.history + private def history = in.history /** The context class loader at the time this object was created */ protected val originalClassLoader = Utils.getContextOrSparkClassLoader // classpath entries added via :cp - var addedClasspath: String = "" + private var addedClasspath: String = "" /** A reverse list of commands to replay if the user requests a :replay */ - var replayCommandStack: List[String] = Nil + private var replayCommandStack: List[String] = Nil /** A list of commands to replay if the user requests a :replay */ - def replayCommands = replayCommandStack.reverse + private def replayCommands = replayCommandStack.reverse /** Record a command for replay should the user request a :replay */ - def addReplay(cmd: String) = replayCommandStack ::= cmd + private def addReplay(cmd: String) = replayCommandStack ::= cmd - def savingReplayStack[T](body: => T): T = { + private def savingReplayStack[T](body: => T): T = { val saved = replayCommandStack try body finally replayCommandStack = saved } - def savingReader[T](body: => T): T = { + private def savingReader[T](body: => T): T = { val saved = in try body finally in = saved } - def sparkCleanUp(){ + private def sparkCleanUp(){ echo("Stopping spark context.") intp.beQuietDuring { command("sc.stop()") } } /** Close the interpreter and set the var to null. */ - def closeInterpreter() { + private def closeInterpreter() { if (intp ne null) { sparkCleanUp() intp.close() @@ -179,14 +185,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, class SparkILoopInterpreter extends SparkIMain(settings, out) { outer => - override lazy val formatting = new Formatting { + override private[repl] lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } - /** Create a new interpreter. */ - def createInterpreter() { + /** + * Constructs a new interpreter. + */ + protected def createInterpreter() { require(settings != null) if (addedClasspath != "") settings.classpath.append(addedClasspath) @@ -207,7 +215,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** print a friendly help message */ - def helpCommand(line: String): Result = { + private def helpCommand(line: String): Result = { if (line == "") helpSummary() else uniqueCommand(line) match { case Some(lc) => echo("\n" + lc.longHelp) @@ -258,7 +266,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** Show the history */ - lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { + private lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 @@ -279,21 +287,21 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // When you know you are most likely breaking into the middle // of a line being typed. This softens the blow. - protected def echoAndRefresh(msg: String) = { + private[repl] def echoAndRefresh(msg: String) = { echo("\n" + msg) in.redrawLine() } - protected def echo(msg: String) = { + private[repl] def echo(msg: String) = { out println msg out.flush() } - protected def echoNoNL(msg: String) = { + private def echoNoNL(msg: String) = { out print msg out.flush() } /** Search the history */ - def searchHistory(_cmdline: String) { + private def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 @@ -302,14 +310,27 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } private var currentPrompt = Properties.shellPromptString + + /** + * Sets the prompt string used by the REPL. + * + * @param prompt The new prompt string + */ + @DeveloperApi def setPrompt(prompt: String) = currentPrompt = prompt - /** Prompt to print when awaiting input */ + + /** + * Represents the current prompt string used by the REPL. + * + * @return The current prompt string + */ + @DeveloperApi def prompt = currentPrompt import LoopCommand.{ cmd, nullary } /** Standard commands */ - lazy val standardCommands = List( + private lazy val standardCommands = List( cmd("cp", "", "add a jar or directory to the classpath", addClasspath), cmd("help", "[command]", "print this summary or command-specific help", helpCommand), historyCommand, @@ -333,7 +354,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, ) /** Power user commands */ - lazy val powerCommands: List[LoopCommand] = List( + private lazy val powerCommands: List[LoopCommand] = List( // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) ) @@ -459,7 +480,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { + private def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { override def tryClass(path: String): Array[Byte] = { val hd :: rest = path split '.' toList; // If there are dots in the name, the first segment is the @@ -581,7 +602,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // } // } - /** Available commands */ + /** + * Provides a list of available commands. + * + * @return The list of commands + */ + @DeveloperApi def commands: List[LoopCommand] = standardCommands /*++ ( if (isReplPower) powerCommands else Nil )*/ @@ -613,7 +639,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * command() for each line of input, and stops when * command() returns false. */ - def loop() { + private def loop() { def readOneLine() = { out.flush() in readLine prompt @@ -642,7 +668,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { + private def interpretAllFrom(file: File) { savingReader { savingReplayStack { file applyReader { reader => @@ -655,7 +681,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } /** create a new interpreter and replay the given commands */ - def replay() { + private def replay() { reset() if (replayCommandStack.isEmpty) echo("Nothing to replay.") @@ -665,7 +691,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, echo("") } } - def resetCommand() { + private def resetCommand() { echo("Resetting repl state.") if (replayCommandStack.nonEmpty) { echo("Forgetting this session history:\n") @@ -681,13 +707,13 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, reset() } - def reset() { + private def reset() { intp.reset() // unleashAndSetPhase() } /** fork a shell and run a command */ - lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { + private lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { override def usage = "" def apply(line: String): Result = line match { case "" => showUsage() @@ -698,14 +724,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - def withFile(filename: String)(action: File => Unit) { + private def withFile(filename: String)(action: File => Unit) { val f = File(filename) if (f.exists) action(f) else echo("That file does not exist") } - def loadCommand(arg: String) = { + private def loadCommand(arg: String) = { var shouldReplay: Option[String] = None withFile(arg)(f => { interpretAllFrom(f) @@ -714,7 +740,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, Result(true, shouldReplay) } - def addAllClasspath(args: Seq[String]): Unit = { + private def addAllClasspath(args: Seq[String]): Unit = { var added = false var totalClasspath = "" for (arg <- args) { @@ -729,7 +755,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - def addClasspath(arg: String): Unit = { + private def addClasspath(arg: String): Unit = { val f = File(arg).normalize if (f.exists) { addedClasspath = ClassPath.join(addedClasspath, f.path) @@ -741,12 +767,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } - def powerCmd(): Result = { + private def powerCmd(): Result = { if (isReplPower) "Already in power mode." else enablePowerMode(false) } - def enablePowerMode(isDuringInit: Boolean) = { + private[repl] def enablePowerMode(isDuringInit: Boolean) = { // replProps.power setValue true // unleashAndSetPhase() // asyncEcho(isDuringInit, power.banner) @@ -759,12 +785,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, // } // } - def asyncEcho(async: Boolean, msg: => String) { + private def asyncEcho(async: Boolean, msg: => String) { if (async) asyncMessage(msg) else echo(msg) } - def verbosity() = { + private def verbosity() = { // val old = intp.printResults // intp.printResults = !old // echo("Switched " + (if (old) "off" else "on") + " result printing.") @@ -773,7 +799,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, /** Run one command submitted by the user. Two values are returned: * (1) whether to keep running, (2) the line to record for replay, * if any. */ - def command(line: String): Result = { + private[repl] def command(line: String): Result = { if (line startsWith ":") { val cmd = line.tail takeWhile (x => !x.isWhitespace) uniqueCommand(cmd) match { @@ -789,7 +815,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - def pasteCommand(): Result = { + private def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" echo("\n// Exiting paste mode, now interpreting.\n") @@ -820,7 +846,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * read, go ahead and interpret it. Return the full string * to be recorded for replay, if any. */ - def interpretStartingWith(code: String): Option[String] = { + private def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() @@ -874,7 +900,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } // runs :load `file` on any files passed via -i - def loadFiles(settings: Settings) = settings match { + private def loadFiles(settings: Settings) = settings match { case settings: SparkRunnerSettings => for (filename <- settings.loadfiles.value) { val cmd = ":load " + filename @@ -889,7 +915,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, * unless settings or properties are such that it should start * with SimpleReader. */ - def chooseReader(settings: Settings): InteractiveReader = { + private def chooseReader(settings: Settings): InteractiveReader = { if (settings.Xnojline.value || Properties.isEmacsShell) SimpleReader() else try new SparkJLineReader( @@ -903,8 +929,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } - val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - val m = u.runtimeMirror(Utils.getSparkClassLoader) + private val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + private val m = u.runtimeMirror(Utils.getSparkClassLoader) private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = u.TypeTag[T]( m, @@ -913,7 +939,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] }) - def process(settings: Settings): Boolean = savingContextLoader { + private def process(settings: Settings): Boolean = savingContextLoader { if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") this.settings = settings @@ -972,6 +998,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, true } + // NOTE: Must be public for visibility + @DeveloperApi def createSparkContext(): SparkContext = { val execUri = System.getenv("SPARK_EXECUTOR_URI") val jars = SparkILoop.getAddedJars @@ -979,7 +1007,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, .setMaster(getMaster()) .setAppName("Spark shell") .setJars(jars) - .set("spark.repl.class.uri", intp.classServer.uri) + .set("spark.repl.class.uri", intp.classServerUri) if (execUri != null) { conf.set("spark.executor.uri", execUri) } @@ -1014,7 +1042,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } @deprecated("Use `process` instead", "2.9.0") - def main(settings: Settings): Unit = process(settings) + private def main(settings: Settings): Unit = process(settings) } object SparkILoop { @@ -1033,7 +1061,7 @@ object SparkILoop { // Designed primarily for use by test code: take a String with a // bunch of code, and prints out a transcript of what it would look // like if you'd just typed it into the repl. - def runForTranscript(code: String, settings: Settings): String = { + private[repl] def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } stringFromStream { ostream => @@ -1071,7 +1099,7 @@ object SparkILoop { /** Creates an interpreter loop with default settings and feeds * the given code to it as input. */ - def run(code: String, sets: Settings = new Settings): String = { + private[repl] def run(code: String, sets: Settings = new Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } stringFromStream { ostream => @@ -1087,5 +1115,5 @@ object SparkILoop { } } } - def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) + private[repl] def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) } diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index da4286c5e4874..99bd777c04fdb 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -19,7 +19,7 @@ import org.apache.spark.SPARK_VERSION /** * Machinery for the asynchronous initialization of the repl. */ -trait SparkILoopInit { +private[repl] trait SparkILoopInit { self: SparkILoop => /** Print a welcome message */ diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala index b646f0b6f0868..35fb625645022 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -39,6 +39,7 @@ import scala.util.control.ControlThrowable import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf} import org.apache.spark.util.Utils +import org.apache.spark.annotation.DeveloperApi // /** directory to save .class files to */ // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { @@ -84,17 +85,18 @@ import org.apache.spark.util.Utils * @author Moez A. Abdel-Gawad * @author Lex Spoon */ + @DeveloperApi class SparkIMain( initialSettings: Settings, val out: JPrintWriter, propagateExceptions: Boolean = false) extends SparkImports with Logging { imain => - val conf = new SparkConf() + private val conf = new SparkConf() - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") + private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") /** Local directory to save .class files too */ - lazy val outputDir = { + private lazy val outputDir = { val tmp = System.getProperty("java.io.tmpdir") val rootDir = conf.get("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) @@ -103,13 +105,20 @@ import org.apache.spark.util.Utils echo("Output directory: " + outputDir) } - val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles + /** + * Returns the path to the output directory containing all generated + * class files that will be served by the REPL class server. + */ + @DeveloperApi + lazy val getClassOutputDirectory = outputDir + + private val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles /** Jetty server that will serve our classes to worker nodes */ - val classServerPort = conf.getInt("spark.replClassServer.port", 0) - val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") + private val classServerPort = conf.getInt("spark.replClassServer.port", 0) + private val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") private var currentSettings: Settings = initialSettings - var printResults = true // whether to print result lines - var totalSilence = false // whether to print anything + private var printResults = true // whether to print result lines + private var totalSilence = false // whether to print anything private var _initializeComplete = false // compiler is initialized private var _isInitialized: Future[Boolean] = null // set up initialization future private var bindExceptions = true // whether to bind the lastException variable @@ -123,6 +132,14 @@ import org.apache.spark.util.Utils echo("Class server started, URI = " + classServer.uri) } + /** + * URI of the class server used to feed REPL compiled classes. + * + * @return The string representing the class server uri + */ + @DeveloperApi + def classServerUri = classServer.uri + /** We're going to go to some trouble to initialize the compiler asynchronously. * It's critical that nothing call into it until it's been initialized or we will * run into unrecoverable issues, but the perceived repl startup time goes @@ -141,17 +158,18 @@ import org.apache.spark.util.Utils () => { counter += 1 ; counter } } - def compilerClasspath: Seq[URL] = ( + private def compilerClasspath: Seq[URL] = ( if (isInitializeComplete) global.classPath.asURLs else new PathResolver(settings).result.asURLs // the compiler's classpath ) - def settings = currentSettings - def mostRecentLine = prevRequestList match { + // NOTE: Exposed to repl package since accessed indirectly from SparkIMain + private[repl] def settings = currentSettings + private def mostRecentLine = prevRequestList match { case Nil => "" case req :: _ => req.originalLine } // Run the code body with the given boolean settings flipped to true. - def withoutWarnings[T](body: => T): T = beQuietDuring { + private def withoutWarnings[T](body: => T): T = beQuietDuring { val saved = settings.nowarn.value if (!saved) settings.nowarn.value = true @@ -164,16 +182,28 @@ import org.apache.spark.util.Utils def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) def this() = this(new Settings()) - lazy val repllog: Logger = new Logger { + private lazy val repllog: Logger = new Logger { val out: JPrintWriter = imain.out val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" } - lazy val formatting: Formatting = new Formatting { + private[repl] lazy val formatting: Formatting = new Formatting { val prompt = Properties.shellPromptString } - lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + // NOTE: Exposed to repl package since used by SparkExprTyper and SparkILoop + private[repl] lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + /** + * Determines if errors were reported (typically during compilation). + * + * @note This is not for runtime errors + * + * @return True if had errors, otherwise false + */ + @DeveloperApi + def isReportingErrors = reporter.hasErrors import formatting._ import reporter.{ printMessage, withoutTruncating } @@ -193,7 +223,8 @@ import org.apache.spark.util.Utils private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" // argument is a thunk to execute after init is done - def initialize(postInitSignal: => Unit) { + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def initialize(postInitSignal: => Unit) { synchronized { if (_isInitialized == null) { _isInitialized = io.spawn { @@ -203,15 +234,27 @@ import org.apache.spark.util.Utils } } } + + /** + * Initializes the underlying compiler/interpreter in a blocking fashion. + * + * @note Must be executed before using SparkIMain! + */ + @DeveloperApi def initializeSynchronous(): Unit = { if (!isInitializeComplete) { _initialize() assert(global != null, global) } } - def isInitializeComplete = _initializeComplete + private def isInitializeComplete = _initializeComplete /** the public, go through the future compiler */ + + /** + * The underlying compiler used to generate ASTs and execute code. + */ + @DeveloperApi lazy val global: Global = { if (isInitializeComplete) _compiler else { @@ -226,13 +269,13 @@ import org.apache.spark.util.Utils } } @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - lazy val compiler: global.type = global + private lazy val compiler: global.type = global import global._ import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} - implicit class ReplTypeOps(tp: Type) { + private implicit class ReplTypeOps(tp: Type) { def orElse(other: => Type): Type = if (tp ne NoType) tp else other def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) } @@ -240,7 +283,8 @@ import org.apache.spark.util.Utils // TODO: If we try to make naming a lazy val, we run into big time // scalac unhappiness with what look like cycles. It has not been easy to // reduce, but name resolution clearly takes different paths. - object naming extends { + // NOTE: Exposed to repl package since used by SparkExprTyper + private[repl] object naming extends { val global: imain.global.type = imain.global } with Naming { // make sure we don't overwrite their unwisely named res3 etc. @@ -254,22 +298,43 @@ import org.apache.spark.util.Utils } import naming._ - object deconstruct extends { + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] object deconstruct extends { val global: imain.global.type = imain.global } with StructuredTypeStrings - lazy val memberHandlers = new { + // NOTE: Exposed to repl package since used by SparkImports + private[repl] lazy val memberHandlers = new { val intp: imain.type = imain } with SparkMemberHandlers import memberHandlers._ - /** Temporarily be quiet */ + /** + * Suppresses overwriting print results during the operation. + * + * @param body The block to execute + * @tparam T The return type of the block + * + * @return The result from executing the block + */ + @DeveloperApi def beQuietDuring[T](body: => T): T = { val saved = printResults printResults = false try body finally printResults = saved } + + /** + * Completely masks all output during the operation (minus JVM standard + * out and error). + * + * @param operation The block to execute + * @tparam T The return type of the block + * + * @return The result from executing the block + */ + @DeveloperApi def beSilentDuring[T](operation: => T): T = { val saved = totalSilence totalSilence = true @@ -277,10 +342,10 @@ import org.apache.spark.util.Utils finally totalSilence = saved } - def quietRun[T](code: String) = beQuietDuring(interpret(code)) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { + private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { case t: ControlThrowable => throw t case t: Throwable => logDebug(label + ": " + unwrap(t)) @@ -298,14 +363,44 @@ import org.apache.spark.util.Utils finally bindExceptions = true } + /** + * Contains the code (in string form) representing a wrapper around all + * code executed by this instance. + * + * @return The wrapper code as a string + */ + @DeveloperApi def executionWrapper = _executionWrapper + + /** + * Sets the code to use as a wrapper around all code executed by this + * instance. + * + * @param code The wrapper code as a string + */ + @DeveloperApi def setExecutionWrapper(code: String) = _executionWrapper = code + + /** + * Clears the code used as a wrapper around all code executed by + * this instance. + */ + @DeveloperApi def clearExecutionWrapper() = _executionWrapper = "" /** interpreter settings */ - lazy val isettings = new SparkISettings(this) + private lazy val isettings = new SparkISettings(this) - /** Instantiate a compiler. Overridable. */ + /** + * Instantiates a new compiler used by SparkIMain. Overridable to provide + * own instance of a compiler. + * + * @param settings The settings to provide the compiler + * @param reporter The reporter to use for compiler output + * + * @return The compiler as a Global + */ + @DeveloperApi protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { settings.outputDirs setSingleOutput virtualDirectory settings.exposeEmptyPackage.value = true @@ -320,13 +415,14 @@ import org.apache.spark.util.Utils * @note Currently only supports jars, not directories * @param urls The list of items to add to the compile and runtime classpaths */ + @DeveloperApi def addUrlsToClassPath(urls: URL*): Unit = { new Run // Needed to force initialization of "something" to correctly load Scala classes from jars urls.foreach(_runtimeClassLoader.addNewUrl) // Add jars/classes to runtime for execution updateCompilerClassPath(urls: _*) // Add jars/classes to compile time for compiling } - protected def updateCompilerClassPath(urls: URL*): Unit = { + private def updateCompilerClassPath(urls: URL*): Unit = { require(!global.forMSIL) // Only support JavaPlatform val platform = global.platform.asInstanceOf[JavaPlatform] @@ -342,7 +438,7 @@ import org.apache.spark.util.Utils global.invalidateClassPathEntries(urls.map(_.getPath): _*) } - protected def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { + private def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { // Collect our new jars/directories and add them to the existing set of classpaths val allClassPaths = ( platform.classPath.asInstanceOf[MergedClassPath[AbstractFile]].entries ++ @@ -365,7 +461,13 @@ import org.apache.spark.util.Utils new MergedClassPath(allClassPaths, platform.classPath.context) } - /** Parent classloader. Overridable. */ + /** + * Represents the parent classloader used by this instance. Can be + * overridden to provide alternative classloader. + * + * @return The classloader used as the parent loader of this instance + */ + @DeveloperApi protected def parentClassLoader: ClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) @@ -382,16 +484,18 @@ import org.apache.spark.util.Utils shadow the old ones, and old code objects refer to the old definitions. */ - def resetClassLoader() = { + private def resetClassLoader() = { logDebug("Setting new classloader: was " + _classLoader) _classLoader = null ensureClassLoader() } - final def ensureClassLoader() { + private final def ensureClassLoader() { if (_classLoader == null) _classLoader = makeClassLoader() } - def classLoader: AbstractFileClassLoader = { + + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def classLoader: AbstractFileClassLoader = { ensureClassLoader() _classLoader } @@ -418,27 +522,58 @@ import org.apache.spark.util.Utils _runtimeClassLoader }) - def getInterpreterClassLoader() = classLoader + private def getInterpreterClassLoader() = classLoader // Set the current Java "context" class loader to this interpreter's class loader - def setContextClassLoader() = classLoader.setAsContext() + // NOTE: Exposed to repl package since used by SparkILoopInit + private[repl] def setContextClassLoader() = classLoader.setAsContext() - /** Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * {{{ - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - * }}} + /** + * Returns the real name of a class based on its repl-defined name. + * + * ==Example== + * Given a simple repl-defined name, returns the real name of + * the class representing it, e.g. for "Bippy" it may return + * {{{ + * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy + * }}} + * + * @param simpleName The repl-defined name whose real name to retrieve + * + * @return Some real name if the simple name exists, else None */ + @DeveloperApi def generatedName(simpleName: String): Option[String] = { if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) else optFlatName(simpleName) } - def flatName(id: String) = optFlatName(id) getOrElse id - def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def flatName(id: String) = optFlatName(id) getOrElse id + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + + /** + * Retrieves all simple names contained in the current instance. + * + * @return A list of sorted names + */ + @DeveloperApi def allDefinedNames = definedNameMap.keys.toList.sorted - def pathToType(id: String): String = pathToName(newTypeName(id)) - def pathToTerm(id: String): String = pathToName(newTermName(id)) + + private def pathToType(id: String): String = pathToName(newTypeName(id)) + // NOTE: Exposed to repl package since used by SparkILoop + private[repl] def pathToTerm(id: String): String = pathToName(newTermName(id)) + + /** + * Retrieves the full code path to access the specified simple name + * content. + * + * @param name The simple name of the target whose path to determine + * + * @return The full path used to access the specified target (name) + */ + @DeveloperApi def pathToName(name: Name): String = { if (definedNameMap contains name) definedNameMap(name) fullPath name @@ -457,13 +592,13 @@ import org.apache.spark.util.Utils } /** Stubs for work in progress. */ - def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { + private def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) } } - def handleTermRedefinition(name: TermName, old: Request, req: Request) = { + private def handleTermRedefinition(name: TermName, old: Request, req: Request) = { for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { // Printing the types here has a tendency to cause assertion errors, like // assertion failed: fatal: has owner value x, but a class owner is required @@ -473,7 +608,7 @@ import org.apache.spark.util.Utils } } - def recordRequest(req: Request) { + private def recordRequest(req: Request) { if (req == null || referencedNameMap == null) return @@ -504,12 +639,12 @@ import org.apache.spark.util.Utils } } - def replwarn(msg: => String) { + private def replwarn(msg: => String) { if (!settings.nowarnings.value) printMessage(msg) } - def isParseable(line: String): Boolean = { + private def isParseable(line: String): Boolean = { beSilentDuring { try parse(line) match { case Some(xs) => xs.nonEmpty // parses as-is @@ -522,22 +657,32 @@ import org.apache.spark.util.Utils } } - def compileSourcesKeepingRun(sources: SourceFile*) = { + private def compileSourcesKeepingRun(sources: SourceFile*) = { val run = new Run() reporter.reset() run compileSources sources.toList (!reporter.hasErrors, run) } - /** Compile an nsc SourceFile. Returns true if there are - * no compilation errors, or false otherwise. + /** + * Compiles specified source files. + * + * @param sources The sequence of source files to compile + * + * @return True if successful, otherwise false */ + @DeveloperApi def compileSources(sources: SourceFile*): Boolean = compileSourcesKeepingRun(sources: _*)._1 - /** Compile a string. Returns true if there are no - * compilation errors, or false otherwise. + /** + * Compiles a string of code. + * + * @param code The string of code to compile + * + * @return True if successful, otherwise false */ + @DeveloperApi def compileString(code: String): Boolean = compileSources(new BatchSourceFile("