diff --git a/bin/pyspark b/bin/pyspark
index 1d8c94d43d285..0b4f695dd06dd 100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@ -132,7 +132,5 @@ if [[ "$1" =~ \.py$ ]]; then
gatherSparkSubmitOpts "$@"
exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}"
else
- # PySpark shell requires special handling downstream
- export PYSPARK_SHELL=1
exec "$PYSPARK_DRIVER_PYTHON" $PYSPARK_DRIVER_PYTHON_OPTS
fi
diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd
index 59415e9bdec2c..a542ec80b49d6 100644
--- a/bin/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -59,7 +59,6 @@ for /f %%i in ('echo %1^| findstr /R "\.py"') do (
)
if [%PYTHON_FILE%] == [] (
- set PYSPARK_SHELL=1
if [%IPYTHON%] == [1] (
ipython %IPYTHON_OPTS%
) else (
diff --git a/bin/spark-submit b/bin/spark-submit
index c557311b4b20e..f92d90c3a66b0 100755
--- a/bin/spark-submit
+++ b/bin/spark-submit
@@ -22,6 +22,9 @@
export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
ORIG_ARGS=("$@")
+# Set COLUMNS for progress bar
+export COLUMNS=`tput cols`
+
while (($#)); do
if [ "$1" = "--deploy-mode" ]; then
SPARK_SUBMIT_DEPLOY_MODE=$2
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index f8ffbf64278fb..0886b0276fb90 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -28,7 +28,7 @@
# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job.
# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job.
-# Options for the daemons used in the standalone deploy mode:
+# Options for the daemons used in the standalone deploy mode
# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname
# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master
# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y")
@@ -41,3 +41,10 @@
# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y")
# - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y")
# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers
+
+# Generic options for the daemons used in the standalone deploy mode
+# - SPARK_CONF_DIR Alternate conf dir. (Default: ${SPARK_HOME}/conf)
+# - SPARK_LOG_DIR Where log files are stored. (Default: ${SPARK_HOME}/logs)
+# - SPARK_PID_DIR Where the pid file is stored. (Default: /tmp)
+# - SPARK_IDENT_STRING A string representing this instance of spark. (Default: $USER)
+# - SPARK_NICENESS The scheduling priority for daemons. (Default: 0)
diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java
index 04e2247210ecc..fd74321093658 100644
--- a/core/src/main/java/org/apache/spark/SparkStageInfo.java
+++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java
@@ -26,6 +26,7 @@
public interface SparkStageInfo {
int stageId();
int currentAttemptId();
+ long submissionTime();
String name();
int numTasks();
int numActiveTasks();
diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala
index 7f91de653a64a..0f9bac7164162 100644
--- a/core/src/main/java/org/apache/spark/api/java/function/package.scala
+++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala
@@ -22,4 +22,4 @@ package org.apache.spark.api.java
* these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's
* Java programming guide for more details.
*/
-package object function
\ No newline at end of file
+package object function
diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
index badd85ed48c82..d33c5c769d683 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js
@@ -26,13 +26,6 @@ $(function() {
// Switch the class of the arrow from open to closed.
$(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open');
$(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed');
-
- // If clicking caused the metrics to expand, automatically check all options for additional
- // metrics (don't trigger a click when collapsing metrics, because it leads to weird
- // toggling behavior).
- if (!$(additionalMetricsDiv).hasClass('collapsed')) {
- $(this).parent().find('input:checkbox:not(:checked)').trigger('click');
- }
});
$("input:checkbox:not(:checked)").each(function() {
@@ -48,6 +41,16 @@ $(function() {
stripeTables();
});
+ $("#select-all-metrics").click(function() {
+ if (this.checked) {
+ // Toggle all un-checked options.
+ $('input:checkbox:not(:checked)').trigger('click');
+ } else {
+ // Toggle all checked options.
+ $('input:checkbox:checked').trigger('click');
+ }
+ });
+
// Trigger a click on the checkbox if a user clicks the label next to it.
$("span.additional-metric-title").click(function() {
$(this).parent().find('input:checkbox').trigger('click');
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index ef93009a074e7..88adb892998af 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -28,7 +28,9 @@ import org.apache.spark.scheduler._
* the scheduler queue is not drained in N seconds, then new executors are added. If the queue
* persists for another M seconds, then more executors are added and so on. The number added
* in each round increases exponentially from the previous round until an upper bound on the
- * number of executors has been reached.
+ * number of executors has been reached. The upper bound is based both on a configured property
+ * and on the number of tasks pending: the policy will never increase the number of executor
+ * requests past the number needed to handle all pending tasks.
*
* The rationale for the exponential increase is twofold: (1) Executors should be added slowly
* in the beginning in case the number of extra executors needed turns out to be small. Otherwise,
@@ -82,6 +84,12 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
// During testing, the methods to actually kill and add executors are mocked out
private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false)
+ // TODO: The default value of 1 for spark.executor.cores works right now because dynamic
+ // allocation is only supported for YARN and the default number of cores per executor in YARN is
+ // 1, but it might need to be attained differently for different cluster managers
+ private val tasksPerExecutor =
+ conf.getInt("spark.executor.cores", 1) / conf.getInt("spark.task.cpus", 1)
+
validateSettings()
// Number of executors to add in the next round
@@ -110,6 +118,9 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
// Clock used to schedule when executors should be added and removed
private var clock: Clock = new RealClock
+ // Listener for Spark events that impact the allocation policy
+ private val listener = new ExecutorAllocationListener(this)
+
/**
* Verify that the settings specified through the config are valid.
* If not, throw an appropriate exception.
@@ -141,6 +152,9 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
throw new SparkException("Dynamic allocation of executors requires the external " +
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
}
+ if (tasksPerExecutor == 0) {
+ throw new SparkException("spark.executor.cores must not be less than spark.task.cpus.cores")
+ }
}
/**
@@ -154,7 +168,6 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
* Register for scheduler callbacks to decide when to add and remove executors.
*/
def start(): Unit = {
- val listener = new ExecutorAllocationListener(this)
sc.addSparkListener(listener)
startPolling()
}
@@ -218,13 +231,27 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
return 0
}
- // Request executors with respect to the upper bound
- val actualNumExecutorsToAdd =
- if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) {
- numExecutorsToAdd
- } else {
- maxNumExecutors - numExistingExecutors
- }
+ // The number of executors needed to satisfy all pending tasks is the number of tasks pending
+ // divided by the number of tasks each executor can fit, rounded up.
+ val maxNumExecutorsPending =
+ (listener.totalPendingTasks() + tasksPerExecutor - 1) / tasksPerExecutor
+ if (numExecutorsPending >= maxNumExecutorsPending) {
+ logDebug(s"Not adding executors because there are already $numExecutorsPending " +
+ s"pending and pending tasks could only fill $maxNumExecutorsPending")
+ numExecutorsToAdd = 1
+ return 0
+ }
+
+ // It's never useful to request more executors than could satisfy all the pending tasks, so
+ // cap request at that amount.
+ // Also cap request with respect to the configured upper bound.
+ val maxNumExecutorsToAdd = math.min(
+ maxNumExecutorsPending - numExecutorsPending,
+ maxNumExecutors - numExistingExecutors)
+ assert(maxNumExecutorsToAdd > 0)
+
+ val actualNumExecutorsToAdd = math.min(numExecutorsToAdd, maxNumExecutorsToAdd)
+
val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd
val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd)
if (addRequestAcknowledged) {
@@ -445,6 +472,16 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging
blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = {
allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId)
}
+
+ /**
+ * 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.
+ */
+ def totalPendingTasks(): Int = {
+ stageIdToNumTasks.map { case (stageId, numTasks) =>
+ numTasks - stageIdToTaskIndices.get(stageId).map(_.size).getOrElse(0)
+ }.sum
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 03ea672c813d1..37013121c572a 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -25,6 +25,7 @@ import java.util.{Arrays, Properties, UUID}
import java.util.concurrent.atomic.AtomicInteger
import java.util.UUID.randomUUID
import scala.collection.{Map, Set}
+import scala.collection.JavaConversions._
import scala.collection.generic.Growable
import scala.collection.mutable.HashMap
import scala.reflect.{ClassTag, classTag}
@@ -49,7 +50,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage._
-import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.{SparkUI, ConsoleProgressBar}
import org.apache.spark.ui.jobs.JobProgressListener
import org.apache.spark.util._
@@ -57,11 +58,25 @@ import org.apache.spark.util._
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
*
+ * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
+ * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
+ *
* @param config a Spark Config object describing the application configuration. Any settings in
* this config overrides the default configs as well as system properties.
*/
+class SparkContext(config: SparkConf) extends Logging {
+
+ // The call site where this SparkContext was constructed.
+ private val creationSite: CallSite = Utils.getCallSite()
+
+ // If true, log warnings instead of throwing exceptions when multiple SparkContexts are active
+ private val allowMultipleContexts: Boolean =
+ config.getBoolean("spark.driver.allowMultipleContexts", false)
-class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
+ // In order to prevent multiple SparkContexts from being active at the same time, mark this
+ // context as having started construction.
+ // NOTE: this must be placed at the beginning of the SparkContext constructor.
+ SparkContext.markPartiallyConstructed(this, allowMultipleContexts)
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
// etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
@@ -228,6 +243,15 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
private[spark] val jobProgressListener = new JobProgressListener(conf)
listenerBus.addListener(jobProgressListener)
+ val statusTracker = new SparkStatusTracker(this)
+
+ private[spark] val progressBar: Option[ConsoleProgressBar] =
+ if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) {
+ Some(new ConsoleProgressBar(this))
+ } else {
+ None
+ }
+
// Initialize the Spark UI
private[spark] val ui: Option[SparkUI] =
if (conf.getBoolean("spark.ui.enabled", true)) {
@@ -1001,6 +1025,69 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
/** The version of Spark on which this application is running. */
def version = SPARK_VERSION
+ /**
+ * Return a map from the slave to the max memory available for caching and the remaining
+ * memory available for caching.
+ */
+ def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
+ env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
+ (blockManagerId.host + ":" + blockManagerId.port, mem)
+ }
+ }
+
+ /**
+ * :: DeveloperApi ::
+ * Return information about what RDDs are cached, if they are in mem or on disk, how much space
+ * they take, etc.
+ */
+ @DeveloperApi
+ def getRDDStorageInfo: Array[RDDInfo] = {
+ val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
+ StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
+ rddInfos.filter(_.isCached)
+ }
+
+ /**
+ * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
+ * Note that this does not necessarily mean the caching or computation was successful.
+ */
+ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
+
+ /**
+ * :: DeveloperApi ::
+ * Return information about blocks stored in all of the slaves
+ */
+ @DeveloperApi
+ def getExecutorStorageStatus: Array[StorageStatus] = {
+ env.blockManager.master.getStorageStatus
+ }
+
+ /**
+ * :: DeveloperApi ::
+ * Return pools for fair scheduler
+ */
+ @DeveloperApi
+ def getAllPools: Seq[Schedulable] = {
+ // TODO(xiajunluan): We should take nested pools into account
+ taskScheduler.rootPool.schedulableQueue.toSeq
+ }
+
+ /**
+ * :: DeveloperApi ::
+ * Return the pool associated with the given name, if one exists
+ */
+ @DeveloperApi
+ def getPoolForName(pool: String): Option[Schedulable] = {
+ Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool))
+ }
+
+ /**
+ * Return current scheduling mode
+ */
+ def getSchedulingMode: SchedulingMode.SchedulingMode = {
+ taskScheduler.schedulingMode
+ }
+
/**
* Clear the job's list of files added by `addFile` so that they do not get downloaded to
* any new nodes.
@@ -1100,27 +1187,30 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
/** Shut down the SparkContext. */
def stop() {
- postApplicationEnd()
- ui.foreach(_.stop())
- // Do this only if not stopped already - best case effort.
- // prevent NPE if stopped more than once.
- val dagSchedulerCopy = dagScheduler
- dagScheduler = null
- if (dagSchedulerCopy != null) {
- env.metricsSystem.report()
- metadataCleaner.cancel()
- env.actorSystem.stop(heartbeatReceiver)
- cleaner.foreach(_.stop())
- dagSchedulerCopy.stop()
- taskScheduler = null
- // TODO: Cache.stop()?
- env.stop()
- SparkEnv.set(null)
- listenerBus.stop()
- eventLogger.foreach(_.stop())
- logInfo("Successfully stopped SparkContext")
- } else {
- logInfo("SparkContext already stopped")
+ SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ postApplicationEnd()
+ ui.foreach(_.stop())
+ // Do this only if not stopped already - best case effort.
+ // prevent NPE if stopped more than once.
+ val dagSchedulerCopy = dagScheduler
+ dagScheduler = null
+ if (dagSchedulerCopy != null) {
+ env.metricsSystem.report()
+ metadataCleaner.cancel()
+ env.actorSystem.stop(heartbeatReceiver)
+ cleaner.foreach(_.stop())
+ dagSchedulerCopy.stop()
+ taskScheduler = null
+ // TODO: Cache.stop()?
+ env.stop()
+ SparkEnv.set(null)
+ listenerBus.stop()
+ eventLogger.foreach(_.stop())
+ logInfo("Successfully stopped SparkContext")
+ SparkContext.clearActiveContext()
+ } else {
+ logInfo("SparkContext already stopped")
+ }
}
}
@@ -1191,6 +1281,7 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
logInfo("Starting job: " + callSite.shortForm)
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
resultHandler, localProperties.get)
+ progressBar.foreach(_.finishAll())
rdd.doCheckpoint()
}
@@ -1409,6 +1500,11 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
private[spark] def cleanup(cleanupTime: Long) {
persistentRdds.clearOldValues(cleanupTime)
}
+
+ // In order to prevent multiple SparkContexts from being active at the same time, mark this
+ // context as having finished construction.
+ // NOTE: this must be placed at the end of the SparkContext constructor.
+ SparkContext.setActiveContext(this, allowMultipleContexts)
}
/**
@@ -1417,6 +1513,107 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging {
*/
object SparkContext extends Logging {
+ /**
+ * Lock that guards access to global variables that track SparkContext construction.
+ */
+ private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object()
+
+ /**
+ * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`.
+ *
+ * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
+ */
+ private var activeContext: Option[SparkContext] = None
+
+ /**
+ * Points to a partially-constructed SparkContext if some thread is in the SparkContext
+ * constructor, or `None` if no SparkContext is being constructed.
+ *
+ * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
+ */
+ private var contextBeingConstructed: Option[SparkContext] = None
+
+ /**
+ * Called to ensure that no other SparkContext is running in this JVM.
+ *
+ * Throws an exception if a running context is detected and logs a warning if another thread is
+ * constructing a SparkContext. This warning is necessary because the current locking scheme
+ * prevents us from reliably distinguishing between cases where another context is being
+ * constructed and cases where another constructor threw an exception.
+ */
+ private def assertNoOtherContextIsRunning(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ contextBeingConstructed.foreach { otherContext =>
+ if (otherContext ne sc) { // checks for reference equality
+ // Since otherContext might point to a partially-constructed context, guard against
+ // its creationSite field being null:
+ val otherContextCreationSite =
+ Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location")
+ val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" +
+ " constructor). This may indicate an error, since only one SparkContext may be" +
+ " running in this JVM (see SPARK-2243)." +
+ s" The other SparkContext was created at:\n$otherContextCreationSite"
+ logWarning(warnMsg)
+ }
+
+ activeContext.foreach { ctx =>
+ val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." +
+ " To ignore this error, set spark.driver.allowMultipleContexts = true. " +
+ s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}"
+ val exception = new SparkException(errMsg)
+ if (allowMultipleContexts) {
+ logWarning("Multiple running SparkContexts detected in the same JVM!", exception)
+ } else {
+ throw exception
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is
+ * running. Throws an exception if a running context is detected and logs a warning if another
+ * thread is constructing a SparkContext. This warning is necessary because the current locking
+ * scheme prevents us from reliably distinguishing between cases where another context is being
+ * constructed and cases where another constructor threw an exception.
+ */
+ private[spark] def markPartiallyConstructed(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ assertNoOtherContextIsRunning(sc, allowMultipleContexts)
+ contextBeingConstructed = Some(sc)
+ }
+ }
+
+ /**
+ * Called at the end of the SparkContext constructor to ensure that no other SparkContext has
+ * raced with this constructor and started.
+ */
+ private[spark] def setActiveContext(
+ sc: SparkContext,
+ allowMultipleContexts: Boolean): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ assertNoOtherContextIsRunning(sc, allowMultipleContexts)
+ contextBeingConstructed = None
+ activeContext = Some(sc)
+ }
+ }
+
+ /**
+ * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's
+ * also called in unit tests to prevent a flood of warnings from test suites that don't / can't
+ * properly clean up their SparkContexts.
+ */
+ private[spark] def clearActiveContext(): Unit = {
+ SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
+ activeContext = None
+ }
+ }
+
private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"
private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala
deleted file mode 100644
index 1982499c5e1d3..0000000000000
--- a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala
+++ /dev/null
@@ -1,142 +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
-
-import scala.collection.Map
-import scala.collection.JavaConversions._
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.scheduler.{SchedulingMode, Schedulable}
-import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo}
-
-/**
- * Trait that implements Spark's status APIs. This trait is designed to be mixed into
- * SparkContext; it allows the status API code to live in its own file.
- */
-private[spark] trait SparkStatusAPI { this: SparkContext =>
-
- /**
- * Return a map from the slave to the max memory available for caching and the remaining
- * memory available for caching.
- */
- def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
- env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
- (blockManagerId.host + ":" + blockManagerId.port, mem)
- }
- }
-
- /**
- * :: DeveloperApi ::
- * Return information about what RDDs are cached, if they are in mem or on disk, how much space
- * they take, etc.
- */
- @DeveloperApi
- def getRDDStorageInfo: Array[RDDInfo] = {
- val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray
- StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus)
- rddInfos.filter(_.isCached)
- }
-
- /**
- * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
- * Note that this does not necessarily mean the caching or computation was successful.
- */
- def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
-
- /**
- * :: DeveloperApi ::
- * Return information about blocks stored in all of the slaves
- */
- @DeveloperApi
- def getExecutorStorageStatus: Array[StorageStatus] = {
- env.blockManager.master.getStorageStatus
- }
-
- /**
- * :: DeveloperApi ::
- * Return pools for fair scheduler
- */
- @DeveloperApi
- def getAllPools: Seq[Schedulable] = {
- // TODO(xiajunluan): We should take nested pools into account
- taskScheduler.rootPool.schedulableQueue.toSeq
- }
-
- /**
- * :: DeveloperApi ::
- * Return the pool associated with the given name, if one exists
- */
- @DeveloperApi
- def getPoolForName(pool: String): Option[Schedulable] = {
- Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool))
- }
-
- /**
- * Return current scheduling mode
- */
- def getSchedulingMode: SchedulingMode.SchedulingMode = {
- taskScheduler.schedulingMode
- }
-
-
- /**
- * Return a list of all known jobs in a particular job group. The returned list may contain
- * running, failed, and completed jobs, and may vary across invocations of this method. This
- * method does not guarantee the order of the elements in its result.
- */
- def getJobIdsForGroup(jobGroup: String): Array[Int] = {
- jobProgressListener.synchronized {
- val jobData = jobProgressListener.jobIdToData.valuesIterator
- jobData.filter(_.jobGroup.exists(_ == jobGroup)).map(_.jobId).toArray
- }
- }
-
- /**
- * Returns job information, or `None` if the job info could not be found or was garbage collected.
- */
- def getJobInfo(jobId: Int): Option[SparkJobInfo] = {
- jobProgressListener.synchronized {
- jobProgressListener.jobIdToData.get(jobId).map { data =>
- new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status)
- }
- }
- }
-
- /**
- * Returns stage information, or `None` if the stage info could not be found or was
- * garbage collected.
- */
- def getStageInfo(stageId: Int): Option[SparkStageInfo] = {
- jobProgressListener.synchronized {
- for (
- info <- jobProgressListener.stageIdToInfo.get(stageId);
- data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId))
- ) yield {
- new SparkStageInfoImpl(
- stageId,
- info.attemptId,
- info.name,
- info.numTasks,
- data.numActiveTasks,
- data.numCompleteTasks,
- data.numFailedTasks)
- }
- }
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
new file mode 100644
index 0000000000000..edbdda8a0bcb6
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala
@@ -0,0 +1,108 @@
+/*
+ * 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
+
+/**
+ * Low-level status reporting APIs for monitoring job and stage progress.
+ *
+ * These APIs intentionally provide very weak consistency semantics; consumers of these APIs should
+ * be prepared to handle empty / missing information. For example, a job's stage ids may be known
+ * but the status API may not have any information about the details of those stages, so
+ * `getStageInfo` could potentially return `None` for a valid stage id.
+ *
+ * To limit memory usage, these APIs only provide information on recent jobs / stages. These APIs
+ * will provide information for the last `spark.ui.retainedStages` stages and
+ * `spark.ui.retainedJobs` jobs.
+ *
+ * NOTE: this class's constructor should be considered private and may be subject to change.
+ */
+class SparkStatusTracker private[spark] (sc: SparkContext) {
+
+ private val jobProgressListener = sc.jobProgressListener
+
+ /**
+ * Return a list of all known jobs in a particular job group. If `jobGroup` is `null`, then
+ * returns all known jobs that are not associated with a job group.
+ *
+ * The returned list may contain running, failed, and completed jobs, and may vary across
+ * invocations of this method. This method does not guarantee the order of the elements in
+ * its result.
+ */
+ def getJobIdsForGroup(jobGroup: String): Array[Int] = {
+ jobProgressListener.synchronized {
+ val jobData = jobProgressListener.jobIdToData.valuesIterator
+ jobData.filter(_.jobGroup.orNull == jobGroup).map(_.jobId).toArray
+ }
+ }
+
+ /**
+ * Returns an array containing the ids of all active stages.
+ *
+ * This method does not guarantee the order of the elements in its result.
+ */
+ def getActiveStageIds(): Array[Int] = {
+ jobProgressListener.synchronized {
+ jobProgressListener.activeStages.values.map(_.stageId).toArray
+ }
+ }
+
+ /**
+ * Returns an array containing the ids of all active jobs.
+ *
+ * This method does not guarantee the order of the elements in its result.
+ */
+ def getActiveJobIds(): Array[Int] = {
+ jobProgressListener.synchronized {
+ jobProgressListener.activeJobs.values.map(_.jobId).toArray
+ }
+ }
+
+ /**
+ * Returns job information, or `None` if the job info could not be found or was garbage collected.
+ */
+ def getJobInfo(jobId: Int): Option[SparkJobInfo] = {
+ jobProgressListener.synchronized {
+ jobProgressListener.jobIdToData.get(jobId).map { data =>
+ new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status)
+ }
+ }
+ }
+
+ /**
+ * Returns stage information, or `None` if the stage info could not be found or was
+ * garbage collected.
+ */
+ def getStageInfo(stageId: Int): Option[SparkStageInfo] = {
+ jobProgressListener.synchronized {
+ for (
+ info <- jobProgressListener.stageIdToInfo.get(stageId);
+ data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId))
+ ) yield {
+ new SparkStageInfoImpl(
+ stageId,
+ info.attemptId,
+ info.submissionTime.getOrElse(0),
+ info.name,
+ info.numTasks,
+ data.numActiveTasks,
+ data.numCompleteTasks,
+ data.numFailedTasks)
+ }
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
index 90b47c847fbca..e5c7c8d0db578 100644
--- a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
+++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala
@@ -26,6 +26,7 @@ private class SparkJobInfoImpl (
private class SparkStageInfoImpl(
val stageId: Int,
val currentAttemptId: Int,
+ val submissionTime: Long,
val name: String,
val numTasks: Int,
val numActiveTasks: Int,
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 5c6e8d32c5c8a..6a6d9bf6857d3 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -42,6 +42,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
/**
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
* [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
+ *
+ * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
+ * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
*/
class JavaSparkContext(val sc: SparkContext)
extends JavaSparkContextVarargsWorkaround with Closeable {
@@ -105,6 +108,8 @@ class JavaSparkContext(val sc: SparkContext)
private[spark] val env = sc.env
+ def statusTracker = new JavaSparkStatusTracker(sc)
+
def isLocal: java.lang.Boolean = sc.isLocal
def sparkUser: String = sc.sparkUser
@@ -134,25 +139,6 @@ class JavaSparkContext(val sc: SparkContext)
/** Default min number of partitions for Hadoop RDDs when not given by user */
def defaultMinPartitions: java.lang.Integer = sc.defaultMinPartitions
-
- /**
- * Return a list of all known jobs in a particular job group. The returned list may contain
- * running, failed, and completed jobs, and may vary across invocations of this method. This
- * method does not guarantee the order of the elements in its result.
- */
- def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.getJobIdsForGroup(jobGroup)
-
- /**
- * Returns job information, or `null` if the job info could not be found or was garbage collected.
- */
- def getJobInfo(jobId: Int): SparkJobInfo = sc.getJobInfo(jobId).orNull
-
- /**
- * Returns stage information, or `null` if the stage info could not be found or was
- * garbage collected.
- */
- def getStageInfo(stageId: Int): SparkStageInfo = sc.getStageInfo(stageId).orNull
-
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = {
implicit val ctag: ClassTag[T] = fakeClassTag
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala
new file mode 100644
index 0000000000000..3300cad9efbab
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkStatusTracker.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.api.java
+
+import org.apache.spark.{SparkStageInfo, SparkJobInfo, SparkContext}
+
+/**
+ * Low-level status reporting APIs for monitoring job and stage progress.
+ *
+ * These APIs intentionally provide very weak consistency semantics; consumers of these APIs should
+ * be prepared to handle empty / missing information. For example, a job's stage ids may be known
+ * but the status API may not have any information about the details of those stages, so
+ * `getStageInfo` could potentially return `null` for a valid stage id.
+ *
+ * To limit memory usage, these APIs only provide information on recent jobs / stages. These APIs
+ * will provide information for the last `spark.ui.retainedStages` stages and
+ * `spark.ui.retainedJobs` jobs.
+ *
+ * NOTE: this class's constructor should be considered private and may be subject to change.
+ */
+class JavaSparkStatusTracker private[spark] (sc: SparkContext) {
+
+ /**
+ * Return a list of all known jobs in a particular job group. If `jobGroup` is `null`, then
+ * returns all known jobs that are not associated with a job group.
+ *
+ * The returned list may contain running, failed, and completed jobs, and may vary across
+ * invocations of this method. This method does not guarantee the order of the elements in
+ * its result.
+ */
+ def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.statusTracker.getJobIdsForGroup(jobGroup)
+
+ /**
+ * Returns an array containing the ids of all active stages.
+ *
+ * This method does not guarantee the order of the elements in its result.
+ */
+ def getActiveStageIds(): Array[Int] = sc.statusTracker.getActiveStageIds()
+
+ /**
+ * Returns an array containing the ids of all active jobs.
+ *
+ * This method does not guarantee the order of the elements in its result.
+ */
+ def getActiveJobIds(): Array[Int] = sc.statusTracker.getActiveJobIds()
+
+ /**
+ * Returns job information, or `null` if the job info could not be found or was garbage collected.
+ */
+ def getJobInfo(jobId: Int): SparkJobInfo = sc.statusTracker.getJobInfo(jobId).orNull
+
+ /**
+ * Returns stage information, or `null` if the stage info could not be found or was
+ * garbage collected.
+ */
+ def getStageInfo(stageId: Int): SparkStageInfo = sc.statusTracker.getStageInfo(stageId).orNull
+}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 45beb8fc8c925..b80c771d58a8f 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -47,7 +47,7 @@ private[spark] class PythonRDD(
pythonIncludes: JList[String],
preservePartitoning: Boolean,
pythonExec: String,
- broadcastVars: JList[Broadcast[Array[Byte]]],
+ broadcastVars: JList[Broadcast[Array[Array[Byte]]]],
accumulator: Accumulator[JList[Array[Byte]]])
extends RDD[Array[Byte]](parent) {
@@ -230,8 +230,8 @@ private[spark] class PythonRDD(
if (!oldBids.contains(broadcast.id)) {
// send new broadcast
dataOut.writeLong(broadcast.id)
- dataOut.writeInt(broadcast.value.length)
- dataOut.write(broadcast.value)
+ dataOut.writeLong(broadcast.value.map(_.length.toLong).sum)
+ broadcast.value.foreach(dataOut.write)
oldBids.add(broadcast.id)
}
}
@@ -368,16 +368,24 @@ private[spark] object PythonRDD extends Logging {
}
}
- def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = {
+ def readBroadcastFromFile(
+ sc: JavaSparkContext,
+ filename: String): Broadcast[Array[Array[Byte]]] = {
+ val size = new File(filename).length()
val file = new DataInputStream(new FileInputStream(filename))
+ val blockSize = 1 << 20
+ val n = ((size + blockSize - 1) / blockSize).toInt
+ val obj = new Array[Array[Byte]](n)
try {
- val length = file.readInt()
- val obj = new Array[Byte](length)
- file.readFully(obj)
- sc.broadcast(obj)
+ for (i <- 0 until n) {
+ val length = if (i < (n - 1)) blockSize else (size % blockSize).toInt
+ obj(i) = new Array[Byte](length)
+ file.readFully(obj(i))
+ }
} finally {
file.close()
}
+ sc.broadcast(obj)
}
def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index 87f5cf944ed85..a5ea478f231d7 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -39,7 +39,7 @@ import scala.reflect.ClassTag
*
* {{{
* scala> val broadcastVar = sc.broadcast(Array(1, 2, 3))
- * broadcastVar: spark.Broadcast[Array[Int]] = spark.Broadcast(b5c40191-a864-4c7d-b9bf-d87e1a4e787c)
+ * broadcastVar: org.apache.spark.broadcast.Broadcast[Array[Int]] = Broadcast(0)
*
* scala> broadcastVar.value
* res0: Array[Int] = Array(1, 2, 3)
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 7dade04273b08..31f0a462f84d8 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -191,10 +191,12 @@ private[broadcast] object HttpBroadcast extends Logging {
logDebug("broadcast security enabled")
val newuri = Utils.constructURIForAuthentication(new URI(url), securityManager)
uc = newuri.toURL.openConnection()
+ uc.setConnectTimeout(httpReadTimeout)
uc.setAllowUserInteraction(false)
} else {
logDebug("broadcast not using security")
uc = new URL(url).openConnection()
+ uc.setConnectTimeout(httpReadTimeout)
}
val in = {
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 4e802e02c4149..2e1e52906ceeb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -75,7 +75,8 @@ private[spark] class ClientArguments(args: Array[String]) {
if (!ClientArguments.isValidJarUrl(_jarUrl)) {
println(s"Jar url '${_jarUrl}' is not in valid format.")
- println(s"Must be a jar file path in URL format (e.g. hdfs://XX.jar, file://XX.jar)")
+ println(s"Must be a jar file path in URL format " +
+ "(e.g. hdfs://host:port/XX.jar, file:///XX.jar)")
printUsageAndExit(-1)
}
@@ -119,7 +120,7 @@ object ClientArguments {
def isValidJarUrl(s: String): Boolean = {
try {
val uri = new URI(s)
- uri.getScheme != null && uri.getAuthority != null && s.endsWith("jar")
+ uri.getScheme != null && uri.getPath != null && uri.getPath.endsWith(".jar")
} catch {
case _: URISyntaxException => false
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
index 2b894a796c8c6..d2687faad62b1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala
@@ -129,6 +129,16 @@ private[spark] object SparkSubmitDriverBootstrapper {
val process = builder.start()
+ // If we kill an app while it's running, its sub-process should be killed too.
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ override def run() = {
+ if (process != null) {
+ process.destroy()
+ process.waitFor()
+ }
+ }
+ })
+
// Redirect stdout and stderr from the child JVM
val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout")
val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr")
@@ -139,14 +149,15 @@ private[spark] object SparkSubmitDriverBootstrapper {
// subprocess there already reads directly from our stdin, so we should avoid spawning a
// thread that contends with the subprocess in reading from System.in.
val isWindows = Utils.isWindows
- val isPySparkShell = sys.env.contains("PYSPARK_SHELL")
+ val isSubprocess = sys.env.contains("IS_SUBPROCESS")
if (!isWindows) {
val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin")
stdinThread.start()
- // For the PySpark shell, Spark submit itself runs as a python subprocess, and so this JVM
- // should terminate on broken pipe, which signals that the parent process has exited. In
- // Windows, the termination logic for the PySpark shell is handled in java_gateway.py
- if (isPySparkShell) {
+ // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on
+ // broken pipe, signaling that the parent process has exited. This is the case if the
+ // application is launched directly from python, as in the PySpark shell. In Windows,
+ // the termination logic is handled in java_gateway.py
+ if (isSubprocess) {
stdinThread.join()
process.destroy()
}
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 f198aa8564a54..df4b085d2251e 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
@@ -18,13 +18,13 @@
package org.apache.spark.network.nio
import java.io.IOException
+import java.lang.ref.WeakReference
import java.net._
import java.nio._
import java.nio.channels._
import java.nio.channels.spi._
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.{LinkedBlockingDeque, ThreadPoolExecutor, TimeUnit}
-import java.util.{Timer, TimerTask}
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, SynchronizedMap, SynchronizedQueue}
import scala.concurrent.duration._
@@ -32,6 +32,7 @@ import scala.concurrent.{Await, ExecutionContext, Future, Promise}
import scala.language.postfixOps
import com.google.common.base.Charsets.UTF_8
+import io.netty.util.{Timeout, TimerTask, HashedWheelTimer}
import org.apache.spark._
import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer}
@@ -77,7 +78,8 @@ private[nio] class ConnectionManager(
}
private val selector = SelectorProvider.provider.openSelector()
- private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true)
+ private val ackTimeoutMonitor =
+ new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor"))
private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60)
@@ -139,7 +141,10 @@ private[nio] class ConnectionManager(
new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection]
private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection]
with SynchronizedMap[ConnectionManagerId, SendingConnection]
- private val messageStatuses = new HashMap[Int, MessageStatus]
+ // Tracks sent messages for which we are awaiting acknowledgements. Entries are added to this
+ // map when messages are sent and are removed when acknowledgement messages are received or when
+ // acknowledgement timeouts expire
+ private val messageStatuses = new HashMap[Int, MessageStatus] // [MessageId, MessageStatus]
private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
private val registerRequests = new SynchronizedQueue[SendingConnection]
@@ -899,22 +904,41 @@ private[nio] class ConnectionManager(
: Future[Message] = {
val promise = Promise[Message]()
- val timeoutTask = new TimerTask {
- override def run(): Unit = {
+ // It's important that the TimerTask doesn't capture a reference to `message`, which can cause
+ // memory leaks since cancelled TimerTasks won't necessarily be garbage collected until the time
+ // at which they would originally be scheduled to run. Therefore, extract the message id
+ // from outside of the TimerTask closure (see SPARK-4393 for more context).
+ val messageId = message.id
+ // Keep a weak reference to the promise so that the completed promise may be garbage-collected
+ val promiseReference = new WeakReference(promise)
+ val timeoutTask: TimerTask = new TimerTask {
+ override def run(timeout: Timeout): Unit = {
messageStatuses.synchronized {
- messageStatuses.remove(message.id).foreach ( s => {
+ messageStatuses.remove(messageId).foreach { s =>
val e = new IOException("sendMessageReliably failed because ack " +
s"was not received within $ackTimeout sec")
- if (!promise.tryFailure(e)) {
- logWarning("Ignore error because promise is completed", e)
+ val p = promiseReference.get
+ if (p != null) {
+ // Attempt to fail the promise with a Timeout exception
+ if (!p.tryFailure(e)) {
+ // If we reach here, then someone else has already signalled success or failure
+ // on this promise, so log a warning:
+ logError("Ignore error because promise is completed", e)
+ }
+ } else {
+ // The WeakReference was empty, which should never happen because
+ // sendMessageReliably's caller should have a strong reference to promise.future;
+ logError("Promise was garbage collected; this should never happen!", e)
}
- })
+ }
}
}
}
+ val timeoutTaskHandle = ackTimeoutMonitor.newTimeout(timeoutTask, ackTimeout, TimeUnit.SECONDS)
+
val status = new MessageStatus(message, connectionManagerId, s => {
- timeoutTask.cancel()
+ timeoutTaskHandle.cancel()
s match {
case scala.util.Failure(e) =>
// Indicates a failure where we either never sent or never got ACK'd
@@ -943,7 +967,6 @@ private[nio] class ConnectionManager(
messageStatuses += ((message.id, status))
}
- ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000)
sendMessage(connectionManagerId, message)
promise.future
}
@@ -953,7 +976,7 @@ private[nio] class ConnectionManager(
}
def stop() {
- ackTimeoutMonitor.cancel()
+ ackTimeoutMonitor.stop()
selectorThread.interrupt()
selectorThread.join()
selector.close()
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 716f2dd17733b..e4025bcf48db6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1202,7 +1202,7 @@ abstract class RDD[T: ClassTag](
*/
def checkpoint() {
if (context.checkpointDir.isEmpty) {
- throw new Exception("Checkpoint directory has not been set in the SparkContext")
+ throw new SparkException("Checkpoint directory has not been set in the SparkContext")
} else if (checkpointData.isEmpty) {
checkpointData = Some(new RDDCheckpointData(this))
checkpointData.get.markForCheckpoint()
@@ -1309,7 +1309,7 @@ abstract class RDD[T: ClassTag](
def debugSelf (rdd: RDD[_]): Seq[String] = {
import Utils.bytesToString
- val persistence = storageLevel.description
+ val persistence = if (storageLevel != StorageLevel.NONE) storageLevel.description else ""
val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info =>
" CachedPartitions: %d; MemorySize: %s; TachyonSize: %s; DiskSize: %s".format(
info.numCachedPartitions, bytesToString(info.memSize),
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
index e2c301603b4a5..8c43a559409f2 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
@@ -39,21 +39,24 @@ class ZippedWithIndexRDDPartition(val prev: Partition, val startIndex: Long)
private[spark]
class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, Long)](prev) {
- override def getPartitions: Array[Partition] = {
+ /** The start index of each partition. */
+ @transient private val startIndices: Array[Long] = {
val n = prev.partitions.size
- val startIndices: Array[Long] =
- if (n == 0) {
- Array[Long]()
- } else if (n == 1) {
- Array(0L)
- } else {
- prev.context.runJob(
- prev,
- Utils.getIteratorSize _,
- 0 until n - 1, // do not need to count the last partition
- false
- ).scanLeft(0L)(_ + _)
- }
+ if (n == 0) {
+ Array[Long]()
+ } else if (n == 1) {
+ Array(0L)
+ } else {
+ prev.context.runJob(
+ prev,
+ Utils.getIteratorSize _,
+ 0 until n - 1, // do not need to count the last partition
+ allowLocal = false
+ ).scanLeft(0L)(_ + _)
+ }
+ }
+
+ override def getPartitions: Array[Partition] = {
firstParent[T].partitions.map(x => new ZippedWithIndexRDDPartition(x, startIndices(x.index)))
}
diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
new file mode 100644
index 0000000000000..27ba9e18237b5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
@@ -0,0 +1,124 @@
+/*
+ * 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.ui
+
+import java.util.{Timer, TimerTask}
+
+import org.apache.spark._
+
+/**
+ * ConsoleProgressBar shows the progress of stages in the next line of the console. It poll the
+ * status of active stages from `sc.statusTracker` periodically, the progress bar will be showed
+ * up after the stage has ran at least 500ms. If multiple stages run in the same time, the status
+ * of them will be combined together, showed in one line.
+ */
+private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging {
+
+ // Carrige return
+ val CR = '\r'
+ // Update period of progress bar, in milliseconds
+ val UPDATE_PERIOD = 200L
+ // Delay to show up a progress bar, in milliseconds
+ val FIRST_DELAY = 500L
+
+ // The width of terminal
+ val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) {
+ sys.env.get("COLUMNS").get.toInt
+ } else {
+ 80
+ }
+
+ var lastFinishTime = 0L
+ var lastUpdateTime = 0L
+ var lastProgressBar = ""
+
+ // Schedule a refresh thread to run periodically
+ private val timer = new Timer("refresh progress", true)
+ timer.schedule(new TimerTask{
+ override def run() {
+ refresh()
+ }
+ }, FIRST_DELAY, UPDATE_PERIOD)
+
+ /**
+ * Try to refresh the progress bar in every cycle
+ */
+ private def refresh(): Unit = synchronized {
+ val now = System.currentTimeMillis()
+ if (now - lastFinishTime < FIRST_DELAY) {
+ return
+ }
+ val stageIds = sc.statusTracker.getActiveStageIds()
+ val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1)
+ .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId())
+ if (stages.size > 0) {
+ show(now, stages.take(3)) // display at most 3 stages in same time
+ }
+ }
+
+ /**
+ * Show progress bar in console. The progress bar is displayed in the next line
+ * after your last output, keeps overwriting itself to hold in one line. The logging will follow
+ * the progress bar, then progress bar will be showed in next line without overwrite logs.
+ */
+ private def show(now: Long, stages: Seq[SparkStageInfo]) {
+ val width = TerminalWidth / stages.size
+ val bar = stages.map { s =>
+ val total = s.numTasks()
+ val header = s"[Stage ${s.stageId()}:"
+ val tailer = s"(${s.numCompletedTasks()} + ${s.numActiveTasks()}) / $total]"
+ val w = width - header.size - tailer.size
+ val bar = if (w > 0) {
+ val percent = w * s.numCompletedTasks() / total
+ (0 until w).map { i =>
+ if (i < percent) "=" else if (i == percent) ">" else " "
+ }.mkString("")
+ } else {
+ ""
+ }
+ header + bar + tailer
+ }.mkString("")
+
+ // only refresh if it's changed of after 1 minute (or the ssh connection will be closed
+ // after idle some time)
+ if (bar != lastProgressBar || now - lastUpdateTime > 60 * 1000L) {
+ System.err.print(CR + bar)
+ lastUpdateTime = now
+ }
+ lastProgressBar = bar
+ }
+
+ /**
+ * Clear the progress bar if showed.
+ */
+ private def clear() {
+ if (!lastProgressBar.isEmpty) {
+ System.err.printf(CR + " " * TerminalWidth + CR)
+ lastProgressBar = ""
+ }
+ }
+
+ /**
+ * Mark all the stages as finished, clear the progress bar if showed, then the progress will not
+ * interweave with output of jobs.
+ */
+ def finishAll(): Unit = synchronized {
+ clear()
+ lastFinishTime = System.currentTimeMillis()
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index 3312671b6f885..7bc1e24d58711 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -175,7 +175,7 @@ private[spark] object UIUtils extends Logging {
val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..."
val header = activeTab.headerTabs.map { tab =>
}
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
index e9c755e36f716..c82730f524eb7 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
@@ -17,6 +17,7 @@
package org.apache.spark.ui.exec
+import java.net.URLDecoder
import javax.servlet.http.HttpServletRequest
import scala.util.Try
@@ -29,7 +30,19 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage
private val sc = parent.sc
def render(request: HttpServletRequest): Seq[Node] = {
- val executorId = Option(request.getParameter("executorId")).getOrElse {
+ val executorId = Option(request.getParameter("executorId")).map {
+ executorId =>
+ // Due to YARN-2844, "" in the url will be encoded to "%25253Cdriver%25253E" when
+ // running in yarn-cluster mode. `request.getParameter("executorId")` will return
+ // "%253Cdriver%253E". Therefore we need to decode it until we get the real id.
+ var id = executorId
+ var decodedId = URLDecoder.decode(id, "UTF-8")
+ while (id != decodedId) {
+ id = decodedId
+ decodedId = URLDecoder.decode(id, "UTF-8")
+ }
+ id
+ }.getOrElse {
return Text(s"Missing executorId parameter")
}
val time = System.currentTimeMillis()
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
index 048fee3ce1ff4..71b59b1d078ca 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -17,6 +17,7 @@
package org.apache.spark.ui.exec
+import java.net.URLEncoder
import javax.servlet.http.HttpServletRequest
import scala.xml.Node
@@ -139,8 +140,9 @@ private[ui] class ExecutorsPage(
{
if (threadDumpEnabled) {
+ val encodedId = URLEncoder.encode(info.id, "UTF-8")
{makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size,
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
index d7dccd4af8c6e..0e4c6d633a4a9 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala
@@ -105,7 +105,8 @@ private[spark] trait Spillable[C] {
*/
@inline private def logSpillage(size: Long) {
val threadId = Thread.currentThread().getId
- logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)"
- .format(threadId, size / (1024 * 1024), _spillCount, if (_spillCount > 1) "s" else ""))
+ logInfo("Thread %d spilling in-memory map of %s to disk (%d time%s so far)"
+ .format(threadId, org.apache.spark.util.Utils.bytesToString(size),
+ _spillCount, if (_spillCount > 1) "s" else ""))
}
}
diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
index 66cf60d25f6d1..ce804f94f3267 100644
--- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala
@@ -37,20 +37,24 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
.set("spark.dynamicAllocation.enabled", "true")
intercept[SparkException] { new SparkContext(conf) }
SparkEnv.get.stop() // cleanup the created environment
+ SparkContext.clearActiveContext()
// Only min
val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
intercept[SparkException] { new SparkContext(conf1) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Only max
val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
intercept[SparkException] { new SparkContext(conf2) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Both min and max, but min > max
intercept[SparkException] { createSparkContext(2, 1) }
SparkEnv.get.stop()
+ SparkContext.clearActiveContext()
// Both min and max, and min == max
val sc1 = createSparkContext(1, 1)
@@ -76,6 +80,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test("add executors") {
sc = createSparkContext(1, 10)
val manager = sc.executorAllocationManager.get
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000)))
// Keep adding until the limit is reached
assert(numExecutorsPending(manager) === 0)
@@ -117,6 +122,51 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
assert(numExecutorsToAdd(manager) === 1)
}
+ test("add executors capped by num pending tasks") {
+ sc = createSparkContext(1, 10)
+ val manager = sc.executorAllocationManager.get
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 5)))
+
+ // Verify that we're capped at number of tasks in the stage
+ assert(numExecutorsPending(manager) === 0)
+ assert(numExecutorsToAdd(manager) === 1)
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 1)
+ assert(numExecutorsToAdd(manager) === 2)
+ assert(addExecutors(manager) === 2)
+ assert(numExecutorsPending(manager) === 3)
+ assert(numExecutorsToAdd(manager) === 4)
+ assert(addExecutors(manager) === 2)
+ assert(numExecutorsPending(manager) === 5)
+ assert(numExecutorsToAdd(manager) === 1)
+
+ // Verify that running a task reduces the cap
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3)))
+ sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1")))
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 6)
+ assert(numExecutorsToAdd(manager) === 2)
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 7)
+ assert(numExecutorsToAdd(manager) === 1)
+
+ // Verify that re-running a task doesn't reduce the cap further
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 3)))
+ sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1")))
+ sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1")))
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 8)
+ assert(numExecutorsToAdd(manager) === 2)
+ assert(addExecutors(manager) === 1)
+ assert(numExecutorsPending(manager) === 9)
+ assert(numExecutorsToAdd(manager) === 1)
+
+ // Verify that running a task once we're at our limit doesn't blow things up
+ sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1")))
+ assert(addExecutors(manager) === 0)
+ assert(numExecutorsPending(manager) === 9)
+ }
+
test("remove executors") {
sc = createSparkContext(5, 10)
val manager = sc.executorAllocationManager.get
@@ -170,6 +220,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
test ("interleaving add and remove") {
sc = createSparkContext(5, 10)
val manager = sc.executorAllocationManager.get
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000)))
// Add a few executors
assert(addExecutors(manager) === 1)
@@ -343,6 +394,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
val clock = new TestClock(2020L)
val manager = sc.executorAllocationManager.get
manager.setClock(clock)
+ sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000)))
// Scheduler queue backlogged
onSchedulerBacklogged(manager)
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
index 31edad1c56c73..9e454ddcc52a6 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala
@@ -21,9 +21,62 @@ import org.scalatest.FunSuite
import org.apache.hadoop.io.BytesWritable
-class SparkContextSuite extends FunSuite {
- //Regression test for SPARK-3121
+class SparkContextSuite extends FunSuite with LocalSparkContext {
+
+ /** Allows system properties to be changed in tests */
+ private def withSystemProperty[T](property: String, value: String)(block: => T): T = {
+ val originalValue = System.getProperty(property)
+ try {
+ System.setProperty(property, value)
+ block
+ } finally {
+ if (originalValue == null) {
+ System.clearProperty(property)
+ } else {
+ System.setProperty(property, originalValue)
+ }
+ }
+ }
+
+ test("Only one SparkContext may be active at a time") {
+ // Regression test for SPARK-4180
+ withSystemProperty("spark.driver.allowMultipleContexts", "false") {
+ val conf = new SparkConf().setAppName("test").setMaster("local")
+ sc = new SparkContext(conf)
+ // A SparkContext is already running, so we shouldn't be able to create a second one
+ intercept[SparkException] { new SparkContext(conf) }
+ // After stopping the running context, we should be able to create a new one
+ resetSparkContext()
+ sc = new SparkContext(conf)
+ }
+ }
+
+ test("Can still construct a new SparkContext after failing to construct a previous one") {
+ withSystemProperty("spark.driver.allowMultipleContexts", "false") {
+ // This is an invalid configuration (no app name or master URL)
+ intercept[SparkException] {
+ new SparkContext(new SparkConf())
+ }
+ // Even though those earlier calls failed, we should still be able to create a new context
+ sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test"))
+ }
+ }
+
+ test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
+ withSystemProperty("spark.driver.allowMultipleContexts", "true") {
+ var secondSparkContext: SparkContext = null
+ try {
+ val conf = new SparkConf().setAppName("test").setMaster("local")
+ sc = new SparkContext(conf)
+ secondSparkContext = new SparkContext(conf)
+ } finally {
+ Option(secondSparkContext).foreach(_.stop())
+ }
+ }
+ }
+
test("BytesWritable implicit conversion is correct") {
+ // Regression test for SPARK-3121
val bytesWritable = new BytesWritable()
val inputArray = (1 to 10).map(_.toByte).toArray
bytesWritable.set(inputArray, 0, 10)
diff --git a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
similarity index 69%
rename from core/src/test/scala/org/apache/spark/StatusAPISuite.scala
rename to core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
index 4468fba8c1dff..8577e4ac7e33e 100644
--- a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala
+++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala
@@ -27,9 +27,10 @@ import org.scalatest.concurrent.Eventually._
import org.apache.spark.JobExecutionStatus._
import org.apache.spark.SparkContext._
-class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
+class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext {
test("basic status API usage") {
+ sc = new SparkContext("local", "test", new SparkConf(false))
val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync()
val jobId: Int = eventually(timeout(10 seconds)) {
val jobIds = jobFuture.jobIds
@@ -37,20 +38,20 @@ class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
jobIds.head
}
val jobInfo = eventually(timeout(10 seconds)) {
- sc.getJobInfo(jobId).get
+ sc.statusTracker.getJobInfo(jobId).get
}
jobInfo.status() should not be FAILED
val stageIds = jobInfo.stageIds()
stageIds.size should be(2)
val firstStageInfo = eventually(timeout(10 seconds)) {
- sc.getStageInfo(stageIds(0)).get
+ sc.statusTracker.getStageInfo(stageIds(0)).get
}
firstStageInfo.stageId() should be(stageIds(0))
firstStageInfo.currentAttemptId() should be(0)
firstStageInfo.numTasks() should be(2)
eventually(timeout(10 seconds)) {
- val updatedFirstStageInfo = sc.getStageInfo(stageIds(0)).get
+ val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds(0)).get
updatedFirstStageInfo.numCompletedTasks() should be(2)
updatedFirstStageInfo.numActiveTasks() should be(0)
updatedFirstStageInfo.numFailedTasks() should be(0)
@@ -58,21 +59,31 @@ class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext {
}
test("getJobIdsForGroup()") {
+ sc = new SparkContext("local", "test", new SparkConf(false))
+ // Passing `null` should return jobs that were not run in a job group:
+ val defaultJobGroupFuture = sc.parallelize(1 to 1000).countAsync()
+ val defaultJobGroupJobId = eventually(timeout(10 seconds)) {
+ defaultJobGroupFuture.jobIds.head
+ }
+ eventually(timeout(10 seconds)) {
+ sc.statusTracker.getJobIdsForGroup(null).toSet should be (Set(defaultJobGroupJobId))
+ }
+ // Test jobs submitted in job groups:
sc.setJobGroup("my-job-group", "description")
- sc.getJobIdsForGroup("my-job-group") should be (Seq.empty)
+ sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq.empty)
val firstJobFuture = sc.parallelize(1 to 1000).countAsync()
val firstJobId = eventually(timeout(10 seconds)) {
firstJobFuture.jobIds.head
}
eventually(timeout(10 seconds)) {
- sc.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId))
+ sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId))
}
val secondJobFuture = sc.parallelize(1 to 1000).countAsync()
val secondJobId = eventually(timeout(10 seconds)) {
secondJobFuture.jobIds.head
}
eventually(timeout(10 seconds)) {
- sc.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
+ sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId))
}
}
}
\ No newline at end of file
diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
index 94a2bdd74e744..d2dae34be7bfb 100644
--- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala
@@ -23,17 +23,26 @@ import org.scalatest.Matchers
class ClientSuite extends FunSuite with Matchers {
test("correctly validates driver jar URL's") {
ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true)
- ClientArguments.isValidJarUrl("file://some/path/to/a/jarFile.jar") should be (true)
+
+ // file scheme with authority and path is valid.
+ ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true)
+
+ // file scheme without path is not valid.
+ // In this case, jarFile.jar is recognized as authority.
+ ClientArguments.isValidJarUrl("file://jarFile.jar") should be (false)
+
+ // file scheme without authority but with triple slash is valid.
+ ClientArguments.isValidJarUrl("file:///some/path/to/a/jarFile.jar") should be (true)
ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo.jar") should be (true)
ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false)
ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false)
ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false)
- // No authority
+ // This URI doesn't have authority and path.
ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false)
- // Invalid syntax
+ // Invalid syntax.
ClientArguments.isValidJarUrl("hdfs:") should be (false)
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 6d2e696dc2fc4..e079ca3b1e896 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -739,6 +739,11 @@ class RDDSuite extends FunSuite with SharedSparkContext {
}
}
+ test("zipWithIndex chained with other RDDs (SPARK-4433)") {
+ val count = sc.parallelize(0 until 10, 2).zipWithIndex().repartition(4).count()
+ assert(count === 10)
+ }
+
test("zipWithUniqueId") {
val n = 10
val data = sc.parallelize(0 until n, 3)
diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
index f9d1af88f3a13..0ea2d13a83505 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -118,7 +118,7 @@ class SizeEstimatorSuite
// TODO: If we sample 100 elements, this should always be 4176 ?
val estimatedSize = SizeEstimator.estimate(Array.fill(1000)(d1))
assert(estimatedSize >= 4000, "Estimated size " + estimatedSize + " should be more than 4000")
- assert(estimatedSize <= 4200, "Estimated size " + estimatedSize + " should be less than 4100")
+ assert(estimatedSize <= 4200, "Estimated size " + estimatedSize + " should be less than 4200")
}
test("32-bit arch") {
diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh
index a6e90a15ee84b..8a0b0348db8c0 100755
--- a/dev/create-release/create-release.sh
+++ b/dev/create-release/create-release.sh
@@ -28,13 +28,19 @@
# - Send output to stderr and have useful logging in stdout
# Note: The following variables must be set before use!
-GIT_USERNAME=${GIT_USERNAME:-pwendell}
-GIT_PASSWORD=${GIT_PASSWORD:-XXX}
+ASF_USERNAME=${ASF_USERNAME:-pwendell}
+ASF_PASSWORD=${ASF_PASSWORD:-XXX}
GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX}
GIT_BRANCH=${GIT_BRANCH:-branch-1.0}
-RELEASE_VERSION=${RELEASE_VERSION:-1.0.0}
+RELEASE_VERSION=${RELEASE_VERSION:-1.2.0}
+NEXT_VERSION=${NEXT_VERSION:-1.2.1}
RC_NAME=${RC_NAME:-rc2}
-USER_NAME=${USER_NAME:-pwendell}
+
+M2_REPO=~/.m2/repository
+SPARK_REPO=$M2_REPO/org/apache/spark
+NEXUS_ROOT=https://repository.apache.org/service/local/staging
+NEXUS_UPLOAD=$NEXUS_ROOT/deploy/maven2
+NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads
if [ -z "$JAVA_HOME" ]; then
echo "Error: JAVA_HOME is not set, cannot proceed."
@@ -47,31 +53,90 @@ set -e
GIT_TAG=v$RELEASE_VERSION-$RC_NAME
if [[ ! "$@" =~ --package-only ]]; then
- echo "Creating and publishing release"
+ echo "Creating release commit and publishing to Apache repository"
# Artifact publishing
- git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH
- cd spark
+ git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \
+ -b $GIT_BRANCH
+ pushd spark
export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g"
- mvn -Pyarn release:clean
-
- mvn -DskipTests \
- -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
- -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \
- -Dmaven.javadoc.skip=true \
- -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
- -Dtag=$GIT_TAG -DautoVersionSubmodules=true \
- -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
- --batch-mode release:prepare
-
- mvn -DskipTests \
- -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \
- -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
- -Dmaven.javadoc.skip=true \
+ # Create release commits and push them to github
+ # NOTE: This is done "eagerly" i.e. we don't check if we can succesfully build
+ # or before we coin the release commit. This helps avoid races where
+ # other people add commits to this branch while we are in the middle of building.
+ old=" ${RELEASE_VERSION}-SNAPSHOT<\/version>"
+ new=" ${RELEASE_VERSION}<\/version>"
+ find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \
+ -e "s/$old/$new/" {}
+ git commit -a -m "Preparing Spark release $GIT_TAG"
+ echo "Creating tag $GIT_TAG at the head of $GIT_BRANCH"
+ git tag $GIT_TAG
+
+ old=" ${RELEASE_VERSION}<\/version>"
+ new=" ${NEXT_VERSION}-SNAPSHOT<\/version>"
+ find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \
+ -e "s/$old/$new/" {}
+ git commit -a -m "Preparing development version ${NEXT_VERSION}-SNAPSHOT"
+ git push origin $GIT_TAG
+ git push origin HEAD:$GIT_BRANCH
+ git checkout -f $GIT_TAG
+
+ # Using Nexus API documented here:
+ # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API
+ echo "Creating Nexus staging repository"
+ repo_request="Apache Spark $GIT_TAG"
+ out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+ -H "Content-Type:application/xml" -v \
+ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start)
+ staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachespark-[0-9]\{4\}\).*/\1/")
+ echo "Created Nexus staging repository: $staged_repo_id"
+
+ rm -rf $SPARK_REPO
+
+ mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
-Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
- release:perform
+ clean install
- cd ..
+ ./dev/change-version-to-2.11.sh
+
+ mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \
+ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \
+ clean install
+
+ ./dev/change-version-to-2.10.sh
+
+ pushd $SPARK_REPO
+
+ # Remove any extra files generated during install
+ find . -type f |grep -v \.jar |grep -v \.pom | xargs rm
+
+ echo "Creating hash and signature files"
+ for file in $(find . -type f)
+ do
+ echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --output $file.asc --detach-sig --armour $file;
+ gpg --print-md MD5 $file > $file.md5;
+ gpg --print-md SHA1 $file > $file.sha1
+ done
+
+ echo "Uplading files to $NEXUS_UPLOAD"
+ for file in $(find . -type f)
+ do
+ # strip leading ./
+ file_short=$(echo $file | sed -e "s/\.\///")
+ dest_url="$NEXUS_UPLOAD/org/apache/spark/$file_short"
+ echo " Uploading $file_short"
+ curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url
+ done
+
+ echo "Closing nexus staging repository"
+ repo_request="$staged_repo_idApache Spark $GIT_TAG"
+ out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+ -H "Content-Type:application/xml" -v \
+ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish)
+ echo "Closed Nexus staging repository: $staged_repo_id"
+
+ popd
+ popd
rm -rf spark
fi
@@ -102,6 +167,12 @@ make_binary_release() {
cp -r spark spark-$RELEASE_VERSION-bin-$NAME
cd spark-$RELEASE_VERSION-bin-$NAME
+
+ # TODO There should probably be a flag to make-distribution to allow 2.11 support
+ if [[ $FLAGS == *scala-2.11* ]]; then
+ ./dev/change-version-to-2.11.sh
+ fi
+
./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log
cd ..
cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz .
@@ -118,11 +189,12 @@ make_binary_release() {
spark-$RELEASE_VERSION-bin-$NAME.tgz.sha
}
+
make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" &
+make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" &
make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" &
make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" &
make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" &
-make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" &
make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" &
make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" &
wait
@@ -130,10 +202,10 @@ wait
# Copy data
echo "Copying release tarballs"
rc_folder=spark-$RELEASE_VERSION-$RC_NAME
-ssh $USER_NAME@people.apache.org \
- mkdir /home/$USER_NAME/public_html/$rc_folder
+ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_folder
scp spark-* \
- $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/
+ $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/
# Docs
cd spark
@@ -143,12 +215,12 @@ cd docs
JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build
echo "Copying release documentation"
rc_docs_folder=${rc_folder}-docs
-ssh $USER_NAME@people.apache.org \
- mkdir /home/$USER_NAME/public_html/$rc_docs_folder
-rsync -r _site/* $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_docs_folder
+ssh $ASF_USERNAME@people.apache.org \
+ mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder
+rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder
echo "Release $RELEASE_VERSION completed:"
echo "Git tag:\t $GIT_TAG"
echo "Release commit:\t $release_hash"
-echo "Binary location:\t http://people.apache.org/~$USER_NAME/$rc_folder"
-echo "Doc location:\t http://people.apache.org/~$USER_NAME/$rc_docs_folder"
+echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder"
+echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder"
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 20ba7da5d71ff..bb18414092aae 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -113,9 +113,9 @@ mvn -Pyarn -Phive -Phive-thriftserver-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0
{% endhighlight %}
# Building for Scala 2.11
-To produce a Spark package compiled with Scala 2.11, use the `-Pscala-2.11` profile:
+To produce a Spark package compiled with Scala 2.11, use the `-Dscala-2.11` property:
- mvn -Pyarn -Phadoop-2.4 -Pscala-2.11 -DskipTests clean package
+ mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package
Scala 2.11 support in Spark is experimental and does not support a few features.
Specifically, Spark's external Kafka library and JDBC component are not yet
diff --git a/docs/configuration.md b/docs/configuration.md
index f0b396e21f198..8839162c3a13e 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -52,7 +52,7 @@ Then, you can supply configuration values at runtime:
--conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" myApp.jar
{% endhighlight %}
-The Spark shell and [`spark-submit`](cluster-overview.html#launching-applications-with-spark-submit)
+The Spark shell and [`spark-submit`](submitting-applications.html)
tool support two ways to load configurations dynamically. The first are command line options,
such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf`
flag, but uses special flags for properties that play a part in launching the Spark application.
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index 18420afb27e3c..49f319ba775e5 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -117,6 +117,8 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/
how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object
that contains information about your application.
+Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one.
+
{% highlight scala %}
val conf = new SparkConf().setAppName(appName).setMaster(master)
new SparkContext(conf)
@@ -1131,7 +1133,7 @@ method. The code below shows this:
{% highlight scala %}
scala> val broadcastVar = sc.broadcast(Array(1, 2, 3))
-broadcastVar: spark.Broadcast[Array[Int]] = spark.Broadcast(b5c40191-a864-4c7d-b9bf-d87e1a4e787c)
+broadcastVar: org.apache.spark.broadcast.Broadcast[Array[Int]] = Broadcast(0)
scala> broadcastVar.value
res0: Array[Int] = Array(1, 2, 3)
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 2f7e4981e5bb9..dfe2db4b3fce8 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -39,7 +39,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes
spark.yarn.preserve.staging.files
false
- Set to true to preserve the staged files (Spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
+ Set to true to preserve the staged files (Spark jar, app jar, distributed cache files) at the end of the job rather than delete them.
@@ -159,7 +159,7 @@ For example:
lib/spark-examples*.jar \
10
-The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs.
+The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Debugging your Application" section below for how to see driver and executor logs.
To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell:
@@ -181,7 +181,7 @@ In YARN terminology, executors and application masters run inside "containers".
yarn logs -applicationId
-will print out the contents of all log files from all containers from the given application.
+will print out the contents of all log files from all containers from the given application. You can also view the container log files directly in HDFS using the HDFS shell or API. The directory where they are located can be found by looking at your YARN configs (`yarn.nodemanager.remote-app-log-dir` and `yarn.nodemanager.remote-app-log-dir-suffix`).
When log aggregation isn't turned on, logs are retained locally on each machine under `YARN_APP_LOGS_DIR`, which is usually configured to `/tmp/logs` or `$HADOOP_HOME/logs/userlogs` depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID.
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 48e8267ac072c..5500da83b2b66 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -14,7 +14,7 @@ title: Spark SQL Programming Guide
Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
Spark. At the core of this component is a new type of RDD,
[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of
-[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with
+[Row](api/scala/index.html#org.apache.spark.sql.package@Row:org.apache.spark.sql.catalyst.expressions.Row.type) objects, along with
a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table
in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io)
file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
diff --git a/examples/pom.xml b/examples/pom.xml
index 2ec5728154abf..85e133779e465 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -217,6 +217,11 @@
org.apache.commonscommons-math3
+
+ com.twitter
+ algebird-core_${scala.binary.version}
+ 0.8.1
+ org.scalatestscalatest_${scala.binary.version}
@@ -389,11 +394,11 @@
-
+
scala-2.10
- true
+ !scala-2.11
@@ -401,11 +406,6 @@
spark-streaming-kafka_${scala.binary.version}${project.version}
-
- com.twitter
- algebird-core_${scala.binary.version}
- 0.1.11
-
@@ -427,65 +427,6 @@
-
- add-scala-test-sources
- generate-test-sources
-
- add-test-source
-
-
-
- src/test/scala
- scala-2.10/src/test/scala
- scala-2.10/src/test/java
-
-
-
-
-
-
-
-
-
- scala-2.11
-
- false
-
-
-
-
-
-
-
- org.codehaus.mojo
- build-helper-maven-plugin
-
-
- add-scala-sources
- generate-sources
-
- add-source
-
-
-
- src/main/scala
- scala-2.11/src/main/scala
-
-
-
-
- add-scala-test-sources
- generate-test-sources
-
- add-test-source
-
-
-
- src/test/scala
- scala-2.11/src/test/scala
-
-
-
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
similarity index 92%
rename from examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java
rename to examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
index 430e96ab14d9d..e68ec74c3ed54 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java
@@ -31,7 +31,7 @@
/**
* Example of using Spark's status APIs from Java.
*/
-public final class JavaStatusAPIDemo {
+public final class JavaStatusTrackerDemo {
public static final String APP_NAME = "JavaStatusAPIDemo";
@@ -58,8 +58,8 @@ public static void main(String[] args) throws Exception {
continue;
}
int currentJobId = jobIds.get(jobIds.size() - 1);
- SparkJobInfo jobInfo = sc.getJobInfo(currentJobId);
- SparkStageInfo stageInfo = sc.getStageInfo(jobInfo.stageIds()[0]);
+ SparkJobInfo jobInfo = sc.statusTracker().getJobInfo(currentJobId);
+ SparkStageInfo stageInfo = sc.statusTracker().getStageInfo(jobInfo.stageIds()[0]);
System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() +
" active, " + stageInfo.numCompletedTasks() + " complete");
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
index 1edd2432a0352..a113653810b93 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
@@ -55,7 +55,7 @@ object BinaryClassification {
stepSize: Double = 1.0,
algorithm: Algorithm = LR,
regType: RegType = L2,
- regParam: Double = 0.1) extends AbstractParams[Params]
+ regParam: Double = 0.01) extends AbstractParams[Params]
def main(args: Array[String]) {
val defaultParams = Params()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
index e1f9622350135..6815b1c052208 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
@@ -47,7 +47,7 @@ object LinearRegression extends App {
numIterations: Int = 100,
stepSize: Double = 1.0,
regType: RegType = L2,
- regParam: Double = 0.1) extends AbstractParams[Params]
+ regParam: Double = 0.01) extends AbstractParams[Params]
val defaultParams = Params()
diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
similarity index 100%
rename from examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
similarity index 100%
rename from examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
index 28ac5929df44a..4d26b640e8d74 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -17,13 +17,12 @@
package org.apache.spark.streaming.kafka
+import java.util.Properties
+
import scala.collection.Map
import scala.reflect.{classTag, ClassTag}
-import java.util.Properties
-import java.util.concurrent.Executors
-
-import kafka.consumer._
+import kafka.consumer.{KafkaStream, Consumer, ConsumerConfig, ConsumerConnector}
import kafka.serializer.Decoder
import kafka.utils.VerifiableProperties
@@ -32,6 +31,7 @@ import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.Receiver
+import org.apache.spark.util.Utils
/**
* Input stream that pulls messages from a Kafka Broker.
@@ -51,12 +51,16 @@ class KafkaInputDStream[
@transient ssc_ : StreamingContext,
kafkaParams: Map[String, String],
topics: Map[String, Int],
+ useReliableReceiver: Boolean,
storageLevel: StorageLevel
) extends ReceiverInputDStream[(K, V)](ssc_) with Logging {
def getReceiver(): Receiver[(K, V)] = {
- new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
- .asInstanceOf[Receiver[(K, V)]]
+ if (!useReliableReceiver) {
+ new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
+ } else {
+ new ReliableKafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
+ }
}
}
@@ -69,14 +73,15 @@ class KafkaReceiver[
kafkaParams: Map[String, String],
topics: Map[String, Int],
storageLevel: StorageLevel
- ) extends Receiver[Any](storageLevel) with Logging {
+ ) extends Receiver[(K, V)](storageLevel) with Logging {
// Connection to Kafka
- var consumerConnector : ConsumerConnector = null
+ var consumerConnector: ConsumerConnector = null
def onStop() {
if (consumerConnector != null) {
consumerConnector.shutdown()
+ consumerConnector = null
}
}
@@ -102,11 +107,11 @@ class KafkaReceiver[
.newInstance(consumerConfig.props)
.asInstanceOf[Decoder[V]]
- // Create Threads for each Topic/Message Stream we are listening
+ // Create threads for each topic/message Stream we are listening
val topicMessageStreams = consumerConnector.createMessageStreams(
topics, keyDecoder, valueDecoder)
- val executorPool = Executors.newFixedThreadPool(topics.values.sum)
+ val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler")
try {
// Start the messages handler for each partition
topicMessageStreams.values.foreach { streams =>
@@ -117,13 +122,15 @@ class KafkaReceiver[
}
}
- // Handles Kafka Messages
- private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V])
+ // Handles Kafka messages
+ private class MessageHandler(stream: KafkaStream[K, V])
extends Runnable {
def run() {
logInfo("Starting MessageHandler.")
try {
- for (msgAndMetadata <- stream) {
+ val streamIterator = stream.iterator()
+ while (streamIterator.hasNext()) {
+ val msgAndMetadata = streamIterator.next()
store((msgAndMetadata.key, msgAndMetadata.message))
}
} catch {
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
index ec812e1ef3b04..b4ac929e0c070 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -70,7 +70,8 @@ object KafkaUtils {
topics: Map[String, Int],
storageLevel: StorageLevel
): ReceiverInputDStream[(K, V)] = {
- new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
+ val walEnabled = ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)
+ new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel)
}
/**
@@ -99,7 +100,6 @@ object KafkaUtils {
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
* @param storageLevel RDD storage level.
- *
*/
def createStream(
jssc: JavaStreamingContext,
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala
new file mode 100644
index 0000000000000..be734b80272d1
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala
@@ -0,0 +1,282 @@
+/*
+ * 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.streaming.kafka
+
+import java.util.Properties
+import java.util.concurrent.{ThreadPoolExecutor, ConcurrentHashMap}
+
+import scala.collection.{Map, mutable}
+import scala.reflect.{ClassTag, classTag}
+
+import kafka.common.TopicAndPartition
+import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream}
+import kafka.message.MessageAndMetadata
+import kafka.serializer.Decoder
+import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils}
+import org.I0Itec.zkclient.ZkClient
+
+import org.apache.spark.{Logging, SparkEnv}
+import org.apache.spark.storage.{StorageLevel, StreamBlockId}
+import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver}
+import org.apache.spark.util.Utils
+
+/**
+ * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss.
+ * It is turned off by default and will be enabled when
+ * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver
+ * is that this receiver manages topic-partition/offset itself and updates the offset information
+ * after data is reliably stored as write-ahead log. Offsets will only be updated when data is
+ * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated.
+ *
+ * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset
+ * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams
+ * will not take effect.
+ */
+private[streaming]
+class ReliableKafkaReceiver[
+ K: ClassTag,
+ V: ClassTag,
+ U <: Decoder[_]: ClassTag,
+ T <: Decoder[_]: ClassTag](
+ kafkaParams: Map[String, String],
+ topics: Map[String, Int],
+ storageLevel: StorageLevel)
+ extends Receiver[(K, V)](storageLevel) with Logging {
+
+ private val groupId = kafkaParams("group.id")
+ private val AUTO_OFFSET_COMMIT = "auto.commit.enable"
+ private def conf = SparkEnv.get.conf
+
+ /** High level consumer to connect to Kafka. */
+ private var consumerConnector: ConsumerConnector = null
+
+ /** zkClient to connect to Zookeeper to commit the offsets. */
+ private var zkClient: ZkClient = null
+
+ /**
+ * A HashMap to manage the offset for each topic/partition, this HashMap is called in
+ * synchronized block, so mutable HashMap will not meet concurrency issue.
+ */
+ private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null
+
+ /** A concurrent HashMap to store the stream block id and related offset snapshot. */
+ private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null
+
+ /**
+ * Manage the BlockGenerator in receiver itself for better managing block store and offset
+ * commit.
+ */
+ private var blockGenerator: BlockGenerator = null
+
+ /** Thread pool running the handlers for receiving message from multiple topics and partitions. */
+ private var messageHandlerThreadPool: ThreadPoolExecutor = null
+
+ override def onStart(): Unit = {
+ logInfo(s"Starting Kafka Consumer Stream with group: $groupId")
+
+ // Initialize the topic-partition / offset hash map.
+ topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long]
+
+ // Initialize the stream block id / offset snapshot hash map.
+ blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]()
+
+ // Initialize the block generator for storing Kafka message.
+ blockGenerator = new BlockGenerator(new GeneratedBlockHandler, streamId, conf)
+
+ if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") {
+ logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " +
+ "otherwise we will manually set it to false to turn off auto offset commit in Kafka")
+ }
+
+ val props = new Properties()
+ kafkaParams.foreach(param => props.put(param._1, param._2))
+ // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true,
+ // we have to make sure this property is set to false to turn off auto commit mechanism in
+ // Kafka.
+ props.setProperty(AUTO_OFFSET_COMMIT, "false")
+
+ val consumerConfig = new ConsumerConfig(props)
+
+ assert(!consumerConfig.autoCommitEnable)
+
+ logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}")
+ consumerConnector = Consumer.create(consumerConfig)
+ logInfo(s"Connected to Zookeeper: ${consumerConfig.zkConnect}")
+
+ zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs,
+ consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer)
+
+ messageHandlerThreadPool = Utils.newDaemonFixedThreadPool(
+ topics.values.sum, "KafkaMessageHandler")
+
+ blockGenerator.start()
+
+ val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
+ .newInstance(consumerConfig.props)
+ .asInstanceOf[Decoder[K]]
+
+ val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
+ .newInstance(consumerConfig.props)
+ .asInstanceOf[Decoder[V]]
+
+ val topicMessageStreams = consumerConnector.createMessageStreams(
+ topics, keyDecoder, valueDecoder)
+
+ topicMessageStreams.values.foreach { streams =>
+ streams.foreach { stream =>
+ messageHandlerThreadPool.submit(new MessageHandler(stream))
+ }
+ }
+ }
+
+ override def onStop(): Unit = {
+ if (messageHandlerThreadPool != null) {
+ messageHandlerThreadPool.shutdown()
+ messageHandlerThreadPool = null
+ }
+
+ if (consumerConnector != null) {
+ consumerConnector.shutdown()
+ consumerConnector = null
+ }
+
+ if (zkClient != null) {
+ zkClient.close()
+ zkClient = null
+ }
+
+ if (blockGenerator != null) {
+ blockGenerator.stop()
+ blockGenerator = null
+ }
+
+ if (topicPartitionOffsetMap != null) {
+ topicPartitionOffsetMap.clear()
+ topicPartitionOffsetMap = null
+ }
+
+ if (blockOffsetMap != null) {
+ blockOffsetMap.clear()
+ blockOffsetMap = null
+ }
+ }
+
+ /** Store a Kafka message and the associated metadata as a tuple. */
+ private def storeMessageAndMetadata(
+ msgAndMetadata: MessageAndMetadata[K, V]): Unit = {
+ val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition)
+ val data = (msgAndMetadata.key, msgAndMetadata.message)
+ val metadata = (topicAndPartition, msgAndMetadata.offset)
+ blockGenerator.addDataWithCallback(data, metadata)
+ }
+
+ /** Update stored offset */
+ private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = {
+ topicPartitionOffsetMap.put(topicAndPartition, offset)
+ }
+
+ /**
+ * Remember the current offsets for each topic and partition. This is called when a block is
+ * generated.
+ */
+ private def rememberBlockOffsets(blockId: StreamBlockId): Unit = {
+ // Get a snapshot of current offset map and store with related block id.
+ val offsetSnapshot = topicPartitionOffsetMap.toMap
+ blockOffsetMap.put(blockId, offsetSnapshot)
+ topicPartitionOffsetMap.clear()
+ }
+
+ /** Store the ready-to-be-stored block and commit the related offsets to zookeeper. */
+ private def storeBlockAndCommitOffset(
+ blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = {
+ store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]])
+ Option(blockOffsetMap.get(blockId)).foreach(commitOffset)
+ blockOffsetMap.remove(blockId)
+ }
+
+ /**
+ * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's
+ * metadata schema in Zookeeper.
+ */
+ private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = {
+ if (zkClient == null) {
+ val thrown = new IllegalStateException("Zookeeper client is unexpectedly null")
+ stop("Zookeeper client is not initialized before commit offsets to ZK", thrown)
+ return
+ }
+
+ for ((topicAndPart, offset) <- offsetMap) {
+ try {
+ val topicDirs = new ZKGroupTopicDirs(groupId, topicAndPart.topic)
+ val zkPath = s"${topicDirs.consumerOffsetDir}/${topicAndPart.partition}"
+
+ ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString)
+ } catch {
+ case e: Exception =>
+ logWarning(s"Exception during commit offset $offset for topic" +
+ s"${topicAndPart.topic}, partition ${topicAndPart.partition}", e)
+ }
+
+ logInfo(s"Committed offset $offset for topic ${topicAndPart.topic}, " +
+ s"partition ${topicAndPart.partition}")
+ }
+ }
+
+ /** Class to handle received Kafka message. */
+ private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable {
+ override def run(): Unit = {
+ while (!isStopped) {
+ try {
+ val streamIterator = stream.iterator()
+ while (streamIterator.hasNext) {
+ storeMessageAndMetadata(streamIterator.next)
+ }
+ } catch {
+ case e: Exception =>
+ logError("Error handling message", e)
+ }
+ }
+ }
+ }
+
+ /** Class to handle blocks generated by the block generator. */
+ private final class GeneratedBlockHandler extends BlockGeneratorListener {
+
+ def onAddData(data: Any, metadata: Any): Unit = {
+ // Update the offset of the data that was added to the generator
+ if (metadata != null) {
+ val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)]
+ updateOffset(topicAndPartition, offset)
+ }
+ }
+
+ def onGenerateBlock(blockId: StreamBlockId): Unit = {
+ // Remember the offsets of topics/partitions when a block has been generated
+ rememberBlockOffsets(blockId)
+ }
+
+ def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = {
+ // Store block and commit the blocks offset
+ storeBlockAndCommitOffset(blockId, arrayBuffer)
+ }
+
+ def onError(message: String, throwable: Throwable): Unit = {
+ reportError(message, throwable)
+ }
+ }
+}
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
index efb0099c7c850..6e1abf3f385ee 100644
--- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -20,7 +20,10 @@
import java.io.Serializable;
import java.util.HashMap;
import java.util.List;
+import java.util.Random;
+import org.apache.spark.SparkConf;
+import org.apache.spark.streaming.Duration;
import scala.Predef;
import scala.Tuple2;
import scala.collection.JavaConverters;
@@ -32,8 +35,6 @@
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.LocalJavaStreamingContext;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
@@ -42,25 +43,27 @@
import org.junit.After;
import org.junit.Before;
-public class JavaKafkaStreamSuite extends LocalJavaStreamingContext implements Serializable {
- private transient KafkaStreamSuite testSuite = new KafkaStreamSuite();
+public class JavaKafkaStreamSuite implements Serializable {
+ private transient JavaStreamingContext ssc = null;
+ private transient Random random = new Random();
+ private transient KafkaStreamSuiteBase suiteBase = null;
@Before
- @Override
public void setUp() {
- testSuite.beforeFunction();
+ suiteBase = new KafkaStreamSuiteBase() { };
+ suiteBase.setupKafka();
System.clearProperty("spark.driver.port");
- //System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock");
- ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ SparkConf sparkConf = new SparkConf()
+ .setMaster("local[4]").setAppName(this.getClass().getSimpleName());
+ ssc = new JavaStreamingContext(sparkConf, new Duration(500));
}
@After
- @Override
public void tearDown() {
ssc.stop();
ssc = null;
System.clearProperty("spark.driver.port");
- testSuite.afterFunction();
+ suiteBase.tearDownKafka();
}
@Test
@@ -74,15 +77,15 @@ public void testKafkaStream() throws InterruptedException {
sent.put("b", 3);
sent.put("c", 10);
- testSuite.createTopic(topic);
+ suiteBase.createTopic(topic);
HashMap tmp = new HashMap(sent);
- testSuite.produceAndSendMessage(topic,
- JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap(
- Predef.>conforms()));
+ suiteBase.produceAndSendMessage(topic,
+ JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap(
+ Predef.>conforms()));
HashMap kafkaParams = new HashMap();
- kafkaParams.put("zookeeper.connect", testSuite.zkHost() + ":" + testSuite.zkPort());
- kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000));
+ kafkaParams.put("zookeeper.connect", suiteBase.zkAddress());
+ kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000));
kafkaParams.put("auto.offset.reset", "smallest");
JavaPairDStream stream = KafkaUtils.createStream(ssc,
@@ -124,11 +127,16 @@ public Void call(JavaPairRDD rdd) throws Exception {
);
ssc.start();
- ssc.awaitTermination(3000);
-
+ long startTime = System.currentTimeMillis();
+ boolean sizeMatches = false;
+ while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) {
+ sizeMatches = sent.size() == result.size();
+ Thread.sleep(200);
+ }
Assert.assertEquals(sent.size(), result.size());
for (String k : sent.keySet()) {
Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue());
}
+ ssc.stop();
}
}
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
index 6943326eb750e..b19c053ebfc44 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -19,51 +19,57 @@ package org.apache.spark.streaming.kafka
import java.io.File
import java.net.InetSocketAddress
-import java.util.{Properties, Random}
+import java.util.Properties
import scala.collection.mutable
+import scala.concurrent.duration._
+import scala.language.postfixOps
+import scala.util.Random
import kafka.admin.CreateTopicCommand
import kafka.common.{KafkaException, TopicAndPartition}
-import kafka.producer.{KeyedMessage, ProducerConfig, Producer}
-import kafka.utils.ZKStringSerializer
+import kafka.producer.{KeyedMessage, Producer, ProducerConfig}
import kafka.serializer.{StringDecoder, StringEncoder}
import kafka.server.{KafkaConfig, KafkaServer}
-
+import kafka.utils.ZKStringSerializer
import org.I0Itec.zkclient.ZkClient
+import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer}
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.scalatest.concurrent.Eventually
-import org.apache.zookeeper.server.ZooKeeperServer
-import org.apache.zookeeper.server.NIOServerCnxnFactory
-
-import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Milliseconds, StreamingContext}
import org.apache.spark.util.Utils
-class KafkaStreamSuite extends TestSuiteBase {
- import KafkaTestUtils._
-
- val zkHost = "localhost"
- var zkPort: Int = 0
- val zkConnectionTimeout = 6000
- val zkSessionTimeout = 6000
-
- protected var brokerPort = 9092
- protected var brokerConf: KafkaConfig = _
- protected var zookeeper: EmbeddedZookeeper = _
- protected var zkClient: ZkClient = _
- protected var server: KafkaServer = _
- protected var producer: Producer[String, String] = _
-
- override def useManualClock = false
-
- override def beforeFunction() {
+/**
+ * This is an abstract base class for Kafka testsuites. This has the functionality to set up
+ * and tear down local Kafka servers, and to push data using Kafka producers.
+ */
+abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging {
+
+ var zkAddress: String = _
+ var zkClient: ZkClient = _
+
+ private val zkHost = "localhost"
+ private val zkConnectionTimeout = 6000
+ private val zkSessionTimeout = 6000
+ private var zookeeper: EmbeddedZookeeper = _
+ private var zkPort: Int = 0
+ private var brokerPort = 9092
+ private var brokerConf: KafkaConfig = _
+ private var server: KafkaServer = _
+ private var producer: Producer[String, String] = _
+
+ def setupKafka() {
// Zookeeper server startup
zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort")
// Get the actual zookeeper binding port
zkPort = zookeeper.actualPort
+ zkAddress = s"$zkHost:$zkPort"
logInfo("==================== 0 ====================")
- zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout,
+ zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout,
ZKStringSerializer)
logInfo("==================== 1 ====================")
@@ -71,7 +77,7 @@ class KafkaStreamSuite extends TestSuiteBase {
var bindSuccess: Boolean = false
while(!bindSuccess) {
try {
- val brokerProps = getBrokerConfig(brokerPort, s"$zkHost:$zkPort")
+ val brokerProps = getBrokerConfig()
brokerConf = new KafkaConfig(brokerProps)
server = new KafkaServer(brokerConf)
logInfo("==================== 2 ====================")
@@ -89,53 +95,30 @@ class KafkaStreamSuite extends TestSuiteBase {
Thread.sleep(2000)
logInfo("==================== 4 ====================")
- super.beforeFunction()
}
- override def afterFunction() {
- producer.close()
- server.shutdown()
- brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) }
-
- zkClient.close()
- zookeeper.shutdown()
-
- super.afterFunction()
- }
-
- test("Kafka input stream") {
- val ssc = new StreamingContext(master, framework, batchDuration)
- val topic = "topic1"
- val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
- createTopic(topic)
- produceAndSendMessage(topic, sent)
+ def tearDownKafka() {
+ if (producer != null) {
+ producer.close()
+ producer = null
+ }
- val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort",
- "group.id" -> s"test-consumer-${random.nextInt(10000)}",
- "auto.offset.reset" -> "smallest")
+ if (server != null) {
+ server.shutdown()
+ server = null
+ }
- val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
- ssc,
- kafkaParams,
- Map(topic -> 1),
- StorageLevel.MEMORY_ONLY)
- val result = new mutable.HashMap[String, Long]()
- stream.map { case (k, v) => v }
- .countByValue()
- .foreachRDD { r =>
- val ret = r.collect()
- ret.toMap.foreach { kv =>
- val count = result.getOrElseUpdate(kv._1, 0) + kv._2
- result.put(kv._1, count)
- }
- }
- ssc.start()
- ssc.awaitTermination(3000)
+ brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) }
- assert(sent.size === result.size)
- sent.keys.foreach { k => assert(sent(k) === result(k).toInt) }
+ if (zkClient != null) {
+ zkClient.close()
+ zkClient = null
+ }
- ssc.stop()
+ if (zookeeper != null) {
+ zookeeper.shutdown()
+ zookeeper = null
+ }
}
private def createTestMessage(topic: String, sent: Map[String, Int])
@@ -150,58 +133,43 @@ class KafkaStreamSuite extends TestSuiteBase {
CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
logInfo("==================== 5 ====================")
// wait until metadata is propagated
- waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000)
+ waitUntilMetadataIsPropagated(topic, 0)
}
def produceAndSendMessage(topic: String, sent: Map[String, Int]) {
- val brokerAddr = brokerConf.hostName + ":" + brokerConf.port
- producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr)))
+ producer = new Producer[String, String](new ProducerConfig(getProducerConfig()))
producer.send(createTestMessage(topic, sent): _*)
+ producer.close()
logInfo("==================== 6 ====================")
}
-}
-
-object KafkaTestUtils {
- val random = new Random()
- def getBrokerConfig(port: Int, zkConnect: String): Properties = {
+ private def getBrokerConfig(): Properties = {
val props = new Properties()
props.put("broker.id", "0")
props.put("host.name", "localhost")
- props.put("port", port.toString)
+ props.put("port", brokerPort.toString)
props.put("log.dir", Utils.createTempDir().getAbsolutePath)
- props.put("zookeeper.connect", zkConnect)
+ props.put("zookeeper.connect", zkAddress)
props.put("log.flush.interval.messages", "1")
props.put("replica.socket.timeout.ms", "1500")
props
}
- def getProducerConfig(brokerList: String): Properties = {
+ private def getProducerConfig(): Properties = {
+ val brokerAddr = brokerConf.hostName + ":" + brokerConf.port
val props = new Properties()
- props.put("metadata.broker.list", brokerList)
+ props.put("metadata.broker.list", brokerAddr)
props.put("serializer.class", classOf[StringEncoder].getName)
props
}
- def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = {
- val startTime = System.currentTimeMillis()
- while (true) {
- if (condition())
- return true
- if (System.currentTimeMillis() > startTime + waitTime)
- return false
- Thread.sleep(waitTime.min(100L))
+ private def waitUntilMetadataIsPropagated(topic: String, partition: Int) {
+ eventually(timeout(1000 milliseconds), interval(100 milliseconds)) {
+ assert(
+ server.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition)),
+ s"Partition [$topic, $partition] metadata not propagated after timeout"
+ )
}
- // Should never go to here
- throw new RuntimeException("unexpected error")
- }
-
- def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int,
- timeout: Long) {
- assert(waitUntilTrue(() =>
- servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains(
- TopicAndPartition(topic, partition))), timeout),
- s"Partition [$topic, $partition] metadata not propagated after timeout")
}
class EmbeddedZookeeper(val zkConnect: String) {
@@ -227,3 +195,53 @@ object KafkaTestUtils {
}
}
}
+
+
+class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
+ var ssc: StreamingContext = _
+
+ before {
+ setupKafka()
+ }
+
+ after {
+ if (ssc != null) {
+ ssc.stop()
+ ssc = null
+ }
+ tearDownKafka()
+ }
+
+ test("Kafka input stream") {
+ val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName)
+ ssc = new StreamingContext(sparkConf, Milliseconds(500))
+ val topic = "topic1"
+ val sent = Map("a" -> 5, "b" -> 3, "c" -> 10)
+ createTopic(topic)
+ produceAndSendMessage(topic, sent)
+
+ val kafkaParams = Map("zookeeper.connect" -> zkAddress,
+ "group.id" -> s"test-consumer-${Random.nextInt(10000)}",
+ "auto.offset.reset" -> "smallest")
+
+ val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY)
+ val result = new mutable.HashMap[String, Long]()
+ stream.map(_._2).countByValue().foreachRDD { r =>
+ val ret = r.collect()
+ ret.toMap.foreach { kv =>
+ val count = result.getOrElseUpdate(kv._1, 0) + kv._2
+ result.put(kv._1, count)
+ }
+ }
+ ssc.start()
+ eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
+ assert(sent.size === result.size)
+ sent.keys.foreach { k =>
+ assert(sent(k) === result(k).toInt)
+ }
+ }
+ ssc.stop()
+ }
+}
+
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
new file mode 100644
index 0000000000000..64ccc92c81fa9
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
@@ -0,0 +1,140 @@
+/*
+ * 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.streaming.kafka
+
+
+import java.io.File
+
+import scala.collection.mutable
+import scala.concurrent.duration._
+import scala.language.postfixOps
+import scala.util.Random
+
+import com.google.common.io.Files
+import kafka.serializer.StringDecoder
+import kafka.utils.{ZKGroupTopicDirs, ZkUtils}
+import org.apache.commons.io.FileUtils
+import org.scalatest.BeforeAndAfter
+import org.scalatest.concurrent.Eventually
+
+import org.apache.spark.SparkConf
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Milliseconds, StreamingContext}
+
+class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually {
+
+ val sparkConf = new SparkConf()
+ .setMaster("local[4]")
+ .setAppName(this.getClass.getSimpleName)
+ .set("spark.streaming.receiver.writeAheadLog.enable", "true")
+ val data = Map("a" -> 10, "b" -> 10, "c" -> 10)
+
+
+ var groupId: String = _
+ var kafkaParams: Map[String, String] = _
+ var ssc: StreamingContext = _
+ var tempDirectory: File = null
+
+ before {
+ setupKafka()
+ groupId = s"test-consumer-${Random.nextInt(10000)}"
+ kafkaParams = Map(
+ "zookeeper.connect" -> zkAddress,
+ "group.id" -> groupId,
+ "auto.offset.reset" -> "smallest"
+ )
+
+ ssc = new StreamingContext(sparkConf, Milliseconds(500))
+ tempDirectory = Files.createTempDir()
+ ssc.checkpoint(tempDirectory.getAbsolutePath)
+ }
+
+ after {
+ if (ssc != null) {
+ ssc.stop()
+ }
+ if (tempDirectory != null && tempDirectory.exists()) {
+ FileUtils.deleteDirectory(tempDirectory)
+ tempDirectory = null
+ }
+ tearDownKafka()
+ }
+
+
+ test("Reliable Kafka input stream with single topic") {
+ var topic = "test-topic"
+ createTopic(topic)
+ produceAndSendMessage(topic, data)
+
+ // Verify whether the offset of this group/topic/partition is 0 before starting.
+ assert(getCommitOffset(groupId, topic, 0) === None)
+
+ val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY)
+ val result = new mutable.HashMap[String, Long]()
+ stream.map { case (k, v) => v }.foreachRDD { r =>
+ val ret = r.collect()
+ ret.foreach { v =>
+ val count = result.getOrElseUpdate(v, 0) + 1
+ result.put(v, count)
+ }
+ }
+ ssc.start()
+ eventually(timeout(20000 milliseconds), interval(200 milliseconds)) {
+ // A basic process verification for ReliableKafkaReceiver.
+ // Verify whether received message number is equal to the sent message number.
+ assert(data.size === result.size)
+ // Verify whether each message is the same as the data to be verified.
+ data.keys.foreach { k => assert(data(k) === result(k).toInt) }
+ // Verify the offset number whether it is equal to the total message number.
+ assert(getCommitOffset(groupId, topic, 0) === Some(29L))
+ }
+ ssc.stop()
+ }
+
+ test("Reliable Kafka input stream with multiple topics") {
+ val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1)
+ topics.foreach { case (t, _) =>
+ createTopic(t)
+ produceAndSendMessage(t, data)
+ }
+
+ // Before started, verify all the group/topic/partition offsets are 0.
+ topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === None) }
+
+ // Consuming all the data sent to the broker which will potential commit the offsets internally.
+ val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY)
+ stream.foreachRDD(_ => Unit)
+ ssc.start()
+ eventually(timeout(20000 milliseconds), interval(100 milliseconds)) {
+ // Verify the offset for each group/topic to see whether they are equal to the expected one.
+ topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) }
+ }
+ ssc.stop()
+ }
+
+
+ /** Getting partition offset from Zookeeper. */
+ private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = {
+ assert(zkClient != null, "Zookeeper client is not initialized")
+ val topicDirs = new ZKGroupTopicDirs(groupId, topic)
+ val zkPath = s"${topicDirs.consumerOffsetDir}/$partition"
+ ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong)
+ }
+}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index 869ef15893eb9..cc70b396a8dd4 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -17,6 +17,7 @@
package org.apache.spark.graphx
+import scala.language.existentials
import scala.reflect.ClassTag
import org.apache.spark.Dependency
@@ -36,16 +37,16 @@ import org.apache.spark.graphx.impl.EdgeRDDImpl
* edge to provide the triplet view. Shipping of the vertex attributes is managed by
* `impl.ReplicatedVertexView`.
*/
-abstract class EdgeRDD[ED, VD](
+abstract class EdgeRDD[ED](
@transient sc: SparkContext,
@transient deps: Seq[Dependency[_]]) extends RDD[Edge[ED]](sc, deps) {
- private[graphx] def partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])]
+ private[graphx] def partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])] forSome { type VD }
override protected def getPartitions: Array[Partition] = partitionsRDD.partitions
override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = {
- val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context)
+ val p = firstParent[(PartitionID, EdgePartition[ED, _])].iterator(part, context)
if (p.hasNext) {
p.next._2.iterator.map(_.copy())
} else {
@@ -60,19 +61,14 @@ abstract class EdgeRDD[ED, VD](
* @param f the function from an edge to a new edge value
* @return a new EdgeRDD containing the new edge values
*/
- def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD]
+ def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2]
/**
* Reverse all the edges in this RDD.
*
* @return a new EdgeRDD containing all the edges reversed
*/
- def reverse: EdgeRDD[ED, VD]
-
- /** Removes all edges but those matching `epred` and where both vertices match `vpred`. */
- def filter(
- epred: EdgeTriplet[VD, ED] => Boolean,
- vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD]
+ def reverse: EdgeRDD[ED]
/**
* Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same
@@ -84,15 +80,8 @@ abstract class EdgeRDD[ED, VD](
* with values supplied by `f`
*/
def innerJoin[ED2: ClassTag, ED3: ClassTag]
- (other: EdgeRDD[ED2, _])
- (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD]
-
- private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag](
- f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2]
-
- /** Replaces the edge partitions while preserving all other properties of the EdgeRDD. */
- private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag](
- partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDD[ED2, VD2]
+ (other: EdgeRDD[ED2])
+ (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3]
/**
* Changes the target storage level while preserving all other properties of the
@@ -101,7 +90,7 @@ abstract class EdgeRDD[ED, VD](
* This does not actually trigger a cache; to do this, call
* [[org.apache.spark.graphx.EdgeRDD#cache]] on the returned EdgeRDD.
*/
- private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): EdgeRDD[ED, VD]
+ private[graphx] def withTargetStorageLevel(targetStorageLevel: StorageLevel): EdgeRDD[ED]
}
object EdgeRDD {
@@ -111,7 +100,7 @@ object EdgeRDD {
* @tparam ED the edge attribute type
* @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD
*/
- def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = {
+ def fromEdges[ED: ClassTag, VD: ClassTag](edges: RDD[Edge[ED]]): EdgeRDDImpl[ED, VD] = {
val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) =>
val builder = new EdgePartitionBuilder[ED, VD]
iter.foreach { e =>
@@ -128,8 +117,8 @@ object EdgeRDD {
* @tparam ED the edge attribute type
* @tparam VD the type of the vertex attributes that may be joined with the returned EdgeRDD
*/
- def fromEdgePartitions[ED: ClassTag, VD: ClassTag](
- edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDD[ED, VD] = {
+ private[graphx] def fromEdgePartitions[ED: ClassTag, VD: ClassTag](
+ edgePartitions: RDD[(Int, EdgePartition[ED, VD])]): EdgeRDDImpl[ED, VD] = {
new EdgeRDDImpl(edgePartitions)
}
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 2c1b9518a3d16..637791543514c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
* along with their vertex data.
*
*/
- @transient val edges: EdgeRDD[ED, VD]
+ @transient val edges: EdgeRDD[ED]
/**
* An RDD containing the edge triplets, which are edges along with the vertex data associated with
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index f8be17669d892..1db3df03c8052 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -207,7 +207,7 @@ abstract class VertexRDD[VD](
def reverseRoutingTables(): VertexRDD[VD]
/** Prepares this VertexRDD for efficient joins with the given EdgeRDD. */
- def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD]
+ def withEdges(edges: EdgeRDD[_]): VertexRDD[VD]
/** Replaces the vertex partitions while preserving all other properties of the VertexRDD. */
private[graphx] def withPartitionsRDD[VD2: ClassTag](
@@ -269,7 +269,7 @@ object VertexRDD {
* @param defaultVal the vertex attribute to use when creating missing vertices
*/
def apply[VD: ClassTag](
- vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = {
+ vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_], defaultVal: VD): VertexRDD[VD] = {
VertexRDD(vertices, edges, defaultVal, (a, b) => a)
}
@@ -286,7 +286,7 @@ object VertexRDD {
* @param mergeFunc the commutative, associative duplicate vertex attribute merge function
*/
def apply[VD: ClassTag](
- vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD, mergeFunc: (VD, VD) => VD
+ vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_], defaultVal: VD, mergeFunc: (VD, VD) => VD
): VertexRDD[VD] = {
val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match {
case Some(p) => vertices
@@ -314,7 +314,7 @@ object VertexRDD {
* @param defaultVal the vertex attribute to use when creating missing vertices
*/
def fromEdges[VD: ClassTag](
- edges: EdgeRDD[_, _], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = {
+ edges: EdgeRDD[_], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = {
val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions))
val vertexPartitions = routingTables.mapPartitions({ routingTableIter =>
val routingTable =
@@ -325,7 +325,7 @@ object VertexRDD {
}
private[graphx] def createRoutingTables(
- edges: EdgeRDD[_, _], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = {
+ edges: EdgeRDD[_], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = {
// Determine which vertices each edge partition needs by creating a mapping from vid to pid.
val vid2pid = edges.partitionsRDD.mapPartitions(_.flatMap(
Function.tupled(RoutingTablePartition.edgePartitionToMsgs)))
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
index 4100a85d17ee3..a8169613b4fd2 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
@@ -28,7 +28,7 @@ import org.apache.spark.graphx._
class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])],
val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY)
- extends EdgeRDD[ED, VD](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
+ extends EdgeRDD[ED](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
override def setName(_name: String): this.type = {
if (partitionsRDD.name != null) {
@@ -75,20 +75,20 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
partitionsRDD.map(_._2.size.toLong).reduce(_ + _)
}
- override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] =
+ override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDDImpl[ED2, VD] =
mapEdgePartitions((pid, part) => part.map(f))
- override def reverse: EdgeRDD[ED, VD] = mapEdgePartitions((pid, part) => part.reverse)
+ override def reverse: EdgeRDDImpl[ED, VD] = mapEdgePartitions((pid, part) => part.reverse)
- override def filter(
+ def filter(
epred: EdgeTriplet[VD, ED] => Boolean,
- vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] = {
+ vpred: (VertexId, VD) => Boolean): EdgeRDDImpl[ED, VD] = {
mapEdgePartitions((pid, part) => part.filter(epred, vpred))
}
override def innerJoin[ED2: ClassTag, ED3: ClassTag]
- (other: EdgeRDD[ED2, _])
- (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = {
+ (other: EdgeRDD[ED2])
+ (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDDImpl[ED3, VD] = {
val ed2Tag = classTag[ED2]
val ed3Tag = classTag[ED3]
this.withPartitionsRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) {
@@ -99,8 +99,8 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
})
}
- override private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag](
- f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = {
+ def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag](
+ f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDDImpl[ED2, VD2] = {
this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter =>
if (iter.hasNext) {
val (pid, ep) = iter.next()
@@ -111,13 +111,13 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
}, preservesPartitioning = true))
}
- override private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag](
- partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDD[ED2, VD2] = {
+ private[graphx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag](
+ partitionsRDD: RDD[(PartitionID, EdgePartition[ED2, VD2])]): EdgeRDDImpl[ED2, VD2] = {
new EdgeRDDImpl(partitionsRDD, this.targetStorageLevel)
}
override private[graphx] def withTargetStorageLevel(
- targetStorageLevel: StorageLevel): EdgeRDD[ED, VD] = {
+ targetStorageLevel: StorageLevel): EdgeRDDImpl[ED, VD] = {
new EdgeRDDImpl(this.partitionsRDD, targetStorageLevel)
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 2b4636a6c6ddf..0eae2a673874a 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -43,7 +43,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
/** Default constructor is provided to support serialization */
protected def this() = this(null, null)
- @transient override val edges: EdgeRDD[ED, VD] = replicatedVertexView.edges
+ @transient override val edges: EdgeRDDImpl[ED, VD] = replicatedVertexView.edges
/** Return a RDD that brings edges together with their source and destination vertices. */
@transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = {
@@ -323,9 +323,10 @@ object GraphImpl {
*/
def apply[VD: ClassTag, ED: ClassTag](
vertices: VertexRDD[VD],
- edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = {
+ edges: EdgeRDD[ED]): GraphImpl[VD, ED] = {
// Convert the vertex partitions in edges to the correct type
- val newEdges = edges.mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD])
+ val newEdges = edges.asInstanceOf[EdgeRDDImpl[ED, _]]
+ .mapEdgePartitions((pid, part) => part.withoutVertexAttributes[VD])
GraphImpl.fromExistingRDDs(vertices, newEdges)
}
@@ -336,8 +337,8 @@ object GraphImpl {
*/
def fromExistingRDDs[VD: ClassTag, ED: ClassTag](
vertices: VertexRDD[VD],
- edges: EdgeRDD[ED, VD]): GraphImpl[VD, ED] = {
- new GraphImpl(vertices, new ReplicatedVertexView(edges))
+ edges: EdgeRDD[ED]): GraphImpl[VD, ED] = {
+ new GraphImpl(vertices, new ReplicatedVertexView(edges.asInstanceOf[EdgeRDDImpl[ED, VD]]))
}
/**
@@ -345,7 +346,7 @@ object GraphImpl {
* `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD.
*/
private def fromEdgeRDD[VD: ClassTag, ED: ClassTag](
- edges: EdgeRDD[ED, VD],
+ edges: EdgeRDDImpl[ED, VD],
defaultVertexAttr: VD,
edgeStorageLevel: StorageLevel,
vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = {
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
index 86b366eb9202b..8ab255bd4038c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
@@ -33,7 +33,7 @@ import org.apache.spark.graphx._
*/
private[impl]
class ReplicatedVertexView[VD: ClassTag, ED: ClassTag](
- var edges: EdgeRDD[ED, VD],
+ var edges: EdgeRDDImpl[ED, VD],
var hasSrcId: Boolean = false,
var hasDstId: Boolean = false) {
@@ -42,7 +42,7 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag](
* shipping level.
*/
def withEdges[VD2: ClassTag, ED2: ClassTag](
- edges_ : EdgeRDD[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = {
+ edges_ : EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = {
new ReplicatedVertexView(edges_, hasSrcId, hasDstId)
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
index 08405629bc052..d92a55a189298 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala
@@ -172,7 +172,7 @@ class VertexRDDImpl[VD] private[graphx] (
override def reverseRoutingTables(): VertexRDD[VD] =
this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse))
- override def withEdges(edges: EdgeRDD[_, _]): VertexRDD[VD] = {
+ override def withEdges(edges: EdgeRDD[_]): VertexRDD[VD] = {
val routingTables = VertexRDD.createRoutingTables(edges, this.partitioner.get)
val vertexPartitions = partitionsRDD.zipPartitions(routingTables, true) {
(partIter, routingTableIter) =>
diff --git a/make-distribution.sh b/make-distribution.sh
index 55cbdc14ac055..2267b1aa08a6c 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -181,7 +181,9 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI
# Copy jars
cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
-cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/"
+# This will fail if the -Pyarn profile is not provided
+# In this case, silence the error and ignore the return code of this command
+cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
# Copy example sources (needed for python and SQL)
mkdir -p "$DISTDIR/examples/src/main"
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index 70d7138e3060f..6f94b7f483ee0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -28,22 +28,22 @@ import net.razorvine.pickle._
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
-import org.apache.spark.api.python.{PythonRDD, SerDeUtil}
+import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.mllib.classification._
import org.apache.spark.mllib.clustering._
import org.apache.spark.mllib.feature._
-import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.linalg._
+import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.random.{RandomRDDs => RG}
import org.apache.spark.mllib.recommendation._
import org.apache.spark.mllib.regression._
-import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
-import org.apache.spark.mllib.tree.DecisionTree
-import org.apache.spark.mllib.tree.impurity._
-import org.apache.spark.mllib.tree.model.DecisionTreeModel
import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics}
import org.apache.spark.mllib.stat.correlation.CorrelationNames
import org.apache.spark.mllib.stat.test.ChiSqTestResult
+import org.apache.spark.mllib.tree.DecisionTree
+import org.apache.spark.mllib.tree.configuration.{Algo, Strategy}
+import org.apache.spark.mllib.tree.impurity._
+import org.apache.spark.mllib.tree.model.DecisionTreeModel
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
@@ -103,9 +103,11 @@ class PythonMLLibAPI extends Serializable {
lrAlg.optimizer.setUpdater(new SquaredL2Updater)
} else if (regType == "l1") {
lrAlg.optimizer.setUpdater(new L1Updater)
- } else if (regType != "none") {
- throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
- + " Can only be initialized using the following string values: [l1, l2, none].")
+ } else if (regType == null) {
+ lrAlg.optimizer.setUpdater(new SimpleUpdater)
+ } else {
+ throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
+ + " Can only be initialized using the following string values: ['l1', 'l2', None].")
}
trainRegressionModel(
lrAlg,
@@ -180,9 +182,11 @@ class PythonMLLibAPI extends Serializable {
SVMAlg.optimizer.setUpdater(new SquaredL2Updater)
} else if (regType == "l1") {
SVMAlg.optimizer.setUpdater(new L1Updater)
- } else if (regType != "none") {
+ } else if (regType == null) {
+ SVMAlg.optimizer.setUpdater(new SimpleUpdater)
+ } else {
throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
- + " Can only be initialized using the following string values: [l1, l2, none].")
+ + " Can only be initialized using the following string values: ['l1', 'l2', None].")
}
trainRegressionModel(
SVMAlg,
@@ -213,9 +217,46 @@ class PythonMLLibAPI extends Serializable {
LogRegAlg.optimizer.setUpdater(new SquaredL2Updater)
} else if (regType == "l1") {
LogRegAlg.optimizer.setUpdater(new L1Updater)
- } else if (regType != "none") {
+ } else if (regType == null) {
+ LogRegAlg.optimizer.setUpdater(new SimpleUpdater)
+ } else {
+ throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
+ + " Can only be initialized using the following string values: ['l1', 'l2', None].")
+ }
+ trainRegressionModel(
+ LogRegAlg,
+ data,
+ initialWeights)
+ }
+
+ /**
+ * Java stub for Python mllib LogisticRegressionWithLBFGS.train()
+ */
+ def trainLogisticRegressionModelWithLBFGS(
+ data: JavaRDD[LabeledPoint],
+ numIterations: Int,
+ initialWeights: Vector,
+ regParam: Double,
+ regType: String,
+ intercept: Boolean,
+ corrections: Int,
+ tolerance: Double): JList[Object] = {
+ val LogRegAlg = new LogisticRegressionWithLBFGS()
+ LogRegAlg.setIntercept(intercept)
+ LogRegAlg.optimizer
+ .setNumIterations(numIterations)
+ .setRegParam(regParam)
+ .setNumCorrections(corrections)
+ .setConvergenceTol(tolerance)
+ if (regType == "l2") {
+ LogRegAlg.optimizer.setUpdater(new SquaredL2Updater)
+ } else if (regType == "l1") {
+ LogRegAlg.optimizer.setUpdater(new L1Updater)
+ } else if (regType == null) {
+ LogRegAlg.optimizer.setUpdater(new SimpleUpdater)
+ } else {
throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter."
- + " Can only be initialized using the following string values: [l1, l2, none].")
+ + " Can only be initialized using the following string values: ['l1', 'l2', None].")
}
trainRegressionModel(
LogRegAlg,
@@ -250,7 +291,7 @@ class PythonMLLibAPI extends Serializable {
.setInitializationMode(initializationMode)
// Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD.
.disableUncachedWarning()
- return kMeansAlg.run(data.rdd)
+ kMeansAlg.run(data.rdd)
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index 84d3c7cebd7c8..94d757bc317ab 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -64,16 +64,17 @@ class LogisticRegressionModel (
val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
val score = 1.0 / (1.0 + math.exp(-margin))
threshold match {
- case Some(t) => if (score < t) 0.0 else 1.0
+ case Some(t) => if (score > t) 1.0 else 0.0
case None => score
}
}
}
/**
- * Train a classification model for Logistic Regression using Stochastic Gradient Descent.
- * NOTE: Labels used in Logistic Regression should be {0, 1}
- *
+ * Train a classification model for Logistic Regression using Stochastic Gradient Descent. By
+ * default L2 regularization is used, which can be changed via
+ * [[LogisticRegressionWithSGD.optimizer]].
+ * NOTE: Labels used in Logistic Regression should be {0, 1}.
* Using [[LogisticRegressionWithLBFGS]] is recommended over this.
*/
class LogisticRegressionWithSGD private (
@@ -93,9 +94,10 @@ class LogisticRegressionWithSGD private (
override protected val validators = List(DataValidators.binaryLabelValidator)
/**
- * Construct a LogisticRegression object with default parameters
+ * Construct a LogisticRegression object with default parameters: {stepSize: 1.0,
+ * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}.
*/
- def this() = this(1.0, 100, 0.0, 1.0)
+ def this() = this(1.0, 100, 0.01, 1.0)
override protected def createModel(weights: Vector, intercept: Double) = {
new LogisticRegressionModel(weights, intercept)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index 80f8a1b2f1e84..dd514ff8a37f2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -65,14 +65,15 @@ class SVMModel (
intercept: Double) = {
val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
threshold match {
- case Some(t) => if (margin < t) 0.0 else 1.0
+ case Some(t) => if (margin > t) 1.0 else 0.0
case None => margin
}
}
}
/**
- * Train a Support Vector Machine (SVM) using Stochastic Gradient Descent.
+ * Train a Support Vector Machine (SVM) using Stochastic Gradient Descent. By default L2
+ * regularization is used, which can be changed via [[SVMWithSGD.optimizer]].
* NOTE: Labels used in SVM should be {0, 1}.
*/
class SVMWithSGD private (
@@ -92,9 +93,10 @@ class SVMWithSGD private (
override protected val validators = List(DataValidators.binaryLabelValidator)
/**
- * Construct a SVM object with default parameters
+ * Construct a SVM object with default parameters: {stepSize: 1.0, numIterations: 100,
+ * regParm: 0.01, miniBatchFraction: 1.0}.
*/
- def this() = this(1.0, 100, 1.0, 1.0)
+ def this() = this(1.0, 100, 0.01, 1.0)
override protected def createModel(weights: Vector, intercept: Double) = {
new SVMModel(weights, intercept)
@@ -185,6 +187,6 @@ object SVMWithSGD {
* @return a SVMModel which has the weights and offset from training.
*/
def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = {
- train(input, numIterations, 1.0, 1.0, 1.0)
+ train(input, numIterations, 1.0, 0.01, 1.0)
}
}
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 9fccd6341ba7d..60ab2aaa8f27a 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
@@ -237,7 +237,7 @@ object Vectors {
private[mllib] def fromBreeze(breezeVector: BV[Double]): Vector = {
breezeVector match {
case v: BDV[Double] =>
- if (v.offset == 0 && v.stride == 1) {
+ if (v.offset == 0 && v.stride == 1 && v.length == v.data.length) {
new DenseVector(v.data)
} else {
new DenseVector(v.toArray) // Can't use underlying array directly, so make a new one
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 84d192db53e26..90ac252226006 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -20,20 +20,20 @@ package org.apache.spark.mllib.recommendation
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.math.{abs, sqrt}
-import scala.util.Random
-import scala.util.Sorting
+import scala.util.{Random, Sorting}
import scala.util.hashing.byteswap32
import org.jblas.{DoubleMatrix, SimpleBlas, Solve}
+import org.apache.spark.{HashPartitioner, Logging, Partitioner}
+import org.apache.spark.SparkContext._
import org.apache.spark.annotation.{DeveloperApi, Experimental}
+import org.apache.spark.api.java.JavaRDD
import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.{Logging, HashPartitioner, Partitioner}
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.mllib.optimization.NNLS
import org.apache.spark.rdd.RDD
-import org.apache.spark.SparkContext._
+import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
-import org.apache.spark.mllib.optimization.NNLS
/**
* Out-link information for a user or product block. This includes the original user/product IDs
@@ -325,6 +325,11 @@ class ALS private (
new MatrixFactorizationModel(rank, usersOut, productsOut)
}
+ /**
+ * Java-friendly version of [[ALS.run]].
+ */
+ def run(ratings: JavaRDD[Rating]): MatrixFactorizationModel = run(ratings.rdd)
+
/**
* Computes the (`rank x rank`) matrix `YtY`, where `Y` is the (`nui x rank`) matrix of factors
* for each user (or product), in a distributed fashion.
@@ -741,7 +746,7 @@ object ALS {
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
* @param blocks level of parallelism to split computation into
- * @param alpha confidence parameter (only applies when immplicitPrefs = true)
+ * @param alpha confidence parameter
* @param seed random seed
*/
def trainImplicit(
@@ -768,7 +773,7 @@ object ALS {
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
* @param blocks level of parallelism to split computation into
- * @param alpha confidence parameter (only applies when immplicitPrefs = true)
+ * @param alpha confidence parameter
*/
def trainImplicit(
ratings: RDD[Rating],
@@ -792,6 +797,7 @@ object ALS {
* @param rank number of features to use
* @param iterations number of iterations of ALS (recommended: 10-20)
* @param lambda regularization factor (recommended: 0.01)
+ * @param alpha confidence parameter
*/
def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double)
: MatrixFactorizationModel = {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index 66b58ba770160..969e23be21623 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -17,13 +17,13 @@
package org.apache.spark.mllib.recommendation
+import java.lang.{Integer => JavaInteger}
+
import org.jblas.DoubleMatrix
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.api.java.JavaRDD
-import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
-import org.apache.spark.mllib.api.python.SerDe
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
+import org.apache.spark.rdd.RDD
/**
* Model representing the result of matrix factorization.
@@ -65,6 +65,13 @@ class MatrixFactorizationModel private[mllib] (
}
}
+ /**
+ * Java-friendly version of [[MatrixFactorizationModel.predict]].
+ */
+ def predict(usersProducts: JavaPairRDD[JavaInteger, JavaInteger]): JavaRDD[Rating] = {
+ predict(usersProducts.rdd.asInstanceOf[RDD[(Int, Int)]]).toJavaRDD()
+ }
+
/**
* Recommends products to a user.
*
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index cb0d39e759a9f..f9791c6571782 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -67,9 +67,9 @@ class LassoWithSGD private (
/**
* Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100,
- * regParam: 1.0, miniBatchFraction: 1.0}.
+ * regParam: 0.01, miniBatchFraction: 1.0}.
*/
- def this() = this(1.0, 100, 1.0, 1.0)
+ def this() = this(1.0, 100, 0.01, 1.0)
override protected def createModel(weights: Vector, intercept: Double) = {
new LassoModel(weights, intercept)
@@ -161,6 +161,6 @@ object LassoWithSGD {
def train(
input: RDD[LabeledPoint],
numIterations: Int): LassoModel = {
- train(input, numIterations, 1.0, 1.0, 1.0)
+ train(input, numIterations, 1.0, 0.01, 1.0)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index a826deb695ee1..c8cad773f5efb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -68,9 +68,9 @@ class RidgeRegressionWithSGD private (
/**
* Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100,
- * regParam: 1.0, miniBatchFraction: 1.0}.
+ * regParam: 0.01, miniBatchFraction: 1.0}.
*/
- def this() = this(1.0, 100, 1.0, 1.0)
+ def this() = this(1.0, 100, 0.01, 1.0)
override protected def createModel(weights: Vector, intercept: Double) = {
new RidgeRegressionModel(weights, intercept)
@@ -143,7 +143,7 @@ object RidgeRegressionWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double): RidgeRegressionModel = {
- train(input, numIterations, stepSize, regParam, 1.0)
+ train(input, numIterations, stepSize, regParam, 0.01)
}
/**
@@ -158,6 +158,6 @@ object RidgeRegressionWithSGD {
def train(
input: RDD[LabeledPoint],
numIterations: Int): RidgeRegressionModel = {
- train(input, numIterations, 1.0, 1.0, 1.0)
+ train(input, numIterations, 1.0, 0.01, 1.0)
}
}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
index f6ca9643227f8..af688c504cf1e 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
@@ -23,13 +23,14 @@
import scala.Tuple2;
import scala.Tuple3;
+import com.google.common.collect.Lists;
import org.jblas.DoubleMatrix;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -47,61 +48,48 @@ public void tearDown() {
sc = null;
}
- static void validatePrediction(
+ void validatePrediction(
MatrixFactorizationModel model,
int users,
int products,
- int features,
DoubleMatrix trueRatings,
double matchThreshold,
boolean implicitPrefs,
DoubleMatrix truePrefs) {
- DoubleMatrix predictedU = new DoubleMatrix(users, features);
- List> userFeatures = model.userFeatures().toJavaRDD().collect();
- for (int i = 0; i < features; ++i) {
- for (Tuple2
-
- scala-2.10
-
-
-
- org.codehaus.mojo
- build-helper-maven-plugin
-
-
- add-scala-sources
- generate-sources
-
- add-source
-
-
-
- src/main/scala
- scala-2.10/src/main/scala
-
-
-
-
- add-scala-test-sources
- generate-test-sources
-
- add-test-source
-
-
-
- src/test/scala
- scala-2.10/src/test/scala
-
-
-
-
-
-
-
- scala-2.11
-
-
-
- org.codehaus.mojo
- build-helper-maven-plugin
-
-
- add-scala-sources
- generate-sources
-
- add-source
-
-
-
- src/main/scala
- scala-2.11/src/main/scala
-
-
-
-
- add-scala-test-sources
- generate-test-sources
-
- add-test-source
-
-
-
- src/test/scala
- scala-2.11/src/test/scala
-
-
-
-
-
-
-
+
+ scala-2.11
+
+
+ scala-2.11/src/main/scala
+ scala-2.11/src/test/scala
+
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index 0cc3175b6a2a2..686d189d810a8 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -100,10 +100,11 @@
+
scala-2.10
- true
+ !scala-2.11
@@ -113,13 +114,5 @@
-
- scala-2.11
-
- false
-
-
-
-
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 a448c794213ae..d3b4cf8e34242 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
@@ -60,7 +60,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
ResolveFunctions ::
GlobalAggregates ::
UnresolvedHavingClauseAttributes ::
- TrimAliases ::
+ TrimGroupingAliases ::
typeCoercionRules ++
extendedRules : _*),
Batch("Check Analysis", Once,
@@ -93,17 +93,10 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
/**
* Removes no-op Alias expressions from the plan.
*/
- object TrimAliases extends Rule[LogicalPlan] {
+ object TrimGroupingAliases extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case Aggregate(groups, aggs, child) =>
- Aggregate(
- groups.map {
- _ transform {
- case Alias(c, _) => c
- }
- },
- aggs,
- child)
+ Aggregate(groups.map(_.transform { case Alias(c, _) => c }), aggs, child)
}
}
@@ -122,10 +115,15 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
case e => e.children.forall(isValidAggregateExpression)
}
- aggregateExprs.foreach { e =>
- if (!isValidAggregateExpression(e)) {
- throw new TreeNodeException(plan, s"Expression not in GROUP BY: $e")
- }
+ aggregateExprs.find { e =>
+ !isValidAggregateExpression(e.transform {
+ // Should trim aliases around `GetField`s. These aliases are introduced while
+ // resolving struct field accesses, because `GetField` is not a `NamedExpression`.
+ // (Should we just turn `GetField` into a `NamedExpression`?)
+ case Alias(g: GetField, _) => g
+ })
+ }.foreach { e =>
+ throw new TreeNodeException(plan, s"Expression not in GROUP BY: $e")
}
aggregatePlan
@@ -328,4 +326,3 @@ object EliminateAnalysisOperators extends Rule[LogicalPlan] {
case Subquery(_, child) => child
}
}
-
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 55319e7a79103..b47865f87a3aa 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
@@ -32,11 +32,15 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
override def nullable = (child.dataType, dataType) match {
case (StringType, _: NumericType) => true
case (StringType, TimestampType) => true
+ case (DoubleType, TimestampType) => true
+ case (FloatType, TimestampType) => true
case (StringType, DateType) => true
case (_: NumericType, DateType) => true
case (BooleanType, DateType) => true
case (DateType, _: NumericType) => true
case (DateType, BooleanType) => true
+ case (DoubleType, _: DecimalType) => true
+ case (FloatType, _: DecimalType) => true
case (_, DecimalType.Fixed(_, _)) => true // TODO: not all upcasts here can really give null
case _ => child.nullable
}
@@ -115,10 +119,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
buildCast[Decimal](_, d => decimalToTimestamp(d))
// TimestampWritable.doubleToTimestamp
case DoubleType =>
- buildCast[Double](_, d => decimalToTimestamp(Decimal(d)))
+ buildCast[Double](_, d => try {
+ decimalToTimestamp(Decimal(d))
+ } catch {
+ case _: NumberFormatException => null
+ })
// TimestampWritable.floatToTimestamp
case FloatType =>
- buildCast[Float](_, f => decimalToTimestamp(Decimal(f)))
+ buildCast[Float](_, f => try {
+ decimalToTimestamp(Decimal(f))
+ } catch {
+ case _: NumberFormatException => null
+ })
}
private[this] def decimalToTimestamp(d: Decimal) = {
@@ -290,7 +302,11 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
case LongType =>
b => changePrecision(Decimal(b.asInstanceOf[Long]), target)
case x: NumericType => // All other numeric types can be represented precisely as Doubles
- b => changePrecision(Decimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)), target)
+ b => try {
+ changePrecision(Decimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)), target)
+ } catch {
+ case _: NumberFormatException => null
+ }
}
// DoubleConverter
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 d00ec39774c35..463f3667fc445 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
@@ -215,9 +215,11 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row {
def copy() = this
}
-class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow {
+class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow {
/** No-arg constructor for serialization. */
- def this() = this(0)
+ def this() = this(null)
+
+ def this(size: Int) = this(new Array[Any](size))
override def setBoolean(ordinal: Int, value: Boolean): Unit = { values(ordinal) = value }
override def setByte(ordinal: Int, value: Byte): Unit = { values(ordinal) = value }
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 2b364fc1df1d8..3ceb5ecaf66e4 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
@@ -304,7 +304,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN
child.dataType match {
case DecimalType.Fixed(_, _) =>
- // Turn the results to unlimited decimals for the divsion, before going back to fixed
+ // Turn the results to unlimited decimals for the division, before going back to fixed
val castedSum = Cast(Sum(partialSum.toAttribute), DecimalType.Unlimited)
val castedCount = Cast(Sum(partialCount.toAttribute), DecimalType.Unlimited)
SplitEvaluation(
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 8574cabc43525..d17c9553ac24e 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
@@ -142,7 +142,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme
* A function that calculates bitwise or(|) of two numbers.
*/
case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic {
- def symbol = "&"
+ def symbol = "|"
override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match {
case ByteType => (evalE1.asInstanceOf[Byte] | evalE2.asInstanceOf[Byte]).toByte
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 7871a62620478..2ff61169a17db 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
@@ -53,8 +53,8 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
val nullFunctions =
q"""
private[this] var nullBits = new Array[Boolean](${expressions.size})
- final def setNullAt(i: Int) = { nullBits(i) = true }
- final def isNullAt(i: Int) = nullBits(i)
+ override def setNullAt(i: Int) = { nullBits(i) = true }
+ override def isNullAt(i: Int) = nullBits(i)
""".children
val tupleElements = expressions.zipWithIndex.flatMap {
@@ -82,7 +82,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
val iLit = ru.Literal(Constant(i))
q"if(isNullAt($iLit)) { null } else { ${newTermName(s"c$i")} }"
}
- q"final def iterator = Iterator[Any](..$allColumns)"
+ q"override def iterator = Iterator[Any](..$allColumns)"
}
val accessorFailure = q"""scala.sys.error("Invalid ordinal:" + i)"""
@@ -94,7 +94,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
q"if(i == $ordinal) { if(isNullAt($i)) return null else return $elementName }"
}
- q"final def apply(i: Int): Any = { ..$cases; $accessorFailure }"
+ q"override def apply(i: Int): Any = { ..$cases; $accessorFailure }"
}
val updateFunction = {
@@ -114,7 +114,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
return
}"""
}
- q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }"
+ q"override def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }"
}
val specificAccessorFunctions = NativeType.all.map { dataType =>
@@ -128,7 +128,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
}
q"""
- final def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = {
+ override def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = {
..$ifStatements;
$accessorFailure
}"""
@@ -145,7 +145,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
}
q"""
- final def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = {
+ override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = {
..$ifStatements;
$accessorFailure
}"""
@@ -193,7 +193,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
val copyFunction =
q"""
- final def copy() = new $genericRowType(this.toArray)
+ override def copy() = new $genericRowType(this.toArray)
"""
val classBody =
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 19421e5667138..917b346086dcb 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
@@ -115,7 +115,9 @@ case class CreateArray(children: Seq[Expression]) extends Expression {
override def dataType: DataType = {
assert(resolved, s"Invalid dataType of mixed ArrayType ${childTypes.mkString(",")}")
- ArrayType(childTypes.headOption.getOrElse(NullType))
+ ArrayType(
+ childTypes.headOption.getOrElse(NullType),
+ containsNull = children.exists(_.nullable))
}
override def nullable: Boolean = false
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 fc90a54a58259..7634d392d4111 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
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.util.Metadata
object NamedExpression {
private val curId = new java.util.concurrent.atomic.AtomicLong()
def newExprId = ExprId(curId.getAndIncrement())
+ def unapply(expr: NamedExpression): Option[(String, DataType)] = Some(expr.name, expr.dataType)
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index 086d0a3e073e5..84a3567895175 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -37,7 +37,9 @@ case class Coalesce(children: Seq[Expression]) extends Expression {
def dataType = if (resolved) {
children.head.dataType
} else {
- throw new UnresolvedException(this, "Coalesce cannot have children of different types.")
+ val childTypes = children.map(c => s"$c: ${c.dataType}").mkString(", ")
+ throw new UnresolvedException(
+ this, s"Coalesce cannot have children of different types. $childTypes")
}
override def eval(input: Row): Any = {
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 1e22b2d03c672..94b6fb084d38a 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
@@ -99,10 +99,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate {
* Optimized version of In clause, when all filter values of In clause are
* static.
*/
-case class InSet(value: Expression, hset: HashSet[Any], child: Seq[Expression])
+case class InSet(value: Expression, hset: Set[Any])
extends Predicate {
- def children = child
+ def children = value :: Nil
def nullable = true // TODO: Figure out correct nullability semantics of IN.
override def toString = s"$value INSET ${hset.mkString("(", ",", ")")}"
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 a4aa322fc52d8..f164a6c68a0de 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
@@ -289,7 +289,7 @@ object OptimizeIn extends Rule[LogicalPlan] {
case q: LogicalPlan => q transformExpressionsDown {
case In(v, list) if !list.exists(!_.isInstanceOf[Literal]) =>
val hSet = list.map(e => e.eval(null))
- InSet(v, HashSet() ++ hSet, v +: list)
+ InSet(v, HashSet() ++ hSet)
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index f0fd9a8b9a46e..310d127506d68 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -151,8 +151,15 @@ object PartialAggregation {
val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp {
case e: Expression if partialEvaluations.contains(new TreeNodeRef(e)) =>
partialEvaluations(new TreeNodeRef(e)).finalEvaluation
- case e: Expression if namedGroupingExpressions.contains(e) =>
- namedGroupingExpressions(e).toAttribute
+
+ case e: Expression =>
+ // Should trim aliases around `GetField`s. These aliases are introduced while
+ // resolving struct field accesses, because `GetField` is not a `NamedExpression`.
+ // (Should we just turn `GetField` into a `NamedExpression`?)
+ namedGroupingExpressions
+ .get(e.transform { case Alias(g: GetField, _) => g })
+ .map(_.toAttribute)
+ .getOrElse(e)
}).asInstanceOf[Seq[NamedExpression]]
val partialComputation =
@@ -188,7 +195,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
logDebug(s"Considering join on: $condition")
// Find equi-join predicates that can be evaluated before the join, and thus can be used
// as join keys.
- val (joinPredicates, otherPredicates) =
+ val (joinPredicates, otherPredicates) =
condition.map(splitConjunctivePredicates).getOrElse(Nil).partition {
case EqualTo(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) ||
(canEvaluate(l, right) && canEvaluate(r, left)) => true
@@ -203,7 +210,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper {
val rightKeys = joinKeys.map(_._2)
if (joinKeys.nonEmpty) {
- logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}")
+ logDebug(s"leftKeys:$leftKeys | rightKeys:$rightKeys")
Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right))
} else {
None
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index d192b151ac1c3..c441f0bf24d85 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -79,7 +79,8 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
}
if (curPlan.fastEquals(lastPlan)) {
- logTrace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
+ logTrace(
+ s"Fixed point reached for batch ${batch.name} after ${iteration - 1} iterations.")
continue = false
}
lastPlan = curPlan
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/catalyst/types/dataTypes.scala
index 5dd19dd12d8dd..ff1dc03069ef1 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/catalyst/types/dataTypes.scala
@@ -171,6 +171,27 @@ object DataType {
case _ =>
}
}
+
+ /**
+ * Compares two types, ignoring nullability of ArrayType, MapType, StructType.
+ */
+ def equalsIgnoreNullability(left: DataType, right: DataType): Boolean = {
+ (left, right) match {
+ case (ArrayType(leftElementType, _), ArrayType(rightElementType, _)) =>
+ equalsIgnoreNullability(leftElementType, rightElementType)
+ case (MapType(leftKeyType, leftValueType, _), MapType(rightKeyType, rightValueType, _)) =>
+ equalsIgnoreNullability(leftKeyType, rightKeyType) &&
+ equalsIgnoreNullability(leftValueType, rightValueType)
+ case (StructType(leftFields), StructType(rightFields)) =>
+ leftFields.size == rightFields.size &&
+ leftFields.zip(rightFields)
+ .forall{
+ case (left, right) =>
+ left.name == right.name && equalsIgnoreNullability(left.dataType, right.dataType)
+ }
+ case (left, right) => left == right
+ }
+ }
}
abstract class DataType {
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/catalyst/util/Metadata.scala
old mode 100644
new mode 100755
index 2f2082fa3c863..8172733e94dd5
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala
@@ -34,6 +34,9 @@ import org.json4s.jackson.JsonMethods._
*/
sealed class Metadata private[util] (private[util] 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)
+
/** Gets a Long. */
def getLong(key: String): Long = get(key)
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 918996f11da2c..3f5b9f698f827 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
@@ -158,13 +158,13 @@ class ExpressionEvaluationSuite extends FunSuite {
val nl = Literal(null)
val s = Seq(one, two)
val nullS = Seq(one, two, null)
- checkEvaluation(InSet(one, hS, one +: s), true)
- checkEvaluation(InSet(two, hS, two +: s), true)
- checkEvaluation(InSet(two, nS, two +: nullS), true)
- checkEvaluation(InSet(nl, nS, nl +: nullS), true)
- checkEvaluation(InSet(three, hS, three +: s), false)
- checkEvaluation(InSet(three, nS, three +: nullS), false)
- checkEvaluation(InSet(one, hS, one +: s) && InSet(two, hS, two +: s), true)
+ checkEvaluation(InSet(one, hS), true)
+ checkEvaluation(InSet(two, hS), true)
+ checkEvaluation(InSet(two, nS), true)
+ checkEvaluation(InSet(nl, nS), true)
+ checkEvaluation(InSet(three, hS), false)
+ checkEvaluation(InSet(three, nS), false)
+ checkEvaluation(InSet(one, hS) && InSet(two, hS), true)
}
test("MaxOf") {
@@ -347,8 +347,8 @@ class ExpressionEvaluationSuite extends FunSuite {
// - Because of this, casts to fixed-precision decimals should be nullable
assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false)
- assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === false)
- assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === false)
+ assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === true)
+ assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === true)
assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false)
assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true)
@@ -396,6 +396,16 @@ class ExpressionEvaluationSuite extends FunSuite {
checkEvaluation(Cast(Literal(-9.95), DecimalType(1, 0)), null)
checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(3, 1)), Decimal(-10.0))
checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(1, 0)), null)
+
+ checkEvaluation(Cast(Literal(Double.NaN), DecimalType.Unlimited), null)
+ checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType.Unlimited), null)
+ checkEvaluation(Cast(Literal(Float.NaN), DecimalType.Unlimited), null)
+ checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType.Unlimited), null)
+
+ checkEvaluation(Cast(Literal(Double.NaN), DecimalType(2, 1)), null)
+ checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType(2, 1)), null)
+ checkEvaluation(Cast(Literal(Float.NaN), DecimalType(2, 1)), null)
+ checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType(2, 1)), null)
}
test("timestamp") {
@@ -440,6 +450,11 @@ class ExpressionEvaluationSuite extends FunSuite {
// A test for higher precision than millis
checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001)
+
+ checkEvaluation(Cast(Literal(Double.NaN), TimestampType), null)
+ checkEvaluation(Cast(Literal(1.0 / 0.0), TimestampType), null)
+ checkEvaluation(Cast(Literal(Float.NaN), TimestampType), null)
+ checkEvaluation(Cast(Literal(1.0f / 0.0f), TimestampType), null)
}
test("null checking") {
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 97a78ec971c39..017b180c574b4 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
@@ -52,8 +52,7 @@ class OptimizeInSuite extends PlanTest {
val optimized = Optimize(originalQuery.analyze)
val correctAnswer =
testRelation
- .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2,
- UnresolvedAttribute("a") +: Seq(Literal(1),Literal(2))))
+ .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2))
.analyze
comparePlans(optimized, correctAnswer)
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
old mode 100644
new mode 100755
index 0063d31666c85..f005b7df21043
--- 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
@@ -56,17 +56,30 @@ class MetadataSuite extends FunSuite {
.build()
test("metadata builder and getters") {
+ assert(age.contains("summary") === false)
+ assert(age.contains("index") === true)
assert(age.getLong("index") === 1L)
+ assert(age.contains("average") === true)
assert(age.getDouble("average") === 45.0)
+ assert(age.contains("categorical") === true)
assert(age.getBoolean("categorical") === false)
+ assert(age.contains("name") === true)
assert(age.getString("name") === "age")
+ assert(metadata.contains("purpose") === true)
assert(metadata.getString("purpose") === "ml")
+ assert(metadata.contains("isBase") === true)
assert(metadata.getBoolean("isBase") === false)
+ assert(metadata.contains("summary") === true)
assert(metadata.getMetadata("summary") === summary)
+ assert(metadata.contains("long[]") === true)
assert(metadata.getLongArray("long[]").toSeq === Seq(0L, 1L))
+ assert(metadata.contains("double[]") === true)
assert(metadata.getDoubleArray("double[]").toSeq === Seq(3.0, 4.0))
+ assert(metadata.contains("boolean[]") === true)
assert(metadata.getBooleanArray("boolean[]").toSeq === Seq(true, false))
+ assert(gender.contains("categories") === true)
assert(gender.getStringArray("categories").toSeq === Seq("male", "female"))
+ assert(metadata.contains("features") === true)
assert(metadata.getMetadataArray("features").toSeq === Seq(age, gender))
}
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 279495aa64755..9697beb132fbb 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
@@ -22,7 +22,6 @@ import scala.collection.JavaConversions._
import java.util.Properties
-
private[spark] object SQLConf {
val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed"
val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize"
@@ -32,11 +31,18 @@ private[spark] object SQLConf {
val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions"
val CODEGEN_ENABLED = "spark.sql.codegen"
val DIALECT = "spark.sql.dialect"
+
val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString"
val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata"
val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec"
+ val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown"
+
val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord"
+ // Options that control which operators can be chosen by the query planner. These should be
+ // considered hints and may be ignored by future versions of Spark SQL.
+ val EXTERNAL_SORT = "spark.sql.planner.externalSort"
+
// This is only used for the thriftserver
val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool"
@@ -90,6 +96,13 @@ private[sql] trait SQLConf {
/** Number of partitions to use for shuffle operators. */
private[spark] def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS, "200").toInt
+ /** When true predicates will be passed to the parquet record reader when possible. */
+ private[spark] def parquetFilterPushDown =
+ getConf(PARQUET_FILTER_PUSHDOWN_ENABLED, "false").toBoolean
+
+ /** When true the planner will use the external sort, which may spill to disk. */
+ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean
+
/**
* When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode
* that evaluates expressions found in queries. In general this custom code runs much faster
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 6d4c0d82ac7af..ddcb5db6c3a21 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
@@ -39,7 +39,7 @@ private[sql] trait UDFRegistration {
envVars: JMap[String, String],
pythonIncludes: JList[String],
pythonExec: String,
- broadcastVars: JList[Broadcast[Array[Byte]]],
+ broadcastVars: JList[Broadcast[Array[Array[Byte]]]],
accumulator: Accumulator[JList[Array[Byte]]],
stringDataType: String): Unit = {
log.debug(
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 455b415d9d959..881d32b105c5f 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
@@ -182,8 +182,8 @@ private[sql] case class InMemoryColumnarTableScan(
// to evaluate to `true' based on statistics collected about this partition batch.
val buildFilter: PartialFunction[Expression, Expression] = {
case And(lhs: Expression, rhs: Expression)
- if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
- buildFilter(lhs) && buildFilter(rhs)
+ if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) =>
+ (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _)
case Or(lhs: Expression, rhs: Expression)
if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
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 927f40063e47e..cff7a012691dc 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
@@ -47,8 +47,8 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
// TODO: Eliminate redundant expressions in grouping key and value.
val rdd = if (sortBasedShuffleOn) {
child.execute().mapPartitions { iter =>
- val hashExpressions = newProjection(expressions, child.output)
- iter.map(r => (hashExpressions(r), r.copy()))
+ val hashExpressions = newMutableProjection(expressions, child.output)()
+ iter.map(r => (hashExpressions(r).copy(), r.copy()))
}
} else {
child.execute().mapPartitions { iter =>
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 cc7e0c05ffc70..1225d18857af2 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
@@ -208,23 +208,16 @@ 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 (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) {
- (filters: Seq[Expression]) => {
- filters.filter { filter =>
- // Note: filters cannot be pushed down to Parquet if they contain more complex
- // expressions than simple "Attribute cmp Literal" comparisons. Here we remove
- // all filters that have been pushed down. Note that a predicate such as
- // "(A AND B) OR C" can result in "A OR C" being pushed down.
- val recordFilter = ParquetFilters.createFilter(filter)
- if (!recordFilter.isDefined) {
- // First case: the pushdown did not result in any record filter.
- true
- } else {
- // Second case: a record filter was created; here we are conservative in
- // the sense that even if "A" was pushed and we check for "A AND B" we
- // still want to keep "A AND B" in the higher-level filter, not just "B".
- !ParquetFilters.findExpression(recordFilter.get, filter).isDefined
- }
+ if (sqlContext.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
+ // filters that have been pushed down. Note that a predicate such as "(A AND B) OR C"
+ // can result in "A OR C" being pushed down. Here we are conservative in the sense
+ // that even if "A" was pushed and we check for "A AND B" we still want to keep
+ // "A AND B" in the higher-level filter, not just "B".
+ predicates.map(p => p -> ParquetFilters.createFilter(p)).collect {
+ case (predicate, None) => predicate
}
}
} else {
@@ -234,7 +227,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
projectList,
filters,
prunePushedDownFilters,
- ParquetTableScan(_, relation, filters)) :: Nil
+ ParquetTableScan(
+ _,
+ relation,
+ if (sqlContext.parquetFilterPushDown) filters else Nil)) :: Nil
case _ => Nil
}
@@ -260,9 +256,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case logical.Distinct(child) =>
execution.Distinct(partial = false,
execution.Distinct(partial = true, planLater(child))) :: Nil
+
+ case logical.Sort(sortExprs, child) if sqlContext.externalSortEnabled =>
+ execution.ExternalSort(sortExprs, global = true, planLater(child)):: Nil
case logical.Sort(sortExprs, child) =>
- // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution.
execution.Sort(sortExprs, global = true, planLater(child)):: Nil
+
case logical.SortPartitions(sortExprs, child) =>
// This sort only sorts tuples within a partition. Its requiredDistribution will be
// an UnspecifiedDistribution.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index 1b8ba3ace2a82..e53723c176569 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution}
import org.apache.spark.util.MutablePair
+import org.apache.spark.util.collection.ExternalSorter
/**
* :: DeveloperApi ::
@@ -189,6 +190,9 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
/**
* :: DeveloperApi ::
+ * Performs a sort on-heap.
+ * @param global when true performs a global sort of all partitions by shuffling the data first
+ * if necessary.
*/
@DeveloperApi
case class Sort(
@@ -199,12 +203,37 @@ case class Sort(
override def requiredChildDistribution =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
+ override def execute() = attachTree(this, "sort") {
+ child.execute().mapPartitions( { iterator =>
+ val ordering = newOrdering(sortOrder, child.output)
+ iterator.map(_.copy()).toArray.sorted(ordering).iterator
+ }, preservesPartitioning = true)
+ }
+
+ override def output = child.output
+}
+
+/**
+ * :: DeveloperApi ::
+ * Performs a sort, spilling to disk as needed.
+ * @param global when true performs a global sort of all partitions by shuffling the data first
+ * if necessary.
+ */
+@DeveloperApi
+case class ExternalSort(
+ sortOrder: Seq[SortOrder],
+ global: Boolean,
+ child: SparkPlan)
+ extends UnaryNode {
+ override def requiredChildDistribution =
+ if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
override def execute() = attachTree(this, "sort") {
- child.execute()
- .mapPartitions( { iterator =>
- val ordering = newOrdering(sortOrder, child.output)
- iterator.map(_.copy()).toArray.sorted(ordering).iterator
+ child.execute().mapPartitions( { iterator =>
+ val ordering = newOrdering(sortOrder, child.output)
+ val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering))
+ sorter.insertAll(iterator.map(r => (r, null)))
+ sorter.iterator.map(_._1)
}, preservesPartitioning = true)
}
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 a83cf5d441d1e..f98cae3f17e4a 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
@@ -45,7 +45,7 @@ private[spark] case class PythonUDF(
envVars: JMap[String, String],
pythonIncludes: JList[String],
pythonExec: String,
- broadcastVars: JList[Broadcast[Array[Byte]]],
+ broadcastVars: JList[Broadcast[Array[Array[Byte]]]],
accumulator: Accumulator[JList[Array[Byte]]],
dataType: DataType,
children: Seq[Expression]) extends Expression with SparkLogging {
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 1e67799e8399a..3a9e1499e2dc4 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
@@ -18,408 +18,152 @@
package org.apache.spark.sql.parquet
import java.nio.ByteBuffer
-import java.sql.{Date, Timestamp}
+import com.google.common.io.BaseEncoding
import org.apache.hadoop.conf.Configuration
-
-import parquet.common.schema.ColumnPath
import parquet.filter2.compat.FilterCompat
import parquet.filter2.compat.FilterCompat._
-import parquet.filter2.predicate.Operators.{Column, SupportsLtGt}
-import parquet.filter2.predicate.{FilterApi, FilterPredicate}
import parquet.filter2.predicate.FilterApi._
+import parquet.filter2.predicate.{FilterApi, FilterPredicate}
import parquet.io.api.Binary
-import parquet.column.ColumnReader
-
-import com.google.common.io.BaseEncoding
import org.apache.spark.SparkEnv
-import org.apache.spark.sql.catalyst.types._
-import org.apache.spark.sql.catalyst.types.decimal.Decimal
-import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate}
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.execution.SparkSqlSerializer
-import org.apache.spark.sql.parquet.ParquetColumns._
+import org.apache.spark.sql.catalyst.types._
private[sql] object ParquetFilters {
val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter"
- // set this to false if pushdown should be disabled
- val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown"
- def createRecordFilter(filterExpressions: Seq[Expression]): Filter = {
- val filters: Seq[CatalystFilter] = filterExpressions.collect {
- case (expression: Expression) if createFilter(expression).isDefined =>
- createFilter(expression).get
- }
- if (filters.length > 0) FilterCompat.get(filters.reduce(FilterApi.and)) else null
+ def createRecordFilter(filterExpressions: Seq[Expression]): Option[Filter] = {
+ filterExpressions.flatMap(createFilter).reduceOption(FilterApi.and).map(FilterCompat.get)
}
- def createFilter(expression: Expression): Option[CatalystFilter] = {
- def createEqualityFilter(
- name: String,
- literal: Literal,
- predicate: CatalystPredicate) = literal.dataType match {
+ def createFilter(predicate: Expression): Option[FilterPredicate] = {
+ val makeEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
case BooleanType =>
- ComparisonFilter.createBooleanEqualityFilter(
- name,
- literal.value.asInstanceOf[Boolean],
- predicate)
- case ByteType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]),
- predicate)
- case ShortType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]),
- predicate)
+ (n: String, v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[java.lang.Boolean])
case IntegerType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(intColumn(name), literal.value.asInstanceOf[Integer]),
- predicate)
+ (n: String, v: Any) => FilterApi.eq(intColumn(n), v.asInstanceOf[Integer])
case LongType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]),
- predicate)
- case DoubleType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]),
- predicate)
+ (n: String, v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[java.lang.Long])
case FloatType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]),
- predicate)
+ (n: String, v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[java.lang.Float])
+ case DoubleType =>
+ (n: String, v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[java.lang.Double])
case StringType =>
- ComparisonFilter.createStringEqualityFilter(
- name,
- literal.value.asInstanceOf[String],
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.eq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
- ComparisonFilter.createBinaryEqualityFilter(
- name,
- literal.value.asInstanceOf[Array[Byte]],
- predicate)
- case DateType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])),
- predicate)
- case TimestampType =>
- new ComparisonFilter(
- name,
- FilterApi.eq(timestampColumn(name),
- new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])),
- predicate)
- case DecimalType.Unlimited =>
- new ComparisonFilter(
- name,
- FilterApi.eq(decimalColumn(name), literal.value.asInstanceOf[Decimal]),
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.eq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]]))
}
- def createLessThanFilter(
- name: String,
- literal: Literal,
- predicate: CatalystPredicate) = literal.dataType match {
- case ByteType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]),
- predicate)
- case ShortType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]),
- predicate)
+ val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
case IntegerType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(intColumn(name), literal.value.asInstanceOf[Integer]),
- predicate)
+ (n: String, v: Any) => FilterApi.lt(intColumn(n), v.asInstanceOf[Integer])
case LongType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]),
- predicate)
- case DoubleType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]),
- predicate)
+ (n: String, v: Any) => FilterApi.lt(longColumn(n), v.asInstanceOf[java.lang.Long])
case FloatType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]),
- predicate)
+ (n: String, v: Any) => FilterApi.lt(floatColumn(n), v.asInstanceOf[java.lang.Float])
+ case DoubleType =>
+ (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double])
case StringType =>
- ComparisonFilter.createStringLessThanFilter(
- name,
- literal.value.asInstanceOf[String],
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
- ComparisonFilter.createBinaryLessThanFilter(
- name,
- literal.value.asInstanceOf[Array[Byte]],
- predicate)
- case DateType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])),
- predicate)
- case TimestampType =>
- new ComparisonFilter(
- name,
- FilterApi.lt(timestampColumn(name),
- new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])),
- predicate)
- case DecimalType.Unlimited =>
- new ComparisonFilter(
- name,
- FilterApi.lt(decimalColumn(name), literal.value.asInstanceOf[Decimal]),
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]]))
}
- def createLessThanOrEqualFilter(
- name: String,
- literal: Literal,
- predicate: CatalystPredicate) = literal.dataType match {
- case ByteType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]),
- predicate)
- case ShortType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]),
- predicate)
+
+ val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
case IntegerType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(intColumn(name), literal.value.asInstanceOf[Integer]),
- predicate)
+ (n: String, v: Any) => FilterApi.ltEq(intColumn(n), v.asInstanceOf[java.lang.Integer])
case LongType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]),
- predicate)
- case DoubleType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]),
- predicate)
+ (n: String, v: Any) => FilterApi.ltEq(longColumn(n), v.asInstanceOf[java.lang.Long])
case FloatType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]),
- predicate)
+ (n: String, v: Any) => FilterApi.ltEq(floatColumn(n), v.asInstanceOf[java.lang.Float])
+ case DoubleType =>
+ (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double])
case StringType =>
- ComparisonFilter.createStringLessThanOrEqualFilter(
- name,
- literal.value.asInstanceOf[String],
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
- ComparisonFilter.createBinaryLessThanOrEqualFilter(
- name,
- literal.value.asInstanceOf[Array[Byte]],
- predicate)
- case DateType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])),
- predicate)
- case TimestampType =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(timestampColumn(name),
- new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])),
- predicate)
- case DecimalType.Unlimited =>
- new ComparisonFilter(
- name,
- FilterApi.ltEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]),
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]]))
}
- // TODO: combine these two types somehow?
- def createGreaterThanFilter(
- name: String,
- literal: Literal,
- predicate: CatalystPredicate) = literal.dataType match {
- case ByteType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]),
- predicate)
- case ShortType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]),
- predicate)
+
+ val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
case IntegerType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(intColumn(name), literal.value.asInstanceOf[Integer]),
- predicate)
+ (n: String, v: Any) => FilterApi.gt(intColumn(n), v.asInstanceOf[java.lang.Integer])
case LongType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]),
- predicate)
- case DoubleType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]),
- predicate)
+ (n: String, v: Any) => FilterApi.gt(longColumn(n), v.asInstanceOf[java.lang.Long])
case FloatType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]),
- predicate)
+ (n: String, v: Any) => FilterApi.gt(floatColumn(n), v.asInstanceOf[java.lang.Float])
+ case DoubleType =>
+ (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double])
case StringType =>
- ComparisonFilter.createStringGreaterThanFilter(
- name,
- literal.value.asInstanceOf[String],
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
- ComparisonFilter.createBinaryGreaterThanFilter(
- name,
- literal.value.asInstanceOf[Array[Byte]],
- predicate)
- case DateType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])),
- predicate)
- case TimestampType =>
- new ComparisonFilter(
- name,
- FilterApi.gt(timestampColumn(name),
- new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])),
- predicate)
- case DecimalType.Unlimited =>
- new ComparisonFilter(
- name,
- FilterApi.gt(decimalColumn(name), literal.value.asInstanceOf[Decimal]),
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]]))
}
- def createGreaterThanOrEqualFilter(
- name: String,
- literal: Literal,
- predicate: CatalystPredicate) = literal.dataType match {
- case ByteType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]),
- predicate)
- case ShortType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]),
- predicate)
+
+ val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = {
case IntegerType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(intColumn(name), literal.value.asInstanceOf[Integer]),
- predicate)
+ (n: String, v: Any) => FilterApi.gtEq(intColumn(n), v.asInstanceOf[java.lang.Integer])
case LongType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]),
- predicate)
- case DoubleType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]),
- predicate)
+ (n: String, v: Any) => FilterApi.gtEq(longColumn(n), v.asInstanceOf[java.lang.Long])
case FloatType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]),
- predicate)
+ (n: String, v: Any) => FilterApi.gtEq(floatColumn(n), v.asInstanceOf[java.lang.Float])
+ case DoubleType =>
+ (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double])
case StringType =>
- ComparisonFilter.createStringGreaterThanOrEqualFilter(
- name,
- literal.value.asInstanceOf[String],
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String]))
case BinaryType =>
- ComparisonFilter.createBinaryGreaterThanOrEqualFilter(
- name,
- literal.value.asInstanceOf[Array[Byte]],
- predicate)
- case DateType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])),
- predicate)
- case TimestampType =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(timestampColumn(name),
- new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])),
- predicate)
- case DecimalType.Unlimited =>
- new ComparisonFilter(
- name,
- FilterApi.gtEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]),
- predicate)
+ (n: String, v: Any) =>
+ FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]]))
}
- /**
- * TODO: we currently only filter on non-nullable (Parquet REQUIRED) attributes until
- * https://github.com/Parquet/parquet-mr/issues/371
- * has been resolved.
- */
- expression match {
- case p @ Or(left: Expression, right: Expression)
- if createFilter(left).isDefined && createFilter(right).isDefined => {
- // If either side of this Or-predicate is empty then this means
- // it contains a more complex comparison than between attribute and literal
- // (e.g., it contained a CAST). The only safe thing to do is then to disregard
- // this disjunction, which could be contained in a conjunction. If it stands
- // alone then it is also safe to drop it, since a Null return value of this
- // function is interpreted as having no filters at all.
- val leftFilter = createFilter(left).get
- val rightFilter = createFilter(right).get
- Some(new OrFilter(leftFilter, rightFilter))
- }
- case p @ And(left: Expression, right: Expression) => {
- // This treats nested conjunctions; since either side of the conjunction
- // may contain more complex filter expressions we may actually generate
- // strictly weaker filter predicates in the process.
- val leftFilter = createFilter(left)
- val rightFilter = createFilter(right)
- (leftFilter, rightFilter) match {
- case (None, Some(filter)) => Some(filter)
- case (Some(filter), None) => Some(filter)
- case (Some(leftF), Some(rightF)) =>
- Some(new AndFilter(leftF, rightF))
- case _ => None
- }
- }
- case p @ EqualTo(left: Literal, right: NamedExpression) if left.dataType != NullType =>
- Some(createEqualityFilter(right.name, left, p))
- case p @ EqualTo(left: NamedExpression, right: Literal) if right.dataType != NullType =>
- Some(createEqualityFilter(left.name, right, p))
- case p @ LessThan(left: Literal, right: NamedExpression) =>
- Some(createLessThanFilter(right.name, left, p))
- case p @ LessThan(left: NamedExpression, right: Literal) =>
- Some(createLessThanFilter(left.name, right, p))
- case p @ LessThanOrEqual(left: Literal, right: NamedExpression) =>
- Some(createLessThanOrEqualFilter(right.name, left, p))
- case p @ LessThanOrEqual(left: NamedExpression, right: Literal) =>
- Some(createLessThanOrEqualFilter(left.name, right, p))
- case p @ GreaterThan(left: Literal, right: NamedExpression) =>
- Some(createGreaterThanFilter(right.name, left, p))
- case p @ GreaterThan(left: NamedExpression, right: Literal) =>
- Some(createGreaterThanFilter(left.name, right, p))
- case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) =>
- Some(createGreaterThanOrEqualFilter(right.name, left, p))
- case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) =>
- Some(createGreaterThanOrEqualFilter(left.name, right, p))
+ predicate match {
+ case EqualTo(NamedExpression(name, _), Literal(value, dataType)) if dataType != NullType =>
+ makeEq.lift(dataType).map(_(name, value))
+ case EqualTo(Literal(value, dataType), NamedExpression(name, _)) if dataType != NullType =>
+ makeEq.lift(dataType).map(_(name, value))
+
+ case LessThan(NamedExpression(name, _), Literal(value, dataType)) =>
+ makeLt.lift(dataType).map(_(name, value))
+ case LessThan(Literal(value, dataType), NamedExpression(name, _)) =>
+ makeLt.lift(dataType).map(_(name, value))
+
+ case LessThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) =>
+ makeLtEq.lift(dataType).map(_(name, value))
+ case LessThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) =>
+ makeLtEq.lift(dataType).map(_(name, value))
+
+ case GreaterThan(NamedExpression(name, _), Literal(value, dataType)) =>
+ makeGt.lift(dataType).map(_(name, value))
+ case GreaterThan(Literal(value, dataType), NamedExpression(name, _)) =>
+ makeGt.lift(dataType).map(_(name, value))
+
+ case GreaterThanOrEqual(NamedExpression(name, _), Literal(value, dataType)) =>
+ makeGtEq.lift(dataType).map(_(name, value))
+ case GreaterThanOrEqual(Literal(value, dataType), NamedExpression(name, _)) =>
+ makeGtEq.lift(dataType).map(_(name, value))
+
+ case And(lhs, rhs) =>
+ (createFilter(lhs) ++ createFilter(rhs)).reduceOption(FilterApi.and)
+
+ case Or(lhs, rhs) =>
+ for {
+ lhsFilter <- createFilter(lhs)
+ rhsFilter <- createFilter(rhs)
+ } yield FilterApi.or(lhsFilter, rhsFilter)
+
+ case Not(pred) =>
+ createFilter(pred).map(FilterApi.not)
+
case _ => None
}
}
@@ -430,7 +174,7 @@ private[sql] object ParquetFilters {
* the actual filter predicate.
*/
def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = {
- if (filters.length > 0) {
+ if (filters.nonEmpty) {
val serialized: Array[Byte] =
SparkEnv.get.closureSerializer.newInstance().serialize(filters).array()
val encoded: String = BaseEncoding.base64().encode(serialized)
@@ -452,245 +196,4 @@ private[sql] object ParquetFilters {
Seq()
}
}
-
- /**
- * Try to find the given expression in the tree of filters in order to
- * determine whether it is safe to remove it from the higher level filters. Note
- * that strictly speaking we could stop the search whenever an expression is found
- * that contains this expression as subexpression (e.g., when searching for "a"
- * and "(a or c)" is found) but we don't care about optimizations here since the
- * filter tree is assumed to be small.
- *
- * @param filter The [[org.apache.spark.sql.parquet.CatalystFilter]] to expand
- * and search
- * @param expression The expression to look for
- * @return An optional [[org.apache.spark.sql.parquet.CatalystFilter]] that
- * contains the expression.
- */
- def findExpression(
- filter: CatalystFilter,
- expression: Expression): Option[CatalystFilter] = filter match {
- case f @ OrFilter(_, leftFilter, rightFilter, _) =>
- if (f.predicate == expression) {
- Some(f)
- } else {
- val left = findExpression(leftFilter, expression)
- if (left.isDefined) left else findExpression(rightFilter, expression)
- }
- case f @ AndFilter(_, leftFilter, rightFilter, _) =>
- if (f.predicate == expression) {
- Some(f)
- } else {
- val left = findExpression(leftFilter, expression)
- if (left.isDefined) left else findExpression(rightFilter, expression)
- }
- case f @ ComparisonFilter(_, _, predicate) =>
- if (predicate == expression) Some(f) else None
- case _ => None
- }
-}
-
-abstract private[parquet] class CatalystFilter(
- @transient val predicate: CatalystPredicate) extends FilterPredicate
-
-private[parquet] case class ComparisonFilter(
- val columnName: String,
- private var filter: FilterPredicate,
- @transient override val predicate: CatalystPredicate)
- extends CatalystFilter(predicate) {
- override def accept[R](visitor: FilterPredicate.Visitor[R]): R = {
- filter.accept(visitor)
- }
-}
-
-private[parquet] case class OrFilter(
- private var filter: FilterPredicate,
- @transient val left: CatalystFilter,
- @transient val right: CatalystFilter,
- @transient override val predicate: Or)
- extends CatalystFilter(predicate) {
- def this(l: CatalystFilter, r: CatalystFilter) =
- this(
- FilterApi.or(l, r),
- l,
- r,
- Or(l.predicate, r.predicate))
-
- override def accept[R](visitor: FilterPredicate.Visitor[R]): R = {
- filter.accept(visitor);
- }
-
-}
-
-private[parquet] case class AndFilter(
- private var filter: FilterPredicate,
- @transient val left: CatalystFilter,
- @transient val right: CatalystFilter,
- @transient override val predicate: And)
- extends CatalystFilter(predicate) {
- def this(l: CatalystFilter, r: CatalystFilter) =
- this(
- FilterApi.and(l, r),
- l,
- r,
- And(l.predicate, r.predicate))
-
- override def accept[R](visitor: FilterPredicate.Visitor[R]): R = {
- filter.accept(visitor);
- }
-
-}
-
-private[parquet] object ComparisonFilter {
- def createBooleanEqualityFilter(
- columnName: String,
- value: Boolean,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.eq(booleanColumn(columnName), value.asInstanceOf[java.lang.Boolean]),
- predicate)
-
- def createStringEqualityFilter(
- columnName: String,
- value: String,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.eq(binaryColumn(columnName), Binary.fromString(value)),
- predicate)
-
- def createStringLessThanFilter(
- columnName: String,
- value: String,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.lt(binaryColumn(columnName), Binary.fromString(value)),
- predicate)
-
- def createStringLessThanOrEqualFilter(
- columnName: String,
- value: String,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.ltEq(binaryColumn(columnName), Binary.fromString(value)),
- predicate)
-
- def createStringGreaterThanFilter(
- columnName: String,
- value: String,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.gt(binaryColumn(columnName), Binary.fromString(value)),
- predicate)
-
- def createStringGreaterThanOrEqualFilter(
- columnName: String,
- value: String,
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.gtEq(binaryColumn(columnName), Binary.fromString(value)),
- predicate)
-
- def createBinaryEqualityFilter(
- columnName: String,
- value: Array[Byte],
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.eq(binaryColumn(columnName), Binary.fromByteArray(value)),
- predicate)
-
- def createBinaryLessThanFilter(
- columnName: String,
- value: Array[Byte],
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.lt(binaryColumn(columnName), Binary.fromByteArray(value)),
- predicate)
-
- def createBinaryLessThanOrEqualFilter(
- columnName: String,
- value: Array[Byte],
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.ltEq(binaryColumn(columnName), Binary.fromByteArray(value)),
- predicate)
-
- def createBinaryGreaterThanFilter(
- columnName: String,
- value: Array[Byte],
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.gt(binaryColumn(columnName), Binary.fromByteArray(value)),
- predicate)
-
- def createBinaryGreaterThanOrEqualFilter(
- columnName: String,
- value: Array[Byte],
- predicate: CatalystPredicate): CatalystFilter =
- new ComparisonFilter(
- columnName,
- FilterApi.gtEq(binaryColumn(columnName), Binary.fromByteArray(value)),
- predicate)
-}
-
-private[spark] object ParquetColumns {
-
- def byteColumn(columnPath: String): ByteColumn = {
- new ByteColumn(ColumnPath.fromDotString(columnPath))
- }
-
- final class ByteColumn(columnPath: ColumnPath)
- extends Column[java.lang.Byte](columnPath, classOf[java.lang.Byte]) with SupportsLtGt
-
- def shortColumn(columnPath: String): ShortColumn = {
- new ShortColumn(ColumnPath.fromDotString(columnPath))
- }
-
- final class ShortColumn(columnPath: ColumnPath)
- extends Column[java.lang.Short](columnPath, classOf[java.lang.Short]) with SupportsLtGt
-
-
- def dateColumn(columnPath: String): DateColumn = {
- new DateColumn(ColumnPath.fromDotString(columnPath))
- }
-
- final class DateColumn(columnPath: ColumnPath)
- extends Column[WrappedDate](columnPath, classOf[WrappedDate]) with SupportsLtGt
-
- def timestampColumn(columnPath: String): TimestampColumn = {
- new TimestampColumn(ColumnPath.fromDotString(columnPath))
- }
-
- final class TimestampColumn(columnPath: ColumnPath)
- extends Column[WrappedTimestamp](columnPath, classOf[WrappedTimestamp]) with SupportsLtGt
-
- def decimalColumn(columnPath: String): DecimalColumn = {
- new DecimalColumn(ColumnPath.fromDotString(columnPath))
- }
-
- final class DecimalColumn(columnPath: ColumnPath)
- extends Column[Decimal](columnPath, classOf[Decimal]) with SupportsLtGt
-
- final class WrappedDate(val date: Date) extends Comparable[WrappedDate] {
-
- override def compareTo(other: WrappedDate): Int = {
- date.compareTo(other.date)
- }
- }
-
- final class WrappedTimestamp(val timestamp: Timestamp) extends Comparable[WrappedTimestamp] {
-
- override def compareTo(other: WrappedTimestamp): Int = {
- timestamp.compareTo(other.timestamp)
- }
- }
}
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 82130b5459174..b237a07c72d07 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
@@ -84,6 +84,21 @@ private[sql] case class ParquetRelation(
private[sql] object ParquetRelation {
def enableLogForwarding() {
+ // Note: the parquet.Log class has a static initializer that
+ // sets the java.util.logging Logger for "parquet". This
+ // checks first to see if there's any handlers already set
+ // and if not it creates them. If this method executes prior
+ // to that class being loaded then:
+ // 1) there's no handlers installed so there's none to
+ // remove. But when it IS finally loaded the desired affect
+ // of removing them is circumvented.
+ // 2) The parquet.Log static initializer calls setUseParentHanders(false)
+ // undoing the attempt to override the logging here.
+ //
+ // Therefore we need to force the class to be loaded.
+ // This should really be resolved by Parquet.
+ Class.forName(classOf[parquet.Log].getName())
+
// Note: Logger.getLogger("parquet") has a default logger
// that appends to Console which needs to be cleared.
val parquetLogger = java.util.logging.Logger.getLogger("parquet")
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 74c43e053b03c..5d0643a64a044 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
@@ -32,26 +32,25 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path}
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
-import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
-
+import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat => NewFileOutputFormat}
import parquet.hadoop._
+import parquet.hadoop.api.ReadSupport.ReadContext
import parquet.hadoop.api.{InitContext, ReadSupport}
import parquet.hadoop.metadata.GlobalMetaData
-import parquet.hadoop.api.ReadSupport.ReadContext
import parquet.hadoop.util.ContextUtil
import parquet.io.ParquetDecodingException
import parquet.schema.MessageType
+import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.SQLConf
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _}
import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
import org.apache.spark.{Logging, SerializableWritable, TaskContext}
/**
+ * :: DeveloperApi ::
* Parquet table scan operator. Imports the file that backs the given
* [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``.
*/
@@ -79,8 +78,6 @@ case class ParquetTableScan(
override def execute(): RDD[Row] = {
import parquet.filter2.compat.FilterCompat.FilterPredicateCompat
- import parquet.filter2.compat.FilterCompat.Filter
- import parquet.filter2.predicate.FilterPredicate
val sc = sqlContext.sparkContext
val job = new Job(sc.hadoopConfiguration)
@@ -108,18 +105,11 @@ case class ParquetTableScan(
// Note 1: the input format ignores all predicates that cannot be expressed
// as simple column predicate filters in Parquet. Here we just record
// the whole pruning predicate.
- // Note 2: you can disable filter predicate pushdown by setting
- // "spark.sql.hints.parquetFilterPushdown" to false inside SparkConf.
- if (columnPruningPred.length > 0 &&
- sc.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) {
-
+ ParquetFilters
+ .createRecordFilter(columnPruningPred)
+ .map(_.asInstanceOf[FilterPredicateCompat].getFilterPredicate)
// Set this in configuration of ParquetInputFormat, needed for RowGroupFiltering
- val filter: Filter = ParquetFilters.createRecordFilter(columnPruningPred)
- if (filter != null){
- val filterPredicate = filter.asInstanceOf[FilterPredicateCompat].getFilterPredicate()
- ParquetInputFormat.setFilterPredicate(conf, filterPredicate)
- }
- }
+ .foreach(ParquetInputFormat.setFilterPredicate(conf, _))
// Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata
conf.set(
@@ -160,7 +150,7 @@ case class ParquetTableScan(
}
} else {
baseRDD.map(_._2)
- }.filter(_ != null) // Parquet's record filters may produce null values
+ }
}
/**
@@ -193,6 +183,7 @@ case class ParquetTableScan(
}
/**
+ * :: DeveloperApi ::
* Operator that acts as a sink for queries on RDDs and can be used to
* store the output inside a directory of Parquet files. This operator
* is similar to Hive's INSERT INTO TABLE operation in the sense that
@@ -208,6 +199,7 @@ case class ParquetTableScan(
* cause unpredicted behaviour and therefore results in a RuntimeException
* (only detected via filename pattern so will not catch all cases).
*/
+@DeveloperApi
case class InsertIntoParquetTable(
relation: ParquetRelation,
child: SparkPlan,
@@ -316,7 +308,7 @@ case class InsertIntoParquetTable(
}
writer.close(hadoopContext)
committer.commitTask(hadoopContext)
- return 1
+ 1
}
val jobFormat = new AppendingParquetOutputFormat(taskIdOffset)
/* apparently we need a TaskAttemptID to construct an OutputCommitter;
@@ -374,9 +366,8 @@ private[parquet] class FilteringParquetRowInputFormat
override def createRecordReader(
inputSplit: InputSplit,
taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = {
-
+
import parquet.filter2.compat.FilterCompat.NoOpFilter
- import parquet.filter2.compat.FilterCompat.Filter
val readSupport: ReadSupport[Row] = new RowReadSupport()
@@ -391,7 +382,7 @@ private[parquet] class FilteringParquetRowInputFormat
}
override def getFooters(jobContext: JobContext): JList[Footer] = {
- import FilteringParquetRowInputFormat.footerCache
+ import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.footerCache
if (footers eq null) {
val conf = ContextUtil.getConfiguration(jobContext)
@@ -441,13 +432,13 @@ private[parquet] class FilteringParquetRowInputFormat
val taskSideMetaData = configuration.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true)
val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue)
val minSplitSize: JLong =
- Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L))
+ Math.max(getFormatMinSplitSize, configuration.getLong("mapred.min.split.size", 0L))
if (maxSplitSize < 0 || minSplitSize < 0) {
throw new ParquetDecodingException(
s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" +
s" minSplitSize = $minSplitSize")
}
-
+
// Uses strict type checking by default
val getGlobalMetaData =
classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]])
@@ -457,29 +448,29 @@ private[parquet] class FilteringParquetRowInputFormat
if (globalMetaData == null) {
val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
return splits
- }
-
+ }
+
val readContext = getReadSupport(configuration).init(
new InitContext(configuration,
- globalMetaData.getKeyValueMetaData(),
- globalMetaData.getSchema()))
-
+ globalMetaData.getKeyValueMetaData,
+ globalMetaData.getSchema))
+
if (taskSideMetaData){
logInfo("Using Task Side Metadata Split Strategy")
- return getTaskSideSplits(configuration,
+ getTaskSideSplits(configuration,
footers,
maxSplitSize,
minSplitSize,
readContext)
} else {
logInfo("Using Client Side Metadata Split Strategy")
- return getClientSideSplits(configuration,
+ getClientSideSplits(configuration,
footers,
maxSplitSize,
minSplitSize,
readContext)
}
-
+
}
def getClientSideSplits(
@@ -488,12 +479,11 @@ private[parquet] class FilteringParquetRowInputFormat
maxSplitSize: JLong,
minSplitSize: JLong,
readContext: ReadContext): JList[ParquetInputSplit] = {
-
- import FilteringParquetRowInputFormat.blockLocationCache
- import parquet.filter2.compat.FilterCompat;
- import parquet.filter2.compat.FilterCompat.Filter;
- import parquet.filter2.compat.RowGroupFilter;
-
+
+ import parquet.filter2.compat.FilterCompat.Filter
+ import parquet.filter2.compat.RowGroupFilter
+ import org.apache.spark.sql.parquet.FilteringParquetRowInputFormat.blockLocationCache
+
val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true)
val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
@@ -502,7 +492,7 @@ private[parquet] class FilteringParquetRowInputFormat
var totalRowGroups: Long = 0
// Ugly hack, stuck with it until PR:
- // https://github.com/apache/incubator-parquet-mr/pull/17
+ // https://github.com/apache/incubator-parquet-mr/pull/17
// is resolved
val generateSplits =
Class.forName("parquet.hadoop.ClientSideMetadataSplitStrategy")
@@ -522,7 +512,7 @@ private[parquet] class FilteringParquetRowInputFormat
blocks,
parquetMetaData.getFileMetaData.getSchema)
rowGroupsDropped = rowGroupsDropped + (blocks.size - filteredBlocks.size)
-
+
if (!filteredBlocks.isEmpty){
var blockLocations: Array[BlockLocation] = null
if (!cacheMetadata) {
@@ -565,7 +555,7 @@ private[parquet] class FilteringParquetRowInputFormat
readContext: ReadContext): JList[ParquetInputSplit] = {
val splits = mutable.ArrayBuffer.empty[ParquetInputSplit]
-
+
// Ugly hack, stuck with it until PR:
// https://github.com/apache/incubator-parquet-mr/pull/17
// is resolved
@@ -575,7 +565,7 @@ private[parquet] class FilteringParquetRowInputFormat
sys.error(
s"Failed to reflectively invoke TaskSideMetadataSplitStrategy.generateTaskSideMDSplits"))
generateSplits.setAccessible(true)
-
+
for (footer <- footers) {
val file = footer.getFile
val fs = file.getFileSystem(configuration)
@@ -593,7 +583,7 @@ private[parquet] class FilteringParquetRowInputFormat
}
splits
- }
+ }
}
@@ -635,11 +625,9 @@ private[parquet] object FileSystemHelper {
files.map(_.getName).map {
case nameP(taskid) => taskid.toInt
case hiddenFileP() => 0
- case other: String => {
+ case other: String =>
sys.error("ERROR: attempting to append to set of Parquet files and found file" +
s"that does not match name pattern: $other")
- 0
- }
case _ => 0
}.reduceLeft((a, b) => if (a < b) b else a)
}
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 7bc249660053a..ef3687e692964 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
@@ -152,14 +152,15 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
}
override def write(record: Row): Unit = {
- if (attributes.size > record.size) {
+ val attributesSize = attributes.size
+ if (attributesSize > record.size) {
throw new IndexOutOfBoundsException(
- s"Trying to write more fields than contained in row (${attributes.size}>${record.size})")
+ s"Trying to write more fields than contained in row (${attributesSize}>${record.size})")
}
var index = 0
writer.startMessage()
- while(index < attributes.size) {
+ while(index < attributesSize) {
// null values indicate optional fields but we do not check currently
if (record(index) != null) {
writer.startField(attributes(index).name, index)
@@ -312,14 +313,15 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
// Optimized for non-nested rows
private[parquet] class MutableRowWriteSupport extends RowWriteSupport {
override def write(record: Row): Unit = {
- if (attributes.size > record.size) {
+ val attributesSize = attributes.size
+ if (attributesSize > record.size) {
throw new IndexOutOfBoundsException(
- s"Trying to write more fields than contained in row (${attributes.size}>${record.size})")
+ s"Trying to write more fields than contained in row (${attributesSize}>${record.size})")
}
var index = 0
writer.startMessage()
- while(index < attributes.size) {
+ while(index < attributesSize) {
// null values indicate optional fields but we do not check currently
if (record(index) != null && record(index) != Nil) {
writer.startField(attributes(index).name, index)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
index 9b8c6a56b94b4..954e86822de17 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
@@ -108,5 +108,7 @@ private[sql] object DataSourceStrategy extends Strategy {
case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) => LessThanOrEqual(a.name, v)
case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) => GreaterThanOrEqual(a.name, v)
+
+ case expressions.InSet(a: Attribute, set) => In(a.name, set.toArray)
}
}
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 82a2cf8402f8f..4d87f6817dcb9 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
@@ -41,8 +41,7 @@ private[sql] case class LogicalRelation(relation: BaseRelation)
}
@transient override lazy val statistics = Statistics(
- // TODO: Allow datasources to provide statistics as well.
- sizeInBytes = BigInt(relation.sqlContext.defaultSizeInBytes)
+ sizeInBytes = BigInt(relation.sizeInBytes)
)
/** Used to lookup original attribute capitalization */
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala
index e72a2aeb8f310..4a9fefc12b9ad 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala
@@ -24,3 +24,4 @@ case class GreaterThan(attribute: String, value: Any) extends Filter
case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter
case class LessThan(attribute: String, value: Any) extends Filter
case class LessThanOrEqual(attribute: String, value: Any) extends Filter
+case class In(attribute: String, values: Array[Any]) extends Filter
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 ac3bf9d8e1a21..861638b1e99b6 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,7 +18,7 @@ package org.apache.spark.sql.sources
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{Row, SQLContext, StructType}
+import org.apache.spark.sql.{SQLConf, Row, SQLContext, StructType}
import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute}
/**
@@ -53,6 +53,15 @@ trait RelationProvider {
abstract class BaseRelation {
def sqlContext: SQLContext
def schema: StructType
+
+ /**
+ * Returns an estimated size of this relation in bytes. This information is used by the planner
+ * to decided when it is safe to broadcast a relation and can be overridden by sources that
+ * know the size ahead of time. By default, the system will assume that tables are too
+ * 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
}
/**
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 8a80724c08c7c..a63515464c688 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
@@ -72,6 +72,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
2.5)
}
+ test("aggregation with codegen") {
+ val originalValue = codegenEnabled
+ setConf(SQLConf.CODEGEN_ENABLED, "true")
+ sql("SELECT key FROM testData GROUP BY key").collect()
+ setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString)
+ }
+
test("SPARK-3176 Added Parser of SQL LAST()") {
checkAnswer(
sql("SELECT LAST(n) FROM lowerCaseData"),
@@ -189,7 +196,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
Seq(Seq("1")))
}
- test("sorting") {
+ def sortTest() = {
checkAnswer(
sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"),
Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2)))
@@ -231,6 +238,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
mapData.collect().sortBy(_.data(1)).reverse.toSeq)
}
+ test("sorting") {
+ val before = externalSortEnabled
+ setConf(SQLConf.EXTERNAL_SORT, "false")
+ sortTest()
+ setConf(SQLConf.EXTERNAL_SORT, before.toString)
+ }
+
+ test("external sorting") {
+ val before = externalSortEnabled
+ setConf(SQLConf.EXTERNAL_SORT, "true")
+ sortTest()
+ setConf(SQLConf.EXTERNAL_SORT, before.toString)
+ }
+
test("limit") {
checkAnswer(
sql("SELECT * FROM testData LIMIT 10"),
@@ -544,7 +565,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil)
}
- test("INTERSECT") {
+ test("INTERSECT") {
checkAnswer(
sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"),
(1, "a") ::
@@ -942,4 +963,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
checkAnswer(sql("SELECT key FROM testData WHERE value not like '100%' order by key"),
(1 to 99).map(i => Seq(i)))
}
+
+ test("SPARK-4322 Grouping field with struct field as sub expression") {
+ jsonRDD(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil)).registerTempTable("data")
+ checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), 1)
+ dropTempTable("data")
+
+ jsonRDD(sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data")
+ checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), 2)
+ dropTempTable("data")
+ }
}
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 9ba3c210171bd..82afa31a99a7e 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
@@ -78,17 +78,23 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be
// Conjunction and disjunction
checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21)
checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100))
+ checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11)
checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) {
Seq(1) ++ (79 to 91)
}
+ checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2) {
+ // Although the `NOT` operator isn't supported directly, the optimizer can transform
+ // `NOT (a < b)` to `b >= a`
+ 88 to 100
+ }
// With unsupported predicate
- checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2)(88 to 100)
- checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11)
-
{
val seq = (1 to 30).mkString(", ")
checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100)
+ checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq)) AND key > 88", 1, 2) {
+ 89 to 100
+ }
}
def checkBatchPruning(
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 3cccafe92d4f3..d31a9d8418dee 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
@@ -17,11 +17,13 @@
package org.apache.spark.sql.parquet
+import _root_.parquet.filter2.predicate.{FilterPredicate, Operators}
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.mapreduce.Job
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
import parquet.hadoop.ParquetFileWriter
import parquet.hadoop.util.ContextUtil
+
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types.IntegerType
@@ -95,6 +97,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA
testRDD.registerTempTable("testsource")
parquetFile(ParquetTestData.testFilterDir.toString)
.registerTempTable("testfiltersource")
+
+ setConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED, "true")
}
override def afterAll() {
@@ -445,44 +449,24 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA
assert(true)
}
- test("create RecordFilter for simple predicates") {
- val attribute1 = new AttributeReference("first", IntegerType, false)()
- val predicate1 = new EqualTo(attribute1, new Literal(1, IntegerType))
- val filter1 = ParquetFilters.createFilter(predicate1)
- assert(filter1.isDefined)
- assert(filter1.get.predicate == predicate1, "predicates do not match")
- assert(filter1.get.isInstanceOf[ComparisonFilter])
- val cmpFilter1 = filter1.get.asInstanceOf[ComparisonFilter]
- assert(cmpFilter1.columnName == "first", "column name incorrect")
-
- val predicate2 = new LessThan(attribute1, new Literal(4, IntegerType))
- val filter2 = ParquetFilters.createFilter(predicate2)
- assert(filter2.isDefined)
- assert(filter2.get.predicate == predicate2, "predicates do not match")
- assert(filter2.get.isInstanceOf[ComparisonFilter])
- val cmpFilter2 = filter2.get.asInstanceOf[ComparisonFilter]
- assert(cmpFilter2.columnName == "first", "column name incorrect")
-
- val predicate3 = new And(predicate1, predicate2)
- val filter3 = ParquetFilters.createFilter(predicate3)
- assert(filter3.isDefined)
- assert(filter3.get.predicate == predicate3, "predicates do not match")
- assert(filter3.get.isInstanceOf[AndFilter])
-
- val predicate4 = new Or(predicate1, predicate2)
- val filter4 = ParquetFilters.createFilter(predicate4)
- assert(filter4.isDefined)
- assert(filter4.get.predicate == predicate4, "predicates do not match")
- assert(filter4.get.isInstanceOf[OrFilter])
-
- val attribute2 = new AttributeReference("second", IntegerType, false)()
- val predicate5 = new GreaterThan(attribute1, attribute2)
- val badfilter = ParquetFilters.createFilter(predicate5)
- assert(badfilter.isDefined === false)
-
- val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2))
- val badfilter2 = ParquetFilters.createFilter(predicate6)
- assert(badfilter2.isDefined === false)
+ test("make RecordFilter for simple predicates") {
+ def checkFilter[T <: FilterPredicate](predicate: Expression, defined: Boolean = true): Unit = {
+ val filter = ParquetFilters.createFilter(predicate)
+ if (defined) {
+ assert(filter.isDefined)
+ assert(filter.get.isInstanceOf[T])
+ } else {
+ assert(filter.isEmpty)
+ }
+ }
+
+ checkFilter[Operators.Eq[Integer]]('a.int === 1)
+ checkFilter[Operators.Lt[Integer]]('a.int < 4)
+ checkFilter[Operators.And]('a.int === 1 && 'a.int < 4)
+ checkFilter[Operators.Or]('a.int === 1 || 'a.int < 4)
+
+ checkFilter('a.int > 'b.int, defined = false)
+ checkFilter(('a.int > 'b.int) && ('a.int > 'b.int), defined = false)
}
test("test filter by predicate pushdown") {
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 8b2f1591d5bf3..939b3c0c66de7 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
@@ -51,6 +51,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL
case LessThanOrEqual("a", v: Int) => (a: Int) => a <= v
case GreaterThan("a", v: Int) => (a: Int) => a > v
case GreaterThanOrEqual("a", v: Int) => (a: Int) => a >= v
+ case In("a", values) => (a: Int) => values.map(_.asInstanceOf[Int]).toSet.contains(a)
}
def eval(a: Int) = !filterFunctions.map(_(a)).contains(false)
@@ -121,6 +122,10 @@ class FilteredScanSuite extends DataSourceTest {
"SELECT * FROM oneToTenFiltered WHERE a = 1",
Seq(1).map(i => Row(i, i * 2)).toSeq)
+ sqlTest(
+ "SELECT * FROM oneToTenFiltered WHERE a IN (1,3,5)",
+ Seq(1,3,5).map(i => Row(i, i * 2)).toSeq)
+
sqlTest(
"SELECT * FROM oneToTenFiltered WHERE A = 1",
Seq(1).map(i => Row(i, i * 2)).toSeq)
@@ -150,6 +155,8 @@ class FilteredScanSuite extends DataSourceTest {
testPushDown("SELECT * FROM oneToTenFiltered WHERE a > 1 AND a < 10", 8)
+ testPushDown("SELECT * FROM oneToTenFiltered WHERE a IN (1,3,5)", 3)
+
testPushDown("SELECT * FROM oneToTenFiltered WHERE a = 20", 0)
testPushDown("SELECT * FROM oneToTenFiltered WHERE b = 1", 10)
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 bba29b2bdca4d..23d12cbff3495 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
@@ -19,9 +19,10 @@ package org.apache.spark.sql.hive.thriftserver
import java.io.File
import java.net.ServerSocket
-import java.sql.{DriverManager, Statement}
+import java.sql.{Date, DriverManager, Statement}
import java.util.concurrent.TimeoutException
+import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.duration._
import scala.concurrent.{Await, Promise}
@@ -51,6 +52,15 @@ import org.apache.spark.sql.hive.HiveShim
class HiveThriftServer2Suite extends FunSuite with Logging {
Class.forName(classOf[HiveDriver].getCanonicalName)
+ object TestData {
+ def getTestDataFilePath(name: String) = {
+ Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name")
+ }
+
+ val smallKv = getTestDataFilePath("small_kv.txt")
+ val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt")
+ }
+
def randomListeningPort = {
// Let the system to choose a random available port to avoid collision with other parallel
// builds.
@@ -145,12 +155,8 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
}
}
- val env = Seq(
- // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths
- "SPARK_TESTING" -> "0",
- // Prevents loading classes out of the assembly jar. Otherwise Utils.sparkVersion can't read
- // proper version information from the jar manifest.
- "SPARK_PREPEND_CLASSES" -> "")
+ // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths
+ val env = Seq("SPARK_TESTING" -> "0")
Process(command, None, env: _*).run(ProcessLogger(
captureThriftServerOutput("stdout"),
@@ -194,15 +200,12 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
test("Test JDBC query execution") {
withJdbcStatement() { statement =>
- val dataFilePath =
- Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")
-
- val queries =
- s"""SET spark.sql.shuffle.partitions=3;
- |CREATE TABLE test(key INT, val STRING);
- |LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test;
- |CACHE TABLE test;
- """.stripMargin.split(";").map(_.trim).filter(_.nonEmpty)
+ val queries = Seq(
+ "SET spark.sql.shuffle.partitions=3",
+ "DROP TABLE IF EXISTS test",
+ "CREATE TABLE test(key INT, val STRING)",
+ s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test",
+ "CACHE TABLE test")
queries.foreach(statement.execute)
@@ -216,14 +219,10 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
test("SPARK-3004 regression: result set containing NULL") {
withJdbcStatement() { statement =>
- val dataFilePath =
- Thread.currentThread().getContextClassLoader.getResource(
- "data/files/small_kv_with_null.txt")
-
val queries = Seq(
"DROP TABLE IF EXISTS test_null",
"CREATE TABLE test_null(key INT, val STRING)",
- s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null")
+ s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null")
queries.foreach(statement.execute)
@@ -270,13 +269,10 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
test("SPARK-4292 regression: result set iterator issue") {
withJdbcStatement() { statement =>
- val dataFilePath =
- Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")
-
val queries = Seq(
"DROP TABLE IF EXISTS test_4292",
"CREATE TABLE test_4292(key INT, val STRING)",
- s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_4292")
+ s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292")
queries.foreach(statement.execute)
@@ -284,10 +280,52 @@ class HiveThriftServer2Suite extends FunSuite with Logging {
Seq(238, 86, 311, 27, 165).foreach { key =>
resultSet.next()
- assert(resultSet.getInt(1) == key)
+ assert(resultSet.getInt(1) === key)
}
statement.executeQuery("DROP TABLE IF EXISTS test_4292")
}
}
+
+ test("SPARK-4309 regression: Date type support") {
+ withJdbcStatement() { statement =>
+ val queries = Seq(
+ "DROP TABLE IF EXISTS test_date",
+ "CREATE TABLE test_date(key INT, value STRING)",
+ s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date")
+
+ queries.foreach(statement.execute)
+
+ assertResult(Date.valueOf("2011-01-01")) {
+ val resultSet = statement.executeQuery(
+ "SELECT CAST('2011-01-01' as date) FROM test_date LIMIT 1")
+ resultSet.next()
+ resultSet.getDate(1)
+ }
+ }
+ }
+
+ test("SPARK-4407 regression: Complex type support") {
+ withJdbcStatement() { statement =>
+ val queries = Seq(
+ "DROP TABLE IF EXISTS test_map",
+ "CREATE TABLE test_map(key INT, value STRING)",
+ s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map")
+
+ queries.foreach(statement.execute)
+
+ assertResult("""{238:"val_238"}""") {
+ val resultSet = statement.executeQuery("SELECT MAP(key, value) FROM test_map LIMIT 1")
+ resultSet.next()
+ resultSet.getString(1)
+ }
+
+ assertResult("""["238","val_238"]""") {
+ val resultSet = statement.executeQuery(
+ "SELECT ARRAY(CAST(key AS STRING), value) FROM test_map LIMIT 1")
+ resultSet.next()
+ resultSet.getString(1)
+ }
+ }
+ }
}
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 aa2e3cab72bb9..9258ad0cdf1d0 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
@@ -17,7 +17,7 @@
package org.apache.spark.sql.hive.thriftserver
-import java.sql.Timestamp
+import java.sql.{Date, Timestamp}
import java.util.{ArrayList => JArrayList, Map => JMap}
import scala.collection.JavaConversions._
@@ -131,14 +131,13 @@ private[hive] class SparkExecuteStatementOperation(
to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal)))
case ShortType =>
to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal)))
+ case DateType =>
+ to.addColumnValue(ColumnValue.dateValue(from(ordinal).asInstanceOf[Date]))
case TimestampType =>
to.addColumnValue(
ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp]))
case BinaryType | _: ArrayType | _: StructType | _: MapType =>
- val hiveString = result
- .queryExecution
- .asInstanceOf[HiveContext#QueryExecution]
- .toHiveString((from.get(ordinal), dataTypes(ordinal)))
+ val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal)))
to.addColumnValue(ColumnValue.stringValue(hiveString))
}
}
@@ -163,6 +162,8 @@ private[hive] class SparkExecuteStatementOperation(
to.addColumnValue(ColumnValue.byteValue(null))
case ShortType =>
to.addColumnValue(ColumnValue.shortValue(null))
+ case DateType =>
+ to.addColumnValue(ColumnValue.dateValue(null))
case TimestampType =>
to.addColumnValue(ColumnValue.timestampValue(null))
case BinaryType | _: ArrayType | _: StructType | _: MapType =>
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 a642478d08857..3c7f62af450d9 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
@@ -18,7 +18,7 @@
package org.apache.spark.sql.hive.thriftserver
import java.security.PrivilegedExceptionAction
-import java.sql.Timestamp
+import java.sql.{Date, Timestamp}
import java.util.concurrent.Future
import java.util.{ArrayList => JArrayList, List => JList, Map => JMap}
@@ -113,7 +113,7 @@ private[hive] class SparkExecuteStatementOperation(
def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) {
dataTypes(ordinal) match {
case StringType =>
- to += from.get(ordinal).asInstanceOf[String]
+ to += from.getString(ordinal)
case IntegerType =>
to += from.getInt(ordinal)
case BooleanType =>
@@ -123,23 +123,20 @@ private[hive] class SparkExecuteStatementOperation(
case FloatType =>
to += from.getFloat(ordinal)
case DecimalType() =>
- to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal
+ to += from.getAs[BigDecimal](ordinal).bigDecimal
case LongType =>
to += from.getLong(ordinal)
case ByteType =>
to += from.getByte(ordinal)
case ShortType =>
to += from.getShort(ordinal)
+ case DateType =>
+ to += from.getAs[Date](ordinal)
case TimestampType =>
- to += from.get(ordinal).asInstanceOf[Timestamp]
- case BinaryType =>
- to += from.get(ordinal).asInstanceOf[String]
- case _: ArrayType =>
- to += from.get(ordinal).asInstanceOf[String]
- case _: StructType =>
- to += from.get(ordinal).asInstanceOf[String]
- case _: MapType =>
- to += from.get(ordinal).asInstanceOf[String]
+ to += from.getAs[Timestamp](ordinal)
+ case BinaryType | _: ArrayType | _: StructType | _: MapType =>
+ val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal)))
+ to += hiveString
}
}
@@ -147,9 +144,9 @@ private[hive] class SparkExecuteStatementOperation(
validateDefaultFetchOrientation(order)
assertState(OperationState.FINISHED)
setHasResultSet(true)
- val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion)
+ val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion)
if (!iter.hasNext) {
- reultRowSet
+ resultRowSet
} else {
// maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int
val maxRows = maxRowsL.toInt
@@ -166,10 +163,10 @@ private[hive] class SparkExecuteStatementOperation(
}
curCol += 1
}
- reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]])
+ resultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]])
curRow += 1
}
- reultRowSet
+ resultRowSet
}
}
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 1a3c24be420e6..7c0be4872d762 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
@@ -103,6 +103,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf5",
"udf_java_method",
"create_merge_compressed",
+ "create_view_partitioned",
"database_location",
"database_properties",
@@ -969,6 +970,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"union_script",
"varchar_2",
"varchar_join1",
- "varchar_union1"
+ "varchar_union1",
+ "view",
+ "view_cast",
+ "view_inputs"
)
}
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 67e36a951e506..fa9a1e64b0f80 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -144,9 +144,6 @@
hive-0.12.0
-
- false
- com.twitter
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 e88afaaf001c0..304b9a73ee91d 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
@@ -19,36 +19,27 @@ package org.apache.spark.sql.hive
import java.io.{BufferedReader, File, InputStreamReader, PrintStream}
import java.sql.{Date, Timestamp}
-import java.util.{ArrayList => JArrayList}
-
-import org.apache.hadoop.hive.common.`type`.HiveDecimal
-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 scala.reflect.runtime.universe.{TypeTag, typeTag}
+import scala.reflect.runtime.universe.TypeTag
-import org.apache.hadoop.fs.FileSystem
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.metadata.Table
import org.apache.hadoop.hive.ql.processors._
import org.apache.hadoop.hive.ql.session.SessionState
-import org.apache.hadoop.hive.serde2.io.TimestampWritable
-import org.apache.hadoop.hive.serde2.io.DateWritable
+import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable}
import org.apache.spark.SparkContext
-import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators}
-import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators, OverrideCatalog, OverrideFunctionRegistry}
import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.ExtractPythonUdfs
-import org.apache.spark.sql.execution.QueryExecutionException
-import org.apache.spark.sql.execution.{Command => PhysicalCommand}
+import org.apache.spark.sql.catalyst.types.DecimalType
+import org.apache.spark.sql.catalyst.types.decimal.Decimal
+import org.apache.spark.sql.execution.{ExtractPythonUdfs, QueryExecutionException, Command => PhysicalCommand}
import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
import org.apache.spark.sql.sources.DataSourceStrategy
@@ -136,7 +127,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val relation = EliminateAnalysisOperators(catalog.lookupRelation(None, tableName))
relation match {
- case relation: MetastoreRelation => {
+ case relation: MetastoreRelation =>
// This method is mainly based on
// org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table)
// in Hive 0.13 (except that we do not use fs.getContentSummary).
@@ -157,7 +148,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
}
def getFileSizeForTable(conf: HiveConf, table: Table): Long = {
- val path = table.getPath()
+ val path = table.getPath
var size: Long = 0L
try {
val fs = path.getFileSystem(conf)
@@ -187,15 +178,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val hiveTTable = relation.hiveQlTable.getTTable
hiveTTable.setParameters(tableParameters)
val tableFullName =
- relation.hiveQlTable.getDbName() + "." + relation.hiveQlTable.getTableName()
+ relation.hiveQlTable.getDbName + "." + relation.hiveQlTable.getTableName
catalog.client.alterTable(tableFullName, new Table(hiveTTable))
}
- }
case otherRelation =>
throw new NotImplementedError(
s"Analyze has only implemented for Hive tables, " +
- s"but ${tableName} is a ${otherRelation.nodeName}")
+ s"but $tableName is a ${otherRelation.nodeName}")
}
}
@@ -374,50 +364,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
/** Extends QueryExecution with hive specific features. */
protected[sql] abstract class QueryExecution extends super.QueryExecution {
- protected val primitiveTypes =
- Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType,
- ShortType, DateType, TimestampType, BinaryType)
-
- protected[sql] def toHiveString(a: (Any, DataType)): String = a match {
- case (struct: Row, StructType(fields)) =>
- struct.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_,_], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "NULL"
- case (d: Date, DateType) => new DateWritable(d).toString
- case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString
- case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8")
- case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString
- HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString
- case (other, tpe) if primitiveTypes contains tpe => other.toString
- }
-
- /** Hive outputs fields of structs slightly differently than top level attributes. */
- protected def toHiveStructString(a: (Any, DataType)): String = a match {
- case (struct: Row, StructType(fields)) =>
- struct.zip(fields).map {
- case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
- }.mkString("{", ",", "}")
- case (seq: Seq[_], ArrayType(typ, _)) =>
- seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
- case (map: Map[_, _], MapType(kType, vType, _)) =>
- map.map {
- case (key, value) =>
- toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType))
- }.toSeq.sorted.mkString("{", ",", "}")
- case (null, _) => "null"
- case (s: String, StringType) => "\"" + s + "\""
- case (decimal, DecimalType()) => decimal.toString
- case (other, tpe) if primitiveTypes contains tpe => other.toString
- }
-
/**
* Returns the result as a hive compatible sequence of strings. For native commands, the
* execution is simply passed back to Hive.
@@ -435,8 +381,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
// We need the types so we can output struct field names
val types = analyzed.output.map(_.dataType)
// Reformat to match hive tab delimited output.
- val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq
- asString
+ result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq
}
override def simpleString: String =
@@ -447,3 +392,49 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
}
}
}
+
+object HiveContext {
+ protected val primitiveTypes =
+ Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType,
+ ShortType, DateType, TimestampType, BinaryType)
+
+ protected[sql] def toHiveString(a: (Any, DataType)): String = a match {
+ case (struct: Row, StructType(fields)) =>
+ struct.zip(fields).map {
+ case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
+ }.mkString("{", ",", "}")
+ case (seq: Seq[_], ArrayType(typ, _)) =>
+ seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
+ case (map: Map[_,_], MapType(kType, vType, _)) =>
+ map.map {
+ case (key, value) =>
+ toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType))
+ }.toSeq.sorted.mkString("{", ",", "}")
+ case (null, _) => "NULL"
+ case (d: Date, DateType) => new DateWritable(d).toString
+ case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString
+ case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8")
+ case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString
+ HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString
+ case (other, tpe) if primitiveTypes contains tpe => other.toString
+ }
+
+ /** Hive outputs fields of structs slightly differently than top level attributes. */
+ protected def toHiveStructString(a: (Any, DataType)): String = a match {
+ case (struct: Row, StructType(fields)) =>
+ struct.zip(fields).map {
+ case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}"""
+ }.mkString("{", ",", "}")
+ case (seq: Seq[_], ArrayType(typ, _)) =>
+ seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]")
+ case (map: Map[_, _], MapType(kType, vType, _)) =>
+ map.map {
+ case (key, value) =>
+ toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType))
+ }.toSeq.sorted.mkString("{", ",", "}")
+ case (null, _) => "null"
+ case (s: String, StringType) => "\"" + s + "\""
+ case (decimal, DecimalType()) => decimal.toString
+ case (other, tpe) if primitiveTypes contains tpe => other.toString
+ }
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 7e76aff642bb5..ada980acb1f77 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
@@ -86,13 +86,41 @@ private[hive] trait HiveInspectors {
* @param data the data in Hive type
* @param oi the ObjectInspector associated with the Hive Type
* @return convert the data into catalyst type
+ * TODO return the function of (data => Any) instead for performance consideration
*/
def unwrap(data: Any, oi: ObjectInspector): Any = oi match {
case _ if data == null => null
- case hvoi: HiveVarcharObjectInspector =>
- if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue
- case hdoi: HiveDecimalObjectInspector =>
- if (data == null) null else HiveShim.toCatalystDecimal(hdoi, data)
+ case poi: VoidObjectInspector => null
+ case poi: WritableConstantHiveVarcharObjectInspector =>
+ poi.getWritableConstantValue.getHiveVarchar.getValue
+ case poi: WritableConstantHiveDecimalObjectInspector =>
+ HiveShim.toCatalystDecimal(
+ PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector,
+ poi.getWritableConstantValue.getHiveDecimal)
+ case poi: WritableConstantTimestampObjectInspector =>
+ poi.getWritableConstantValue.getTimestamp.clone()
+ case poi: WritableConstantIntObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantDoubleObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantBooleanObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantLongObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantFloatObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantShortObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantByteObjectInspector =>
+ poi.getWritableConstantValue.get()
+ case poi: WritableConstantBinaryObjectInspector =>
+ val writable = poi.getWritableConstantValue
+ val temp = new Array[Byte](writable.getLength)
+ System.arraycopy(writable.getBytes, 0, temp, 0, temp.length)
+ temp
+ case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get()
+ case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue
+ case hdoi: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(hdoi, data)
// org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object
// if next timestamp is null, so Timestamp object is cloned
case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone()
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 9ae019842217d..91a157785d5bb 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,17 +20,15 @@ package org.apache.spark.sql.hive
import java.io.IOException
import java.util.{List => JList}
-import scala.util.matching.Regex
import scala.util.parsing.combinator.RegexParsers
import org.apache.hadoop.util.ReflectionUtils
-import org.apache.hadoop.fs.Path
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.ql.metadata.{Hive, Partition, Table, HiveException}
-import org.apache.hadoop.hive.ql.plan.{TableDesc, CreateTableDesc}
+import org.apache.hadoop.hive.ql.plan.CreateTableDesc
import org.apache.hadoop.hive.serde.serdeConstants
import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException}
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
@@ -67,20 +65,26 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
db: Option[String],
tableName: String,
alias: Option[String]): LogicalPlan = synchronized {
- val (databaseName, tblName) = processDatabaseAndTableName(
- db.getOrElse(hive.sessionState.getCurrentDatabase), tableName)
+ val (databaseName, tblName) =
+ processDatabaseAndTableName(db.getOrElse(hive.sessionState.getCurrentDatabase), tableName)
val table = client.getTable(databaseName, tblName)
- val partitions: Seq[Partition] =
- if (table.isPartitioned) {
- HiveShim.getAllPartitionsOf(client, table).toSeq
- } else {
- Nil
- }
+ if (table.isView) {
+ // if the unresolved relation is from hive view
+ // parse the text into logic node.
+ HiveQl.createPlanForView(table, alias)
+ } else {
+ val partitions: Seq[Partition] =
+ if (table.isPartitioned) {
+ HiveShim.getAllPartitionsOf(client, table).toSeq
+ } else {
+ Nil
+ }
- // Since HiveQL is case insensitive for table names we make them all lowercase.
- MetastoreRelation(
- databaseName, tblName, alias)(
- table.getTTable, partitions.map(part => part.getTPartition))(hive)
+ // Since HiveQL is case insensitive for table names we make them all lowercase.
+ MetastoreRelation(
+ databaseName, tblName, alias)(
+ table.getTTable, partitions.map(part => part.getTPartition))(hive)
+ }
}
/**
@@ -282,6 +286,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
if (childOutputDataTypes == tableOutputDataTypes) {
p
+ } else if (childOutputDataTypes.size == tableOutputDataTypes.size &&
+ childOutputDataTypes.zip(tableOutputDataTypes)
+ .forall { case (left, right) => DataType.equalsIgnoreNullability(left, right) }) {
+ // If both types ignoring nullability of ArrayType, MapType, StructType are the same,
+ // use InsertIntoHiveTable instead of InsertIntoTable.
+ InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite)
} else {
// Only do the casting when child output data types differ from table output data types.
val castedChildOutput = child.output.zip(table.output).map {
@@ -312,6 +322,27 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
override def unregisterAllTables() = {}
}
+/**
+ * A logical plan representing insertion into Hive table.
+ * This plan ignores nullability of ArrayType, MapType, StructType unlike InsertIntoTable
+ * because Hive table doesn't have nullability for ARRAY, MAP, STRUCT types.
+ */
+private[hive] case class InsertIntoHiveTable(
+ table: LogicalPlan,
+ partition: Map[String, Option[String]],
+ child: LogicalPlan,
+ overwrite: Boolean)
+ extends LogicalPlan {
+
+ override def children = child :: Nil
+ override def output = child.output
+
+ override lazy val resolved = childrenResolved && child.output.zip(table.output).forall {
+ case (childAttr, tableAttr) =>
+ DataType.equalsIgnoreNullability(childAttr.dataType, tableAttr.dataType)
+ }
+}
+
/**
* :: DeveloperApi ::
* Provides conversions between Spark SQL data types and Hive Metastore types.
@@ -416,6 +447,8 @@ private[hive] case class MetastoreRelation
@transient override lazy val statistics = Statistics(
sizeInBytes = {
+ val totalSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)
+ val rawDataSize = hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize)
// TODO: check if this estimate is valid for tables after partition pruning.
// NOTE: getting `totalSize` directly from params is kind of hacky, but this should be
// relatively cheap if parameters for the table are populated into the metastore. An
@@ -423,9 +456,12 @@ private[hive] case class MetastoreRelation
// of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`,
// `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future.
BigInt(
- Option(hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize))
- .map(_.toLong)
- .getOrElse(sqlContext.defaultSizeInBytes))
+ // When table is external,`totalSize` is always zero, which will influence join strategy
+ // so when `totalSize` is zero, use `rawDataSize` instead
+ // 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)))
}
)
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 74f68d0f95317..1ca0403d6f8ce 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
@@ -21,6 +21,7 @@ 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
+import org.apache.hadoop.hive.ql.metadata.Table
import org.apache.hadoop.hive.ql.parse._
import org.apache.hadoop.hive.ql.plan.PlanUtils
@@ -106,7 +107,6 @@ private[hive] object HiveQl {
"TOK_DROPINDEX",
"TOK_MSCK",
- // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this.
"TOK_ALTERVIEW_ADDPARTS",
"TOK_ALTERVIEW_AS",
"TOK_ALTERVIEW_DROPPARTS",
@@ -259,6 +259,14 @@ private[hive] object HiveQl {
}
}
+ /** Creates LogicalPlan for a given VIEW */
+ def createPlanForView(view: Table, alias: Option[String]) = alias match {
+ // because hive use things like `_c0` to build the expanded text
+ // currently we cannot support view from "create view v1(c1) as ..."
+ case None => Subquery(view.getTableName, createPlan(view.getViewExpandedText))
+ case Some(aliasText) => Subquery(aliasText, createPlan(view.getViewExpandedText))
+ }
+
def parseDdl(ddl: String): Seq[Attribute] = {
val tree =
try {
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 989740c8d43b6..56fc85239e1c0 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
@@ -58,12 +58,18 @@ private[hive] trait HiveStrategies {
def lowerCase =
new SchemaRDD(s.sqlContext, s.logicalPlan)
- def addPartitioningAttributes(attrs: Seq[Attribute]) =
- new SchemaRDD(
- s.sqlContext,
- s.logicalPlan transform {
- case p: ParquetRelation => p.copy(partitioningAttributes = attrs)
- })
+ def addPartitioningAttributes(attrs: Seq[Attribute]) = {
+ // Don't add the partitioning key if its already present in the data.
+ if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) {
+ s
+ } else {
+ new SchemaRDD(
+ s.sqlContext,
+ s.logicalPlan transform {
+ case p: ParquetRelation => p.copy(partitioningAttributes = attrs)
+ })
+ }
+ }
}
implicit class PhysicalPlanHacks(originalPlan: SparkPlan) {
@@ -161,7 +167,11 @@ private[hive] trait HiveStrategies {
object DataSinks extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) =>
- InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil
+ execution.InsertIntoHiveTable(
+ table, partition, planLater(child), overwrite)(hiveContext) :: Nil
+ case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) =>
+ execution.InsertIntoHiveTable(
+ table, partition, planLater(child), overwrite)(hiveContext) :: Nil
case logical.CreateTableAsSelect(
Some(database), tableName, child, allowExisting, Some(extra: ASTNode)) =>
CreateTableAsSelect(
diff --git a/sql/hive/src/test/resources/golden/NaN to Decimal-0-6ca781bc343025635d72321ef0a9d425 b/sql/hive/src/test/resources/golden/NaN to Decimal-0-6ca781bc343025635d72321ef0a9d425
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/NaN to Decimal-0-6ca781bc343025635d72321ef0a9d425
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 b/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 b/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-16-3077fd708f97a03d4151a1a30e4308d8 b/sql/hive/src/test/resources/golden/view-16-3077fd708f97a03d4151a1a30e4308d8
new file mode 100644
index 0000000000000..7aae61e5eb82f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-16-3077fd708f97a03d4151a1a30e4308d8
@@ -0,0 +1,500 @@
+238 val_238
+86 val_86
+311 val_311
+27 val_27
+165 val_165
+409 val_409
+255 val_255
+278 val_278
+98 val_98
+484 val_484
+265 val_265
+193 val_193
+401 val_401
+150 val_150
+273 val_273
+224 val_224
+369 val_369
+66 val_66
+128 val_128
+213 val_213
+146 val_146
+406 val_406
+429 val_429
+374 val_374
+152 val_152
+469 val_469
+145 val_145
+495 val_495
+37 val_37
+327 val_327
+281 val_281
+277 val_277
+209 val_209
+15 val_15
+82 val_82
+403 val_403
+166 val_166
+417 val_417
+430 val_430
+252 val_252
+292 val_292
+219 val_219
+287 val_287
+153 val_153
+193 val_193
+338 val_338
+446 val_446
+459 val_459
+394 val_394
+237 val_237
+482 val_482
+174 val_174
+413 val_413
+494 val_494
+207 val_207
+199 val_199
+466 val_466
+208 val_208
+174 val_174
+399 val_399
+396 val_396
+247 val_247
+417 val_417
+489 val_489
+162 val_162
+377 val_377
+397 val_397
+309 val_309
+365 val_365
+266 val_266
+439 val_439
+342 val_342
+367 val_367
+325 val_325
+167 val_167
+195 val_195
+475 val_475
+17 val_17
+113 val_113
+155 val_155
+203 val_203
+339 val_339
+0 val_0
+455 val_455
+128 val_128
+311 val_311
+316 val_316
+57 val_57
+302 val_302
+205 val_205
+149 val_149
+438 val_438
+345 val_345
+129 val_129
+170 val_170
+20 val_20
+489 val_489
+157 val_157
+378 val_378
+221 val_221
+92 val_92
+111 val_111
+47 val_47
+72 val_72
+4 val_4
+280 val_280
+35 val_35
+427 val_427
+277 val_277
+208 val_208
+356 val_356
+399 val_399
+169 val_169
+382 val_382
+498 val_498
+125 val_125
+386 val_386
+437 val_437
+469 val_469
+192 val_192
+286 val_286
+187 val_187
+176 val_176
+54 val_54
+459 val_459
+51 val_51
+138 val_138
+103 val_103
+239 val_239
+213 val_213
+216 val_216
+430 val_430
+278 val_278
+176 val_176
+289 val_289
+221 val_221
+65 val_65
+318 val_318
+332 val_332
+311 val_311
+275 val_275
+137 val_137
+241 val_241
+83 val_83
+333 val_333
+180 val_180
+284 val_284
+12 val_12
+230 val_230
+181 val_181
+67 val_67
+260 val_260
+404 val_404
+384 val_384
+489 val_489
+353 val_353
+373 val_373
+272 val_272
+138 val_138
+217 val_217
+84 val_84
+348 val_348
+466 val_466
+58 val_58
+8 val_8
+411 val_411
+230 val_230
+208 val_208
+348 val_348
+24 val_24
+463 val_463
+431 val_431
+179 val_179
+172 val_172
+42 val_42
+129 val_129
+158 val_158
+119 val_119
+496 val_496
+0 val_0
+322 val_322
+197 val_197
+468 val_468
+393 val_393
+454 val_454
+100 val_100
+298 val_298
+199 val_199
+191 val_191
+418 val_418
+96 val_96
+26 val_26
+165 val_165
+327 val_327
+230 val_230
+205 val_205
+120 val_120
+131 val_131
+51 val_51
+404 val_404
+43 val_43
+436 val_436
+156 val_156
+469 val_469
+468 val_468
+308 val_308
+95 val_95
+196 val_196
+288 val_288
+481 val_481
+457 val_457
+98 val_98
+282 val_282
+197 val_197
+187 val_187
+318 val_318
+318 val_318
+409 val_409
+470 val_470
+137 val_137
+369 val_369
+316 val_316
+169 val_169
+413 val_413
+85 val_85
+77 val_77
+0 val_0
+490 val_490
+87 val_87
+364 val_364
+179 val_179
+118 val_118
+134 val_134
+395 val_395
+282 val_282
+138 val_138
+238 val_238
+419 val_419
+15 val_15
+118 val_118
+72 val_72
+90 val_90
+307 val_307
+19 val_19
+435 val_435
+10 val_10
+277 val_277
+273 val_273
+306 val_306
+224 val_224
+309 val_309
+389 val_389
+327 val_327
+242 val_242
+369 val_369
+392 val_392
+272 val_272
+331 val_331
+401 val_401
+242 val_242
+452 val_452
+177 val_177
+226 val_226
+5 val_5
+497 val_497
+402 val_402
+396 val_396
+317 val_317
+395 val_395
+58 val_58
+35 val_35
+336 val_336
+95 val_95
+11 val_11
+168 val_168
+34 val_34
+229 val_229
+233 val_233
+143 val_143
+472 val_472
+322 val_322
+498 val_498
+160 val_160
+195 val_195
+42 val_42
+321 val_321
+430 val_430
+119 val_119
+489 val_489
+458 val_458
+78 val_78
+76 val_76
+41 val_41
+223 val_223
+492 val_492
+149 val_149
+449 val_449
+218 val_218
+228 val_228
+138 val_138
+453 val_453
+30 val_30
+209 val_209
+64 val_64
+468 val_468
+76 val_76
+74 val_74
+342 val_342
+69 val_69
+230 val_230
+33 val_33
+368 val_368
+103 val_103
+296 val_296
+113 val_113
+216 val_216
+367 val_367
+344 val_344
+167 val_167
+274 val_274
+219 val_219
+239 val_239
+485 val_485
+116 val_116
+223 val_223
+256 val_256
+263 val_263
+70 val_70
+487 val_487
+480 val_480
+401 val_401
+288 val_288
+191 val_191
+5 val_5
+244 val_244
+438 val_438
+128 val_128
+467 val_467
+432 val_432
+202 val_202
+316 val_316
+229 val_229
+469 val_469
+463 val_463
+280 val_280
+2 val_2
+35 val_35
+283 val_283
+331 val_331
+235 val_235
+80 val_80
+44 val_44
+193 val_193
+321 val_321
+335 val_335
+104 val_104
+466 val_466
+366 val_366
+175 val_175
+403 val_403
+483 val_483
+53 val_53
+105 val_105
+257 val_257
+406 val_406
+409 val_409
+190 val_190
+406 val_406
+401 val_401
+114 val_114
+258 val_258
+90 val_90
+203 val_203
+262 val_262
+348 val_348
+424 val_424
+12 val_12
+396 val_396
+201 val_201
+217 val_217
+164 val_164
+431 val_431
+454 val_454
+478 val_478
+298 val_298
+125 val_125
+431 val_431
+164 val_164
+424 val_424
+187 val_187
+382 val_382
+5 val_5
+70 val_70
+397 val_397
+480 val_480
+291 val_291
+24 val_24
+351 val_351
+255 val_255
+104 val_104
+70 val_70
+163 val_163
+438 val_438
+119 val_119
+414 val_414
+200 val_200
+491 val_491
+237 val_237
+439 val_439
+360 val_360
+248 val_248
+479 val_479
+305 val_305
+417 val_417
+199 val_199
+444 val_444
+120 val_120
+429 val_429
+169 val_169
+443 val_443
+323 val_323
+325 val_325
+277 val_277
+230 val_230
+478 val_478
+178 val_178
+468 val_468
+310 val_310
+317 val_317
+333 val_333
+493 val_493
+460 val_460
+207 val_207
+249 val_249
+265 val_265
+480 val_480
+83 val_83
+136 val_136
+353 val_353
+172 val_172
+214 val_214
+462 val_462
+233 val_233
+406 val_406
+133 val_133
+175 val_175
+189 val_189
+454 val_454
+375 val_375
+401 val_401
+421 val_421
+407 val_407
+384 val_384
+256 val_256
+26 val_26
+134 val_134
+67 val_67
+384 val_384
+379 val_379
+18 val_18
+462 val_462
+492 val_492
+100 val_100
+298 val_298
+9 val_9
+341 val_341
+498 val_498
+146 val_146
+458 val_458
+362 val_362
+186 val_186
+285 val_285
+348 val_348
+167 val_167
+18 val_18
+273 val_273
+183 val_183
+281 val_281
+344 val_344
+97 val_97
+469 val_469
+315 val_315
+84 val_84
+28 val_28
+37 val_37
+448 val_448
+152 val_152
+348 val_348
+307 val_307
+194 val_194
+414 val_414
+477 val_477
+222 val_222
+126 val_126
+90 val_90
+169 val_169
+403 val_403
+400 val_400
+200 val_200
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/view-17-544b822e12afa24d2c64d6149e19c12c b/sql/hive/src/test/resources/golden/view-17-544b822e12afa24d2c64d6149e19c12c
new file mode 100644
index 0000000000000..7aae61e5eb82f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-17-544b822e12afa24d2c64d6149e19c12c
@@ -0,0 +1,500 @@
+238 val_238
+86 val_86
+311 val_311
+27 val_27
+165 val_165
+409 val_409
+255 val_255
+278 val_278
+98 val_98
+484 val_484
+265 val_265
+193 val_193
+401 val_401
+150 val_150
+273 val_273
+224 val_224
+369 val_369
+66 val_66
+128 val_128
+213 val_213
+146 val_146
+406 val_406
+429 val_429
+374 val_374
+152 val_152
+469 val_469
+145 val_145
+495 val_495
+37 val_37
+327 val_327
+281 val_281
+277 val_277
+209 val_209
+15 val_15
+82 val_82
+403 val_403
+166 val_166
+417 val_417
+430 val_430
+252 val_252
+292 val_292
+219 val_219
+287 val_287
+153 val_153
+193 val_193
+338 val_338
+446 val_446
+459 val_459
+394 val_394
+237 val_237
+482 val_482
+174 val_174
+413 val_413
+494 val_494
+207 val_207
+199 val_199
+466 val_466
+208 val_208
+174 val_174
+399 val_399
+396 val_396
+247 val_247
+417 val_417
+489 val_489
+162 val_162
+377 val_377
+397 val_397
+309 val_309
+365 val_365
+266 val_266
+439 val_439
+342 val_342
+367 val_367
+325 val_325
+167 val_167
+195 val_195
+475 val_475
+17 val_17
+113 val_113
+155 val_155
+203 val_203
+339 val_339
+0 val_0
+455 val_455
+128 val_128
+311 val_311
+316 val_316
+57 val_57
+302 val_302
+205 val_205
+149 val_149
+438 val_438
+345 val_345
+129 val_129
+170 val_170
+20 val_20
+489 val_489
+157 val_157
+378 val_378
+221 val_221
+92 val_92
+111 val_111
+47 val_47
+72 val_72
+4 val_4
+280 val_280
+35 val_35
+427 val_427
+277 val_277
+208 val_208
+356 val_356
+399 val_399
+169 val_169
+382 val_382
+498 val_498
+125 val_125
+386 val_386
+437 val_437
+469 val_469
+192 val_192
+286 val_286
+187 val_187
+176 val_176
+54 val_54
+459 val_459
+51 val_51
+138 val_138
+103 val_103
+239 val_239
+213 val_213
+216 val_216
+430 val_430
+278 val_278
+176 val_176
+289 val_289
+221 val_221
+65 val_65
+318 val_318
+332 val_332
+311 val_311
+275 val_275
+137 val_137
+241 val_241
+83 val_83
+333 val_333
+180 val_180
+284 val_284
+12 val_12
+230 val_230
+181 val_181
+67 val_67
+260 val_260
+404 val_404
+384 val_384
+489 val_489
+353 val_353
+373 val_373
+272 val_272
+138 val_138
+217 val_217
+84 val_84
+348 val_348
+466 val_466
+58 val_58
+8 val_8
+411 val_411
+230 val_230
+208 val_208
+348 val_348
+24 val_24
+463 val_463
+431 val_431
+179 val_179
+172 val_172
+42 val_42
+129 val_129
+158 val_158
+119 val_119
+496 val_496
+0 val_0
+322 val_322
+197 val_197
+468 val_468
+393 val_393
+454 val_454
+100 val_100
+298 val_298
+199 val_199
+191 val_191
+418 val_418
+96 val_96
+26 val_26
+165 val_165
+327 val_327
+230 val_230
+205 val_205
+120 val_120
+131 val_131
+51 val_51
+404 val_404
+43 val_43
+436 val_436
+156 val_156
+469 val_469
+468 val_468
+308 val_308
+95 val_95
+196 val_196
+288 val_288
+481 val_481
+457 val_457
+98 val_98
+282 val_282
+197 val_197
+187 val_187
+318 val_318
+318 val_318
+409 val_409
+470 val_470
+137 val_137
+369 val_369
+316 val_316
+169 val_169
+413 val_413
+85 val_85
+77 val_77
+0 val_0
+490 val_490
+87 val_87
+364 val_364
+179 val_179
+118 val_118
+134 val_134
+395 val_395
+282 val_282
+138 val_138
+238 val_238
+419 val_419
+15 val_15
+118 val_118
+72 val_72
+90 val_90
+307 val_307
+19 val_19
+435 val_435
+10 val_10
+277 val_277
+273 val_273
+306 val_306
+224 val_224
+309 val_309
+389 val_389
+327 val_327
+242 val_242
+369 val_369
+392 val_392
+272 val_272
+331 val_331
+401 val_401
+242 val_242
+452 val_452
+177 val_177
+226 val_226
+5 val_5
+497 val_497
+402 val_402
+396 val_396
+317 val_317
+395 val_395
+58 val_58
+35 val_35
+336 val_336
+95 val_95
+11 val_11
+168 val_168
+34 val_34
+229 val_229
+233 val_233
+143 val_143
+472 val_472
+322 val_322
+498 val_498
+160 val_160
+195 val_195
+42 val_42
+321 val_321
+430 val_430
+119 val_119
+489 val_489
+458 val_458
+78 val_78
+76 val_76
+41 val_41
+223 val_223
+492 val_492
+149 val_149
+449 val_449
+218 val_218
+228 val_228
+138 val_138
+453 val_453
+30 val_30
+209 val_209
+64 val_64
+468 val_468
+76 val_76
+74 val_74
+342 val_342
+69 val_69
+230 val_230
+33 val_33
+368 val_368
+103 val_103
+296 val_296
+113 val_113
+216 val_216
+367 val_367
+344 val_344
+167 val_167
+274 val_274
+219 val_219
+239 val_239
+485 val_485
+116 val_116
+223 val_223
+256 val_256
+263 val_263
+70 val_70
+487 val_487
+480 val_480
+401 val_401
+288 val_288
+191 val_191
+5 val_5
+244 val_244
+438 val_438
+128 val_128
+467 val_467
+432 val_432
+202 val_202
+316 val_316
+229 val_229
+469 val_469
+463 val_463
+280 val_280
+2 val_2
+35 val_35
+283 val_283
+331 val_331
+235 val_235
+80 val_80
+44 val_44
+193 val_193
+321 val_321
+335 val_335
+104 val_104
+466 val_466
+366 val_366
+175 val_175
+403 val_403
+483 val_483
+53 val_53
+105 val_105
+257 val_257
+406 val_406
+409 val_409
+190 val_190
+406 val_406
+401 val_401
+114 val_114
+258 val_258
+90 val_90
+203 val_203
+262 val_262
+348 val_348
+424 val_424
+12 val_12
+396 val_396
+201 val_201
+217 val_217
+164 val_164
+431 val_431
+454 val_454
+478 val_478
+298 val_298
+125 val_125
+431 val_431
+164 val_164
+424 val_424
+187 val_187
+382 val_382
+5 val_5
+70 val_70
+397 val_397
+480 val_480
+291 val_291
+24 val_24
+351 val_351
+255 val_255
+104 val_104
+70 val_70
+163 val_163
+438 val_438
+119 val_119
+414 val_414
+200 val_200
+491 val_491
+237 val_237
+439 val_439
+360 val_360
+248 val_248
+479 val_479
+305 val_305
+417 val_417
+199 val_199
+444 val_444
+120 val_120
+429 val_429
+169 val_169
+443 val_443
+323 val_323
+325 val_325
+277 val_277
+230 val_230
+478 val_478
+178 val_178
+468 val_468
+310 val_310
+317 val_317
+333 val_333
+493 val_493
+460 val_460
+207 val_207
+249 val_249
+265 val_265
+480 val_480
+83 val_83
+136 val_136
+353 val_353
+172 val_172
+214 val_214
+462 val_462
+233 val_233
+406 val_406
+133 val_133
+175 val_175
+189 val_189
+454 val_454
+375 val_375
+401 val_401
+421 val_421
+407 val_407
+384 val_384
+256 val_256
+26 val_26
+134 val_134
+67 val_67
+384 val_384
+379 val_379
+18 val_18
+462 val_462
+492 val_492
+100 val_100
+298 val_298
+9 val_9
+341 val_341
+498 val_498
+146 val_146
+458 val_458
+362 val_362
+186 val_186
+285 val_285
+348 val_348
+167 val_167
+18 val_18
+273 val_273
+183 val_183
+281 val_281
+344 val_344
+97 val_97
+469 val_469
+315 val_315
+84 val_84
+28 val_28
+37 val_37
+448 val_448
+152 val_152
+348 val_348
+307 val_307
+194 val_194
+414 val_414
+477 val_477
+222 val_222
+126 val_126
+90 val_90
+169 val_169
+403 val_403
+400 val_400
+200 val_200
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/view-18-cd6a123a71769b082000669292e57add b/sql/hive/src/test/resources/golden/view-18-cd6a123a71769b082000669292e57add
new file mode 100644
index 0000000000000..68782c7a277bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-18-cd6a123a71769b082000669292e57add
@@ -0,0 +1,1028 @@
+238 val_238 238
+238 val_238 238
+86 val_86 86
+311 val_311 311
+311 val_311 311
+311 val_311 311
+27 val_27 27
+165 val_165 165
+165 val_165 165
+409 val_409 409
+409 val_409 409
+409 val_409 409
+255 val_255 255
+255 val_255 255
+278 val_278 278
+278 val_278 278
+98 val_98 98
+98 val_98 98
+484 val_484 484
+265 val_265 265
+265 val_265 265
+193 val_193 193
+193 val_193 193
+193 val_193 193
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+150 val_150 150
+273 val_273 273
+273 val_273 273
+273 val_273 273
+224 val_224 224
+224 val_224 224
+369 val_369 369
+369 val_369 369
+369 val_369 369
+66 val_66 66
+128 val_128 128
+128 val_128 128
+128 val_128 128
+213 val_213 213
+213 val_213 213
+146 val_146 146
+146 val_146 146
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+429 val_429 429
+429 val_429 429
+374 val_374 374
+152 val_152 152
+152 val_152 152
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+145 val_145 145
+495 val_495 495
+37 val_37 37
+37 val_37 37
+327 val_327 327
+327 val_327 327
+327 val_327 327
+281 val_281 281
+281 val_281 281
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+209 val_209 209
+209 val_209 209
+15 val_15 15
+15 val_15 15
+82 val_82 82
+403 val_403 403
+403 val_403 403
+403 val_403 403
+166 val_166 166
+417 val_417 417
+417 val_417 417
+417 val_417 417
+430 val_430 430
+430 val_430 430
+430 val_430 430
+252 val_252 252
+292 val_292 292
+219 val_219 219
+219 val_219 219
+287 val_287 287
+153 val_153 153
+193 val_193 193
+193 val_193 193
+193 val_193 193
+338 val_338 338
+446 val_446 446
+459 val_459 459
+459 val_459 459
+394 val_394 394
+237 val_237 237
+237 val_237 237
+482 val_482 482
+174 val_174 174
+174 val_174 174
+413 val_413 413
+413 val_413 413
+494 val_494 494
+207 val_207 207
+207 val_207 207
+199 val_199 199
+199 val_199 199
+199 val_199 199
+466 val_466 466
+466 val_466 466
+466 val_466 466
+208 val_208 208
+208 val_208 208
+208 val_208 208
+174 val_174 174
+174 val_174 174
+399 val_399 399
+399 val_399 399
+396 val_396 396
+396 val_396 396
+396 val_396 396
+247 val_247 247
+417 val_417 417
+417 val_417 417
+417 val_417 417
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+162 val_162 162
+377 val_377 377
+397 val_397 397
+397 val_397 397
+309 val_309 309
+309 val_309 309
+365 val_365 365
+266 val_266 266
+439 val_439 439
+439 val_439 439
+342 val_342 342
+342 val_342 342
+367 val_367 367
+367 val_367 367
+325 val_325 325
+325 val_325 325
+167 val_167 167
+167 val_167 167
+167 val_167 167
+195 val_195 195
+195 val_195 195
+475 val_475 475
+17 val_17 17
+113 val_113 113
+113 val_113 113
+155 val_155 155
+203 val_203 203
+203 val_203 203
+339 val_339 339
+0 val_0 0
+0 val_0 0
+0 val_0 0
+455 val_455 455
+128 val_128 128
+128 val_128 128
+128 val_128 128
+311 val_311 311
+311 val_311 311
+311 val_311 311
+316 val_316 316
+316 val_316 316
+316 val_316 316
+57 val_57 57
+302 val_302 302
+205 val_205 205
+205 val_205 205
+149 val_149 149
+149 val_149 149
+438 val_438 438
+438 val_438 438
+438 val_438 438
+345 val_345 345
+129 val_129 129
+129 val_129 129
+170 val_170 170
+20 val_20 20
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+157 val_157 157
+378 val_378 378
+221 val_221 221
+221 val_221 221
+92 val_92 92
+111 val_111 111
+47 val_47 47
+72 val_72 72
+72 val_72 72
+4 val_4 4
+280 val_280 280
+280 val_280 280
+35 val_35 35
+35 val_35 35
+35 val_35 35
+427 val_427 427
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+208 val_208 208
+208 val_208 208
+208 val_208 208
+356 val_356 356
+399 val_399 399
+399 val_399 399
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+382 val_382 382
+382 val_382 382
+498 val_498 498
+498 val_498 498
+498 val_498 498
+125 val_125 125
+125 val_125 125
+386 val_386 386
+437 val_437 437
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+192 val_192 192
+286 val_286 286
+187 val_187 187
+187 val_187 187
+187 val_187 187
+176 val_176 176
+176 val_176 176
+54 val_54 54
+459 val_459 459
+459 val_459 459
+51 val_51 51
+51 val_51 51
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+103 val_103 103
+103 val_103 103
+239 val_239 239
+239 val_239 239
+213 val_213 213
+213 val_213 213
+216 val_216 216
+216 val_216 216
+430 val_430 430
+430 val_430 430
+430 val_430 430
+278 val_278 278
+278 val_278 278
+176 val_176 176
+176 val_176 176
+289 val_289 289
+221 val_221 221
+221 val_221 221
+65 val_65 65
+318 val_318 318
+318 val_318 318
+318 val_318 318
+332 val_332 332
+311 val_311 311
+311 val_311 311
+311 val_311 311
+275 val_275 275
+137 val_137 137
+137 val_137 137
+241 val_241 241
+83 val_83 83
+83 val_83 83
+333 val_333 333
+333 val_333 333
+180 val_180 180
+284 val_284 284
+12 val_12 12
+12 val_12 12
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+181 val_181 181
+67 val_67 67
+67 val_67 67
+260 val_260 260
+404 val_404 404
+404 val_404 404
+384 val_384 384
+384 val_384 384
+384 val_384 384
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+353 val_353 353
+353 val_353 353
+373 val_373 373
+272 val_272 272
+272 val_272 272
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+217 val_217 217
+217 val_217 217
+84 val_84 84
+84 val_84 84
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+466 val_466 466
+466 val_466 466
+466 val_466 466
+58 val_58 58
+58 val_58 58
+8 val_8 8
+411 val_411 411
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+208 val_208 208
+208 val_208 208
+208 val_208 208
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+24 val_24 24
+24 val_24 24
+463 val_463 463
+463 val_463 463
+431 val_431 431
+431 val_431 431
+431 val_431 431
+179 val_179 179
+179 val_179 179
+172 val_172 172
+172 val_172 172
+42 val_42 42
+42 val_42 42
+129 val_129 129
+129 val_129 129
+158 val_158 158
+119 val_119 119
+119 val_119 119
+119 val_119 119
+496 val_496 496
+0 val_0 0
+0 val_0 0
+0 val_0 0
+322 val_322 322
+322 val_322 322
+197 val_197 197
+197 val_197 197
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+393 val_393 393
+454 val_454 454
+454 val_454 454
+454 val_454 454
+100 val_100 100
+100 val_100 100
+298 val_298 298
+298 val_298 298
+298 val_298 298
+199 val_199 199
+199 val_199 199
+199 val_199 199
+191 val_191 191
+191 val_191 191
+418 val_418 418
+96 val_96 96
+26 val_26 26
+26 val_26 26
+165 val_165 165
+165 val_165 165
+327 val_327 327
+327 val_327 327
+327 val_327 327
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+205 val_205 205
+205 val_205 205
+120 val_120 120
+120 val_120 120
+131 val_131 131
+51 val_51 51
+51 val_51 51
+404 val_404 404
+404 val_404 404
+43 val_43 43
+436 val_436 436
+156 val_156 156
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+308 val_308 308
+95 val_95 95
+95 val_95 95
+196 val_196 196
+288 val_288 288
+288 val_288 288
+481 val_481 481
+457 val_457 457
+98 val_98 98
+98 val_98 98
+282 val_282 282
+282 val_282 282
+197 val_197 197
+197 val_197 197
+187 val_187 187
+187 val_187 187
+187 val_187 187
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+409 val_409 409
+409 val_409 409
+409 val_409 409
+470 val_470 470
+137 val_137 137
+137 val_137 137
+369 val_369 369
+369 val_369 369
+369 val_369 369
+316 val_316 316
+316 val_316 316
+316 val_316 316
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+413 val_413 413
+413 val_413 413
+85 val_85 85
+77 val_77 77
+0 val_0 0
+0 val_0 0
+0 val_0 0
+490 val_490 490
+87 val_87 87
+364 val_364 364
+179 val_179 179
+179 val_179 179
+118 val_118 118
+118 val_118 118
+134 val_134 134
+134 val_134 134
+395 val_395 395
+395 val_395 395
+282 val_282 282
+282 val_282 282
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+238 val_238 238
+238 val_238 238
+419 val_419 419
+15 val_15 15
+15 val_15 15
+118 val_118 118
+118 val_118 118
+72 val_72 72
+72 val_72 72
+90 val_90 90
+90 val_90 90
+90 val_90 90
+307 val_307 307
+307 val_307 307
+19 val_19 19
+435 val_435 435
+10 val_10 10
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+273 val_273 273
+273 val_273 273
+273 val_273 273
+306 val_306 306
+224 val_224 224
+224 val_224 224
+309 val_309 309
+309 val_309 309
+389 val_389 389
+327 val_327 327
+327 val_327 327
+327 val_327 327
+242 val_242 242
+242 val_242 242
+369 val_369 369
+369 val_369 369
+369 val_369 369
+392 val_392 392
+272 val_272 272
+272 val_272 272
+331 val_331 331
+331 val_331 331
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+242 val_242 242
+242 val_242 242
+452 val_452 452
+177 val_177 177
+226 val_226 226
+5 val_5 5
+5 val_5 5
+5 val_5 5
+497 val_497 497
+402 val_402 402
+396 val_396 396
+396 val_396 396
+396 val_396 396
+317 val_317 317
+317 val_317 317
+395 val_395 395
+395 val_395 395
+58 val_58 58
+58 val_58 58
+35 val_35 35
+35 val_35 35
+35 val_35 35
+336 val_336 336
+95 val_95 95
+95 val_95 95
+11 val_11 11
+168 val_168 168
+34 val_34 34
+229 val_229 229
+229 val_229 229
+233 val_233 233
+233 val_233 233
+143 val_143 143
+472 val_472 472
+322 val_322 322
+322 val_322 322
+498 val_498 498
+498 val_498 498
+498 val_498 498
+160 val_160 160
+195 val_195 195
+195 val_195 195
+42 val_42 42
+42 val_42 42
+321 val_321 321
+321 val_321 321
+430 val_430 430
+430 val_430 430
+430 val_430 430
+119 val_119 119
+119 val_119 119
+119 val_119 119
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+458 val_458 458
+458 val_458 458
+78 val_78 78
+76 val_76 76
+76 val_76 76
+41 val_41 41
+223 val_223 223
+223 val_223 223
+492 val_492 492
+492 val_492 492
+149 val_149 149
+149 val_149 149
+449 val_449 449
+218 val_218 218
+228 val_228 228
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+453 val_453 453
+30 val_30 30
+209 val_209 209
+209 val_209 209
+64 val_64 64
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+76 val_76 76
+76 val_76 76
+74 val_74 74
+342 val_342 342
+342 val_342 342
+69 val_69 69
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+33 val_33 33
+368 val_368 368
+103 val_103 103
+103 val_103 103
+296 val_296 296
+113 val_113 113
+113 val_113 113
+216 val_216 216
+216 val_216 216
+367 val_367 367
+367 val_367 367
+344 val_344 344
+344 val_344 344
+167 val_167 167
+167 val_167 167
+167 val_167 167
+274 val_274 274
+219 val_219 219
+219 val_219 219
+239 val_239 239
+239 val_239 239
+485 val_485 485
+116 val_116 116
+223 val_223 223
+223 val_223 223
+256 val_256 256
+256 val_256 256
+263 val_263 263
+70 val_70 70
+70 val_70 70
+70 val_70 70
+487 val_487 487
+480 val_480 480
+480 val_480 480
+480 val_480 480
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+288 val_288 288
+288 val_288 288
+191 val_191 191
+191 val_191 191
+5 val_5 5
+5 val_5 5
+5 val_5 5
+244 val_244 244
+438 val_438 438
+438 val_438 438
+438 val_438 438
+128 val_128 128
+128 val_128 128
+128 val_128 128
+467 val_467 467
+432 val_432 432
+202 val_202 202
+316 val_316 316
+316 val_316 316
+316 val_316 316
+229 val_229 229
+229 val_229 229
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+463 val_463 463
+463 val_463 463
+280 val_280 280
+280 val_280 280
+2 val_2 2
+35 val_35 35
+35 val_35 35
+35 val_35 35
+283 val_283 283
+331 val_331 331
+331 val_331 331
+235 val_235 235
+80 val_80 80
+44 val_44 44
+193 val_193 193
+193 val_193 193
+193 val_193 193
+321 val_321 321
+321 val_321 321
+335 val_335 335
+104 val_104 104
+104 val_104 104
+466 val_466 466
+466 val_466 466
+466 val_466 466
+366 val_366 366
+175 val_175 175
+175 val_175 175
+403 val_403 403
+403 val_403 403
+403 val_403 403
+483 val_483 483
+53 val_53 53
+105 val_105 105
+257 val_257 257
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+409 val_409 409
+409 val_409 409
+409 val_409 409
+190 val_190 190
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+114 val_114 114
+258 val_258 258
+90 val_90 90
+90 val_90 90
+90 val_90 90
+203 val_203 203
+203 val_203 203
+262 val_262 262
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+424 val_424 424
+424 val_424 424
+12 val_12 12
+12 val_12 12
+396 val_396 396
+396 val_396 396
+396 val_396 396
+201 val_201 201
+217 val_217 217
+217 val_217 217
+164 val_164 164
+164 val_164 164
+431 val_431 431
+431 val_431 431
+431 val_431 431
+454 val_454 454
+454 val_454 454
+454 val_454 454
+478 val_478 478
+478 val_478 478
+298 val_298 298
+298 val_298 298
+298 val_298 298
+125 val_125 125
+125 val_125 125
+431 val_431 431
+431 val_431 431
+431 val_431 431
+164 val_164 164
+164 val_164 164
+424 val_424 424
+424 val_424 424
+187 val_187 187
+187 val_187 187
+187 val_187 187
+382 val_382 382
+382 val_382 382
+5 val_5 5
+5 val_5 5
+5 val_5 5
+70 val_70 70
+70 val_70 70
+70 val_70 70
+397 val_397 397
+397 val_397 397
+480 val_480 480
+480 val_480 480
+480 val_480 480
+291 val_291 291
+24 val_24 24
+24 val_24 24
+351 val_351 351
+255 val_255 255
+255 val_255 255
+104 val_104 104
+104 val_104 104
+70 val_70 70
+70 val_70 70
+70 val_70 70
+163 val_163 163
+438 val_438 438
+438 val_438 438
+438 val_438 438
+119 val_119 119
+119 val_119 119
+119 val_119 119
+414 val_414 414
+414 val_414 414
+200 val_200 200
+200 val_200 200
+491 val_491 491
+237 val_237 237
+237 val_237 237
+439 val_439 439
+439 val_439 439
+360 val_360 360
+248 val_248 248
+479 val_479 479
+305 val_305 305
+417 val_417 417
+417 val_417 417
+417 val_417 417
+199 val_199 199
+199 val_199 199
+199 val_199 199
+444 val_444 444
+120 val_120 120
+120 val_120 120
+429 val_429 429
+429 val_429 429
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+443 val_443 443
+323 val_323 323
+325 val_325 325
+325 val_325 325
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+478 val_478 478
+478 val_478 478
+178 val_178 178
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+310 val_310 310
+317 val_317 317
+317 val_317 317
+333 val_333 333
+333 val_333 333
+493 val_493 493
+460 val_460 460
+207 val_207 207
+207 val_207 207
+249 val_249 249
+265 val_265 265
+265 val_265 265
+480 val_480 480
+480 val_480 480
+480 val_480 480
+83 val_83 83
+83 val_83 83
+136 val_136 136
+353 val_353 353
+353 val_353 353
+172 val_172 172
+172 val_172 172
+214 val_214 214
+462 val_462 462
+462 val_462 462
+233 val_233 233
+233 val_233 233
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+133 val_133 133
+175 val_175 175
+175 val_175 175
+189 val_189 189
+454 val_454 454
+454 val_454 454
+454 val_454 454
+375 val_375 375
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+421 val_421 421
+407 val_407 407
+384 val_384 384
+384 val_384 384
+384 val_384 384
+256 val_256 256
+256 val_256 256
+26 val_26 26
+26 val_26 26
+134 val_134 134
+134 val_134 134
+67 val_67 67
+67 val_67 67
+384 val_384 384
+384 val_384 384
+384 val_384 384
+379 val_379 379
+18 val_18 18
+18 val_18 18
+462 val_462 462
+462 val_462 462
+492 val_492 492
+492 val_492 492
+100 val_100 100
+100 val_100 100
+298 val_298 298
+298 val_298 298
+298 val_298 298
+9 val_9 9
+341 val_341 341
+498 val_498 498
+498 val_498 498
+498 val_498 498
+146 val_146 146
+146 val_146 146
+458 val_458 458
+458 val_458 458
+362 val_362 362
+186 val_186 186
+285 val_285 285
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+167 val_167 167
+167 val_167 167
+167 val_167 167
+18 val_18 18
+18 val_18 18
+273 val_273 273
+273 val_273 273
+273 val_273 273
+183 val_183 183
+281 val_281 281
+281 val_281 281
+344 val_344 344
+344 val_344 344
+97 val_97 97
+97 val_97 97
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+315 val_315 315
+84 val_84 84
+84 val_84 84
+28 val_28 28
+37 val_37 37
+37 val_37 37
+448 val_448 448
+152 val_152 152
+152 val_152 152
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+307 val_307 307
+307 val_307 307
+194 val_194 194
+414 val_414 414
+414 val_414 414
+477 val_477 477
+222 val_222 222
+126 val_126 126
+90 val_90 90
+90 val_90 90
+90 val_90 90
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+403 val_403 403
+403 val_403 403
+403 val_403 403
+400 val_400 400
+200 val_200 200
+200 val_200 200
+97 val_97 97
+97 val_97 97
diff --git a/sql/hive/src/test/resources/golden/view-19-4a8f62f10a8a7b1f6e0d1c15481590a6 b/sql/hive/src/test/resources/golden/view-19-4a8f62f10a8a7b1f6e0d1c15481590a6
new file mode 100644
index 0000000000000..7aae61e5eb82f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-19-4a8f62f10a8a7b1f6e0d1c15481590a6
@@ -0,0 +1,500 @@
+238 val_238
+86 val_86
+311 val_311
+27 val_27
+165 val_165
+409 val_409
+255 val_255
+278 val_278
+98 val_98
+484 val_484
+265 val_265
+193 val_193
+401 val_401
+150 val_150
+273 val_273
+224 val_224
+369 val_369
+66 val_66
+128 val_128
+213 val_213
+146 val_146
+406 val_406
+429 val_429
+374 val_374
+152 val_152
+469 val_469
+145 val_145
+495 val_495
+37 val_37
+327 val_327
+281 val_281
+277 val_277
+209 val_209
+15 val_15
+82 val_82
+403 val_403
+166 val_166
+417 val_417
+430 val_430
+252 val_252
+292 val_292
+219 val_219
+287 val_287
+153 val_153
+193 val_193
+338 val_338
+446 val_446
+459 val_459
+394 val_394
+237 val_237
+482 val_482
+174 val_174
+413 val_413
+494 val_494
+207 val_207
+199 val_199
+466 val_466
+208 val_208
+174 val_174
+399 val_399
+396 val_396
+247 val_247
+417 val_417
+489 val_489
+162 val_162
+377 val_377
+397 val_397
+309 val_309
+365 val_365
+266 val_266
+439 val_439
+342 val_342
+367 val_367
+325 val_325
+167 val_167
+195 val_195
+475 val_475
+17 val_17
+113 val_113
+155 val_155
+203 val_203
+339 val_339
+0 val_0
+455 val_455
+128 val_128
+311 val_311
+316 val_316
+57 val_57
+302 val_302
+205 val_205
+149 val_149
+438 val_438
+345 val_345
+129 val_129
+170 val_170
+20 val_20
+489 val_489
+157 val_157
+378 val_378
+221 val_221
+92 val_92
+111 val_111
+47 val_47
+72 val_72
+4 val_4
+280 val_280
+35 val_35
+427 val_427
+277 val_277
+208 val_208
+356 val_356
+399 val_399
+169 val_169
+382 val_382
+498 val_498
+125 val_125
+386 val_386
+437 val_437
+469 val_469
+192 val_192
+286 val_286
+187 val_187
+176 val_176
+54 val_54
+459 val_459
+51 val_51
+138 val_138
+103 val_103
+239 val_239
+213 val_213
+216 val_216
+430 val_430
+278 val_278
+176 val_176
+289 val_289
+221 val_221
+65 val_65
+318 val_318
+332 val_332
+311 val_311
+275 val_275
+137 val_137
+241 val_241
+83 val_83
+333 val_333
+180 val_180
+284 val_284
+12 val_12
+230 val_230
+181 val_181
+67 val_67
+260 val_260
+404 val_404
+384 val_384
+489 val_489
+353 val_353
+373 val_373
+272 val_272
+138 val_138
+217 val_217
+84 val_84
+348 val_348
+466 val_466
+58 val_58
+8 val_8
+411 val_411
+230 val_230
+208 val_208
+348 val_348
+24 val_24
+463 val_463
+431 val_431
+179 val_179
+172 val_172
+42 val_42
+129 val_129
+158 val_158
+119 val_119
+496 val_496
+0 val_0
+322 val_322
+197 val_197
+468 val_468
+393 val_393
+454 val_454
+100 val_100
+298 val_298
+199 val_199
+191 val_191
+418 val_418
+96 val_96
+26 val_26
+165 val_165
+327 val_327
+230 val_230
+205 val_205
+120 val_120
+131 val_131
+51 val_51
+404 val_404
+43 val_43
+436 val_436
+156 val_156
+469 val_469
+468 val_468
+308 val_308
+95 val_95
+196 val_196
+288 val_288
+481 val_481
+457 val_457
+98 val_98
+282 val_282
+197 val_197
+187 val_187
+318 val_318
+318 val_318
+409 val_409
+470 val_470
+137 val_137
+369 val_369
+316 val_316
+169 val_169
+413 val_413
+85 val_85
+77 val_77
+0 val_0
+490 val_490
+87 val_87
+364 val_364
+179 val_179
+118 val_118
+134 val_134
+395 val_395
+282 val_282
+138 val_138
+238 val_238
+419 val_419
+15 val_15
+118 val_118
+72 val_72
+90 val_90
+307 val_307
+19 val_19
+435 val_435
+10 val_10
+277 val_277
+273 val_273
+306 val_306
+224 val_224
+309 val_309
+389 val_389
+327 val_327
+242 val_242
+369 val_369
+392 val_392
+272 val_272
+331 val_331
+401 val_401
+242 val_242
+452 val_452
+177 val_177
+226 val_226
+5 val_5
+497 val_497
+402 val_402
+396 val_396
+317 val_317
+395 val_395
+58 val_58
+35 val_35
+336 val_336
+95 val_95
+11 val_11
+168 val_168
+34 val_34
+229 val_229
+233 val_233
+143 val_143
+472 val_472
+322 val_322
+498 val_498
+160 val_160
+195 val_195
+42 val_42
+321 val_321
+430 val_430
+119 val_119
+489 val_489
+458 val_458
+78 val_78
+76 val_76
+41 val_41
+223 val_223
+492 val_492
+149 val_149
+449 val_449
+218 val_218
+228 val_228
+138 val_138
+453 val_453
+30 val_30
+209 val_209
+64 val_64
+468 val_468
+76 val_76
+74 val_74
+342 val_342
+69 val_69
+230 val_230
+33 val_33
+368 val_368
+103 val_103
+296 val_296
+113 val_113
+216 val_216
+367 val_367
+344 val_344
+167 val_167
+274 val_274
+219 val_219
+239 val_239
+485 val_485
+116 val_116
+223 val_223
+256 val_256
+263 val_263
+70 val_70
+487 val_487
+480 val_480
+401 val_401
+288 val_288
+191 val_191
+5 val_5
+244 val_244
+438 val_438
+128 val_128
+467 val_467
+432 val_432
+202 val_202
+316 val_316
+229 val_229
+469 val_469
+463 val_463
+280 val_280
+2 val_2
+35 val_35
+283 val_283
+331 val_331
+235 val_235
+80 val_80
+44 val_44
+193 val_193
+321 val_321
+335 val_335
+104 val_104
+466 val_466
+366 val_366
+175 val_175
+403 val_403
+483 val_483
+53 val_53
+105 val_105
+257 val_257
+406 val_406
+409 val_409
+190 val_190
+406 val_406
+401 val_401
+114 val_114
+258 val_258
+90 val_90
+203 val_203
+262 val_262
+348 val_348
+424 val_424
+12 val_12
+396 val_396
+201 val_201
+217 val_217
+164 val_164
+431 val_431
+454 val_454
+478 val_478
+298 val_298
+125 val_125
+431 val_431
+164 val_164
+424 val_424
+187 val_187
+382 val_382
+5 val_5
+70 val_70
+397 val_397
+480 val_480
+291 val_291
+24 val_24
+351 val_351
+255 val_255
+104 val_104
+70 val_70
+163 val_163
+438 val_438
+119 val_119
+414 val_414
+200 val_200
+491 val_491
+237 val_237
+439 val_439
+360 val_360
+248 val_248
+479 val_479
+305 val_305
+417 val_417
+199 val_199
+444 val_444
+120 val_120
+429 val_429
+169 val_169
+443 val_443
+323 val_323
+325 val_325
+277 val_277
+230 val_230
+478 val_478
+178 val_178
+468 val_468
+310 val_310
+317 val_317
+333 val_333
+493 val_493
+460 val_460
+207 val_207
+249 val_249
+265 val_265
+480 val_480
+83 val_83
+136 val_136
+353 val_353
+172 val_172
+214 val_214
+462 val_462
+233 val_233
+406 val_406
+133 val_133
+175 val_175
+189 val_189
+454 val_454
+375 val_375
+401 val_401
+421 val_421
+407 val_407
+384 val_384
+256 val_256
+26 val_26
+134 val_134
+67 val_67
+384 val_384
+379 val_379
+18 val_18
+462 val_462
+492 val_492
+100 val_100
+298 val_298
+9 val_9
+341 val_341
+498 val_498
+146 val_146
+458 val_458
+362 val_362
+186 val_186
+285 val_285
+348 val_348
+167 val_167
+18 val_18
+273 val_273
+183 val_183
+281 val_281
+344 val_344
+97 val_97
+469 val_469
+315 val_315
+84 val_84
+28 val_28
+37 val_37
+448 val_448
+152 val_152
+348 val_348
+307 val_307
+194 val_194
+414 val_414
+477 val_477
+222 val_222
+126 val_126
+90 val_90
+169 val_169
+403 val_403
+400 val_400
+200 val_200
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-20-92aa822cb9dc29a1d3ad37d3ebaa344a b/sql/hive/src/test/resources/golden/view-20-92aa822cb9dc29a1d3ad37d3ebaa344a
new file mode 100644
index 0000000000000..7aae61e5eb82f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-20-92aa822cb9dc29a1d3ad37d3ebaa344a
@@ -0,0 +1,500 @@
+238 val_238
+86 val_86
+311 val_311
+27 val_27
+165 val_165
+409 val_409
+255 val_255
+278 val_278
+98 val_98
+484 val_484
+265 val_265
+193 val_193
+401 val_401
+150 val_150
+273 val_273
+224 val_224
+369 val_369
+66 val_66
+128 val_128
+213 val_213
+146 val_146
+406 val_406
+429 val_429
+374 val_374
+152 val_152
+469 val_469
+145 val_145
+495 val_495
+37 val_37
+327 val_327
+281 val_281
+277 val_277
+209 val_209
+15 val_15
+82 val_82
+403 val_403
+166 val_166
+417 val_417
+430 val_430
+252 val_252
+292 val_292
+219 val_219
+287 val_287
+153 val_153
+193 val_193
+338 val_338
+446 val_446
+459 val_459
+394 val_394
+237 val_237
+482 val_482
+174 val_174
+413 val_413
+494 val_494
+207 val_207
+199 val_199
+466 val_466
+208 val_208
+174 val_174
+399 val_399
+396 val_396
+247 val_247
+417 val_417
+489 val_489
+162 val_162
+377 val_377
+397 val_397
+309 val_309
+365 val_365
+266 val_266
+439 val_439
+342 val_342
+367 val_367
+325 val_325
+167 val_167
+195 val_195
+475 val_475
+17 val_17
+113 val_113
+155 val_155
+203 val_203
+339 val_339
+0 val_0
+455 val_455
+128 val_128
+311 val_311
+316 val_316
+57 val_57
+302 val_302
+205 val_205
+149 val_149
+438 val_438
+345 val_345
+129 val_129
+170 val_170
+20 val_20
+489 val_489
+157 val_157
+378 val_378
+221 val_221
+92 val_92
+111 val_111
+47 val_47
+72 val_72
+4 val_4
+280 val_280
+35 val_35
+427 val_427
+277 val_277
+208 val_208
+356 val_356
+399 val_399
+169 val_169
+382 val_382
+498 val_498
+125 val_125
+386 val_386
+437 val_437
+469 val_469
+192 val_192
+286 val_286
+187 val_187
+176 val_176
+54 val_54
+459 val_459
+51 val_51
+138 val_138
+103 val_103
+239 val_239
+213 val_213
+216 val_216
+430 val_430
+278 val_278
+176 val_176
+289 val_289
+221 val_221
+65 val_65
+318 val_318
+332 val_332
+311 val_311
+275 val_275
+137 val_137
+241 val_241
+83 val_83
+333 val_333
+180 val_180
+284 val_284
+12 val_12
+230 val_230
+181 val_181
+67 val_67
+260 val_260
+404 val_404
+384 val_384
+489 val_489
+353 val_353
+373 val_373
+272 val_272
+138 val_138
+217 val_217
+84 val_84
+348 val_348
+466 val_466
+58 val_58
+8 val_8
+411 val_411
+230 val_230
+208 val_208
+348 val_348
+24 val_24
+463 val_463
+431 val_431
+179 val_179
+172 val_172
+42 val_42
+129 val_129
+158 val_158
+119 val_119
+496 val_496
+0 val_0
+322 val_322
+197 val_197
+468 val_468
+393 val_393
+454 val_454
+100 val_100
+298 val_298
+199 val_199
+191 val_191
+418 val_418
+96 val_96
+26 val_26
+165 val_165
+327 val_327
+230 val_230
+205 val_205
+120 val_120
+131 val_131
+51 val_51
+404 val_404
+43 val_43
+436 val_436
+156 val_156
+469 val_469
+468 val_468
+308 val_308
+95 val_95
+196 val_196
+288 val_288
+481 val_481
+457 val_457
+98 val_98
+282 val_282
+197 val_197
+187 val_187
+318 val_318
+318 val_318
+409 val_409
+470 val_470
+137 val_137
+369 val_369
+316 val_316
+169 val_169
+413 val_413
+85 val_85
+77 val_77
+0 val_0
+490 val_490
+87 val_87
+364 val_364
+179 val_179
+118 val_118
+134 val_134
+395 val_395
+282 val_282
+138 val_138
+238 val_238
+419 val_419
+15 val_15
+118 val_118
+72 val_72
+90 val_90
+307 val_307
+19 val_19
+435 val_435
+10 val_10
+277 val_277
+273 val_273
+306 val_306
+224 val_224
+309 val_309
+389 val_389
+327 val_327
+242 val_242
+369 val_369
+392 val_392
+272 val_272
+331 val_331
+401 val_401
+242 val_242
+452 val_452
+177 val_177
+226 val_226
+5 val_5
+497 val_497
+402 val_402
+396 val_396
+317 val_317
+395 val_395
+58 val_58
+35 val_35
+336 val_336
+95 val_95
+11 val_11
+168 val_168
+34 val_34
+229 val_229
+233 val_233
+143 val_143
+472 val_472
+322 val_322
+498 val_498
+160 val_160
+195 val_195
+42 val_42
+321 val_321
+430 val_430
+119 val_119
+489 val_489
+458 val_458
+78 val_78
+76 val_76
+41 val_41
+223 val_223
+492 val_492
+149 val_149
+449 val_449
+218 val_218
+228 val_228
+138 val_138
+453 val_453
+30 val_30
+209 val_209
+64 val_64
+468 val_468
+76 val_76
+74 val_74
+342 val_342
+69 val_69
+230 val_230
+33 val_33
+368 val_368
+103 val_103
+296 val_296
+113 val_113
+216 val_216
+367 val_367
+344 val_344
+167 val_167
+274 val_274
+219 val_219
+239 val_239
+485 val_485
+116 val_116
+223 val_223
+256 val_256
+263 val_263
+70 val_70
+487 val_487
+480 val_480
+401 val_401
+288 val_288
+191 val_191
+5 val_5
+244 val_244
+438 val_438
+128 val_128
+467 val_467
+432 val_432
+202 val_202
+316 val_316
+229 val_229
+469 val_469
+463 val_463
+280 val_280
+2 val_2
+35 val_35
+283 val_283
+331 val_331
+235 val_235
+80 val_80
+44 val_44
+193 val_193
+321 val_321
+335 val_335
+104 val_104
+466 val_466
+366 val_366
+175 val_175
+403 val_403
+483 val_483
+53 val_53
+105 val_105
+257 val_257
+406 val_406
+409 val_409
+190 val_190
+406 val_406
+401 val_401
+114 val_114
+258 val_258
+90 val_90
+203 val_203
+262 val_262
+348 val_348
+424 val_424
+12 val_12
+396 val_396
+201 val_201
+217 val_217
+164 val_164
+431 val_431
+454 val_454
+478 val_478
+298 val_298
+125 val_125
+431 val_431
+164 val_164
+424 val_424
+187 val_187
+382 val_382
+5 val_5
+70 val_70
+397 val_397
+480 val_480
+291 val_291
+24 val_24
+351 val_351
+255 val_255
+104 val_104
+70 val_70
+163 val_163
+438 val_438
+119 val_119
+414 val_414
+200 val_200
+491 val_491
+237 val_237
+439 val_439
+360 val_360
+248 val_248
+479 val_479
+305 val_305
+417 val_417
+199 val_199
+444 val_444
+120 val_120
+429 val_429
+169 val_169
+443 val_443
+323 val_323
+325 val_325
+277 val_277
+230 val_230
+478 val_478
+178 val_178
+468 val_468
+310 val_310
+317 val_317
+333 val_333
+493 val_493
+460 val_460
+207 val_207
+249 val_249
+265 val_265
+480 val_480
+83 val_83
+136 val_136
+353 val_353
+172 val_172
+214 val_214
+462 val_462
+233 val_233
+406 val_406
+133 val_133
+175 val_175
+189 val_189
+454 val_454
+375 val_375
+401 val_401
+421 val_421
+407 val_407
+384 val_384
+256 val_256
+26 val_26
+134 val_134
+67 val_67
+384 val_384
+379 val_379
+18 val_18
+462 val_462
+492 val_492
+100 val_100
+298 val_298
+9 val_9
+341 val_341
+498 val_498
+146 val_146
+458 val_458
+362 val_362
+186 val_186
+285 val_285
+348 val_348
+167 val_167
+18 val_18
+273 val_273
+183 val_183
+281 val_281
+344 val_344
+97 val_97
+469 val_469
+315 val_315
+84 val_84
+28 val_28
+37 val_37
+448 val_448
+152 val_152
+348 val_348
+307 val_307
+194 val_194
+414 val_414
+477 val_477
+222 val_222
+126 val_126
+90 val_90
+169 val_169
+403 val_403
+400 val_400
+200 val_200
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/view-21-4fa118ed540dfe42748bbed1e7fb513d b/sql/hive/src/test/resources/golden/view-21-4fa118ed540dfe42748bbed1e7fb513d
new file mode 100644
index 0000000000000..68782c7a277bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-21-4fa118ed540dfe42748bbed1e7fb513d
@@ -0,0 +1,1028 @@
+238 val_238 238
+238 val_238 238
+86 val_86 86
+311 val_311 311
+311 val_311 311
+311 val_311 311
+27 val_27 27
+165 val_165 165
+165 val_165 165
+409 val_409 409
+409 val_409 409
+409 val_409 409
+255 val_255 255
+255 val_255 255
+278 val_278 278
+278 val_278 278
+98 val_98 98
+98 val_98 98
+484 val_484 484
+265 val_265 265
+265 val_265 265
+193 val_193 193
+193 val_193 193
+193 val_193 193
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+150 val_150 150
+273 val_273 273
+273 val_273 273
+273 val_273 273
+224 val_224 224
+224 val_224 224
+369 val_369 369
+369 val_369 369
+369 val_369 369
+66 val_66 66
+128 val_128 128
+128 val_128 128
+128 val_128 128
+213 val_213 213
+213 val_213 213
+146 val_146 146
+146 val_146 146
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+429 val_429 429
+429 val_429 429
+374 val_374 374
+152 val_152 152
+152 val_152 152
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+145 val_145 145
+495 val_495 495
+37 val_37 37
+37 val_37 37
+327 val_327 327
+327 val_327 327
+327 val_327 327
+281 val_281 281
+281 val_281 281
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+209 val_209 209
+209 val_209 209
+15 val_15 15
+15 val_15 15
+82 val_82 82
+403 val_403 403
+403 val_403 403
+403 val_403 403
+166 val_166 166
+417 val_417 417
+417 val_417 417
+417 val_417 417
+430 val_430 430
+430 val_430 430
+430 val_430 430
+252 val_252 252
+292 val_292 292
+219 val_219 219
+219 val_219 219
+287 val_287 287
+153 val_153 153
+193 val_193 193
+193 val_193 193
+193 val_193 193
+338 val_338 338
+446 val_446 446
+459 val_459 459
+459 val_459 459
+394 val_394 394
+237 val_237 237
+237 val_237 237
+482 val_482 482
+174 val_174 174
+174 val_174 174
+413 val_413 413
+413 val_413 413
+494 val_494 494
+207 val_207 207
+207 val_207 207
+199 val_199 199
+199 val_199 199
+199 val_199 199
+466 val_466 466
+466 val_466 466
+466 val_466 466
+208 val_208 208
+208 val_208 208
+208 val_208 208
+174 val_174 174
+174 val_174 174
+399 val_399 399
+399 val_399 399
+396 val_396 396
+396 val_396 396
+396 val_396 396
+247 val_247 247
+417 val_417 417
+417 val_417 417
+417 val_417 417
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+162 val_162 162
+377 val_377 377
+397 val_397 397
+397 val_397 397
+309 val_309 309
+309 val_309 309
+365 val_365 365
+266 val_266 266
+439 val_439 439
+439 val_439 439
+342 val_342 342
+342 val_342 342
+367 val_367 367
+367 val_367 367
+325 val_325 325
+325 val_325 325
+167 val_167 167
+167 val_167 167
+167 val_167 167
+195 val_195 195
+195 val_195 195
+475 val_475 475
+17 val_17 17
+113 val_113 113
+113 val_113 113
+155 val_155 155
+203 val_203 203
+203 val_203 203
+339 val_339 339
+0 val_0 0
+0 val_0 0
+0 val_0 0
+455 val_455 455
+128 val_128 128
+128 val_128 128
+128 val_128 128
+311 val_311 311
+311 val_311 311
+311 val_311 311
+316 val_316 316
+316 val_316 316
+316 val_316 316
+57 val_57 57
+302 val_302 302
+205 val_205 205
+205 val_205 205
+149 val_149 149
+149 val_149 149
+438 val_438 438
+438 val_438 438
+438 val_438 438
+345 val_345 345
+129 val_129 129
+129 val_129 129
+170 val_170 170
+20 val_20 20
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+157 val_157 157
+378 val_378 378
+221 val_221 221
+221 val_221 221
+92 val_92 92
+111 val_111 111
+47 val_47 47
+72 val_72 72
+72 val_72 72
+4 val_4 4
+280 val_280 280
+280 val_280 280
+35 val_35 35
+35 val_35 35
+35 val_35 35
+427 val_427 427
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+208 val_208 208
+208 val_208 208
+208 val_208 208
+356 val_356 356
+399 val_399 399
+399 val_399 399
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+382 val_382 382
+382 val_382 382
+498 val_498 498
+498 val_498 498
+498 val_498 498
+125 val_125 125
+125 val_125 125
+386 val_386 386
+437 val_437 437
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+192 val_192 192
+286 val_286 286
+187 val_187 187
+187 val_187 187
+187 val_187 187
+176 val_176 176
+176 val_176 176
+54 val_54 54
+459 val_459 459
+459 val_459 459
+51 val_51 51
+51 val_51 51
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+103 val_103 103
+103 val_103 103
+239 val_239 239
+239 val_239 239
+213 val_213 213
+213 val_213 213
+216 val_216 216
+216 val_216 216
+430 val_430 430
+430 val_430 430
+430 val_430 430
+278 val_278 278
+278 val_278 278
+176 val_176 176
+176 val_176 176
+289 val_289 289
+221 val_221 221
+221 val_221 221
+65 val_65 65
+318 val_318 318
+318 val_318 318
+318 val_318 318
+332 val_332 332
+311 val_311 311
+311 val_311 311
+311 val_311 311
+275 val_275 275
+137 val_137 137
+137 val_137 137
+241 val_241 241
+83 val_83 83
+83 val_83 83
+333 val_333 333
+333 val_333 333
+180 val_180 180
+284 val_284 284
+12 val_12 12
+12 val_12 12
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+181 val_181 181
+67 val_67 67
+67 val_67 67
+260 val_260 260
+404 val_404 404
+404 val_404 404
+384 val_384 384
+384 val_384 384
+384 val_384 384
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+353 val_353 353
+353 val_353 353
+373 val_373 373
+272 val_272 272
+272 val_272 272
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+217 val_217 217
+217 val_217 217
+84 val_84 84
+84 val_84 84
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+466 val_466 466
+466 val_466 466
+466 val_466 466
+58 val_58 58
+58 val_58 58
+8 val_8 8
+411 val_411 411
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+208 val_208 208
+208 val_208 208
+208 val_208 208
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+24 val_24 24
+24 val_24 24
+463 val_463 463
+463 val_463 463
+431 val_431 431
+431 val_431 431
+431 val_431 431
+179 val_179 179
+179 val_179 179
+172 val_172 172
+172 val_172 172
+42 val_42 42
+42 val_42 42
+129 val_129 129
+129 val_129 129
+158 val_158 158
+119 val_119 119
+119 val_119 119
+119 val_119 119
+496 val_496 496
+0 val_0 0
+0 val_0 0
+0 val_0 0
+322 val_322 322
+322 val_322 322
+197 val_197 197
+197 val_197 197
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+393 val_393 393
+454 val_454 454
+454 val_454 454
+454 val_454 454
+100 val_100 100
+100 val_100 100
+298 val_298 298
+298 val_298 298
+298 val_298 298
+199 val_199 199
+199 val_199 199
+199 val_199 199
+191 val_191 191
+191 val_191 191
+418 val_418 418
+96 val_96 96
+26 val_26 26
+26 val_26 26
+165 val_165 165
+165 val_165 165
+327 val_327 327
+327 val_327 327
+327 val_327 327
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+205 val_205 205
+205 val_205 205
+120 val_120 120
+120 val_120 120
+131 val_131 131
+51 val_51 51
+51 val_51 51
+404 val_404 404
+404 val_404 404
+43 val_43 43
+436 val_436 436
+156 val_156 156
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+308 val_308 308
+95 val_95 95
+95 val_95 95
+196 val_196 196
+288 val_288 288
+288 val_288 288
+481 val_481 481
+457 val_457 457
+98 val_98 98
+98 val_98 98
+282 val_282 282
+282 val_282 282
+197 val_197 197
+197 val_197 197
+187 val_187 187
+187 val_187 187
+187 val_187 187
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+318 val_318 318
+409 val_409 409
+409 val_409 409
+409 val_409 409
+470 val_470 470
+137 val_137 137
+137 val_137 137
+369 val_369 369
+369 val_369 369
+369 val_369 369
+316 val_316 316
+316 val_316 316
+316 val_316 316
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+413 val_413 413
+413 val_413 413
+85 val_85 85
+77 val_77 77
+0 val_0 0
+0 val_0 0
+0 val_0 0
+490 val_490 490
+87 val_87 87
+364 val_364 364
+179 val_179 179
+179 val_179 179
+118 val_118 118
+118 val_118 118
+134 val_134 134
+134 val_134 134
+395 val_395 395
+395 val_395 395
+282 val_282 282
+282 val_282 282
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+238 val_238 238
+238 val_238 238
+419 val_419 419
+15 val_15 15
+15 val_15 15
+118 val_118 118
+118 val_118 118
+72 val_72 72
+72 val_72 72
+90 val_90 90
+90 val_90 90
+90 val_90 90
+307 val_307 307
+307 val_307 307
+19 val_19 19
+435 val_435 435
+10 val_10 10
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+273 val_273 273
+273 val_273 273
+273 val_273 273
+306 val_306 306
+224 val_224 224
+224 val_224 224
+309 val_309 309
+309 val_309 309
+389 val_389 389
+327 val_327 327
+327 val_327 327
+327 val_327 327
+242 val_242 242
+242 val_242 242
+369 val_369 369
+369 val_369 369
+369 val_369 369
+392 val_392 392
+272 val_272 272
+272 val_272 272
+331 val_331 331
+331 val_331 331
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+242 val_242 242
+242 val_242 242
+452 val_452 452
+177 val_177 177
+226 val_226 226
+5 val_5 5
+5 val_5 5
+5 val_5 5
+497 val_497 497
+402 val_402 402
+396 val_396 396
+396 val_396 396
+396 val_396 396
+317 val_317 317
+317 val_317 317
+395 val_395 395
+395 val_395 395
+58 val_58 58
+58 val_58 58
+35 val_35 35
+35 val_35 35
+35 val_35 35
+336 val_336 336
+95 val_95 95
+95 val_95 95
+11 val_11 11
+168 val_168 168
+34 val_34 34
+229 val_229 229
+229 val_229 229
+233 val_233 233
+233 val_233 233
+143 val_143 143
+472 val_472 472
+322 val_322 322
+322 val_322 322
+498 val_498 498
+498 val_498 498
+498 val_498 498
+160 val_160 160
+195 val_195 195
+195 val_195 195
+42 val_42 42
+42 val_42 42
+321 val_321 321
+321 val_321 321
+430 val_430 430
+430 val_430 430
+430 val_430 430
+119 val_119 119
+119 val_119 119
+119 val_119 119
+489 val_489 489
+489 val_489 489
+489 val_489 489
+489 val_489 489
+458 val_458 458
+458 val_458 458
+78 val_78 78
+76 val_76 76
+76 val_76 76
+41 val_41 41
+223 val_223 223
+223 val_223 223
+492 val_492 492
+492 val_492 492
+149 val_149 149
+149 val_149 149
+449 val_449 449
+218 val_218 218
+228 val_228 228
+138 val_138 138
+138 val_138 138
+138 val_138 138
+138 val_138 138
+453 val_453 453
+30 val_30 30
+209 val_209 209
+209 val_209 209
+64 val_64 64
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+76 val_76 76
+76 val_76 76
+74 val_74 74
+342 val_342 342
+342 val_342 342
+69 val_69 69
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+33 val_33 33
+368 val_368 368
+103 val_103 103
+103 val_103 103
+296 val_296 296
+113 val_113 113
+113 val_113 113
+216 val_216 216
+216 val_216 216
+367 val_367 367
+367 val_367 367
+344 val_344 344
+344 val_344 344
+167 val_167 167
+167 val_167 167
+167 val_167 167
+274 val_274 274
+219 val_219 219
+219 val_219 219
+239 val_239 239
+239 val_239 239
+485 val_485 485
+116 val_116 116
+223 val_223 223
+223 val_223 223
+256 val_256 256
+256 val_256 256
+263 val_263 263
+70 val_70 70
+70 val_70 70
+70 val_70 70
+487 val_487 487
+480 val_480 480
+480 val_480 480
+480 val_480 480
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+288 val_288 288
+288 val_288 288
+191 val_191 191
+191 val_191 191
+5 val_5 5
+5 val_5 5
+5 val_5 5
+244 val_244 244
+438 val_438 438
+438 val_438 438
+438 val_438 438
+128 val_128 128
+128 val_128 128
+128 val_128 128
+467 val_467 467
+432 val_432 432
+202 val_202 202
+316 val_316 316
+316 val_316 316
+316 val_316 316
+229 val_229 229
+229 val_229 229
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+463 val_463 463
+463 val_463 463
+280 val_280 280
+280 val_280 280
+2 val_2 2
+35 val_35 35
+35 val_35 35
+35 val_35 35
+283 val_283 283
+331 val_331 331
+331 val_331 331
+235 val_235 235
+80 val_80 80
+44 val_44 44
+193 val_193 193
+193 val_193 193
+193 val_193 193
+321 val_321 321
+321 val_321 321
+335 val_335 335
+104 val_104 104
+104 val_104 104
+466 val_466 466
+466 val_466 466
+466 val_466 466
+366 val_366 366
+175 val_175 175
+175 val_175 175
+403 val_403 403
+403 val_403 403
+403 val_403 403
+483 val_483 483
+53 val_53 53
+105 val_105 105
+257 val_257 257
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+409 val_409 409
+409 val_409 409
+409 val_409 409
+190 val_190 190
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+114 val_114 114
+258 val_258 258
+90 val_90 90
+90 val_90 90
+90 val_90 90
+203 val_203 203
+203 val_203 203
+262 val_262 262
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+424 val_424 424
+424 val_424 424
+12 val_12 12
+12 val_12 12
+396 val_396 396
+396 val_396 396
+396 val_396 396
+201 val_201 201
+217 val_217 217
+217 val_217 217
+164 val_164 164
+164 val_164 164
+431 val_431 431
+431 val_431 431
+431 val_431 431
+454 val_454 454
+454 val_454 454
+454 val_454 454
+478 val_478 478
+478 val_478 478
+298 val_298 298
+298 val_298 298
+298 val_298 298
+125 val_125 125
+125 val_125 125
+431 val_431 431
+431 val_431 431
+431 val_431 431
+164 val_164 164
+164 val_164 164
+424 val_424 424
+424 val_424 424
+187 val_187 187
+187 val_187 187
+187 val_187 187
+382 val_382 382
+382 val_382 382
+5 val_5 5
+5 val_5 5
+5 val_5 5
+70 val_70 70
+70 val_70 70
+70 val_70 70
+397 val_397 397
+397 val_397 397
+480 val_480 480
+480 val_480 480
+480 val_480 480
+291 val_291 291
+24 val_24 24
+24 val_24 24
+351 val_351 351
+255 val_255 255
+255 val_255 255
+104 val_104 104
+104 val_104 104
+70 val_70 70
+70 val_70 70
+70 val_70 70
+163 val_163 163
+438 val_438 438
+438 val_438 438
+438 val_438 438
+119 val_119 119
+119 val_119 119
+119 val_119 119
+414 val_414 414
+414 val_414 414
+200 val_200 200
+200 val_200 200
+491 val_491 491
+237 val_237 237
+237 val_237 237
+439 val_439 439
+439 val_439 439
+360 val_360 360
+248 val_248 248
+479 val_479 479
+305 val_305 305
+417 val_417 417
+417 val_417 417
+417 val_417 417
+199 val_199 199
+199 val_199 199
+199 val_199 199
+444 val_444 444
+120 val_120 120
+120 val_120 120
+429 val_429 429
+429 val_429 429
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+443 val_443 443
+323 val_323 323
+325 val_325 325
+325 val_325 325
+277 val_277 277
+277 val_277 277
+277 val_277 277
+277 val_277 277
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+230 val_230 230
+478 val_478 478
+478 val_478 478
+178 val_178 178
+468 val_468 468
+468 val_468 468
+468 val_468 468
+468 val_468 468
+310 val_310 310
+317 val_317 317
+317 val_317 317
+333 val_333 333
+333 val_333 333
+493 val_493 493
+460 val_460 460
+207 val_207 207
+207 val_207 207
+249 val_249 249
+265 val_265 265
+265 val_265 265
+480 val_480 480
+480 val_480 480
+480 val_480 480
+83 val_83 83
+83 val_83 83
+136 val_136 136
+353 val_353 353
+353 val_353 353
+172 val_172 172
+172 val_172 172
+214 val_214 214
+462 val_462 462
+462 val_462 462
+233 val_233 233
+233 val_233 233
+406 val_406 406
+406 val_406 406
+406 val_406 406
+406 val_406 406
+133 val_133 133
+175 val_175 175
+175 val_175 175
+189 val_189 189
+454 val_454 454
+454 val_454 454
+454 val_454 454
+375 val_375 375
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+401 val_401 401
+421 val_421 421
+407 val_407 407
+384 val_384 384
+384 val_384 384
+384 val_384 384
+256 val_256 256
+256 val_256 256
+26 val_26 26
+26 val_26 26
+134 val_134 134
+134 val_134 134
+67 val_67 67
+67 val_67 67
+384 val_384 384
+384 val_384 384
+384 val_384 384
+379 val_379 379
+18 val_18 18
+18 val_18 18
+462 val_462 462
+462 val_462 462
+492 val_492 492
+492 val_492 492
+100 val_100 100
+100 val_100 100
+298 val_298 298
+298 val_298 298
+298 val_298 298
+9 val_9 9
+341 val_341 341
+498 val_498 498
+498 val_498 498
+498 val_498 498
+146 val_146 146
+146 val_146 146
+458 val_458 458
+458 val_458 458
+362 val_362 362
+186 val_186 186
+285 val_285 285
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+167 val_167 167
+167 val_167 167
+167 val_167 167
+18 val_18 18
+18 val_18 18
+273 val_273 273
+273 val_273 273
+273 val_273 273
+183 val_183 183
+281 val_281 281
+281 val_281 281
+344 val_344 344
+344 val_344 344
+97 val_97 97
+97 val_97 97
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+469 val_469 469
+315 val_315 315
+84 val_84 84
+84 val_84 84
+28 val_28 28
+37 val_37 37
+37 val_37 37
+448 val_448 448
+152 val_152 152
+152 val_152 152
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+348 val_348 348
+307 val_307 307
+307 val_307 307
+194 val_194 194
+414 val_414 414
+414 val_414 414
+477 val_477 477
+222 val_222 222
+126 val_126 126
+90 val_90 90
+90 val_90 90
+90 val_90 90
+169 val_169 169
+169 val_169 169
+169 val_169 169
+169 val_169 169
+403 val_403 403
+403 val_403 403
+403 val_403 403
+400 val_400 400
+200 val_200 200
+200 val_200 200
+97 val_97 97
+97 val_97 97
diff --git a/sql/hive/src/test/resources/golden/view-22-f83b15c828d4ec599d7827af8b25f578 b/sql/hive/src/test/resources/golden/view-22-f83b15c828d4ec599d7827af8b25f578
new file mode 100644
index 0000000000000..e34118512c1d7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-22-f83b15c828d4ec599d7827af8b25f578
@@ -0,0 +1,500 @@
+238
+86
+311
+27
+165
+409
+255
+278
+98
+484
+265
+193
+401
+150
+273
+224
+369
+66
+128
+213
+146
+406
+429
+374
+152
+469
+145
+495
+37
+327
+281
+277
+209
+15
+82
+403
+166
+417
+430
+252
+292
+219
+287
+153
+193
+338
+446
+459
+394
+237
+482
+174
+413
+494
+207
+199
+466
+208
+174
+399
+396
+247
+417
+489
+162
+377
+397
+309
+365
+266
+439
+342
+367
+325
+167
+195
+475
+17
+113
+155
+203
+339
+0
+455
+128
+311
+316
+57
+302
+205
+149
+438
+345
+129
+170
+20
+489
+157
+378
+221
+92
+111
+47
+72
+4
+280
+35
+427
+277
+208
+356
+399
+169
+382
+498
+125
+386
+437
+469
+192
+286
+187
+176
+54
+459
+51
+138
+103
+239
+213
+216
+430
+278
+176
+289
+221
+65
+318
+332
+311
+275
+137
+241
+83
+333
+180
+284
+12
+230
+181
+67
+260
+404
+384
+489
+353
+373
+272
+138
+217
+84
+348
+466
+58
+8
+411
+230
+208
+348
+24
+463
+431
+179
+172
+42
+129
+158
+119
+496
+0
+322
+197
+468
+393
+454
+100
+298
+199
+191
+418
+96
+26
+165
+327
+230
+205
+120
+131
+51
+404
+43
+436
+156
+469
+468
+308
+95
+196
+288
+481
+457
+98
+282
+197
+187
+318
+318
+409
+470
+137
+369
+316
+169
+413
+85
+77
+0
+490
+87
+364
+179
+118
+134
+395
+282
+138
+238
+419
+15
+118
+72
+90
+307
+19
+435
+10
+277
+273
+306
+224
+309
+389
+327
+242
+369
+392
+272
+331
+401
+242
+452
+177
+226
+5
+497
+402
+396
+317
+395
+58
+35
+336
+95
+11
+168
+34
+229
+233
+143
+472
+322
+498
+160
+195
+42
+321
+430
+119
+489
+458
+78
+76
+41
+223
+492
+149
+449
+218
+228
+138
+453
+30
+209
+64
+468
+76
+74
+342
+69
+230
+33
+368
+103
+296
+113
+216
+367
+344
+167
+274
+219
+239
+485
+116
+223
+256
+263
+70
+487
+480
+401
+288
+191
+5
+244
+438
+128
+467
+432
+202
+316
+229
+469
+463
+280
+2
+35
+283
+331
+235
+80
+44
+193
+321
+335
+104
+466
+366
+175
+403
+483
+53
+105
+257
+406
+409
+190
+406
+401
+114
+258
+90
+203
+262
+348
+424
+12
+396
+201
+217
+164
+431
+454
+478
+298
+125
+431
+164
+424
+187
+382
+5
+70
+397
+480
+291
+24
+351
+255
+104
+70
+163
+438
+119
+414
+200
+491
+237
+439
+360
+248
+479
+305
+417
+199
+444
+120
+429
+169
+443
+323
+325
+277
+230
+478
+178
+468
+310
+317
+333
+493
+460
+207
+249
+265
+480
+83
+136
+353
+172
+214
+462
+233
+406
+133
+175
+189
+454
+375
+401
+421
+407
+384
+256
+26
+134
+67
+384
+379
+18
+462
+492
+100
+298
+9
+341
+498
+146
+458
+362
+186
+285
+348
+167
+18
+273
+183
+281
+344
+97
+469
+315
+84
+28
+37
+448
+152
+348
+307
+194
+414
+477
+222
+126
+90
+169
+403
+400
+200
+97
diff --git a/sql/hive/src/test/resources/golden/view-23-f6a52dd2ff5b11ea3bba2feb867f00c2 b/sql/hive/src/test/resources/golden/view-23-f6a52dd2ff5b11ea3bba2feb867f00c2
new file mode 100644
index 0000000000000..e34118512c1d7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view-23-f6a52dd2ff5b11ea3bba2feb867f00c2
@@ -0,0 +1,500 @@
+238
+86
+311
+27
+165
+409
+255
+278
+98
+484
+265
+193
+401
+150
+273
+224
+369
+66
+128
+213
+146
+406
+429
+374
+152
+469
+145
+495
+37
+327
+281
+277
+209
+15
+82
+403
+166
+417
+430
+252
+292
+219
+287
+153
+193
+338
+446
+459
+394
+237
+482
+174
+413
+494
+207
+199
+466
+208
+174
+399
+396
+247
+417
+489
+162
+377
+397
+309
+365
+266
+439
+342
+367
+325
+167
+195
+475
+17
+113
+155
+203
+339
+0
+455
+128
+311
+316
+57
+302
+205
+149
+438
+345
+129
+170
+20
+489
+157
+378
+221
+92
+111
+47
+72
+4
+280
+35
+427
+277
+208
+356
+399
+169
+382
+498
+125
+386
+437
+469
+192
+286
+187
+176
+54
+459
+51
+138
+103
+239
+213
+216
+430
+278
+176
+289
+221
+65
+318
+332
+311
+275
+137
+241
+83
+333
+180
+284
+12
+230
+181
+67
+260
+404
+384
+489
+353
+373
+272
+138
+217
+84
+348
+466
+58
+8
+411
+230
+208
+348
+24
+463
+431
+179
+172
+42
+129
+158
+119
+496
+0
+322
+197
+468
+393
+454
+100
+298
+199
+191
+418
+96
+26
+165
+327
+230
+205
+120
+131
+51
+404
+43
+436
+156
+469
+468
+308
+95
+196
+288
+481
+457
+98
+282
+197
+187
+318
+318
+409
+470
+137
+369
+316
+169
+413
+85
+77
+0
+490
+87
+364
+179
+118
+134
+395
+282
+138
+238
+419
+15
+118
+72
+90
+307
+19
+435
+10
+277
+273
+306
+224
+309
+389
+327
+242
+369
+392
+272
+331
+401
+242
+452
+177
+226
+5
+497
+402
+396
+317
+395
+58
+35
+336
+95
+11
+168
+34
+229
+233
+143
+472
+322
+498
+160
+195
+42
+321
+430
+119
+489
+458
+78
+76
+41
+223
+492
+149
+449
+218
+228
+138
+453
+30
+209
+64
+468
+76
+74
+342
+69
+230
+33
+368
+103
+296
+113
+216
+367
+344
+167
+274
+219
+239
+485
+116
+223
+256
+263
+70
+487
+480
+401
+288
+191
+5
+244
+438
+128
+467
+432
+202
+316
+229
+469
+463
+280
+2
+35
+283
+331
+235
+80
+44
+193
+321
+335
+104
+466
+366
+175
+403
+483
+53
+105
+257
+406
+409
+190
+406
+401
+114
+258
+90
+203
+262
+348
+424
+12
+396
+201
+217
+164
+431
+454
+478
+298
+125
+431
+164
+424
+187
+382
+5
+70
+397
+480
+291
+24
+351
+255
+104
+70
+163
+438
+119
+414
+200
+491
+237
+439
+360
+248
+479
+305
+417
+199
+444
+120
+429
+169
+443
+323
+325
+277
+230
+478
+178
+468
+310
+317
+333
+493
+460
+207
+249
+265
+480
+83
+136
+353
+172
+214
+462
+233
+406
+133
+175
+189
+454
+375
+401
+421
+407
+384
+256
+26
+134
+67
+384
+379
+18
+462
+492
+100
+298
+9
+341
+498
+146
+458
+362
+186
+285
+348
+167
+18
+273
+183
+281
+344
+97
+469
+315
+84
+28
+37
+448
+152
+348
+307
+194
+414
+477
+222
+126
+90
+169
+403
+400
+200
+97
diff --git a/sql/hive/src/test/resources/golden/view-3-e7dd3b24daa60d8955b22f0441f01a6a b/sql/hive/src/test/resources/golden/view-3-e7dd3b24daa60d8955b22f0441f01a6a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-5-7abee38ed087f13f03ac216ef0decf4c b/sql/hive/src/test/resources/golden/view-5-7abee38ed087f13f03ac216ef0decf4c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-11-b809f5d793b072146ccf577abf286003 b/sql/hive/src/test/resources/golden/view_cast-11-b809f5d793b072146ccf577abf286003
new file mode 100644
index 0000000000000..da7e68de78227
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view_cast-11-b809f5d793b072146ccf577abf286003
@@ -0,0 +1,8 @@
+1111 abc fun bar NULL 2222 NULL 99999.0 99999.0
+1111 abc fun bar NULL 2222 NULL 99999.0 99999.0
+2222 abc fun bar NULL 4444 NULL 99999.0 99999.0
+2222 abc fun bar NULL 4444 NULL 99999.0 99999.0
+3333 abc fun bar NULL 6666 NULL 99999.0 10.0
+3333 abc fun bar NULL 6666 NULL 99999.0 10.0
+3333 abc fun bar NULL 6666 NULL 99999.0 10.0
+4444 abc fun bar NULL 8888 NULL 99999.0 99999.0
diff --git a/sql/hive/src/test/resources/golden/view_cast-2-635031c0752d5b30c44dfb3dec759a6c b/sql/hive/src/test/resources/golden/view_cast-2-635031c0752d5b30c44dfb3dec759a6c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-3-9f675e7edd64a33713e91282dc201867 b/sql/hive/src/test/resources/golden/view_cast-3-9f675e7edd64a33713e91282dc201867
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-6-b18da53e46b62d6d91efac88ba62f308 b/sql/hive/src/test/resources/golden/view_cast-6-b18da53e46b62d6d91efac88ba62f308
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/view_inputs-2-626fa3664754125edc44b7ca7f8630db b/sql/hive/src/test/resources/golden/view_inputs-2-626fa3664754125edc44b7ca7f8630db
new file mode 100644
index 0000000000000..1f3d8a7a1fc08
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/view_inputs-2-626fa3664754125edc44b7ca7f8630db
@@ -0,0 +1 @@
+1028
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 5dbfb923139fa..fb481edc853b7 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
@@ -121,4 +121,54 @@ class InsertIntoHiveTableSuite extends QueryTest {
sql("DROP TABLE table_with_partition")
sql("DROP TABLE tmp_table")
}
+
+ test("Insert ArrayType.containsNull == false") {
+ val schema = StructType(Seq(
+ StructField("a", ArrayType(StringType, containsNull = false))))
+ val rowRDD = TestHive.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i"))))
+ val schemaRDD = applySchema(rowRDD, schema)
+ schemaRDD.registerTempTable("tableWithArrayValue")
+ sql("CREATE TABLE hiveTableWithArrayValue(a Array )")
+ sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue")
+
+ checkAnswer(
+ sql("SELECT * FROM hiveTableWithArrayValue"),
+ rowRDD.collect().toSeq)
+
+ sql("DROP TABLE hiveTableWithArrayValue")
+ }
+
+ test("Insert MapType.valueContainsNull == false") {
+ val schema = StructType(Seq(
+ StructField("m", MapType(StringType, StringType, valueContainsNull = false))))
+ val rowRDD = TestHive.sparkContext.parallelize(
+ (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i"))))
+ val schemaRDD = applySchema(rowRDD, schema)
+ schemaRDD.registerTempTable("tableWithMapValue")
+ sql("CREATE TABLE hiveTableWithMapValue(m Map )")
+ sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue")
+
+ checkAnswer(
+ sql("SELECT * FROM hiveTableWithMapValue"),
+ rowRDD.collect().toSeq)
+
+ sql("DROP TABLE hiveTableWithMapValue")
+ }
+
+ test("Insert StructType.fields.exists(_.nullable == false)") {
+ val schema = StructType(Seq(
+ StructField("s", StructType(Seq(StructField("f", StringType, nullable = false))))))
+ val rowRDD = TestHive.sparkContext.parallelize(
+ (1 to 100).map(i => Row(Row(s"value$i"))))
+ val schemaRDD = applySchema(rowRDD, schema)
+ schemaRDD.registerTempTable("tableWithStructValue")
+ sql("CREATE TABLE hiveTableWithStructValue(s Struct )")
+ sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue")
+
+ checkAnswer(
+ sql("SELECT * FROM hiveTableWithStructValue"),
+ rowRDD.collect().toSeq)
+
+ sql("DROP TABLE hiveTableWithStructValue")
+ }
}
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 684d22807c0c6..0dd766f25348d 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
@@ -56,6 +56,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
Locale.setDefault(originalLocale)
}
+ createQueryTest("NaN to Decimal",
+ "SELECT CAST(CAST('NaN' AS DOUBLE) AS DECIMAL(1,1)) FROM src LIMIT 1")
+
createQueryTest("constant null testing",
"""SELECT
|IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1,
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala
index 86adbbf3ad2d8..cc65242c0da9b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala
@@ -27,7 +27,11 @@ import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.hive.execution.HiveTableScan
import org.apache.spark.sql.hive.test.TestHive._
+// The data where the partitioning key exists only in the directory structure.
case class ParquetData(intField: Int, stringField: String)
+// The data that also includes the partitioning key
+case class ParquetDataWithKey(p: Int, intField: Int, stringField: String)
+
/**
* Tests for our SerDe -> Native parquet scan conversion.
@@ -45,6 +49,17 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll {
.saveAsParquetFile(partDir.getCanonicalPath)
}
+ val partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql")
+ partitionedTableDirWithKey.delete()
+ partitionedTableDirWithKey.mkdir()
+
+ (1 to 10).foreach { p =>
+ val partDir = new File(partitionedTableDirWithKey, s"p=$p")
+ sparkContext.makeRDD(1 to 10)
+ .map(i => ParquetDataWithKey(p, i, s"part-$p"))
+ .saveAsParquetFile(partDir.getCanonicalPath)
+ }
+
sql(s"""
create external table partitioned_parquet
(
@@ -59,6 +74,20 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll {
location '${partitionedTableDir.getCanonicalPath}'
""")
+ sql(s"""
+ create external table partitioned_parquet_with_key
+ (
+ intField INT,
+ stringField STRING
+ )
+ PARTITIONED BY (p int)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
+ STORED AS
+ INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
+ OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
+ location '${partitionedTableDirWithKey.getCanonicalPath}'
+ """)
+
sql(s"""
create external table normal_parquet
(
@@ -76,6 +105,10 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll {
sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)")
}
+ (1 to 10).foreach { p =>
+ sql(s"ALTER TABLE partitioned_parquet_with_key ADD PARTITION (p=$p)")
+ }
+
setConf("spark.sql.hive.convertMetastoreParquet", "true")
}
@@ -83,75 +116,76 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll {
setConf("spark.sql.hive.convertMetastoreParquet", "false")
}
- test("project the partitioning column") {
- checkAnswer(
- sql("SELECT p, count(*) FROM partitioned_parquet group by p"),
- (1, 10) ::
- (2, 10) ::
- (3, 10) ::
- (4, 10) ::
- (5, 10) ::
- (6, 10) ::
- (7, 10) ::
- (8, 10) ::
- (9, 10) ::
- (10, 10) :: Nil
- )
- }
+ Seq("partitioned_parquet", "partitioned_parquet_with_key").foreach { table =>
+ test(s"project the partitioning column $table") {
+ checkAnswer(
+ sql(s"SELECT p, count(*) FROM $table group by p"),
+ (1, 10) ::
+ (2, 10) ::
+ (3, 10) ::
+ (4, 10) ::
+ (5, 10) ::
+ (6, 10) ::
+ (7, 10) ::
+ (8, 10) ::
+ (9, 10) ::
+ (10, 10) :: Nil
+ )
+ }
- test("project partitioning and non-partitioning columns") {
- checkAnswer(
- sql("SELECT stringField, p, count(intField) " +
- "FROM partitioned_parquet GROUP BY p, stringField"),
- ("part-1", 1, 10) ::
- ("part-2", 2, 10) ::
- ("part-3", 3, 10) ::
- ("part-4", 4, 10) ::
- ("part-5", 5, 10) ::
- ("part-6", 6, 10) ::
- ("part-7", 7, 10) ::
- ("part-8", 8, 10) ::
- ("part-9", 9, 10) ::
- ("part-10", 10, 10) :: Nil
- )
- }
+ test(s"project partitioning and non-partitioning columns $table") {
+ checkAnswer(
+ sql(s"SELECT stringField, p, count(intField) FROM $table GROUP BY p, stringField"),
+ ("part-1", 1, 10) ::
+ ("part-2", 2, 10) ::
+ ("part-3", 3, 10) ::
+ ("part-4", 4, 10) ::
+ ("part-5", 5, 10) ::
+ ("part-6", 6, 10) ::
+ ("part-7", 7, 10) ::
+ ("part-8", 8, 10) ::
+ ("part-9", 9, 10) ::
+ ("part-10", 10, 10) :: Nil
+ )
+ }
- test("simple count") {
- checkAnswer(
- sql("SELECT COUNT(*) FROM partitioned_parquet"),
- 100)
- }
+ test(s"simple count $table") {
+ checkAnswer(
+ sql(s"SELECT COUNT(*) FROM $table"),
+ 100)
+ }
- test("pruned count") {
- checkAnswer(
- sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p = 1"),
- 10)
- }
+ test(s"pruned count $table") {
+ checkAnswer(
+ sql(s"SELECT COUNT(*) FROM $table WHERE p = 1"),
+ 10)
+ }
- test("multi-partition pruned count") {
- checkAnswer(
- sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p IN (1,2,3)"),
- 30)
- }
+ test(s"multi-partition pruned count $table") {
+ checkAnswer(
+ sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"),
+ 30)
+ }
- test("non-partition predicates") {
- checkAnswer(
- sql("SELECT COUNT(*) FROM partitioned_parquet WHERE intField IN (1,2,3)"),
- 30)
- }
+ test(s"non-partition predicates $table") {
+ checkAnswer(
+ sql(s"SELECT COUNT(*) FROM $table WHERE intField IN (1,2,3)"),
+ 30)
+ }
- test("sum") {
- checkAnswer(
- sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"),
- 1 + 2 + 3)
- }
+ test(s"sum $table") {
+ checkAnswer(
+ sql(s"SELECT SUM(intField) FROM $table WHERE intField IN (1,2,3) AND p = 1"),
+ 1 + 2 + 3)
+ }
- test("hive udfs") {
- checkAnswer(
- sql("SELECT concat(stringField, stringField) FROM partitioned_parquet"),
- sql("SELECT stringField FROM partitioned_parquet").map {
- case Row(s: String) => Row(s + s)
- }.collect().toSeq)
+ test(s"hive udfs $table") {
+ checkAnswer(
+ sql(s"SELECT concat(stringField, stringField) FROM $table"),
+ sql(s"SELECT stringField FROM $table").map {
+ case Row(s: String) => Row(s + s)
+ }.collect().toSeq)
+ }
}
test("non-part select(*)") {
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 8ba25f889d176..76f09cbcdec99 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
@@ -136,6 +136,8 @@ private[hive] object HiveShim {
def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
+ def getStatsSetupConstRawDataSize = StatsSetupConst.RAW_DATA_SIZE
+
def createDefaultDBIfNeeded(context: HiveContext) = { }
def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
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 e4aee57f0ad9f..91f7ceac21177 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
@@ -154,6 +154,8 @@ private[hive] object HiveShim {
def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
+ def getStatsSetupConstRawDataSize = StatsSetupConst.RAW_DATA_SIZE
+
def createDefaultDBIfNeeded(context: HiveContext) = {
context.runSqlHive("CREATE DATABASE default")
context.runSqlHive("USE default")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
index 0316b6862f195..55765dc90698b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
@@ -27,9 +27,38 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
/** Listener object for BlockGenerator events */
private[streaming] trait BlockGeneratorListener {
- /** Called when a new block needs to be pushed */
+ /**
+ * Called after a data item is added into the BlockGenerator. The data addition and this
+ * callback are synchronized with the block generation and its associated callback,
+ * so block generation waits for the active data addition+callback to complete. This is useful
+ * for updating metadata on successful buffering of a data item, specifically that metadata
+ * that will be useful when a block is generated. Any long blocking operation in this callback
+ * will hurt the throughput.
+ */
+ def onAddData(data: Any, metadata: Any)
+
+ /**
+ * Called when a new block of data is generated by the block generator. The block generation
+ * and this callback are synchronized with the data addition and its associated callback, so
+ * the data addition waits for the block generation+callback to complete. This is useful
+ * for updating metadata when a block has been generated, specifically metadata that will
+ * be useful when the block has been successfully stored. Any long blocking operation in this
+ * callback will hurt the throughput.
+ */
+ def onGenerateBlock(blockId: StreamBlockId)
+
+ /**
+ * Called when a new block is ready to be pushed. Callers are supposed to store the block into
+ * Spark in this method. Internally this is called from a single
+ * thread, that is not synchronized with any other callbacks. Hence it is okay to do long
+ * blocking operation in this callback.
+ */
def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_])
- /** Called when an error has occurred in BlockGenerator */
+
+ /**
+ * Called when an error has occurred in the BlockGenerator. Can be called form many places
+ * so better to not do any long block operation in this callback.
+ */
def onError(message: String, throwable: Throwable)
}
@@ -80,9 +109,20 @@ private[streaming] class BlockGenerator(
* Push a single data item into the buffer. All received data items
* will be periodically pushed into BlockManager.
*/
- def += (data: Any): Unit = synchronized {
+ def addData (data: Any): Unit = synchronized {
+ waitToPush()
+ currentBuffer += data
+ }
+
+ /**
+ * Push a single data item into the buffer. After buffering the data, the
+ * `BlockGeneratorListnere.onAddData` callback will be called. All received data items
+ * will be periodically pushed into BlockManager.
+ */
+ def addDataWithCallback(data: Any, metadata: Any) = synchronized {
waitToPush()
currentBuffer += data
+ listener.onAddData(data, metadata)
}
/** Change the buffer to which single records are added to. */
@@ -93,14 +133,15 @@ private[streaming] class BlockGenerator(
if (newBlockBuffer.size > 0) {
val blockId = StreamBlockId(receiverId, time - blockInterval)
val newBlock = new Block(blockId, newBlockBuffer)
+ listener.onGenerateBlock(blockId)
blocksForPushing.put(newBlock) // put is blocking when queue is full
logDebug("Last element in " + blockId + " is " + newBlockBuffer.last)
}
} catch {
case ie: InterruptedException =>
logInfo("Block updating timer thread was interrupted")
- case t: Throwable =>
- reportError("Error in block updating thread", t)
+ case e: Exception =>
+ reportError("Error in block updating thread", e)
}
}
@@ -126,8 +167,8 @@ private[streaming] class BlockGenerator(
} catch {
case ie: InterruptedException =>
logInfo("Block pushing thread was interrupted")
- case t: Throwable =>
- reportError("Error in block pushing thread", t)
+ case e: Exception =>
+ reportError("Error in block pushing thread", e)
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
index 5360412330d37..3b1233e86c210 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala
@@ -27,10 +27,10 @@ import akka.actor.{Actor, Props}
import akka.pattern.ask
import com.google.common.base.Throwables
import org.apache.hadoop.conf.Configuration
+
import org.apache.spark.{Logging, SparkEnv, SparkException}
import org.apache.spark.storage.StreamBlockId
import org.apache.spark.streaming.scheduler._
-import org.apache.spark.streaming.util.WriteAheadLogFileSegment
import org.apache.spark.util.{AkkaUtils, Utils}
/**
@@ -99,6 +99,10 @@ private[streaming] class ReceiverSupervisorImpl(
/** Divides received data records into data blocks for pushing in BlockManager. */
private val blockGenerator = new BlockGenerator(new BlockGeneratorListener {
+ def onAddData(data: Any, metadata: Any): Unit = { }
+
+ def onGenerateBlock(blockId: StreamBlockId): Unit = { }
+
def onError(message: String, throwable: Throwable) {
reportError(message, throwable)
}
@@ -110,7 +114,7 @@ private[streaming] class ReceiverSupervisorImpl(
/** Push a single record of received data into block generator. */
def pushSingle(data: Any) {
- blockGenerator += (data)
+ blockGenerator.addData(data)
}
/** Store an ArrayBuffer of received data as a data block into Spark's memory. */
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 30a359677cc74..86b96785d7b87 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -470,32 +470,31 @@ class BasicOperationsSuite extends TestSuiteBase {
}
test("slice") {
- val ssc = new StreamingContext(conf, Seconds(1))
- val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
- val stream = new TestInputStream[Int](ssc, input, 2)
- stream.foreachRDD(_ => {}) // Dummy output stream
- ssc.start()
- Thread.sleep(2000)
- def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
- stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
- }
+ withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc =>
+ val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
+ val stream = new TestInputStream[Int](ssc, input, 2)
+ stream.foreachRDD(_ => {}) // Dummy output stream
+ ssc.start()
+ Thread.sleep(2000)
+ def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
+ stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
+ }
- assert(getInputFromSlice(0, 1000) == Set(1))
- assert(getInputFromSlice(0, 2000) == Set(1, 2))
- assert(getInputFromSlice(1000, 2000) == Set(1, 2))
- assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
- ssc.stop()
- Thread.sleep(1000)
+ assert(getInputFromSlice(0, 1000) == Set(1))
+ assert(getInputFromSlice(0, 2000) == Set(1, 2))
+ assert(getInputFromSlice(1000, 2000) == Set(1, 2))
+ assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
+ }
}
-
test("slice - has not been initialized") {
- val ssc = new StreamingContext(conf, Seconds(1))
- val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
- val stream = new TestInputStream[Int](ssc, input, 2)
- val thrown = intercept[SparkException] {
- stream.slice(new Time(0), new Time(1000))
+ withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc =>
+ val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
+ val stream = new TestInputStream[Int](ssc, input, 2)
+ val thrown = intercept[SparkException] {
+ stream.slice(new Time(0), new Time(1000))
+ }
+ assert(thrown.getMessage.contains("has not been initialized"))
}
- assert(thrown.getMessage.contains("has not been initialized"))
}
val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq
@@ -555,73 +554,72 @@ class BasicOperationsSuite extends TestSuiteBase {
test("rdd cleanup - input blocks and persisted RDDs") {
// Actually receive data over through receiver to create BlockRDDs
- // Start the server
- val testServer = new TestServer()
- testServer.start()
-
- // Set up the streaming context and input streams
- val ssc = new StreamingContext(conf, batchDuration)
- val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
- val mappedStream = networkStream.map(_ + ".").persist()
- val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
- val outputStream = new TestOutputStream(mappedStream, outputBuffer)
-
- outputStream.register()
- ssc.start()
-
- // Feed data to the server to send to the network receiver
- val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
- val input = Seq(1, 2, 3, 4, 5, 6)
+ withTestServer(new TestServer()) { testServer =>
+ withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc =>
+ testServer.start()
+ // Set up the streaming context and input streams
+ val networkStream =
+ ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
+ val mappedStream = networkStream.map(_ + ".").persist()
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
+ val outputStream = new TestOutputStream(mappedStream, outputBuffer)
+
+ outputStream.register()
+ ssc.start()
+
+ // Feed data to the server to send to the network receiver
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5, 6)
+
+ val blockRdds = new mutable.HashMap[Time, BlockRDD[_]]
+ val persistentRddIds = new mutable.HashMap[Time, Int]
+
+ def collectRddInfo() { // get all RDD info required for verification
+ networkStream.generatedRDDs.foreach { case (time, rdd) =>
+ blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]]
+ }
+ mappedStream.generatedRDDs.foreach { case (time, rdd) =>
+ persistentRddIds(time) = rdd.id
+ }
+ }
- val blockRdds = new mutable.HashMap[Time, BlockRDD[_]]
- val persistentRddIds = new mutable.HashMap[Time, Int]
+ Thread.sleep(200)
+ for (i <- 0 until input.size) {
+ testServer.send(input(i).toString + "\n")
+ Thread.sleep(200)
+ clock.addToTime(batchDuration.milliseconds)
+ collectRddInfo()
+ }
- def collectRddInfo() { // get all RDD info required for verification
- networkStream.generatedRDDs.foreach { case (time, rdd) =>
- blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]]
- }
- mappedStream.generatedRDDs.foreach { case (time, rdd) =>
- persistentRddIds(time) = rdd.id
+ Thread.sleep(200)
+ collectRddInfo()
+ logInfo("Stopping server")
+ testServer.stop()
+
+ // verify data has been received
+ assert(outputBuffer.size > 0)
+ assert(blockRdds.size > 0)
+ assert(persistentRddIds.size > 0)
+
+ import Time._
+
+ val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max)
+ val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min)
+ val latestBlockRdd = blockRdds(blockRdds.keySet.max)
+ val earliestBlockRdd = blockRdds(blockRdds.keySet.min)
+ // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted
+ assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId))
+ assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId))
+
+ // verify that the latest input blocks are present but the earliest blocks have been removed
+ assert(latestBlockRdd.isValid)
+ assert(latestBlockRdd.collect != null)
+ assert(!earliestBlockRdd.isValid)
+ earliestBlockRdd.blockIds.foreach { blockId =>
+ assert(!ssc.sparkContext.env.blockManager.master.contains(blockId))
+ }
}
}
-
- Thread.sleep(200)
- for (i <- 0 until input.size) {
- testServer.send(input(i).toString + "\n")
- Thread.sleep(200)
- clock.addToTime(batchDuration.milliseconds)
- collectRddInfo()
- }
-
- Thread.sleep(200)
- collectRddInfo()
- logInfo("Stopping server")
- testServer.stop()
- logInfo("Stopping context")
-
- // verify data has been received
- assert(outputBuffer.size > 0)
- assert(blockRdds.size > 0)
- assert(persistentRddIds.size > 0)
-
- import Time._
-
- val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max)
- val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min)
- val latestBlockRdd = blockRdds(blockRdds.keySet.max)
- val earliestBlockRdd = blockRdds(blockRdds.keySet.min)
- // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted
- assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId))
- assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId))
-
- // verify that the latest input blocks are present but the earliest blocks have been removed
- assert(latestBlockRdd.isValid)
- assert(latestBlockRdd.collect != null)
- assert(!earliestBlockRdd.isValid)
- earliestBlockRdd.blockIds.foreach { blockId =>
- assert(!ssc.sparkContext.env.blockManager.master.contains(blockId))
- }
- ssc.stop()
}
/** Test cleanup of RDDs in DStream metadata */
@@ -635,13 +633,15 @@ class BasicOperationsSuite extends TestSuiteBase {
// Setup the stream computation
assert(batchDuration === Seconds(1),
"Batch duration has changed from 1 second, check cleanup tests")
- val ssc = setupStreams(cleanupTestInput, operation)
- val operatedStream = ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]]
- if (rememberDuration != null) ssc.remember(rememberDuration)
- val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput)
- val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
- assert(clock.time === Seconds(10).milliseconds)
- assert(output.size === numExpectedOutput)
- operatedStream
+ withStreamingContext(setupStreams(cleanupTestInput, operation)) { ssc =>
+ val operatedStream =
+ ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]]
+ if (rememberDuration != null) ssc.remember(rememberDuration)
+ val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput)
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ assert(clock.time === Seconds(10).milliseconds)
+ assert(output.size === numExpectedOutput)
+ operatedStream
+ }
}
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
index 0f6a9489dbe0d..e26c0c6859e57 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala
@@ -138,7 +138,7 @@ class ReceiverSuite extends FunSuite with Timeouts {
blockGenerator.start()
var count = 0
while(System.currentTimeMillis - startTime < waitTime) {
- blockGenerator += count
+ blockGenerator.addData(count)
generatedData += count
count += 1
Thread.sleep(10)
@@ -168,7 +168,7 @@ class ReceiverSuite extends FunSuite with Timeouts {
blockGenerator.start()
var count = 0
while(System.currentTimeMillis - startTime < waitTime) {
- blockGenerator += count
+ blockGenerator.addData(count)
generatedData += count
count += 1
Thread.sleep(1)
@@ -299,6 +299,10 @@ class ReceiverSuite extends FunSuite with Timeouts {
val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]]
val errors = new ArrayBuffer[Throwable]
+ def onAddData(data: Any, metadata: Any) { }
+
+ def onGenerateBlock(blockId: StreamBlockId) { }
+
def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) {
val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int])
arrayBuffers += bufferOfInts
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 2154c24abda3a..52972f63c6c5c 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -163,6 +163,40 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
before(beforeFunction)
after(afterFunction)
+ /**
+ * Run a block of code with the given StreamingContext and automatically
+ * stop the context when the block completes or when an exception is thrown.
+ */
+ def withStreamingContext[R](ssc: StreamingContext)(block: StreamingContext => R): R = {
+ try {
+ block(ssc)
+ } finally {
+ try {
+ ssc.stop(stopSparkContext = true)
+ } catch {
+ case e: Exception =>
+ logError("Error stopping StreamingContext", e)
+ }
+ }
+ }
+
+ /**
+ * Run a block of code with the given TestServer and automatically
+ * stop the server when the block completes or when an exception is thrown.
+ */
+ def withTestServer[R](testServer: TestServer)(block: TestServer => R): R = {
+ try {
+ block(testServer)
+ } finally {
+ try {
+ testServer.stop()
+ } catch {
+ case e: Exception =>
+ logError("Error stopping TestServer", e)
+ }
+ }
+ }
+
/**
* Set up required DStreams to test the DStream operation using the two sequences
* of input collections.
@@ -282,10 +316,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
assert(output.size === numExpectedOutput, "Unexpected number of outputs generated")
Thread.sleep(100) // Give some time for the forgetting old RDDs to complete
- } catch {
- case e: Exception => {e.printStackTrace(); throw e}
} finally {
- ssc.stop()
+ ssc.stop(stopSparkContext = true)
}
output
}
@@ -351,9 +383,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
- val ssc = setupStreams[U, V](input, operation)
- val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
- verifyOutput[V](output, expectedOutput, useSet)
+ withStreamingContext(setupStreams[U, V](input, operation)) { ssc =>
+ val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[V](output, expectedOutput, useSet)
+ }
}
/**
@@ -389,8 +422,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
- val ssc = setupStreams[U, V, W](input1, input2, operation)
- val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
- verifyOutput[W](output, expectedOutput, useSet)
+ withStreamingContext(setupStreams[U, V, W](input1, input2, operation)) { ssc =>
+ val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[W](output, expectedOutput, useSet)
+ }
}
}