diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java
index 2dff241900e82..a7bd4b3799a25 100644
--- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java
+++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java
@@ -88,7 +88,7 @@ public void spill() throws IOException {
* `LongArray` is too large to fit in a single page. The caller side should take care of these
* two exceptions, or make sure the `size` is small enough that won't trigger exceptions.
*
- * @throws OutOfMemoryError
+ * @throws SparkOutOfMemoryError
* @throws TooLargePageException
*/
public LongArray allocateArray(long size) {
@@ -154,6 +154,6 @@ private void throwOom(final MemoryBlock page, final long required) {
taskMemoryManager.freePage(page, this);
}
taskMemoryManager.showMemoryUsage();
- throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
+ throw new SparkOutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got);
}
}
diff --git a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java
new file mode 100644
index 0000000000000..ca00ca58e9713
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.memory;
+
+import org.apache.spark.annotation.Private;
+
+/**
+ * This exception is thrown when a task can not acquire memory from the Memory manager.
+ * Instead of throwing {@link OutOfMemoryError}, which kills the executor,
+ * we should use throw this exception, which just kills the current task.
+ */
+@Private
+public final class SparkOutOfMemoryError extends OutOfMemoryError {
+
+ public SparkOutOfMemoryError(String s) {
+ super(s);
+ }
+
+ public SparkOutOfMemoryError(OutOfMemoryError e) {
+ super(e.getMessage());
+ }
+}
diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
index f6b5ea3c0ad26..e8d3730daa7a4 100644
--- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
+++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
@@ -192,7 +192,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
throw new RuntimeException(e.getMessage());
} catch (IOException e) {
logger.error("error while calling spill() on " + c, e);
- throw new OutOfMemoryError("error while calling spill() on " + c + " : "
+ throw new SparkOutOfMemoryError("error while calling spill() on " + c + " : "
+ e.getMessage());
}
}
@@ -213,7 +213,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) {
throw new RuntimeException(e.getMessage());
} catch (IOException e) {
logger.error("error while calling spill() on " + consumer, e);
- throw new OutOfMemoryError("error while calling spill() on " + consumer + " : "
+ throw new SparkOutOfMemoryError("error while calling spill() on " + consumer + " : "
+ e.getMessage());
}
}
diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
index e80f9734ecf7b..c3a07b2abf896 100644
--- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java
@@ -33,6 +33,7 @@
import org.apache.spark.executor.ShuffleWriteMetrics;
import org.apache.spark.internal.config.package$;
import org.apache.spark.memory.MemoryConsumer;
+import org.apache.spark.memory.SparkOutOfMemoryError;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.memory.TooLargePageException;
import org.apache.spark.serializer.DummySerializerInstance;
@@ -337,7 +338,7 @@ private void growPointerArrayIfNecessary() throws IOException {
// The pointer array is too big to fix in a single page, spill.
spill();
return;
- } catch (OutOfMemoryError e) {
+ } catch (SparkOutOfMemoryError e) {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index 8b8e15e3f78ed..66118f454159b 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -25,6 +25,7 @@
import java.util.function.Supplier;
import com.google.common.annotations.VisibleForTesting;
+import org.apache.spark.memory.SparkOutOfMemoryError;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -349,7 +350,7 @@ private void growPointerArrayIfNecessary() throws IOException {
// The pointer array is too big to fix in a single page, spill.
spill();
return;
- } catch (OutOfMemoryError e) {
+ } catch (SparkOutOfMemoryError e) {
// should have trigger spilling
if (!inMemSorter.hasSpaceForAnotherRecord()) {
logger.error("Unable to grow the pointer array");
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index 3bb87a6ed653d..951d076420ee6 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -24,6 +24,7 @@
import org.apache.spark.TaskContext;
import org.apache.spark.memory.MemoryConsumer;
+import org.apache.spark.memory.SparkOutOfMemoryError;
import org.apache.spark.memory.TaskMemoryManager;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.UnsafeAlignedOffset;
@@ -212,7 +213,7 @@ public boolean hasSpaceForAnotherRecord() {
public void expandPointerArray(LongArray newArray) {
if (newArray.size() < array.size()) {
- throw new OutOfMemoryError("Not enough memory to grow pointer array");
+ throw new SparkOutOfMemoryError("Not enough memory to grow pointer array");
}
Platform.copyMemory(
array.getBaseObject(),
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index a8e1becc56ab7..fa2c5194aa41b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -44,6 +44,7 @@ import org.apache.spark.scheduler.ReplayListenerBus._
import org.apache.spark.status._
import org.apache.spark.status.KVUtils._
import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo}
+import org.apache.spark.status.config._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
import org.apache.spark.util.kvstore._
@@ -304,6 +305,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
val (kvstore, needReplay) = uiStorePath match {
case Some(path) =>
try {
+ // The store path is not guaranteed to exist - maybe it hasn't been created, or was
+ // invalidated because changes to the event log were detected. Need to replay in that
+ // case.
val _replay = !path.isDirectory()
(createDiskStore(path, conf), _replay)
} catch {
@@ -318,24 +322,23 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
(new InMemoryStore(), true)
}
+ val trackingStore = new ElementTrackingStore(kvstore, conf)
if (needReplay) {
val replayBus = new ReplayListenerBus()
- val listener = new AppStatusListener(kvstore, conf, false,
+ val listener = new AppStatusListener(trackingStore, conf, false,
lastUpdateTime = Some(attempt.info.lastUpdated.getTime()))
replayBus.addListener(listener)
AppStatusPlugin.loadPlugins().foreach { plugin =>
- plugin.setupListeners(conf, kvstore, l => replayBus.addListener(l), false)
+ plugin.setupListeners(conf, trackingStore, l => replayBus.addListener(l), false)
}
try {
val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath))
replay(fileStatus, isApplicationCompleted(fileStatus), replayBus)
- listener.flush()
+ trackingStore.close(false)
} catch {
case e: Exception =>
- try {
- kvstore.close()
- } catch {
- case _e: Exception => logInfo("Error closing store.", _e)
+ Utils.tryLogNonFatalError {
+ trackingStore.close()
}
uiStorePath.foreach(Utils.deleteRecursively)
if (e.isInstanceOf[FileNotFoundException]) {
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index af0a0ab656564..2c3a8ef74800b 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -35,7 +35,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
-import org.apache.spark.memory.TaskMemoryManager
+import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
import org.apache.spark.shuffle.FetchFailedException
@@ -553,10 +553,9 @@ private[spark] class Executor(
// Don't forcibly exit unless the exception was inherently fatal, to avoid
// stopping other tasks unnecessarily.
- if (Utils.isFatalError(t)) {
+ if (!t.isInstanceOf[SparkOutOfMemoryError] && Utils.isFatalError(t)) {
uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), t)
}
-
} finally {
runningTasks.remove(taskId)
}
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 172ba85359da7..eb12ddf961314 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -240,11 +240,6 @@ package object config {
.stringConf
.createOptional
- // To limit memory usage, we only track information for a fixed number of tasks
- private[spark] val UI_RETAINED_TASKS = ConfigBuilder("spark.ui.retainedTasks")
- .intConf
- .createWithDefault(100000)
-
// To limit how many applications are shown in the History Server summary ui
private[spark] val HISTORY_UI_MAX_APPS =
ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE)
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index 6da44cbc44c4d..1fb7b76d43d04 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -32,7 +32,6 @@ import org.apache.spark.status.api.v1
import org.apache.spark.storage._
import org.apache.spark.ui.SparkUI
import org.apache.spark.ui.scope._
-import org.apache.spark.util.kvstore.KVStore
/**
* A Spark listener that writes application information to a data store. The types written to the
@@ -42,7 +41,7 @@ import org.apache.spark.util.kvstore.KVStore
* unfinished tasks can be more accurately calculated (see SPARK-21922).
*/
private[spark] class AppStatusListener(
- kvstore: KVStore,
+ kvstore: ElementTrackingStore,
conf: SparkConf,
live: Boolean,
lastUpdateTime: Option[Long] = None) extends SparkListener with Logging {
@@ -51,6 +50,7 @@ private[spark] class AppStatusListener(
private var sparkVersion = SPARK_VERSION
private var appInfo: v1.ApplicationInfo = null
+ private var appSummary = new AppSummary(0, 0)
private var coresPerTask: Int = 1
// How often to update live entities. -1 means "never update" when replaying applications,
@@ -58,6 +58,7 @@ private[spark] class AppStatusListener(
// operations that we can live without when rapidly processing incoming task events.
private val liveUpdatePeriodNs = if (live) conf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L
+ private val maxTasksPerStage = conf.get(MAX_RETAINED_TASKS_PER_STAGE)
private val maxGraphRootNodes = conf.get(MAX_RETAINED_ROOT_NODES)
// Keep track of live entities, so that task metrics can be efficiently updated (without
@@ -68,10 +69,25 @@ private[spark] class AppStatusListener(
private val liveTasks = new HashMap[Long, LiveTask]()
private val liveRDDs = new HashMap[Int, LiveRDD]()
private val pools = new HashMap[String, SchedulerPool]()
+ // Keep the active executor count as a separate variable to avoid having to do synchronization
+ // around liveExecutors.
+ @volatile private var activeExecutorCount = 0
- override def onOtherEvent(event: SparkListenerEvent): Unit = event match {
- case SparkListenerLogStart(version) => sparkVersion = version
- case _ =>
+ kvstore.addTrigger(classOf[ExecutorSummaryWrapper], conf.get(MAX_RETAINED_DEAD_EXECUTORS))
+ { count => cleanupExecutors(count) }
+
+ kvstore.addTrigger(classOf[JobDataWrapper], conf.get(MAX_RETAINED_JOBS)) { count =>
+ cleanupJobs(count)
+ }
+
+ kvstore.addTrigger(classOf[StageDataWrapper], conf.get(MAX_RETAINED_STAGES)) { count =>
+ cleanupStages(count)
+ }
+
+ kvstore.onFlush {
+ if (!live) {
+ flush()
+ }
}
override def onApplicationStart(event: SparkListenerApplicationStart): Unit = {
@@ -97,6 +113,7 @@ private[spark] class AppStatusListener(
Seq(attempt))
kvstore.write(new ApplicationInfoWrapper(appInfo))
+ kvstore.write(appSummary)
}
override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = {
@@ -158,10 +175,11 @@ private[spark] class AppStatusListener(
override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = {
liveExecutors.remove(event.executorId).foreach { exec =>
val now = System.nanoTime()
+ activeExecutorCount = math.max(0, activeExecutorCount - 1)
exec.isActive = false
exec.removeTime = new Date(event.time)
exec.removeReason = event.reason
- update(exec, now)
+ update(exec, now, last = true)
// Remove all RDD distributions that reference the removed executor, in case there wasn't
// a corresponding event.
@@ -290,8 +308,11 @@ private[spark] class AppStatusListener(
}
job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None
- update(job, now)
+ update(job, now, last = true)
}
+
+ appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages)
+ kvstore.write(appSummary)
}
override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = {
@@ -350,6 +371,13 @@ private[spark] class AppStatusListener(
job.activeTasks += 1
maybeUpdate(job, now)
}
+
+ if (stage.savedTasks.incrementAndGet() > maxTasksPerStage && !stage.cleaning) {
+ stage.cleaning = true
+ kvstore.doAsync {
+ cleanupTasks(stage)
+ }
+ }
}
liveExecutors.get(event.taskInfo.executorId).foreach { exec =>
@@ -449,6 +477,13 @@ private[spark] class AppStatusListener(
esummary.metrics.update(metricsDelta)
}
maybeUpdate(esummary, now)
+
+ if (!stage.cleaning && stage.savedTasks.get() > maxTasksPerStage) {
+ stage.cleaning = true
+ kvstore.doAsync {
+ cleanupTasks(stage)
+ }
+ }
}
liveExecutors.get(event.taskInfo.executorId).foreach { exec =>
@@ -516,8 +551,11 @@ private[spark] class AppStatusListener(
}
stage.executorSummaries.values.foreach(update(_, now))
- update(stage, now)
+ update(stage, now, last = true)
}
+
+ appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1)
+ kvstore.write(appSummary)
}
override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = {
@@ -573,7 +611,7 @@ private[spark] class AppStatusListener(
}
/** Flush all live entities' data to the underlying store. */
- def flush(): Unit = {
+ private def flush(): Unit = {
val now = System.nanoTime()
liveStages.values.asScala.foreach { stage =>
update(stage, now)
@@ -708,7 +746,10 @@ private[spark] class AppStatusListener(
}
private def getOrCreateExecutor(executorId: String, addTime: Long): LiveExecutor = {
- liveExecutors.getOrElseUpdate(executorId, new LiveExecutor(executorId, addTime))
+ liveExecutors.getOrElseUpdate(executorId, {
+ activeExecutorCount += 1
+ new LiveExecutor(executorId, addTime)
+ })
}
private def updateStreamBlock(event: SparkListenerBlockUpdated, stream: StreamBlockId): Unit = {
@@ -754,8 +795,8 @@ private[spark] class AppStatusListener(
}
}
- private def update(entity: LiveEntity, now: Long): Unit = {
- entity.write(kvstore, now)
+ private def update(entity: LiveEntity, now: Long, last: Boolean = false): Unit = {
+ entity.write(kvstore, now, checkTriggers = last)
}
/** Update a live entity only if it hasn't been updated in the last configured period. */
@@ -772,4 +813,127 @@ private[spark] class AppStatusListener(
}
}
+ private def cleanupExecutors(count: Long): Unit = {
+ // Because the limit is on the number of *dead* executors, we need to calculate whether
+ // there are actually enough dead executors to be deleted.
+ val threshold = conf.get(MAX_RETAINED_DEAD_EXECUTORS)
+ val dead = count - activeExecutorCount
+
+ if (dead > threshold) {
+ val countToDelete = calculateNumberToRemove(dead, threshold)
+ val toDelete = kvstore.view(classOf[ExecutorSummaryWrapper]).index("active")
+ .max(countToDelete).first(false).last(false).asScala.toSeq
+ toDelete.foreach { e => kvstore.delete(e.getClass(), e.info.id) }
+ }
+ }
+
+ private def cleanupJobs(count: Long): Unit = {
+ val countToDelete = calculateNumberToRemove(count, conf.get(MAX_RETAINED_JOBS))
+ if (countToDelete <= 0L) {
+ return
+ }
+
+ val toDelete = KVUtils.viewToSeq(kvstore.view(classOf[JobDataWrapper]),
+ countToDelete.toInt) { j =>
+ j.info.status != JobExecutionStatus.RUNNING && j.info.status != JobExecutionStatus.UNKNOWN
+ }
+ toDelete.foreach { j => kvstore.delete(j.getClass(), j.info.jobId) }
+ }
+
+ private def cleanupStages(count: Long): Unit = {
+ val countToDelete = calculateNumberToRemove(count, conf.get(MAX_RETAINED_STAGES))
+ if (countToDelete <= 0L) {
+ return
+ }
+
+ val stages = KVUtils.viewToSeq(kvstore.view(classOf[StageDataWrapper]),
+ countToDelete.toInt) { s =>
+ s.info.status != v1.StageStatus.ACTIVE && s.info.status != v1.StageStatus.PENDING
+ }
+
+ stages.foreach { s =>
+ val key = s.id
+ kvstore.delete(s.getClass(), key)
+
+ val execSummaries = kvstore.view(classOf[ExecutorStageSummaryWrapper])
+ .index("stage")
+ .first(key)
+ .last(key)
+ .asScala
+ .toSeq
+ execSummaries.foreach { e =>
+ kvstore.delete(e.getClass(), e.id)
+ }
+
+ val tasks = kvstore.view(classOf[TaskDataWrapper])
+ .index("stage")
+ .first(key)
+ .last(key)
+ .asScala
+
+ tasks.foreach { t =>
+ kvstore.delete(t.getClass(), t.info.taskId)
+ }
+
+ // Check whether there are remaining attempts for the same stage. If there aren't, then
+ // also delete the RDD graph data.
+ val remainingAttempts = kvstore.view(classOf[StageDataWrapper])
+ .index("stageId")
+ .first(s.stageId)
+ .last(s.stageId)
+ .closeableIterator()
+
+ val hasMoreAttempts = try {
+ remainingAttempts.asScala.exists { other =>
+ other.info.attemptId != s.info.attemptId
+ }
+ } finally {
+ remainingAttempts.close()
+ }
+
+ if (!hasMoreAttempts) {
+ kvstore.delete(classOf[RDDOperationGraphWrapper], s.stageId)
+ }
+ }
+ }
+
+ private def cleanupTasks(stage: LiveStage): Unit = {
+ val countToDelete = calculateNumberToRemove(stage.savedTasks.get(), maxTasksPerStage).toInt
+ if (countToDelete > 0) {
+ val stageKey = Array(stage.info.stageId, stage.info.attemptId)
+ val view = kvstore.view(classOf[TaskDataWrapper]).index("stage").first(stageKey)
+ .last(stageKey)
+
+ // Try to delete finished tasks only.
+ val toDelete = KVUtils.viewToSeq(view, countToDelete) { t =>
+ !live || t.info.status != TaskState.RUNNING.toString()
+ }
+ toDelete.foreach { t => kvstore.delete(t.getClass(), t.info.taskId) }
+ stage.savedTasks.addAndGet(-toDelete.size)
+
+ // If there are more running tasks than the configured limit, delete running tasks. This
+ // should be extremely rare since the limit should generally far exceed the number of tasks
+ // that can run in parallel.
+ val remaining = countToDelete - toDelete.size
+ if (remaining > 0) {
+ val runningTasksToDelete = view.max(remaining).iterator().asScala.toList
+ runningTasksToDelete.foreach { t => kvstore.delete(t.getClass(), t.info.taskId) }
+ stage.savedTasks.addAndGet(-remaining)
+ }
+ }
+ stage.cleaning = false
+ }
+
+ /**
+ * Remove at least (retainedSize / 10) items to reduce friction. Because tracking may be done
+ * asynchronously, this method may return 0 in case enough items have been deleted already.
+ */
+ private def calculateNumberToRemove(dataSize: Long, retainedSize: Long): Long = {
+ if (dataSize > retainedSize) {
+ math.max(retainedSize / 10L, dataSize - retainedSize)
+ } else {
+ 0L
+ }
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala
index 69ca02ec76293..4cada5c7b0de4 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala
@@ -48,7 +48,7 @@ private[spark] trait AppStatusPlugin {
*/
def setupListeners(
conf: SparkConf,
- store: KVStore,
+ store: ElementTrackingStore,
addListenerFn: SparkListener => Unit,
live: Boolean): Unit
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 22d768b3cb990..9987419b170f6 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -330,6 +330,10 @@ private[spark] class AppStatusStore(
store.read(classOf[PoolData], name)
}
+ def appSummary(): AppSummary = {
+ store.read(classOf[AppSummary], classOf[AppSummary].getName())
+ }
+
def close(): Unit = {
store.close()
}
@@ -347,7 +351,7 @@ private[spark] object AppStatusStore {
* @param addListenerFn Function to register a listener with a bus.
*/
def createLiveStore(conf: SparkConf, addListenerFn: SparkListener => Unit): AppStatusStore = {
- val store = new InMemoryStore()
+ val store = new ElementTrackingStore(new InMemoryStore(), conf)
val listener = new AppStatusListener(store, conf, true)
addListenerFn(listener)
AppStatusPlugin.loadPlugins().foreach { p =>
diff --git a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
new file mode 100644
index 0000000000000..863b0967f765e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status
+
+import java.util.concurrent.TimeUnit
+
+import scala.collection.mutable.{HashMap, ListBuffer}
+
+import com.google.common.util.concurrent.MoreExecutors
+
+import org.apache.spark.SparkConf
+import org.apache.spark.util.{ThreadUtils, Utils}
+import org.apache.spark.util.kvstore._
+
+/**
+ * A KVStore wrapper that allows tracking the number of elements of specific types, and triggering
+ * actions once they reach a threshold. This allows writers, for example, to control how much data
+ * is stored by potentially deleting old data as new data is added.
+ *
+ * This store is used when populating data either from a live UI or an event log. On top of firing
+ * triggers when elements reach a certain threshold, it provides two extra bits of functionality:
+ *
+ * - a generic worker thread that can be used to run expensive tasks asynchronously; the tasks can
+ * be configured to run on the calling thread when more determinism is desired (e.g. unit tests).
+ * - a generic flush mechanism so that listeners can be notified about when they should flush
+ * internal state to the store (e.g. after the SHS finishes parsing an event log).
+ *
+ * The configured triggers are run on a separate thread by default; they can be forced to run on
+ * the calling thread by setting the `ASYNC_TRACKING_ENABLED` configuration to `false`.
+ */
+private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) extends KVStore {
+
+ import config._
+
+ private val triggers = new HashMap[Class[_], Seq[Trigger[_]]]()
+ private val flushTriggers = new ListBuffer[() => Unit]()
+ private val executor = if (conf.get(ASYNC_TRACKING_ENABLED)) {
+ ThreadUtils.newDaemonSingleThreadExecutor("element-tracking-store-worker")
+ } else {
+ MoreExecutors.sameThreadExecutor()
+ }
+
+ @volatile private var stopped = false
+
+ /**
+ * Register a trigger that will be fired once the number of elements of a given type reaches
+ * the given threshold.
+ *
+ * @param klass The type to monitor.
+ * @param threshold The number of elements that should trigger the action.
+ * @param action Action to run when the threshold is reached; takes as a parameter the number
+ * of elements of the registered type currently known to be in the store.
+ */
+ def addTrigger(klass: Class[_], threshold: Long)(action: Long => Unit): Unit = {
+ val existing = triggers.getOrElse(klass, Seq())
+ triggers(klass) = existing :+ Trigger(threshold, action)
+ }
+
+ /**
+ * Adds a trigger to be executed before the store is flushed. This normally happens before
+ * closing, and is useful for flushing intermediate state to the store, e.g. when replaying
+ * in-progress applications through the SHS.
+ *
+ * Flush triggers are called synchronously in the same thread that is closing the store.
+ */
+ def onFlush(action: => Unit): Unit = {
+ flushTriggers += { () => action }
+ }
+
+ /**
+ * Enqueues an action to be executed asynchronously. The task will run on the calling thread if
+ * `ASYNC_TRACKING_ENABLED` is `false`.
+ */
+ def doAsync(fn: => Unit): Unit = {
+ executor.submit(new Runnable() {
+ override def run(): Unit = Utils.tryLog { fn }
+ })
+ }
+
+ override def read[T](klass: Class[T], naturalKey: Any): T = store.read(klass, naturalKey)
+
+ override def write(value: Any): Unit = store.write(value)
+
+ /** Write an element to the store, optionally checking for whether to fire triggers. */
+ def write(value: Any, checkTriggers: Boolean): Unit = {
+ write(value)
+
+ if (checkTriggers && !stopped) {
+ triggers.get(value.getClass()).foreach { list =>
+ doAsync {
+ val count = store.count(value.getClass())
+ list.foreach { t =>
+ if (count > t.threshold) {
+ t.action(count)
+ }
+ }
+ }
+ }
+ }
+ }
+
+ override def delete(klass: Class[_], naturalKey: Any): Unit = store.delete(klass, naturalKey)
+
+ override def getMetadata[T](klass: Class[T]): T = store.getMetadata(klass)
+
+ override def setMetadata(value: Any): Unit = store.setMetadata(value)
+
+ override def view[T](klass: Class[T]): KVStoreView[T] = store.view(klass)
+
+ override def count(klass: Class[_]): Long = store.count(klass)
+
+ override def count(klass: Class[_], index: String, indexedValue: Any): Long = {
+ store.count(klass, index, indexedValue)
+ }
+
+ override def close(): Unit = {
+ close(true)
+ }
+
+ /** A close() method that optionally leaves the parent store open. */
+ def close(closeParent: Boolean): Unit = synchronized {
+ if (stopped) {
+ return
+ }
+
+ stopped = true
+ executor.shutdown()
+ if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+ executor.shutdownNow()
+ }
+
+ flushTriggers.foreach { trigger =>
+ Utils.tryLog(trigger())
+ }
+
+ if (closeParent) {
+ store.close()
+ }
+ }
+
+ private case class Trigger[T](
+ threshold: Long,
+ action: Long => Unit)
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala
index 4638511944c61..99b1843d8e1c0 100644
--- a/core/src/main/scala/org/apache/spark/status/KVUtils.scala
+++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala
@@ -20,6 +20,7 @@ package org.apache.spark.status
import java.io.File
import scala.annotation.meta.getter
+import scala.collection.JavaConverters._
import scala.language.implicitConversions
import scala.reflect.{classTag, ClassTag}
@@ -68,6 +69,19 @@ private[spark] object KVUtils extends Logging {
db
}
+ /** Turns a KVStoreView into a Scala sequence, applying a filter. */
+ def viewToSeq[T](
+ view: KVStoreView[T],
+ max: Int)
+ (filter: T => Boolean): Seq[T] = {
+ val iter = view.closeableIterator()
+ try {
+ iter.asScala.filter(filter).take(max).toList
+ } finally {
+ iter.close()
+ }
+ }
+
private[spark] class MetadataMismatchException extends Exception
}
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index 983c58a607aa8..52e83f250d34e 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -18,6 +18,7 @@
package org.apache.spark.status
import java.util.Date
+import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.HashMap
@@ -38,10 +39,12 @@ import org.apache.spark.util.kvstore.KVStore
*/
private[spark] abstract class LiveEntity {
- var lastWriteTime = 0L
+ var lastWriteTime = -1L
- def write(store: KVStore, now: Long): Unit = {
- store.write(doUpdate())
+ def write(store: ElementTrackingStore, now: Long, checkTriggers: Boolean = false): Unit = {
+ // Always check triggers on the first write, since adding an element to the store may
+ // cause the maximum count for the element type to be exceeded.
+ store.write(doUpdate(), checkTriggers || lastWriteTime == -1L)
lastWriteTime = now
}
@@ -403,6 +406,10 @@ private class LiveStage extends LiveEntity {
val executorSummaries = new HashMap[String, LiveExecutorStageSummary]()
+ // Used for cleanup of tasks after they reach the configured limit. Not written to the store.
+ @volatile var cleaning = false
+ var savedTasks = new AtomicInteger(0)
+
def executorSummary(executorId: String): LiveExecutorStageSummary = {
executorSummaries.getOrElseUpdate(executorId,
new LiveExecutorStageSummary(info.stageId, info.attemptId, executorId))
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
index b3561109bc636..3b879545b3d2e 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala
@@ -59,7 +59,15 @@ private[v1] class StagesResource extends BaseAppResource {
ui.store.stageAttempt(stageId, stageAttemptId, details = details)
} catch {
case _: NoSuchElementException =>
- throw new NotFoundException(s"unknown attempt $stageAttemptId for stage $stageId.")
+ // Change the message depending on whether there are any attempts for the requested stage.
+ val all = ui.store.stageData(stageId)
+ val msg = if (all.nonEmpty) {
+ val ids = all.map(_.attemptId)
+ s"unknown attempt for stage $stageId. Found attempts: [${ids.mkString(",")}]"
+ } else {
+ s"unknown stage: $stageId"
+ }
+ throw new NotFoundException(msg)
}
}
diff --git a/core/src/main/scala/org/apache/spark/status/config.scala b/core/src/main/scala/org/apache/spark/status/config.scala
index 7af9dff977a86..67801b8f046f4 100644
--- a/core/src/main/scala/org/apache/spark/status/config.scala
+++ b/core/src/main/scala/org/apache/spark/status/config.scala
@@ -23,10 +23,30 @@ import org.apache.spark.internal.config._
private[spark] object config {
+ val ASYNC_TRACKING_ENABLED = ConfigBuilder("spark.appStateStore.asyncTracking.enable")
+ .booleanConf
+ .createWithDefault(true)
+
val LIVE_ENTITY_UPDATE_PERIOD = ConfigBuilder("spark.ui.liveUpdate.period")
.timeConf(TimeUnit.NANOSECONDS)
.createWithDefaultString("100ms")
+ val MAX_RETAINED_JOBS = ConfigBuilder("spark.ui.retainedJobs")
+ .intConf
+ .createWithDefault(1000)
+
+ val MAX_RETAINED_STAGES = ConfigBuilder("spark.ui.retainedStages")
+ .intConf
+ .createWithDefault(1000)
+
+ val MAX_RETAINED_TASKS_PER_STAGE = ConfigBuilder("spark.ui.retainedTasks")
+ .intConf
+ .createWithDefault(100000)
+
+ val MAX_RETAINED_DEAD_EXECUTORS = ConfigBuilder("spark.ui.retainedDeadExecutors")
+ .intConf
+ .createWithDefault(100)
+
val MAX_RETAINED_ROOT_NODES = ConfigBuilder("spark.ui.dagGraph.retainedRootRDDs")
.intConf
.createWithDefault(Int.MaxValue)
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
index c1ea87542d6cc..d9ead0071d3bf 100644
--- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -112,6 +112,9 @@ private[spark] class TaskDataWrapper(
Array(stageId: JInteger, stageAttemptId: JInteger, info.launchTime.getTime(): JLong)
}
+ @JsonIgnore @KVIndex("active")
+ def active: Boolean = info.duration.isEmpty
+
}
private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) {
@@ -187,3 +190,16 @@ private[spark] class RDDOperationGraphWrapper(
private[spark] class PoolData(
@KVIndexParam val name: String,
val stageIds: Set[Int])
+
+/**
+ * A class with information about an app, to be used by the UI. There's only one instance of
+ * this summary per application, so its ID in the store is the class name.
+ */
+private[spark] class AppSummary(
+ val numCompletedJobs: Int,
+ val numCompletedStages: Int) {
+
+ @KVIndex
+ def id: String = classOf[AppSummary].getName()
+
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 35da3c3bfd1a2..b44ac0ea1febc 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -154,8 +154,6 @@ private[spark] object SparkUI {
val DEFAULT_PORT = 4040
val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
val DEFAULT_POOL_NAME = "default"
- val DEFAULT_RETAINED_STAGES = 1000
- val DEFAULT_RETAINED_JOBS = 1000
def getUIPort(conf: SparkConf): Int = {
conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT)
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index b60d39b21b4bf..37e3b3b304a63 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -300,7 +300,13 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We
val shouldShowCompletedJobs = completedJobs.nonEmpty
val shouldShowFailedJobs = failedJobs.nonEmpty
- val completedJobNumStr = s"${completedJobs.size}"
+ val appSummary = store.appSummary()
+ val completedJobNumStr = if (completedJobs.size == appSummary.numCompletedJobs) {
+ s"${completedJobs.size}"
+ } else {
+ s"${appSummary.numCompletedJobs}, only showing ${completedJobs.size}"
+ }
+
val schedulingMode = store.environmentInfo().sparkProperties.toMap
.get("spark.scheduler.mode")
.map { mode => SchedulingMode.withName(mode).toString }
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
index e4cf99e7b9e04..b1e343451e28e 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
@@ -39,7 +39,6 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
val completedStages = allStages.filter(_.status == StageStatus.COMPLETE)
val failedStages = allStages.filter(_.status == StageStatus.FAILED).reverse
- val numCompletedStages = completedStages.size
val numFailedStages = failedStages.size
val subPath = "stages"
@@ -69,10 +68,11 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
val shouldShowCompletedStages = completedStages.nonEmpty
val shouldShowFailedStages = failedStages.nonEmpty
- val completedStageNumStr = if (numCompletedStages == completedStages.size) {
- s"$numCompletedStages"
+ val appSummary = parent.store.appSummary()
+ val completedStageNumStr = if (appSummary.numCompletedStages == completedStages.size) {
+ s"${appSummary.numCompletedStages}"
} else {
- s"$numCompletedStages, only showing ${completedStages.size}"
+ s"${appSummary.numCompletedStages}, only showing ${completedStages.size}"
}
val summary: NodeSeq =
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 8871870eb8681..5853302973140 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -2650,15 +2650,29 @@ private[spark] object Utils extends Logging {
redact(redactionPattern, kvs)
}
+ /**
+ * Redact the sensitive values in the given map. If a map key matches the redaction pattern then
+ * its value is replaced with a dummy text.
+ */
+ def redact(regex: Option[Regex], kvs: Seq[(String, String)]): Seq[(String, String)] = {
+ regex match {
+ case None => kvs
+ case Some(r) => redact(r, kvs)
+ }
+ }
+
/**
* Redact the sensitive information in the given string.
*/
- def redact(conf: SparkConf, text: String): String = {
- if (text == null || text.isEmpty || conf == null || !conf.contains(STRING_REDACTION_PATTERN)) {
- text
- } else {
- val regex = conf.get(STRING_REDACTION_PATTERN).get
- regex.replaceAllIn(text, REDACTION_REPLACEMENT_TEXT)
+ def redact(regex: Option[Regex], text: String): String = {
+ regex match {
+ case None => text
+ case Some(r) =>
+ if (text == null || text.isEmpty) {
+ text
+ } else {
+ r.replaceAllIn(text, REDACTION_REPLACEMENT_TEXT)
+ }
}
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 2eb8a1fee104c..27dd435332348 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -396,7 +396,7 @@ class SparkSubmitSuite
"--class", "org.SomeClass",
"--driver-memory", "4g",
"--conf", "spark.kubernetes.namespace=spark",
- "--conf", "spark.kubernetes.driver.docker.image=bar",
+ "--conf", "spark.kubernetes.driver.container.image=bar",
"/home/thejar.jar",
"arg1")
val appArgs = new SparkSubmitArguments(clArgs)
@@ -412,7 +412,7 @@ class SparkSubmitSuite
conf.get("spark.executor.memory") should be ("5g")
conf.get("spark.driver.memory") should be ("4g")
conf.get("spark.kubernetes.namespace") should be ("spark")
- conf.get("spark.kubernetes.driver.docker.image") should be ("bar")
+ conf.get("spark.kubernetes.driver.container.image") should be ("bar")
}
test("handles confs with flag equivalents") {
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 3a9790cd57270..3738f85da5831 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -264,7 +264,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1")
badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND)
- badStageAttemptId._3 should be (Some("unknown attempt 1 for stage 1."))
+ badStageAttemptId._3 should be (Some("unknown attempt for stage 1. Found attempts: [0]"))
val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam")
badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND)
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index 88fe6bd70a14e..9cf4f7efb24a8 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -39,16 +39,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
import config._
- private val conf = new SparkConf().set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
+ private val conf = new SparkConf()
+ .set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
+ .set(ASYNC_TRACKING_ENABLED, false)
private var time: Long = _
private var testDir: File = _
- private var store: KVStore = _
+ private var store: ElementTrackingStore = _
+ private var taskIdTracker = -1L
before {
time = 0L
testDir = Utils.createTempDir()
- store = KVUtils.open(testDir, getClass().getName())
+ store = new ElementTrackingStore(KVUtils.open(testDir, getClass().getName()), conf)
+ taskIdTracker = -1L
}
after {
@@ -185,22 +189,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
// Start tasks from stage 1
time += 1
- var _taskIdTracker = -1L
- def nextTaskId(): Long = {
- _taskIdTracker += 1
- _taskIdTracker
- }
-
- def createTasks(count: Int, time: Long): Seq[TaskInfo] = {
- (1 to count).map { id =>
- val exec = execIds(id.toInt % execIds.length)
- val taskId = nextTaskId()
- new TaskInfo(taskId, taskId.toInt, 1, time, exec, s"$exec.example.com",
- TaskLocality.PROCESS_LOCAL, id % 2 == 0)
- }
- }
- val s1Tasks = createTasks(4, time)
+ val s1Tasks = createTasks(4, execIds)
s1Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, task))
}
@@ -419,7 +409,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
// Start and fail all tasks of stage 2.
time += 1
- val s2Tasks = createTasks(4, time)
+ val s2Tasks = createTasks(4, execIds)
s2Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId, stages.last.attemptId, task))
}
@@ -470,7 +460,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
listener.onStageSubmitted(SparkListenerStageSubmitted(newS2, jobProps))
assert(store.count(classOf[StageDataWrapper]) === 3)
- val newS2Tasks = createTasks(4, time)
+ val newS2Tasks = createTasks(4, execIds)
newS2Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptId, task))
@@ -526,7 +516,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
assert(store.count(classOf[StageDataWrapper]) === 5)
time += 1
- val j2s2Tasks = createTasks(4, time)
+ val j2s2Tasks = createTasks(4, execIds)
j2s2Tasks.foreach { task =>
listener.onTaskStart(SparkListenerTaskStart(j2Stages.last.stageId, j2Stages.last.attemptId,
@@ -587,8 +577,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
// Stop executors.
- listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "1", "Test"))
- listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "2", "Test"))
+ time += 1
+ listener.onExecutorRemoved(SparkListenerExecutorRemoved(time, "1", "Test"))
+ listener.onExecutorRemoved(SparkListenerExecutorRemoved(time, "2", "Test"))
Seq("1", "2").foreach { id =>
check[ExecutorSummaryWrapper](id) { exec =>
@@ -851,6 +842,103 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
}
}
+ test("eviction of old data") {
+ val testConf = conf.clone()
+ .set(MAX_RETAINED_JOBS, 2)
+ .set(MAX_RETAINED_STAGES, 2)
+ .set(MAX_RETAINED_TASKS_PER_STAGE, 2)
+ .set(MAX_RETAINED_DEAD_EXECUTORS, 1)
+ val listener = new AppStatusListener(store, testConf, true)
+
+ // Start 3 jobs, all should be kept. Stop one, it should be evicted.
+ time += 1
+ listener.onJobStart(SparkListenerJobStart(1, time, Nil, null))
+ listener.onJobStart(SparkListenerJobStart(2, time, Nil, null))
+ listener.onJobStart(SparkListenerJobStart(3, time, Nil, null))
+ assert(store.count(classOf[JobDataWrapper]) === 3)
+
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded))
+ assert(store.count(classOf[JobDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[JobDataWrapper], 2)
+ }
+
+ // Start 3 stages, all should be kept. Stop 2 of them, the stopped one with the lowest id should
+ // be deleted. Start a new attempt of the second stopped one, and verify that the stage graph
+ // data is not deleted.
+ time += 1
+ val stages = Seq(
+ new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"),
+ new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2"),
+ new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3"))
+
+ // Graph data is generated by the job start event, so fire it.
+ listener.onJobStart(SparkListenerJobStart(4, time, stages, null))
+
+ stages.foreach { s =>
+ time += 1
+ s.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(s, new Properties()))
+ }
+
+ assert(store.count(classOf[StageDataWrapper]) === 3)
+ assert(store.count(classOf[RDDOperationGraphWrapper]) === 3)
+
+ stages.drop(1).foreach { s =>
+ time += 1
+ s.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(s))
+ }
+
+ assert(store.count(classOf[StageDataWrapper]) === 2)
+ assert(store.count(classOf[RDDOperationGraphWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[StageDataWrapper], Array(2, 0))
+ }
+
+ val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3")
+ time += 1
+ attempt2.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(attempt2, new Properties()))
+
+ assert(store.count(classOf[StageDataWrapper]) === 2)
+ assert(store.count(classOf[RDDOperationGraphWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[StageDataWrapper], Array(2, 0))
+ }
+ intercept[NoSuchElementException] {
+ store.read(classOf[StageDataWrapper], Array(3, 0))
+ }
+ store.read(classOf[StageDataWrapper], Array(3, 1))
+
+ // Start 2 tasks. Finish the second one.
+ time += 1
+ val tasks = createTasks(2, Array("1"))
+ tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ }
+ assert(store.count(classOf[TaskDataWrapper]) === 2)
+
+ // Start a 3rd task. The finished tasks should be deleted.
+ createTasks(1, Array("1")).foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ }
+ assert(store.count(classOf[TaskDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[TaskDataWrapper], tasks.last.id)
+ }
+
+ // Start a 4th task. The first task should be deleted, even if it's still running.
+ createTasks(1, Array("1")).foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(attempt2.stageId, attempt2.attemptId, task))
+ }
+ assert(store.count(classOf[TaskDataWrapper]) === 2)
+ intercept[NoSuchElementException] {
+ store.read(classOf[TaskDataWrapper], tasks.head.id)
+ }
+ }
+
private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptId)
private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = {
@@ -864,6 +952,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative)
}
+ private def createTasks(count: Int, execs: Array[String]): Seq[TaskInfo] = {
+ (1 to count).map { id =>
+ val exec = execs(id.toInt % execs.length)
+ val taskId = nextTaskId()
+ new TaskInfo(taskId, taskId.toInt, 1, time, exec, s"$exec.example.com",
+ TaskLocality.PROCESS_LOCAL, id % 2 == 0)
+ }
+ }
+
+ private def nextTaskId(): Long = {
+ taskIdTracker += 1
+ taskIdTracker
+ }
+
private case class RddBlock(
rddId: Int,
partId: Int,
diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
new file mode 100644
index 0000000000000..07a7b58404c29
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status
+
+import org.mockito.Mockito._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.util.kvstore._
+
+class ElementTrackingStoreSuite extends SparkFunSuite {
+
+ import config._
+
+ test("tracking for multiple types") {
+ val store = mock(classOf[KVStore])
+ val tracking = new ElementTrackingStore(store, new SparkConf()
+ .set(ASYNC_TRACKING_ENABLED, false))
+
+ var type1 = 0L
+ var type2 = 0L
+ var flushed = false
+
+ tracking.addTrigger(classOf[Type1], 100) { count =>
+ type1 = count
+ }
+ tracking.addTrigger(classOf[Type2], 1000) { count =>
+ type2 = count
+ }
+ tracking.onFlush {
+ flushed = true
+ }
+
+ when(store.count(classOf[Type1])).thenReturn(1L)
+ tracking.write(new Type1, true)
+ assert(type1 === 0L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type1])).thenReturn(100L)
+ tracking.write(new Type1, true)
+ assert(type1 === 0L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type1])).thenReturn(101L)
+ tracking.write(new Type1, true)
+ assert(type1 === 101L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type1])).thenReturn(200L)
+ tracking.write(new Type1, true)
+ assert(type1 === 200L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type2])).thenReturn(500L)
+ tracking.write(new Type2, true)
+ assert(type1 === 200L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type2])).thenReturn(1000L)
+ tracking.write(new Type2, true)
+ assert(type1 === 200L)
+ assert(type2 === 0L)
+
+ when(store.count(classOf[Type2])).thenReturn(2000L)
+ tracking.write(new Type2, true)
+ assert(type1 === 200L)
+ assert(type2 === 2000L)
+
+ tracking.close(false)
+ assert(flushed)
+ verify(store, never()).close()
+ }
+
+ private class Type1
+ private class Type2
+
+}
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index df5f0b5335e82..326546787ab6c 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -42,6 +42,7 @@ import org.apache.spark.deploy.history.HistoryServerSuite
import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus}
+import org.apache.spark.status.config._
private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler {
@@ -525,14 +526,15 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
}
}
- ignore("stage & job retention") {
+ test("stage & job retention") {
val conf = new SparkConf()
.setMaster("local")
.setAppName("test")
.set("spark.ui.enabled", "true")
.set("spark.ui.port", "0")
- .set("spark.ui.retainedStages", "3")
- .set("spark.ui.retainedJobs", "2")
+ .set(MAX_RETAINED_STAGES, 3)
+ .set(MAX_RETAINED_JOBS, 2)
+ .set(ASYNC_TRACKING_ENABLED, false)
val sc = new SparkContext(conf)
assert(sc.ui.isDefined)
diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
index 48a824499acb9..7245163ea2a51 100755
--- a/dev/make-distribution.sh
+++ b/dev/make-distribution.sh
@@ -168,12 +168,18 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE"
# Copy jars
cp "$SPARK_HOME"/assembly/target/scala*/jars/* "$DISTDIR/jars/"
-# Only create the yarn directory if the yarn artifacts were build.
+# Only create the yarn directory if the yarn artifacts were built.
if [ -f "$SPARK_HOME"/common/network-yarn/target/scala*/spark-*-yarn-shuffle.jar ]; then
mkdir "$DISTDIR/yarn"
cp "$SPARK_HOME"/common/network-yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/yarn"
fi
+# Only create and copy the dockerfiles directory if the kubernetes artifacts were built.
+if [ -d "$SPARK_HOME"/resource-managers/kubernetes/core/target/ ]; then
+ mkdir -p "$DISTDIR/kubernetes/"
+ cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src/main/dockerfiles "$DISTDIR/kubernetes/"
+fi
+
# Copy examples and dependencies
mkdir -p "$DISTDIR/examples/jars"
cp "$SPARK_HOME"/examples/target/scala*/jars/* "$DISTDIR/examples/jars"
diff --git a/dev/run-tests.py b/dev/run-tests.py
index ef0e788a91606..7e6f7ff060351 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -253,9 +253,9 @@ def kill_zinc_on_port(zinc_port):
"""
Kill the Zinc process running on the given port, if one exists.
"""
- cmd = ("/usr/sbin/lsof -P |grep %s | grep LISTEN "
- "| awk '{ print $2; }' | xargs kill") % zinc_port
- subprocess.check_call(cmd, shell=True)
+ cmd = "%s -P |grep %s | grep LISTEN | awk '{ print $2; }' | xargs kill"
+ lsof_exe = which("lsof")
+ subprocess.check_call(cmd % (lsof_exe if lsof_exe else "/usr/sbin/lsof", zinc_port), shell=True)
def exec_maven(mvn_args=()):
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 67b05ecf7a858..e5af5ae4561c7 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -99,6 +99,7 @@
Spark Standalone
Mesos
YARN
+ Kubernetes
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 98f7df155456f..c391255a91596 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -49,7 +49,7 @@ To create a Spark distribution like those distributed by the
to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured
with Maven profile settings and so on like the direct Maven build. Example:
- ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn
+ ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes
This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help`
@@ -90,6 +90,10 @@ like ZooKeeper and Hadoop itself.
## Building with Mesos support
./build/mvn -Pmesos -DskipTests clean package
+
+## Building with Kubernetes support
+
+ ./build/mvn -Pkubernetes -DskipTests clean package
## Building with Kafka 0.8 support
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index c42bb4bb8377e..658e67f99dd71 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -52,11 +52,8 @@ The system currently supports three cluster managers:
* [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
and service applications.
* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
-* [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- In addition to the above,
-there is experimental support for Kubernetes. Kubernetes is an open-source platform
-for providing container-centric infrastructure. Kubernetes support is being actively
-developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization.
-For documentation, refer to that project's README.
+* [Kubernetes](running-on-kubernetes.html) -- [Kubernetes](https://kubernetes.io/docs/concepts/overview/what-is-kubernetes/)
+is an open-source platform that provides container-centric infrastructure.
A third-party project (not supported by the Spark project) exists to add support for
[Nomad](https://github.com/hashicorp/nomad-spark) as a cluster manager.
diff --git a/docs/configuration.md b/docs/configuration.md
index d70bac134808f..1189aea2aa71f 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -2376,6 +2376,8 @@ can be found on the pages for each mode:
#### [Mesos](running-on-mesos.html#configuration)
+#### [Kubernetes](running-on-kubernetes.html#configuration)
+
#### [Standalone Mode](spark-standalone.html#cluster-launch-scripts)
# Environment Variables
diff --git a/docs/img/k8s-cluster-mode.png b/docs/img/k8s-cluster-mode.png
new file mode 100644
index 0000000000000..12a6288ed5823
Binary files /dev/null and b/docs/img/k8s-cluster-mode.png differ
diff --git a/docs/index.md b/docs/index.md
index b867c972b4b48..2f009417fafb0 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -81,6 +81,7 @@ options for deployment:
* [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster
* [Apache Mesos](running-on-mesos.html)
* [Hadoop YARN](running-on-yarn.html)
+* [Kubernetes](running-on-kubernetes.html)
# Where to Go from Here
@@ -112,7 +113,7 @@ options for deployment:
* [Mesos](running-on-mesos.html): deploy a private cluster using
[Apache Mesos](http://mesos.apache.org)
* [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
- * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark): deploy Spark on top of Kubernetes
+ * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes
**Other Documents:**
diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
new file mode 100644
index 0000000000000..0048bd90b48ae
--- /dev/null
+++ b/docs/running-on-kubernetes.md
@@ -0,0 +1,578 @@
+---
+layout: global
+title: Running Spark on Kubernetes
+---
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+Spark can run on clusters managed by [Kubernetes](https://kubernetes.io). This feature makes use of native
+Kubernetes scheduler that has been added to Spark.
+
+# Prerequisites
+
+* A runnable distribution of Spark 2.3 or above.
+* A running Kubernetes cluster at version >= 1.6 with access configured to it using
+[kubectl](https://kubernetes.io/docs/user-guide/prereqs/). If you do not already have a working Kubernetes cluster,
+you may setup a test cluster on your local machine using
+[minikube](https://kubernetes.io/docs/getting-started-guides/minikube/).
+ * We recommend using the latest release of minikube with the DNS addon enabled.
+* You must have appropriate permissions to list, create, edit and delete
+[pods](https://kubernetes.io/docs/user-guide/pods/) in your cluster. You can verify that you can list these resources
+by running `kubectl auth can-i pods`.
+ * The service account credentials used by the driver pods must be allowed to create pods, services and configmaps.
+* You must have [Kubernetes DNS](https://kubernetes.io/docs/concepts/services-networking/dns-pod-service/) configured in your cluster.
+
+# How it works
+
+
+
+
+
+spark-submit
can be directly used to submit a Spark application to a Kubernetes cluster.
+The submission mechanism works as follows:
+
+* Spark creates a Spark driver running within a [Kubernetes pod](https://kubernetes.io/docs/concepts/workloads/pods/pod/).
+* The driver creates executors which are also running within Kubernetes pods and connects to them, and executes application code.
+* When the application completes, the executor pods terminate and are cleaned up, but the driver pod persists
+logs and remains in "completed" state in the Kubernetes API until it's eventually garbage collected or manually cleaned up.
+
+Note that in the completed state, the driver pod does *not* use any computational or memory resources.
+
+The driver and executor pod scheduling is handled by Kubernetes. It will be possible to affect Kubernetes scheduling
+decisions for driver and executor pods using advanced primitives like
+[node selectors](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector)
+and [node/pod affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity)
+in a future release.
+
+# Submitting Applications to Kubernetes
+
+## Docker Images
+
+Kubernetes requires users to supply images that can be deployed into containers within pods. The images are built to
+be run in a container runtime environment that Kubernetes supports. Docker is a container runtime environment that is
+frequently used with Kubernetes. With Spark 2.3, there are Dockerfiles provided in the runnable distribution that can be customized
+and built for your usage.
+
+You may build these docker images from sources.
+There is a script, `sbin/build-push-docker-images.sh` that you can use to build and push
+customized Spark distribution images consisting of all the above components.
+
+Example usage is:
+
+ ./sbin/build-push-docker-images.sh -r -t my-tag build
+ ./sbin/build-push-docker-images.sh -r -t my-tag push
+
+Docker files are under the `kubernetes/dockerfiles/` directory and can be customized further before
+building using the supplied script, or manually.
+
+## Cluster Mode
+
+To launch Spark Pi in cluster mode,
+
+{% highlight bash %}
+$ bin/spark-submit \
+ --master k8s://https://: \
+ --deploy-mode cluster \
+ --name spark-pi \
+ --class org.apache.spark.examples.SparkPi \
+ --conf spark.executor.instances=5 \
+ --conf spark.kubernetes.driver.docker.image= \
+ --conf spark.kubernetes.executor.docker.image= \
+ local:///path/to/examples.jar
+{% endhighlight %}
+
+The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting
+`spark.master` in the application's configuration, must be a URL with the format `k8s://`. Prefixing the
+master string with `k8s://` will cause the Spark application to launch on the Kubernetes cluster, with the API server
+being contacted at `api_server_url`. If no HTTP protocol is specified in the URL, it defaults to `https`. For example,
+setting the master to `k8s://example.com:443` is equivalent to setting it to `k8s://https://example.com:443`, but to
+connect without TLS on a different port, the master would be set to `k8s://http://example.com:8080`.
+
+In Kubernetes mode, the Spark application name that is specified by `spark.app.name` or the `--name` argument to
+`spark-submit` is used by default to name the Kubernetes resources created like drivers and executors. So, application names
+must consist of lower case alphanumeric characters, `-`, and `.` and must start and end with an alphanumeric character.
+
+If you have a Kubernetes cluster setup, one way to discover the apiserver URL is by executing `kubectl cluster-info`.
+
+```bash
+kubectl cluster-info
+Kubernetes master is running at http://127.0.0.1:6443
+```
+
+In the above example, the specific Kubernetes cluster can be used with spark-submit
by specifying
+`--master k8s://http://127.0.0.1:6443` as an argument to spark-submit. Additionally, it is also possible to use the
+authenticating proxy, `kubectl proxy` to communicate to the Kubernetes API.
+
+The local proxy can be started by:
+
+```bash
+kubectl proxy
+```
+
+If the local proxy is running at localhost:8001, `--master k8s://http://127.0.0.1:8001` can be used as the argument to
+spark-submit. Finally, notice that in the above example we specify a jar with a specific URI with a scheme of `local://`.
+This URI is the location of the example jar that is already in the Docker image.
+
+## Dependency Management
+
+If your application's dependencies are all hosted in remote locations like HDFS or HTTP servers, they may be referred to
+by their appropriate remote URIs. Also, application dependencies can be pre-mounted into custom-built Docker images.
+Those dependencies can be added to the classpath by referencing them with `local://` URIs and/or setting the
+`SPARK_EXTRA_CLASSPATH` environment variable in your Dockerfiles.
+
+## Introspection and Debugging
+
+These are the different ways in which you can investigate a running/completed Spark application, monitor progress, and
+take actions.
+
+### Accessing Logs
+
+Logs can be accessed using the Kubernetes API and the `kubectl` CLI. When a Spark application is running, it's possible
+to stream logs from the application using:
+
+```bash
+kubectl -n= logs -f
+```
+
+The same logs can also be accessed through the
+[Kubernetes dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) if installed on
+the cluster.
+
+### Accessing Driver UI
+
+The UI associated with any application can be accessed locally using
+[`kubectl port-forward`](https://kubernetes.io/docs/tasks/access-application-cluster/port-forward-access-application-cluster/#forward-a-local-port-to-a-port-on-the-pod).
+
+```bash
+kubectl port-forward 4040:4040
+```
+
+Then, the Spark driver UI can be accessed on `http://localhost:4040`.
+
+### Debugging
+
+There may be several kinds of failures. If the Kubernetes API server rejects the request made from spark-submit, or the
+connection is refused for a different reason, the submission logic should indicate the error encountered. However, if there
+are errors during the running of the application, often, the best way to investigate may be through the Kubernetes CLI.
+
+To get some basic information about the scheduling decisions made around the driver pod, you can run:
+
+```bash
+kubectl describe pod
+```
+
+If the pod has encountered a runtime error, the status can be probed further using:
+
+```bash
+kubectl logs
+```
+
+Status and logs of failed executor pods can be checked in similar ways. Finally, deleting the driver pod will clean up the entire spark
+application, includling all executors, associated service, etc. The driver pod can be thought of as the Kubernetes representation of
+the Spark application.
+
+## Kubernetes Features
+
+### Namespaces
+
+Kubernetes has the concept of [namespaces](https://kubernetes.io/docs/concepts/overview/working-with-objects/namespaces/).
+Namespaces are ways to divide cluster resources between multiple users (via resource quota). Spark on Kubernetes can
+use namespaces to launch Spark applications. This can be made use of through the `spark.kubernetes.namespace` configuration.
+
+Kubernetes allows using [ResourceQuota](https://kubernetes.io/docs/concepts/policy/resource-quotas/) to set limits on
+resources, number of objects, etc on individual namespaces. Namespaces and ResourceQuota can be used in combination by
+administrator to control sharing and resource allocation in a Kubernetes cluster running Spark applications.
+
+### RBAC
+
+In Kubernetes clusters with [RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) enabled, users can configure
+Kubernetes RBAC roles and service accounts used by the various Spark on Kubernetes components to access the Kubernetes
+API server.
+
+The Spark driver pod uses a Kubernetes service account to access the Kubernetes API server to create and watch executor
+pods. The service account used by the driver pod must have the appropriate permission for the driver to be able to do
+its work. Specifically, at minimum, the service account must be granted a
+[`Role` or `ClusterRole`](https://kubernetes.io/docs/admin/authorization/rbac/#role-and-clusterrole) that allows driver
+pods to create pods and services. By default, the driver pod is automatically assigned the `default` service account in
+the namespace specified by `spark.kubernetes.namespace`, if no service account is specified when the pod gets created.
+
+Depending on the version and setup of Kubernetes deployed, this `default` service account may or may not have the role
+that allows driver pods to create pods and services under the default Kubernetes
+[RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) policies. Sometimes users may need to specify a custom
+service account that has the right role granted. Spark on Kubernetes supports specifying a custom service account to
+be used by the driver pod through the configuration property
+`spark.kubernetes.authenticate.driver.serviceAccountName=`. For example to make the driver pod
+use the `spark` service account, a user simply adds the following option to the `spark-submit` command:
+
+```
+--conf spark.kubernetes.authenticate.driver.serviceAccountName=spark
+```
+
+To create a custom service account, a user can use the `kubectl create serviceaccount` command. For example, the
+following command creates a service account named `spark`:
+
+```bash
+kubectl create serviceaccount spark
+```
+
+To grant a service account a `Role` or `ClusterRole`, a `RoleBinding` or `ClusterRoleBinding` is needed. To create
+a `RoleBinding` or `ClusterRoleBinding`, a user can use the `kubectl create rolebinding` (or `clusterrolebinding`
+for `ClusterRoleBinding`) command. For example, the following command creates an `edit` `ClusterRole` in the `default`
+namespace and grants it to the `spark` service account created above:
+
+```bash
+kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=default:spark --namespace=default
+```
+
+Note that a `Role` can only be used to grant access to resources (like pods) within a single namespace, whereas a
+`ClusterRole` can be used to grant access to cluster-scoped resources (like nodes) as well as namespaced resources
+(like pods) across all namespaces. For Spark on Kubernetes, since the driver always creates executor pods in the
+same namespace, a `Role` is sufficient, although users may use a `ClusterRole` instead. For more information on
+RBAC authorization and how to configure Kubernetes service accounts for pods, please refer to
+[Using RBAC Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and
+[Configure Service Accounts for Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/).
+
+## Client Mode
+
+Client mode is not currently supported.
+
+## Future Work
+
+There are several Spark on Kubernetes features that are currently being incubated in a fork -
+[apache-spark-on-k8s/spark](https://github.com/apache-spark-on-k8s/spark), which are expected to eventually make it into
+future versions of the spark-kubernetes integration.
+
+Some of these include:
+
+* PySpark
+* R
+* Dynamic Executor Scaling
+* Local File Dependency Management
+* Spark Application Management
+* Job Queues and Resource Management
+
+You can refer to the [documentation](https://apache-spark-on-k8s.github.io/userdocs/) if you want to try these features
+and provide feedback to the development team.
+
+# Configuration
+
+See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are
+specific to Spark on Kubernetes.
+
+#### Spark Properties
+
+
+Property Name | Default | Meaning |
+
+ spark.kubernetes.namespace |
+ default |
+
+ The namespace that will be used for running the driver and executor pods.
+ |
+
+
+ spark.kubernetes.driver.container.image |
+ (none) |
+
+ Container image to use for the driver.
+ This is usually of the form `example.com/repo/spark-driver:v1.0.0`.
+ This configuration is required and must be provided by the user.
+ |
+
+
+ spark.kubernetes.executor.container.image |
+ (none) |
+
+ Container image to use for the executors.
+ This is usually of the form `example.com/repo/spark-executor:v1.0.0`.
+ This configuration is required and must be provided by the user.
+ |
+
+
+ spark.kubernetes.container.image.pullPolicy |
+ IfNotPresent |
+
+ Container image pull policy used when pulling images within Kubernetes.
+ |
+
+
+ spark.kubernetes.allocation.batch.size |
+ 5 |
+
+ Number of pods to launch at once in each round of executor pod allocation.
+ |
+
+
+ spark.kubernetes.allocation.batch.delay |
+ 1s |
+
+ Time to wait between each round of executor pod allocation. Specifying values less than 1 second may lead to
+ excessive CPU usage on the spark driver.
+ |
+
+
+ spark.kubernetes.authenticate.submission.caCertFile |
+ (none) |
+
+ Path to the CA cert file for connecting to the Kubernetes API server over TLS when starting the driver. This file
+ must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide
+ a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.submission.clientKeyFile |
+ (none) |
+
+ Path to the client key file for authenticating against the Kubernetes API server when starting the driver. This file
+ must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide
+ a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.submission.clientCertFile |
+ (none) |
+
+ Path to the client cert file for authenticating against the Kubernetes API server when starting the driver. This
+ file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not
+ provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.submission.oauthToken |
+ (none) |
+
+ OAuth token to use when authenticating against the Kubernetes API server when starting the driver. Note
+ that unlike the other authentication options, this is expected to be the exact string value of the token to use for
+ the authentication.
+ |
+
+
+ spark.kubernetes.authenticate.submission.oauthTokenFile |
+ (none) |
+
+ Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server when starting the driver.
+ This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not
+ provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.caCertFile |
+ (none) |
+
+ Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting
+ executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod.
+ Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.clientKeyFile |
+ (none) |
+
+ Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting
+ executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod.
+ Specify this as a path as opposed to a URI (i.e. do not provide a scheme). If this is specified, it is highly
+ recommended to set up TLS for the driver submission server, as this value is sensitive information that would be
+ passed to the driver pod in plaintext otherwise.
+ |
+
+
+ spark.kubernetes.authenticate.driver.clientCertFile |
+ (none) |
+
+ Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when
+ requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the
+ driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.oauthToken |
+ (none) |
+
+ OAuth token to use when authenticating against the Kubernetes API server from the driver pod when
+ requesting executors. Note that unlike the other authentication options, this must be the exact string value of
+ the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is
+ highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would
+ be passed to the driver pod in plaintext otherwise.
+ |
+
+
+ spark.kubernetes.authenticate.driver.oauthTokenFile |
+ (none) |
+
+ Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server from the driver pod when
+ requesting executors. Note that unlike the other authentication options, this file must contain the exact string value of
+ the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is
+ highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would
+ be passed to the driver pod in plaintext otherwise.
+ |
+
+
+ spark.kubernetes.authenticate.driver.mounted.caCertFile |
+ (none) |
+
+ Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting
+ executors. This path must be accessible from the driver pod.
+ Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.mounted.clientKeyFile |
+ (none) |
+
+ Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting
+ executors. This path must be accessible from the driver pod.
+ Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.mounted.clientCertFile |
+ (none) |
+
+ Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when
+ requesting executors. This path must be accessible from the driver pod.
+ Specify this as a path as opposed to a URI (i.e. do not provide a scheme).
+ |
+
+
+ spark.kubernetes.authenticate.driver.mounted.oauthTokenFile |
+ (none) |
+
+ Path to the file containing the OAuth token to use when authenticating against the Kubernetes API server from the driver pod when
+ requesting executors. This path must be accessible from the driver pod.
+ Note that unlike the other authentication options, this file must contain the exact string value of the token to use for the authentication.
+ |
+
+
+ spark.kubernetes.authenticate.driver.serviceAccountName |
+ default |
+
+ Service account that is used when running the driver pod. The driver pod uses this service account when requesting
+ executor pods from the API server. Note that this cannot be specified alongside a CA cert file, client key file,
+ client cert file, and/or OAuth token.
+ |
+
+
+ spark.kubernetes.driver.label.[LabelName] |
+ (none) |
+
+ Add the label specified by LabelName to the driver pod.
+ For example, spark.kubernetes.driver.label.something=true .
+ Note that Spark also adds its own labels to the driver pod
+ for bookkeeping purposes.
+ |
+
+
+ spark.kubernetes.driver.annotation.[AnnotationName] |
+ (none) |
+
+ Add the annotation specified by AnnotationName to the driver pod.
+ For example, spark.kubernetes.driver.annotation.something=true .
+ |
+
+
+ spark.kubernetes.executor.label.[LabelName] |
+ (none) |
+
+ Add the label specified by LabelName to the executor pods.
+ For example, spark.kubernetes.executor.label.something=true .
+ Note that Spark also adds its own labels to the driver pod
+ for bookkeeping purposes.
+ |
+
+
+ spark.kubernetes.executor.annotation.[AnnotationName] |
+ (none) |
+
+ Add the annotation specified by AnnotationName to the executor pods.
+ For example, spark.kubernetes.executor.annotation.something=true .
+ |
+
+
+ spark.kubernetes.driver.pod.name |
+ (none) |
+
+ Name of the driver pod. If not set, the driver pod name is set to "spark.app.name" suffixed by the current timestamp
+ to avoid name conflicts.
+ |
+
+
+ spark.kubernetes.executor.podNamePrefix |
+ (none) |
+
+ Prefix for naming the executor pods.
+ If not set, the executor pod name is set to driver pod name suffixed by an integer.
+ |
+
+
+ spark.kubernetes.executor.lostCheck.maxAttempts |
+ 10 |
+
+ Number of times that the driver will try to ascertain the loss reason for a specific executor.
+ The loss reason is used to ascertain whether the executor failure is due to a framework or an application error
+ which in turn decides whether the executor is removed and replaced, or placed into a failed state for debugging.
+ |
+
+
+ spark.kubernetes.submission.waitAppCompletion |
+ true |
+
+ In cluster mode, whether to wait for the application to finish before exiting the launcher process. When changed to
+ false, the launcher has a "fire-and-forget" behavior when launching the Spark job.
+ |
+
+
+ spark.kubernetes.report.interval |
+ 1s |
+
+ Interval between reports of the current Spark job status in cluster mode.
+ |
+
+
+ spark.kubernetes.driver.limit.cores |
+ (none) |
+
+ Specify the hard CPU [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for the driver pod.
+ |
+
+
+ spark.kubernetes.executor.limit.cores |
+ (none) |
+
+ Specify the hard CPU [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for each executor pod launched for the Spark Application.
+ |
+
+
+ spark.kubernetes.node.selector.[labelKey] |
+ (none) |
+
+ Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the
+ configuration's value. For example, setting spark.kubernetes.node.selector.identifier to myIdentifier
+ will result in the driver pod and executors having a node selector with key identifier and value
+ myIdentifier . Multiple node selector keys can be added by setting multiple configurations with this prefix.
+ |
+
+
+ spark.kubernetes.driverEnv.[EnvironmentVariableName] |
+ (none) |
+
+ Add the environment variable specified by EnvironmentVariableName to
+ the Driver process. The user can specify multiple of these to set multiple environment variables.
+ |
+
+
+ spark.kubernetes.mountDependencies.jarsDownloadDir |
+ /var/spark-data/spark-jars |
+
+ Location to download jars to in the driver and executors.
+ This directory must be empty and will be mounted as an empty directory volume on the driver and executor pods.
+ |
+
+
+ spark.kubernetes.mountDependencies.filesDownloadDir |
+ /var/spark-data/spark-files |
+
+ Location to download jars to in the driver and executors.
+ This directory must be empty and will be mounted as an empty directory volume on the driver and executor pods.
+ |
+
+
\ No newline at end of file
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 7e2386f33b583..e7edec5990363 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -18,7 +18,9 @@ Spark application's configuration (driver, executors, and the AM when running in
There are two deploy modes that can be used to launch Spark applications on YARN. In `cluster` mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In `client` mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN.
-Unlike [Spark standalone](spark-standalone.html) and [Mesos](running-on-mesos.html) modes, in which the master's address is specified in the `--master` parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the `--master` parameter is `yarn`.
+Unlike other cluster managers supported by Spark in which the master's address is specified in the `--master`
+parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration.
+Thus, the `--master` parameter is `yarn`.
To launch a Spark application in `cluster` mode:
diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md
index 93bef8d5bb7e2..31fcfabb9cacc 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -493,7 +493,7 @@ returned by `SparkSession.readStream()`. In [R](api/R/read.stream.html), with th
#### Input Sources
There are a few built-in sources.
- - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations.
+ - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, orc, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations.
- **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details.
diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md
index 866d6e527549c..0473ab73a5e6c 100644
--- a/docs/submitting-applications.md
+++ b/docs/submitting-applications.md
@@ -127,6 +127,16 @@ export HADOOP_CONF_DIR=XXX
http://path/to/examples.jar \
1000
+# Run on a Kubernetes cluster in cluster deploy mode
+./bin/spark-submit \
+ --class org.apache.spark.examples.SparkPi \
+ --master k8s://xx.yy.zz.ww:443 \
+ --deploy-mode cluster \
+ --executor-memory 20G \
+ --num-executors 50 \
+ http://path/to/examples.jar \
+ 1000
+
{% endhighlight %}
# Master URLs
@@ -155,6 +165,12 @@ The master URL passed to Spark can be in one of the following formats:
client
or cluster
mode depending on the value of --deploy-mode
.
The cluster location will be found based on the HADOOP_CONF_DIR
or YARN_CONF_DIR
variable.
+ k8s://HOST:PORT | Connect to a Kubernetes cluster in
+ cluster mode. Client mode is currently unsupported and will be supported in future releases.
+ The HOST and PORT refer to the [Kubernetes API Server](https://kubernetes.io/docs/reference/generated/kube-apiserver/).
+ It connects using TLS by default. In order to force it to use an unsecured connection, you can use
+ k8s://http://HOST:PORT .
+ |
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 9864dc98c1f33..75395a754a831 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -368,6 +368,20 @@ def checkpoint(self, eager=True):
jdf = self._jdf.checkpoint(eager)
return DataFrame(jdf, self.sql_ctx)
+ @since(2.3)
+ def localCheckpoint(self, eager=True):
+ """Returns a locally checkpointed version of this Dataset. Checkpointing can be used to
+ truncate the logical plan of this DataFrame, which is especially useful in iterative
+ algorithms where the plan may grow exponentially. Local checkpoints are stored in the
+ executors using the caching subsystem and therefore they are not reliable.
+
+ :param eager: Whether to checkpoint this DataFrame immediately
+
+ .. note:: Experimental
+ """
+ jdf = self._jdf.localCheckpoint(eager)
+ return DataFrame(jdf, self.sql_ctx)
+
@since(2.1)
def withWatermark(self, eventTime, delayThreshold):
"""Defines an event time watermark for this :class:`DataFrame`. A watermark tracks a point
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 4e0faddb1c0df..54530055dfa85 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -1099,7 +1099,7 @@ def trunc(date, format):
"""
Returns date truncated to the unit specified by the format.
- :param format: 'year', 'YYYY', 'yy' or 'month', 'mon', 'mm'
+ :param format: 'year', 'yyyy', 'yy' or 'month', 'mon', 'mm'
>>> df = spark.createDataFrame([('1997-02-28',)], ['d'])
>>> df.select(trunc(df.d, 'year').alias('year')).collect()
@@ -1111,6 +1111,24 @@ def trunc(date, format):
return Column(sc._jvm.functions.trunc(_to_java_column(date), format))
+@since(2.3)
+def date_trunc(format, timestamp):
+ """
+ Returns timestamp truncated to the unit specified by the format.
+
+ :param format: 'year', 'yyyy', 'yy', 'month', 'mon', 'mm',
+ 'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
+
+ >>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['t'])
+ >>> df.select(date_trunc('year', df.t).alias('year')).collect()
+ [Row(year=datetime.datetime(1997, 1, 1, 0, 0))]
+ >>> df.select(date_trunc('mon', df.t).alias('month')).collect()
+ [Row(month=datetime.datetime(1997, 2, 1, 0, 0))]
+ """
+ sc = SparkContext._active_spark_context
+ return Column(sc._jvm.functions.date_trunc(format, _to_java_column(timestamp)))
+
+
@since(1.5)
def next_day(date, dayOfWeek):
"""
diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py
index 0cf702143c773..d0aba28788ac9 100644
--- a/python/pyspark/sql/streaming.py
+++ b/python/pyspark/sql/streaming.py
@@ -490,6 +490,23 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
else:
raise TypeError("path can be only a single string")
+ @since(2.3)
+ def orc(self, path):
+ """Loads a ORC file stream, returning the result as a :class:`DataFrame`.
+
+ .. note:: Evolving.
+
+ >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp())
+ >>> orc_sdf.isStreaming
+ True
+ >>> orc_sdf.schema == sdf_schema
+ True
+ """
+ if isinstance(path, basestring):
+ return self._df(self._jreader.orc(path))
+ else:
+ raise TypeError("path can be only a single string")
+
@since(2.0)
def parquet(self, path):
"""Loads a Parquet file stream, returning the result as a :class:`DataFrame`.
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index f35fb38798218..04aadb4b06af4 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -30,21 +30,20 @@ private[spark] object Config extends Logging {
.stringConf
.createWithDefault("default")
- val DRIVER_DOCKER_IMAGE =
- ConfigBuilder("spark.kubernetes.driver.docker.image")
- .doc("Docker image to use for the driver. Specify this using the standard Docker tag format.")
+ val DRIVER_CONTAINER_IMAGE =
+ ConfigBuilder("spark.kubernetes.driver.container.image")
+ .doc("Container image to use for the driver.")
.stringConf
.createOptional
- val EXECUTOR_DOCKER_IMAGE =
- ConfigBuilder("spark.kubernetes.executor.docker.image")
- .doc("Docker image to use for the executors. Specify this using the standard Docker tag " +
- "format.")
+ val EXECUTOR_CONTAINER_IMAGE =
+ ConfigBuilder("spark.kubernetes.executor.container.image")
+ .doc("Container image to use for the executors.")
.stringConf
.createOptional
- val DOCKER_IMAGE_PULL_POLICY =
- ConfigBuilder("spark.kubernetes.docker.image.pullPolicy")
+ val CONTAINER_IMAGE_PULL_POLICY =
+ ConfigBuilder("spark.kubernetes.container.image.pullPolicy")
.doc("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent.")
.stringConf
.checkValues(Set("Always", "Never", "IfNotPresent"))
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala
index c563fc5bfbadf..1411e6f40b468 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala
@@ -49,7 +49,7 @@ private[spark] class DriverConfigurationStepsOrchestrator(
s"$appName-$uuid".toLowerCase.replaceAll("\\.", "-")
}
- private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY)
+ private val imagePullPolicy = submissionSparkConf.get(CONTAINER_IMAGE_PULL_POLICY)
private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION)
private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION)
@@ -72,7 +72,7 @@ private[spark] class DriverConfigurationStepsOrchestrator(
kubernetesAppId,
kubernetesResourceNamePrefix,
allDriverLabels,
- dockerImagePullPolicy,
+ imagePullPolicy,
appName,
mainClass,
appArgs,
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala
index ba2a11b9e6689..c335fcce4036e 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala
@@ -34,7 +34,7 @@ private[spark] class BaseDriverConfigurationStep(
kubernetesAppId: String,
kubernetesResourceNamePrefix: String,
driverLabels: Map[String, String],
- dockerImagePullPolicy: String,
+ imagePullPolicy: String,
appName: String,
mainClass: String,
appArgs: Array[String],
@@ -46,9 +46,9 @@ private[spark] class BaseDriverConfigurationStep(
private val driverExtraClasspath = submissionSparkConf.get(
DRIVER_CLASS_PATH)
- private val driverDockerImage = submissionSparkConf
- .get(DRIVER_DOCKER_IMAGE)
- .getOrElse(throw new SparkException("Must specify the driver Docker image"))
+ private val driverContainerImage = submissionSparkConf
+ .get(DRIVER_CONTAINER_IMAGE)
+ .getOrElse(throw new SparkException("Must specify the driver container image"))
// CPU settings
private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1")
@@ -110,8 +110,8 @@ private[spark] class BaseDriverConfigurationStep(
val driverContainer = new ContainerBuilder(driverSpec.driverContainer)
.withName(DRIVER_CONTAINER_NAME)
- .withImage(driverDockerImage)
- .withImagePullPolicy(dockerImagePullPolicy)
+ .withImage(driverContainerImage)
+ .withImagePullPolicy(imagePullPolicy)
.addAllToEnv(driverCustomEnvs.asJava)
.addToEnv(driverExtraClasspathEnv.toSeq: _*)
.addNewEnv()
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
index 9d8f3b912c33d..70226157dd68b 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala
@@ -72,10 +72,10 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
sparkConf,
KUBERNETES_NODE_SELECTOR_PREFIX)
- private val executorDockerImage = sparkConf
- .get(EXECUTOR_DOCKER_IMAGE)
- .getOrElse(throw new SparkException("Must specify the executor Docker image"))
- private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY)
+ private val executorContainerImage = sparkConf
+ .get(EXECUTOR_CONTAINER_IMAGE)
+ .getOrElse(throw new SparkException("Must specify the executor container image"))
+ private val imagePullPolicy = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY)
private val blockManagerPort = sparkConf
.getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
@@ -166,8 +166,8 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf)
val executorContainer = new ContainerBuilder()
.withName("executor")
- .withImage(executorDockerImage)
- .withImagePullPolicy(dockerImagePullPolicy)
+ .withImage(executorContainerImage)
+ .withImagePullPolicy(imagePullPolicy)
.withNewResources()
.addToRequests("memory", executorMemoryQuantity)
.addToLimits("memory", executorMemoryLimitQuantity)
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala
index c7291d49b465e..98f9f27da5cde 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark.deploy.k8s.submit
import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.k8s.Config.DRIVER_DOCKER_IMAGE
+import org.apache.spark.deploy.k8s.Config.DRIVER_CONTAINER_IMAGE
import org.apache.spark.deploy.k8s.submit.steps._
class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
@@ -32,7 +32,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
test("Base submission steps with a main app resource.") {
val sparkConf = new SparkConf(false)
- .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE)
+ .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
val orchestrator = new DriverConfigurationStepsOrchestrator(
NAMESPACE,
@@ -54,7 +54,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
test("Base submission steps without a main app resource.") {
val sparkConf = new SparkConf(false)
- .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE)
+ .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE)
val orchestrator = new DriverConfigurationStepsOrchestrator(
NAMESPACE,
APP_ID,
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala
index 83c5f98254829..f7c1b3142cf71 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala
@@ -30,7 +30,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite {
private val APP_ID = "spark-app-id"
private val RESOURCE_NAME_PREFIX = "spark"
private val DRIVER_LABELS = Map("labelkey" -> "labelvalue")
- private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent"
+ private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent"
private val APP_NAME = "spark-test"
private val MAIN_CLASS = "org.apache.spark.examples.SparkPi"
private val APP_ARGS = Array("arg1", "arg2", "arg 3")
@@ -47,7 +47,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite {
.set(KUBERNETES_DRIVER_LIMIT_CORES, "4")
.set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "256M")
.set(org.apache.spark.internal.config.DRIVER_MEMORY_OVERHEAD, 200L)
- .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest")
+ .set(DRIVER_CONTAINER_IMAGE, "spark-driver:latest")
.set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE)
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1")
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2")
@@ -56,7 +56,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite {
APP_ID,
RESOURCE_NAME_PREFIX,
DRIVER_LABELS,
- DOCKER_IMAGE_PULL_POLICY,
+ CONTAINER_IMAGE_PULL_POLICY,
APP_NAME,
MAIN_CLASS,
APP_ARGS,
@@ -71,7 +71,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite {
assert(preparedDriverSpec.driverContainer.getName === DRIVER_CONTAINER_NAME)
assert(preparedDriverSpec.driverContainer.getImage === "spark-driver:latest")
- assert(preparedDriverSpec.driverContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY)
+ assert(preparedDriverSpec.driverContainer.getImagePullPolicy === CONTAINER_IMAGE_PULL_POLICY)
assert(preparedDriverSpec.driverContainer.getEnv.size === 7)
val envs = preparedDriverSpec.driverContainer
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala
index 1c7717c238096..3a55d7cb37b1f 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala
@@ -50,7 +50,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
baseConf = new SparkConf()
.set(KUBERNETES_DRIVER_POD_NAME, driverPodName)
.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix)
- .set(EXECUTOR_DOCKER_IMAGE, executorImage)
+ .set(EXECUTOR_CONTAINER_IMAGE, executorImage)
}
test("basic executor pod has reasonable defaults") {
diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh
old mode 100644
new mode 100755
diff --git a/sbin/build-push-docker-images.sh b/sbin/build-push-docker-images.sh
new file mode 100755
index 0000000000000..4546e98dc2074
--- /dev/null
+++ b/sbin/build-push-docker-images.sh
@@ -0,0 +1,68 @@
+#!/usr/bin/env bash
+
+# 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.
+#
+
+# This script builds and pushes docker images when run from a release of Spark
+# with Kubernetes support.
+
+declare -A path=( [spark-driver]=kubernetes/dockerfiles/driver/Dockerfile \
+ [spark-executor]=kubernetes/dockerfiles/executor/Dockerfile )
+
+function build {
+ docker build -t spark-base -f kubernetes/dockerfiles/spark-base/Dockerfile .
+ for image in "${!path[@]}"; do
+ docker build -t ${REPO}/$image:${TAG} -f ${path[$image]} .
+ done
+}
+
+
+function push {
+ for image in "${!path[@]}"; do
+ docker push ${REPO}/$image:${TAG}
+ done
+}
+
+function usage {
+ echo "This script must be run from a runnable distribution of Apache Spark."
+ echo "Usage: ./sbin/build-push-docker-images.sh -r -t build"
+ echo " ./sbin/build-push-docker-images.sh -r -t push"
+ echo "for example: ./sbin/build-push-docker-images.sh -r docker.io/myrepo -t v2.3.0 push"
+}
+
+if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
+ usage
+ exit 0
+fi
+
+while getopts r:t: option
+do
+ case "${option}"
+ in
+ r) REPO=${OPTARG};;
+ t) TAG=${OPTARG};;
+ esac
+done
+
+if [ -z "$REPO" ] || [ -z "$TAG" ]; then
+ usage
+else
+ case "${@: -1}" in
+ build) build;;
+ push) push;;
+ *) usage;;
+ esac
+fi
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 0d5e866c0683e..10b237fb22b96 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
@@ -222,22 +222,20 @@ class Analyzer(
*/
object ResolveAliases extends Rule[LogicalPlan] {
private def assignAliases(exprs: Seq[NamedExpression]) = {
- exprs.zipWithIndex.map {
- case (expr, i) =>
- expr.transformUp { case u @ UnresolvedAlias(child, optGenAliasFunc) =>
- child match {
- case ne: NamedExpression => ne
- case go @ GeneratorOuter(g: Generator) if g.resolved => MultiAlias(go, Nil)
- case e if !e.resolved => u
- case g: Generator => MultiAlias(g, Nil)
- case c @ Cast(ne: NamedExpression, _, _) => Alias(c, ne.name)()
- case e: ExtractValue => Alias(e, toPrettySQL(e))()
- case e if optGenAliasFunc.isDefined =>
- Alias(child, optGenAliasFunc.get.apply(e))()
- case e => Alias(e, toPrettySQL(e))()
- }
+ exprs.map(_.transformUp { case u @ UnresolvedAlias(child, optGenAliasFunc) =>
+ child match {
+ case ne: NamedExpression => ne
+ case go @ GeneratorOuter(g: Generator) if g.resolved => MultiAlias(go, Nil)
+ case e if !e.resolved => u
+ case g: Generator => MultiAlias(g, Nil)
+ case c @ Cast(ne: NamedExpression, _, _) => Alias(c, ne.name)()
+ case e: ExtractValue => Alias(e, toPrettySQL(e))()
+ case e if optGenAliasFunc.isDefined =>
+ Alias(child, optGenAliasFunc.get.apply(e))()
+ case e => Alias(e, toPrettySQL(e))()
}
- }.asInstanceOf[Seq[NamedExpression]]
+ }
+ ).asInstanceOf[Seq[NamedExpression]]
}
private def hasUnresolvedAlias(exprs: Seq[NamedExpression]) =
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index 11538bd31b4fd..5ddb39822617d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -392,6 +392,7 @@ object FunctionRegistry {
expression[ToUnixTimestamp]("to_unix_timestamp"),
expression[ToUTCTimestamp]("to_utc_timestamp"),
expression[TruncDate]("trunc"),
+ expression[TruncTimestamp]("date_trunc"),
expression[UnixTimestamp]("unix_timestamp"),
expression[DayOfWeek]("dayofweek"),
expression[WeekOfYear]("weekofyear"),
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
index 072dc954879ca..7a0aa08289efa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis
import scala.util.control.NonFatal
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.expressions.{Add, AttributeReference, AttributeSet, Cast, CheckOverflow, Expression, ExpressionSet, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Literal, Multiply, PreciseTimestampConversion, PredicateHelper, Subtract, TimeAdd, TimeSub, UnaryMinus}
+import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._
@@ -238,6 +238,8 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
collect(child, !negate)
case CheckOverflow(child, _) =>
collect(child, negate)
+ case PromotePrecision(child) =>
+ collect(child, negate)
case Cast(child, dataType, _) =>
dataType match {
case _: NumericType | _: TimestampType => collect(child, negate)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 743782a6453e9..4568714933095 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -119,8 +119,7 @@ abstract class Expression extends TreeNode[Expression] {
// TODO: support whole stage codegen too
if (eval.code.trim.length > 1024 && ctx.INPUT_ROW != null && ctx.currentVars == null) {
val setIsNull = if (eval.isNull != "false" && eval.isNull != "true") {
- val globalIsNull = ctx.freshName("globalIsNull")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, globalIsNull)
+ val globalIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "globalIsNull")
val localIsNull = eval.isNull
eval.isNull = globalIsNull
s"$globalIsNull = $localIsNull;"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
index 821d784a01342..784eaf8195194 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala
@@ -65,10 +65,8 @@ case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterminis
}
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val countTerm = ctx.freshName("count")
- val partitionMaskTerm = ctx.freshName("partitionMask")
- ctx.addMutableState(ctx.JAVA_LONG, countTerm)
- ctx.addMutableState(ctx.JAVA_LONG, partitionMaskTerm)
+ val countTerm = ctx.addMutableState(ctx.JAVA_LONG, "count")
+ val partitionMaskTerm = ctx.addMutableState(ctx.JAVA_LONG, "partitionMask")
ctx.addPartitionInitializationStatement(s"$countTerm = 0L;")
ctx.addPartitionInitializationStatement(s"$partitionMaskTerm = ((long) partitionIndex) << 33;")
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
index 4fa18d6b3209b..736ca37c6d54a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala
@@ -43,8 +43,7 @@ case class SparkPartitionID() extends LeafExpression with Nondeterministic {
override protected def evalInternal(input: InternalRow): Int = partitionId
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val idTerm = ctx.freshName("partitionId")
- ctx.addMutableState(ctx.JAVA_INT, idTerm)
+ val idTerm = ctx.addMutableState(ctx.JAVA_INT, "partitionId")
ctx.addPartitionInitializationStatement(s"$idTerm = partitionIndex;")
ev.copy(code = s"final ${ctx.javaType(dataType)} ${ev.value} = $idTerm;", isNull = "false")
}
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 1893eec22b65d..d3a8cb5804717 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
@@ -602,8 +602,7 @@ case class Least(children: Seq[Expression]) extends Expression {
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val evalChildren = children.map(_.genCode(ctx))
- val tmpIsNull = ctx.freshName("leastTmpIsNull")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, tmpIsNull)
+ val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "leastTmpIsNull")
val evals = evalChildren.map(eval =>
s"""
|${eval.code}
@@ -683,8 +682,7 @@ case class Greatest(children: Seq[Expression]) extends Expression {
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val evalChildren = children.map(_.genCode(ctx))
- val tmpIsNull = ctx.freshName("greatestTmpIsNull")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, tmpIsNull)
+ val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "greatestTmpIsNull")
val evals = evalChildren.map(eval =>
s"""
|${eval.code}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 3a03a65e1af92..41a920ba3d677 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -137,22 +137,63 @@ class CodegenContext {
var currentVars: Seq[ExprCode] = null
/**
- * Holding expressions' mutable states like `MonotonicallyIncreasingID.count` as a
- * 3-tuple: java type, variable name, code to init it.
- * As an example, ("int", "count", "count = 0;") will produce code:
+ * Holding expressions' inlined mutable states like `MonotonicallyIncreasingID.count` as a
+ * 2-tuple: java type, variable name.
+ * As an example, ("int", "count") will produce code:
* {{{
* private int count;
* }}}
- * as a member variable, and add
- * {{{
- * count = 0;
- * }}}
- * to the constructor.
+ * as a member variable
*
* They will be kept as member variables in generated classes like `SpecificProjection`.
*/
- val mutableStates: mutable.ArrayBuffer[(String, String, String)] =
- mutable.ArrayBuffer.empty[(String, String, String)]
+ val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] =
+ mutable.ArrayBuffer.empty[(String, String)]
+
+ /**
+ * The mapping between mutable state types and corrseponding compacted arrays.
+ * The keys are java type string. The values are [[MutableStateArrays]] which encapsulates
+ * the compacted arrays for the mutable states with the same java type.
+ */
+ val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] =
+ mutable.Map.empty[String, MutableStateArrays]
+
+ // An array holds the code that will initialize each state
+ val mutableStateInitCode: mutable.ArrayBuffer[String] =
+ mutable.ArrayBuffer.empty[String]
+
+ /**
+ * This class holds a set of names of mutableStateArrays that is used for compacting mutable
+ * states for a certain type, and holds the next available slot of the current compacted array.
+ */
+ class MutableStateArrays {
+ val arrayNames = mutable.ListBuffer.empty[String]
+ createNewArray()
+
+ private[this] var currentIndex = 0
+
+ private def createNewArray() = arrayNames.append(freshName("mutableStateArray"))
+
+ def getCurrentIndex: Int = currentIndex
+
+ /**
+ * Returns the reference of next available slot in current compacted array. The size of each
+ * compacted array is controlled by the config `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
+ * Once reaching the threshold, new compacted array is created.
+ */
+ def getNextSlot(): String = {
+ if (currentIndex < CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT) {
+ val res = s"${arrayNames.last}[$currentIndex]"
+ currentIndex += 1
+ res
+ } else {
+ createNewArray()
+ currentIndex = 1
+ s"${arrayNames.last}[0]"
+ }
+ }
+
+ }
/**
* Add a mutable state as a field to the generated class. c.f. the comments above.
@@ -163,11 +204,52 @@ class CodegenContext {
* the list of default imports available.
* Also, generic type arguments are accepted but ignored.
* @param variableName Name of the field.
- * @param initCode The statement(s) to put into the init() method to initialize this field.
+ * @param initFunc Function includes statement(s) to put into the init() method to initialize
+ * this field. The argument is the name of the mutable state variable.
* If left blank, the field will be default-initialized.
+ * @param forceInline whether the declaration and initialization code may be inlined rather than
+ * compacted. Please set `true` into forceInline for one of the followings:
+ * 1. use the original name of the status
+ * 2. expect to non-frequently generate the status
+ * (e.g. not much sort operators in one stage)
+ * @param useFreshName If this is false and the mutable state ends up inlining in the outer
+ * class, the name is not changed
+ * @return the name of the mutable state variable, which is the original name or fresh name if
+ * the variable is inlined to the outer class, or an array access if the variable is to
+ * be stored in an array of variables of the same type.
+ * A variable will be inlined into the outer class when one of the following conditions
+ * are satisfied:
+ * 1. forceInline is true
+ * 2. its type is primitive type and the total number of the inlined mutable variables
+ * is less than `CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD`
+ * 3. its type is multi-dimensional array
+ * When a variable is compacted into an array, the max size of the array for compaction
+ * is given by `CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT`.
*/
- def addMutableState(javaType: String, variableName: String, initCode: String = ""): Unit = {
- mutableStates += ((javaType, variableName, initCode))
+ def addMutableState(
+ javaType: String,
+ variableName: String,
+ initFunc: String => String = _ => "",
+ forceInline: Boolean = false,
+ useFreshName: Boolean = true): String = {
+
+ // want to put a primitive type variable at outerClass for performance
+ val canInlinePrimitive = isPrimitiveType(javaType) &&
+ (inlinedMutableStates.length < CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD)
+ if (forceInline || canInlinePrimitive || javaType.contains("[][]")) {
+ val varName = if (useFreshName) freshName(variableName) else variableName
+ val initCode = initFunc(varName)
+ inlinedMutableStates += ((javaType, varName))
+ mutableStateInitCode += initCode
+ varName
+ } else {
+ val arrays = arrayCompactedMutableStates.getOrElseUpdate(javaType, new MutableStateArrays)
+ val element = arrays.getNextSlot()
+
+ val initCode = initFunc(element)
+ mutableStateInitCode += initCode
+ element
+ }
}
/**
@@ -176,8 +258,7 @@ class CodegenContext {
* data types like: UTF8String, ArrayData, MapData & InternalRow.
*/
def addBufferedState(dataType: DataType, variableName: String, initCode: String): ExprCode = {
- val value = freshName(variableName)
- addMutableState(javaType(dataType), value, "")
+ val value = addMutableState(javaType(dataType), variableName)
val code = dataType match {
case StringType => s"$value = $initCode.clone();"
case _: StructType | _: ArrayType | _: MapType => s"$value = $initCode.copy();"
@@ -189,15 +270,37 @@ class CodegenContext {
def declareMutableStates(): String = {
// It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in
// `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones.
- mutableStates.distinct.map { case (javaType, variableName, _) =>
+ val inlinedStates = inlinedMutableStates.distinct.map { case (javaType, variableName) =>
s"private $javaType $variableName;"
- }.mkString("\n")
+ }
+
+ val arrayStates = arrayCompactedMutableStates.flatMap { case (javaType, mutableStateArrays) =>
+ val numArrays = mutableStateArrays.arrayNames.size
+ mutableStateArrays.arrayNames.zipWithIndex.map { case (arrayName, index) =>
+ val length = if (index + 1 == numArrays) {
+ mutableStateArrays.getCurrentIndex
+ } else {
+ CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT
+ }
+ if (javaType.contains("[]")) {
+ // initializer had an one-dimensional array variable
+ val baseType = javaType.substring(0, javaType.length - 2)
+ s"private $javaType[] $arrayName = new $baseType[$length][];"
+ } else {
+ // initializer had a scalar variable
+ s"private $javaType[] $arrayName = new $javaType[$length];"
+ }
+ }
+ }
+
+ (inlinedStates ++ arrayStates).mkString("\n")
}
def initMutableStates(): String = {
// It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in
// `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones.
- val initCodes = mutableStates.distinct.map(_._3 + "\n")
+ val initCodes = mutableStateInitCode.distinct
+
// The generated initialization code may exceed 64kb function size limit in JVM if there are too
// many mutable states, so split it into multiple functions.
splitExpressions(expressions = initCodes, funcName = "init", arguments = Nil)
@@ -1011,9 +1114,9 @@ class CodegenContext {
val commonExprs = equivalentExpressions.getAllEquivalentExprs.filter(_.size > 1)
commonExprs.foreach { e =>
val expr = e.head
- val fnName = freshName("evalExpr")
- val isNull = s"${fnName}IsNull"
- val value = s"${fnName}Value"
+ val fnName = freshName("subExpr")
+ val isNull = addMutableState(JAVA_BOOLEAN, "subExprIsNull")
+ val value = addMutableState(javaType(expr.dataType), "subExprValue")
// Generate the code for this expression tree and wrap it in a function.
val eval = expr.genCode(this)
@@ -1039,9 +1142,6 @@ class CodegenContext {
// 2. Less code.
// Currently, we will do this for all non-leaf only expression trees (i.e. expr trees with
// at least two nodes) as the cost of doing it is expected to be low.
- addMutableState(JAVA_BOOLEAN, isNull, s"$isNull = false;")
- addMutableState(javaType(expr.dataType), value,
- s"$value = ${defaultValue(expr.dataType)};")
subexprFunctions += s"${addNewFunction(fnName, fn)}($INPUT_ROW);"
val state = SubExprEliminationState(isNull, value)
@@ -1165,6 +1265,15 @@ object CodeGenerator extends Logging {
// class.
val GENERATED_CLASS_SIZE_THRESHOLD = 1000000
+ // This is the threshold for the number of global variables, whose types are primitive type or
+ // complex type (e.g. more than one-dimensional array), that will be placed at the outer class
+ val OUTER_CLASS_VARIABLES_THRESHOLD = 10000
+
+ // This is the maximum number of array elements to keep global variables in one Java array
+ // 32767 is the maximum integer value that does not require a constant pool entry in a Java
+ // bytecode instruction
+ val MUTABLESTATEARRAY_SIZE_LIMIT = 32768
+
/**
* Compile the Java source code into a Java class, using Janino.
*
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
index bd8312eb8b7fe..b53c0087e7e2d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
@@ -57,41 +57,37 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP
case _ => true
}.unzip
val exprVals = ctx.generateExpressions(validExpr, useSubexprElimination)
- val projectionCodes = exprVals.zip(index).map {
+
+ // 4-tuples: (code for projection, isNull variable name, value variable name, column index)
+ val projectionCodes: Seq[(String, String, String, Int)] = exprVals.zip(index).map {
case (ev, i) =>
val e = expressions(i)
+ val value = ctx.addMutableState(ctx.javaType(e.dataType), "value")
if (e.nullable) {
- val isNull = s"isNull_$i"
- val value = s"value_$i"
- ctx.addMutableState(ctx.JAVA_BOOLEAN, isNull, s"$isNull = true;")
- ctx.addMutableState(ctx.javaType(e.dataType), value,
- s"$value = ${ctx.defaultValue(e.dataType)};")
- s"""
- ${ev.code}
- $isNull = ${ev.isNull};
- $value = ${ev.value};
- """
+ val isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "isNull")
+ (s"""
+ |${ev.code}
+ |$isNull = ${ev.isNull};
+ |$value = ${ev.value};
+ """.stripMargin, isNull, value, i)
} else {
- val value = s"value_$i"
- ctx.addMutableState(ctx.javaType(e.dataType), value,
- s"$value = ${ctx.defaultValue(e.dataType)};")
- s"""
- ${ev.code}
- $value = ${ev.value};
- """
+ (s"""
+ |${ev.code}
+ |$value = ${ev.value};
+ """.stripMargin, ev.isNull, value, i)
}
}
// Evaluate all the subexpressions.
val evalSubexpr = ctx.subexprFunctions.mkString("\n")
- val updates = validExpr.zip(index).map {
- case (e, i) =>
- val ev = ExprCode("", s"isNull_$i", s"value_$i")
+ val updates = validExpr.zip(projectionCodes).map {
+ case (e, (_, isNull, value, i)) =>
+ val ev = ExprCode("", isNull, value)
ctx.updateColumn("mutableRow", e.dataType, i, ev, e.nullable)
}
- val allProjections = ctx.splitExpressionsWithCurrentInputs(projectionCodes)
+ val allProjections = ctx.splitExpressionsWithCurrentInputs(projectionCodes.map(_._1))
val allUpdates = ctx.splitExpressionsWithCurrentInputs(updates)
val codeBody = s"""
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
index b022457865d50..36ffa8dcdd2b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
@@ -73,9 +73,8 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
bufferHolder: String,
isTopLevel: Boolean = false): String = {
val rowWriterClass = classOf[UnsafeRowWriter].getName
- val rowWriter = ctx.freshName("rowWriter")
- ctx.addMutableState(rowWriterClass, rowWriter,
- s"$rowWriter = new $rowWriterClass($bufferHolder, ${inputs.length});")
+ val rowWriter = ctx.addMutableState(rowWriterClass, "rowWriter",
+ v => s"$v = new $rowWriterClass($bufferHolder, ${inputs.length});")
val resetWriter = if (isTopLevel) {
// For top level row writer, it always writes to the beginning of the global buffer holder,
@@ -186,9 +185,8 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
// Puts `input` in a local variable to avoid to re-evaluate it if it's a statement.
val tmpInput = ctx.freshName("tmpInput")
val arrayWriterClass = classOf[UnsafeArrayWriter].getName
- val arrayWriter = ctx.freshName("arrayWriter")
- ctx.addMutableState(arrayWriterClass, arrayWriter,
- s"$arrayWriter = new $arrayWriterClass();")
+ val arrayWriter = ctx.addMutableState(arrayWriterClass, "arrayWriter",
+ v => s"$v = new $arrayWriterClass();")
val numElements = ctx.freshName("numElements")
val index = ctx.freshName("index")
@@ -318,13 +316,12 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro
case _ => true
}
- val result = ctx.freshName("result")
- ctx.addMutableState("UnsafeRow", result, s"$result = new UnsafeRow(${expressions.length});")
+ val result = ctx.addMutableState("UnsafeRow", "result",
+ v => s"$v = new UnsafeRow(${expressions.length});")
- val holder = ctx.freshName("holder")
val holderClass = classOf[BufferHolder].getName
- ctx.addMutableState(holderClass, holder,
- s"$holder = new $holderClass($result, ${numVarLenFields * 32});")
+ val holder = ctx.addMutableState(holderClass, "holder",
+ v => s"$v = new $holderClass($result, ${numVarLenFields * 32});")
val resetBufferHolder = if (numVarLenFields == 0) {
""
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
index 53c3b226895ec..1a9b68222a7f4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
@@ -190,8 +190,7 @@ case class CaseWhen(
// It is initialized to `NOT_MATCHED`, and if it's set to `HAS_NULL` or `HAS_NONNULL`,
// We won't go on anymore on the computation.
val resultState = ctx.freshName("caseWhenResultState")
- val tmpResult = ctx.freshName("caseWhenTmpResult")
- ctx.addMutableState(ctx.javaType(dataType), tmpResult)
+ val tmpResult = ctx.addMutableState(ctx.javaType(dataType), "caseWhenTmpResult")
// these blocks are meant to be inside a
// do {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
index 44d54a20844a3..59c3e3d9947a3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
@@ -442,9 +442,9 @@ case class DayOfWeek(child: Expression) extends UnaryExpression with ImplicitCas
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
nullSafeCodeGen(ctx, ev, time => {
val cal = classOf[Calendar].getName
- val c = ctx.freshName("cal")
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
- ctx.addMutableState(cal, c, s"""$c = $cal.getInstance($dtu.getTimeZone("UTC"));""")
+ val c = ctx.addMutableState(cal, "cal",
+ v => s"""$v = $cal.getInstance($dtu.getTimeZone("UTC"));""")
s"""
$c.setTimeInMillis($time * 1000L * 3600L * 24L);
${ev.value} = $c.get($cal.DAY_OF_WEEK);
@@ -484,18 +484,17 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
nullSafeCodeGen(ctx, ev, time => {
val cal = classOf[Calendar].getName
- val c = ctx.freshName("cal")
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
- ctx.addMutableState(cal, c,
+ val c = ctx.addMutableState(cal, "cal", v =>
s"""
- $c = $cal.getInstance($dtu.getTimeZone("UTC"));
- $c.setFirstDayOfWeek($cal.MONDAY);
- $c.setMinimalDaysInFirstWeek(4);
- """)
+ |$v = $cal.getInstance($dtu.getTimeZone("UTC"));
+ |$v.setFirstDayOfWeek($cal.MONDAY);
+ |$v.setMinimalDaysInFirstWeek(4);
+ """.stripMargin)
s"""
- $c.setTimeInMillis($time * 1000L * 3600L * 24L);
- ${ev.value} = $c.get($cal.WEEK_OF_YEAR);
- """
+ |$c.setTimeInMillis($time * 1000L * 3600L * 24L);
+ |${ev.value} = $c.get($cal.WEEK_OF_YEAR);
+ """.stripMargin
})
}
}
@@ -1014,12 +1013,12 @@ case class FromUTCTimestamp(left: Expression, right: Expression)
|long ${ev.value} = 0;
""".stripMargin)
} else {
- val tzTerm = ctx.freshName("tz")
- val utcTerm = ctx.freshName("utc")
val tzClass = classOf[TimeZone].getName
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
- ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $dtu.getTimeZone("$tz");""")
- ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $dtu.getTimeZone("UTC");""")
+ val tzTerm = ctx.addMutableState(tzClass, "tz",
+ v => s"""$v = $dtu.getTimeZone("$tz");""")
+ val utcTerm = ctx.addMutableState(tzClass, "utc",
+ v => s"""$v = $dtu.getTimeZone("UTC");""")
val eval = left.genCode(ctx)
ev.copy(code = s"""
|${eval.code}
@@ -1190,12 +1189,12 @@ case class ToUTCTimestamp(left: Expression, right: Expression)
|long ${ev.value} = 0;
""".stripMargin)
} else {
- val tzTerm = ctx.freshName("tz")
- val utcTerm = ctx.freshName("utc")
val tzClass = classOf[TimeZone].getName
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
- ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $dtu.getTimeZone("$tz");""")
- ctx.addMutableState(tzClass, utcTerm, s"""$utcTerm = $dtu.getTimeZone("UTC");""")
+ val tzTerm = ctx.addMutableState(tzClass, "tz",
+ v => s"""$v = $dtu.getTimeZone("$tz");""")
+ val utcTerm = ctx.addMutableState(tzClass, "utc",
+ v => s"""$v = $dtu.getTimeZone("UTC");""")
val eval = left.genCode(ctx)
ev.copy(code = s"""
|${eval.code}
@@ -1295,80 +1294,79 @@ case class ParseToTimestamp(left: Expression, format: Option[Expression], child:
override def dataType: DataType = TimestampType
}
-/**
- * Returns date truncated to the unit specified by the format.
- */
-// scalastyle:off line.size.limit
-@ExpressionDescription(
- usage = "_FUNC_(date, fmt) - Returns `date` with the time portion of the day truncated to the unit specified by the format model `fmt`.",
- examples = """
- Examples:
- > SELECT _FUNC_('2009-02-12', 'MM');
- 2009-02-01
- > SELECT _FUNC_('2015-10-27', 'YEAR');
- 2015-01-01
- """,
- since = "1.5.0")
-// scalastyle:on line.size.limit
-case class TruncDate(date: Expression, format: Expression)
- extends BinaryExpression with ImplicitCastInputTypes {
- override def left: Expression = date
- override def right: Expression = format
-
- override def inputTypes: Seq[AbstractDataType] = Seq(DateType, StringType)
- override def dataType: DataType = DateType
+trait TruncInstant extends BinaryExpression with ImplicitCastInputTypes {
+ val instant: Expression
+ val format: Expression
override def nullable: Boolean = true
- override def prettyName: String = "trunc"
private lazy val truncLevel: Int =
DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String])
- override def eval(input: InternalRow): Any = {
+ /**
+ * @param input internalRow (time)
+ * @param maxLevel Maximum level that can be used for truncation (e.g MONTH for Date input)
+ * @param truncFunc function: (time, level) => time
+ */
+ protected def evalHelper(input: InternalRow, maxLevel: Int)(
+ truncFunc: (Any, Int) => Any): Any = {
val level = if (format.foldable) {
truncLevel
} else {
DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String])
}
- if (level == -1) {
- // unknown format
+ if (level == DateTimeUtils.TRUNC_INVALID || level > maxLevel) {
+ // unknown format or too large level
null
} else {
- val d = date.eval(input)
- if (d == null) {
+ val t = instant.eval(input)
+ if (t == null) {
null
} else {
- DateTimeUtils.truncDate(d.asInstanceOf[Int], level)
+ truncFunc(t, level)
}
}
}
- override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ protected def codeGenHelper(
+ ctx: CodegenContext,
+ ev: ExprCode,
+ maxLevel: Int,
+ orderReversed: Boolean = false)(
+ truncFunc: (String, String) => String)
+ : ExprCode = {
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
if (format.foldable) {
- if (truncLevel == -1) {
+ if (truncLevel == DateTimeUtils.TRUNC_INVALID || truncLevel > maxLevel) {
ev.copy(code = s"""
boolean ${ev.isNull} = true;
${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};""")
} else {
- val d = date.genCode(ctx)
+ val t = instant.genCode(ctx)
+ val truncFuncStr = truncFunc(t.value, truncLevel.toString)
ev.copy(code = s"""
- ${d.code}
- boolean ${ev.isNull} = ${d.isNull};
+ ${t.code}
+ boolean ${ev.isNull} = ${t.isNull};
${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)};
if (!${ev.isNull}) {
- ${ev.value} = $dtu.truncDate(${d.value}, $truncLevel);
+ ${ev.value} = $dtu.$truncFuncStr;
}""")
}
} else {
- nullSafeCodeGen(ctx, ev, (dateVal, fmt) => {
+ nullSafeCodeGen(ctx, ev, (left, right) => {
val form = ctx.freshName("form")
+ val (dateVal, fmt) = if (orderReversed) {
+ (right, left)
+ } else {
+ (left, right)
+ }
+ val truncFuncStr = truncFunc(dateVal, form)
s"""
int $form = $dtu.parseTruncLevel($fmt);
- if ($form == -1) {
+ if ($form == -1 || $form > $maxLevel) {
${ev.isNull} = true;
} else {
- ${ev.value} = $dtu.truncDate($dateVal, $form);
+ ${ev.value} = $dtu.$truncFuncStr
}
"""
})
@@ -1376,6 +1374,101 @@ case class TruncDate(date: Expression, format: Expression)
}
}
+/**
+ * Returns date truncated to the unit specified by the format.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(date, fmt) - Returns `date` with the time portion of the day truncated to the unit specified by the format model `fmt`.
+ `fmt` should be one of ["year", "yyyy", "yy", "mon", "month", "mm"]
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('2009-02-12', 'MM');
+ 2009-02-01
+ > SELECT _FUNC_('2015-10-27', 'YEAR');
+ 2015-01-01
+ """,
+ since = "1.5.0")
+// scalastyle:on line.size.limit
+case class TruncDate(date: Expression, format: Expression)
+ extends TruncInstant {
+ override def left: Expression = date
+ override def right: Expression = format
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(DateType, StringType)
+ override def dataType: DataType = DateType
+ override def prettyName: String = "trunc"
+ override val instant = date
+
+ override def eval(input: InternalRow): Any = {
+ evalHelper(input, maxLevel = DateTimeUtils.TRUNC_TO_MONTH) { (d: Any, level: Int) =>
+ DateTimeUtils.truncDate(d.asInstanceOf[Int], level)
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ codeGenHelper(ctx, ev, maxLevel = DateTimeUtils.TRUNC_TO_MONTH) { (date: String, fmt: String) =>
+ s"truncDate($date, $fmt);"
+ }
+ }
+}
+
+/**
+ * Returns timestamp truncated to the unit specified by the format.
+ */
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+ usage = """
+ _FUNC_(fmt, ts) - Returns timestamp `ts` truncated to the unit specified by the format model `fmt`.
+ `fmt` should be one of ["YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", "SECOND", "WEEK", "QUARTER"]
+ """,
+ examples = """
+ Examples:
+ > SELECT _FUNC_('2015-03-05T09:32:05.359', 'YEAR');
+ 2015-01-01T00:00:00
+ > SELECT _FUNC_('2015-03-05T09:32:05.359', 'MM');
+ 2015-03-01T00:00:00
+ > SELECT _FUNC_('2015-03-05T09:32:05.359', 'DD');
+ 2015-03-05T00:00:00
+ > SELECT _FUNC_('2015-03-05T09:32:05.359', 'HOUR');
+ 2015-03-05T09:00:00
+ """,
+ since = "2.3.0")
+// scalastyle:on line.size.limit
+case class TruncTimestamp(
+ format: Expression,
+ timestamp: Expression,
+ timeZoneId: Option[String] = None)
+ extends TruncInstant with TimeZoneAwareExpression {
+ override def left: Expression = format
+ override def right: Expression = timestamp
+
+ override def inputTypes: Seq[AbstractDataType] = Seq(StringType, TimestampType)
+ override def dataType: TimestampType = TimestampType
+ override def prettyName: String = "date_trunc"
+ override val instant = timestamp
+ override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
+ copy(timeZoneId = Option(timeZoneId))
+
+ def this(format: Expression, timestamp: Expression) = this(format, timestamp, None)
+
+ override def eval(input: InternalRow): Any = {
+ evalHelper(input, maxLevel = DateTimeUtils.TRUNC_TO_SECOND) { (t: Any, level: Int) =>
+ DateTimeUtils.truncTimestamp(t.asInstanceOf[Long], level, timeZone)
+ }
+ }
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val tz = ctx.addReferenceObj("timeZone", timeZone)
+ codeGenHelper(ctx, ev, maxLevel = DateTimeUtils.TRUNC_TO_SECOND, true) {
+ (date: String, fmt: String) =>
+ s"truncTimestamp($date, $fmt, $tz);"
+ }
+ }
+}
+
/**
* Returns the number of days from startDate to endDate.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
index 752dea23e1f7a..db1579ba28671 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
@@ -70,10 +70,12 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un
case class PromotePrecision(child: Expression) extends UnaryExpression {
override def dataType: DataType = child.dataType
override def eval(input: InternalRow): Any = child.eval(input)
+ /** Just a simple pass-through for code generation. */
override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx)
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = ev.copy("")
override def prettyName: String = "promote_precision"
override def sql: String = child.sql
+ override lazy val canonicalized: Expression = child.canonicalized
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index cd38783a731ad..1cd73a92a8635 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -199,8 +199,8 @@ case class Stack(children: Seq[Expression]) extends Generator {
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
// Rows - we write these into an array.
- val rowData = ctx.freshName("rows")
- ctx.addMutableState("InternalRow[]", rowData, s"$rowData = new InternalRow[$numRows];")
+ val rowData = ctx.addMutableState("InternalRow[]", "rows",
+ v => s"$v = new InternalRow[$numRows];")
val values = children.tail
val dataTypes = values.take(numFields).map(_.dataType)
val code = ctx.splitExpressionsWithCurrentInputs(Seq.tabulate(numRows) { row =>
@@ -217,7 +217,7 @@ case class Stack(children: Seq[Expression]) extends Generator {
ctx.addMutableState(
s"$wrapperClass",
ev.value,
- s"${ev.value} = $wrapperClass$$.MODULE$$.make($rowData);")
+ v => s"$v = $wrapperClass$$.MODULE$$.make($rowData);", useFreshName = false)
ev.copy(code = code, isNull = "false")
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
index 294cdcb2e9546..b4f895fffda38 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
@@ -72,8 +72,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression {
}
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val tmpIsNull = ctx.freshName("coalesceTmpIsNull")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, tmpIsNull)
+ val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "coalesceTmpIsNull")
// all the evals are meant to be in a do { ... } while (false); loop
val evals = children.map { e =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index 4bd395eadcf19..a59aad5be8715 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -62,15 +62,13 @@ trait InvokeLike extends Expression with NonSQLExpression {
def prepareArguments(ctx: CodegenContext): (String, String, String) = {
val resultIsNull = if (needNullCheck) {
- val resultIsNull = ctx.freshName("resultIsNull")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, resultIsNull)
+ val resultIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "resultIsNull")
resultIsNull
} else {
"false"
}
val argValues = arguments.map { e =>
- val argValue = ctx.freshName("argValue")
- ctx.addMutableState(ctx.javaType(e.dataType), argValue)
+ val argValue = ctx.addMutableState(ctx.javaType(e.dataType), "argValue")
argValue
}
@@ -548,7 +546,7 @@ case class MapObjects private(
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val elementJavaType = ctx.javaType(loopVarDataType)
- ctx.addMutableState(elementJavaType, loopValue)
+ ctx.addMutableState(elementJavaType, loopValue, forceInline = true, useFreshName = false)
val genInputData = inputData.genCode(ctx)
val genFunction = lambdaFunction.genCode(ctx)
val dataLength = ctx.freshName("dataLength")
@@ -644,7 +642,7 @@ case class MapObjects private(
}
val loopNullCheck = if (loopIsNull != "false") {
- ctx.addMutableState(ctx.JAVA_BOOLEAN, loopIsNull)
+ ctx.addMutableState(ctx.JAVA_BOOLEAN, loopIsNull, forceInline = true, useFreshName = false)
inputDataType match {
case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);"
case _ => s"$loopIsNull = $loopValue == null;"
@@ -808,10 +806,11 @@ case class CatalystToExternalMap private(
val mapType = inputDataType(inputData.dataType).asInstanceOf[MapType]
val keyElementJavaType = ctx.javaType(mapType.keyType)
- ctx.addMutableState(keyElementJavaType, keyLoopValue)
+ ctx.addMutableState(keyElementJavaType, keyLoopValue, forceInline = true, useFreshName = false)
val genKeyFunction = keyLambdaFunction.genCode(ctx)
val valueElementJavaType = ctx.javaType(mapType.valueType)
- ctx.addMutableState(valueElementJavaType, valueLoopValue)
+ ctx.addMutableState(valueElementJavaType, valueLoopValue, forceInline = true,
+ useFreshName = false)
val genValueFunction = valueLambdaFunction.genCode(ctx)
val genInputData = inputData.genCode(ctx)
val dataLength = ctx.freshName("dataLength")
@@ -844,7 +843,8 @@ case class CatalystToExternalMap private(
val genValueFunctionValue = genFunctionValue(valueLambdaFunction, genValueFunction)
val valueLoopNullCheck = if (valueLoopIsNull != "false") {
- ctx.addMutableState(ctx.JAVA_BOOLEAN, valueLoopIsNull)
+ ctx.addMutableState(ctx.JAVA_BOOLEAN, valueLoopIsNull, forceInline = true,
+ useFreshName = false)
s"$valueLoopIsNull = $valueArray.isNullAt($loopIndex);"
} else {
""
@@ -994,8 +994,8 @@ case class ExternalMapToCatalyst private(
val keyElementJavaType = ctx.javaType(keyType)
val valueElementJavaType = ctx.javaType(valueType)
- ctx.addMutableState(keyElementJavaType, key)
- ctx.addMutableState(valueElementJavaType, value)
+ ctx.addMutableState(keyElementJavaType, key, forceInline = true, useFreshName = false)
+ ctx.addMutableState(valueElementJavaType, value, forceInline = true, useFreshName = false)
val (defineEntries, defineKeyValue) = child.dataType match {
case ObjectType(cls) if classOf[java.util.Map[_, _]].isAssignableFrom(cls) =>
@@ -1031,14 +1031,14 @@ case class ExternalMapToCatalyst private(
}
val keyNullCheck = if (keyIsNull != "false") {
- ctx.addMutableState(ctx.JAVA_BOOLEAN, keyIsNull)
+ ctx.addMutableState(ctx.JAVA_BOOLEAN, keyIsNull, forceInline = true, useFreshName = false)
s"$keyIsNull = $key == null;"
} else {
""
}
val valueNullCheck = if (valueIsNull != "false") {
- ctx.addMutableState(ctx.JAVA_BOOLEAN, valueIsNull)
+ ctx.addMutableState(ctx.JAVA_BOOLEAN, valueIsNull, forceInline = true, useFreshName = false)
s"$valueIsNull = $value == null;"
} else {
""
@@ -1148,7 +1148,6 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean)
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
// Code to initialize the serializer.
- val serializer = ctx.freshName("serializer")
val (serializerClass, serializerInstanceClass) = {
if (kryo) {
(classOf[KryoSerializer].getName, classOf[KryoSerializerInstance].getName)
@@ -1159,14 +1158,14 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean)
// try conf from env, otherwise create a new one
val env = s"${classOf[SparkEnv].getName}.get()"
val sparkConf = s"new ${classOf[SparkConf].getName}()"
- val serializerInit = s"""
- if ($env == null) {
- $serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();
- } else {
- $serializer = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance();
- }
- """
- ctx.addMutableState(serializerInstanceClass, serializer, serializerInit)
+ val serializer = ctx.addMutableState(serializerInstanceClass, "serializerForEncode", v =>
+ s"""
+ |if ($env == null) {
+ | $v = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();
+ |} else {
+ | $v = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance();
+ |}
+ """.stripMargin)
// Code to serialize.
val input = child.genCode(ctx)
@@ -1194,7 +1193,6 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
// Code to initialize the serializer.
- val serializer = ctx.freshName("serializer")
val (serializerClass, serializerInstanceClass) = {
if (kryo) {
(classOf[KryoSerializer].getName, classOf[KryoSerializerInstance].getName)
@@ -1205,14 +1203,14 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B
// try conf from env, otherwise create a new one
val env = s"${classOf[SparkEnv].getName}.get()"
val sparkConf = s"new ${classOf[SparkConf].getName}()"
- val serializerInit = s"""
- if ($env == null) {
- $serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();
- } else {
- $serializer = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance();
- }
- """
- ctx.addMutableState(serializerInstanceClass, serializer, serializerInit)
+ val serializer = ctx.addMutableState(serializerInstanceClass, "serializerForDecode", v =>
+ s"""
+ |if ($env == null) {
+ | $v = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();
+ |} else {
+ | $v = ($serializerInstanceClass) new $serializerClass($env.conf()).newInstance();
+ |}
+ """.stripMargin)
// Code to deserialize.
val input = child.genCode(ctx)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
index b4aefe6cff73e..8bc936fcbfc31 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
@@ -77,9 +77,8 @@ case class Rand(child: Expression) extends RDG {
override protected def evalInternal(input: InternalRow): Double = rng.nextDouble()
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val rngTerm = ctx.freshName("rng")
val className = classOf[XORShiftRandom].getName
- ctx.addMutableState(className, rngTerm)
+ val rngTerm = ctx.addMutableState(className, "rng")
ctx.addPartitionInitializationStatement(
s"$rngTerm = new $className(${seed}L + partitionIndex);")
ev.copy(code = s"""
@@ -112,9 +111,8 @@ case class Randn(child: Expression) extends RDG {
override protected def evalInternal(input: InternalRow): Double = rng.nextGaussian()
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val rngTerm = ctx.freshName("rng")
val className = classOf[XORShiftRandom].getName
- ctx.addMutableState(className, rngTerm)
+ val rngTerm = ctx.addMutableState(className, "rng")
ctx.addPartitionInitializationStatement(
s"$rngTerm = new $className(${seed}L + partitionIndex);")
ev.copy(code = s"""
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
index 53d7096dd87d3..fa5425c77ebba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala
@@ -112,15 +112,15 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val patternClass = classOf[Pattern].getName
val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex"
- val pattern = ctx.freshName("pattern")
if (right.foldable) {
val rVal = right.eval()
if (rVal != null) {
val regexStr =
StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString()))
- ctx.addMutableState(patternClass, pattern,
- s"""$pattern = ${patternClass}.compile("$regexStr");""")
+ // inline mutable state since not many Like operations in a task
+ val pattern = ctx.addMutableState(patternClass, "patternLike",
+ v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
// We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
val eval = left.genCode(ctx)
@@ -139,6 +139,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi
""")
}
} else {
+ val pattern = ctx.freshName("pattern")
val rightStr = ctx.freshName("rightStr")
nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
s"""
@@ -187,15 +188,15 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
val patternClass = classOf[Pattern].getName
- val pattern = ctx.freshName("pattern")
if (right.foldable) {
val rVal = right.eval()
if (rVal != null) {
val regexStr =
StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString())
- ctx.addMutableState(patternClass, pattern,
- s"""$pattern = ${patternClass}.compile("$regexStr");""")
+ // inline mutable state since not many RLike operations in a task
+ val pattern = ctx.addMutableState(patternClass, "patternRLike",
+ v => s"""$v = ${patternClass}.compile("$regexStr");""", forceInline = true)
// We don't use nullSafeCodeGen here because we don't want to re-evaluate right again.
val eval = left.genCode(ctx)
@@ -215,6 +216,7 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress
}
} else {
val rightStr = ctx.freshName("rightStr")
+ val pattern = ctx.freshName("pattern")
nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
s"""
String $rightStr = ${eval2}.toString();
@@ -316,11 +318,6 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio
override def prettyName: String = "regexp_replace"
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val termLastRegex = ctx.freshName("lastRegex")
- val termPattern = ctx.freshName("pattern")
-
- val termLastReplacement = ctx.freshName("lastReplacement")
- val termLastReplacementInUTF8 = ctx.freshName("lastReplacementInUTF8")
val termResult = ctx.freshName("termResult")
val classNamePattern = classOf[Pattern].getCanonicalName
@@ -328,11 +325,10 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio
val matcher = ctx.freshName("matcher")
- ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;")
- ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;")
- ctx.addMutableState("String", termLastReplacement, s"${termLastReplacement} = null;")
- ctx.addMutableState("UTF8String",
- termLastReplacementInUTF8, s"${termLastReplacementInUTF8} = null;")
+ val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex")
+ val termPattern = ctx.addMutableState(classNamePattern, "pattern")
+ val termLastReplacement = ctx.addMutableState("String", "lastReplacement")
+ val termLastReplacementInUTF8 = ctx.addMutableState("UTF8String", "lastReplacementInUTF8")
val setEvNotNull = if (nullable) {
s"${ev.isNull} = false;"
@@ -414,14 +410,12 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio
override def prettyName: String = "regexp_extract"
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val termLastRegex = ctx.freshName("lastRegex")
- val termPattern = ctx.freshName("pattern")
val classNamePattern = classOf[Pattern].getCanonicalName
val matcher = ctx.freshName("matcher")
val matchResult = ctx.freshName("matchResult")
- ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;")
- ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;")
+ val termLastRegex = ctx.addMutableState("UTF8String", "lastRegex")
+ val termPattern = ctx.addMutableState(classNamePattern, "pattern")
val setEvNotNull = if (nullable) {
s"${ev.isNull} = false;"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
index 8c4d2fd686be5..c02c41db1668e 100755
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala
@@ -291,8 +291,7 @@ case class Elt(children: Seq[Expression])
val indexVal = ctx.freshName("index")
val indexMatched = ctx.freshName("eltIndexMatched")
- val stringVal = ctx.freshName("stringVal")
- ctx.addMutableState(ctx.javaType(dataType), stringVal)
+ val stringVal = ctx.addMutableState(ctx.javaType(dataType), "stringVal")
val assignStringValue = strings.zipWithIndex.map { case (eval, index) =>
s"""
@@ -532,14 +531,11 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac
}
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
- val termLastMatching = ctx.freshName("lastMatching")
- val termLastReplace = ctx.freshName("lastReplace")
- val termDict = ctx.freshName("dict")
val classNameDict = classOf[JMap[Character, Character]].getCanonicalName
- ctx.addMutableState("UTF8String", termLastMatching, s"$termLastMatching = null;")
- ctx.addMutableState("UTF8String", termLastReplace, s"$termLastReplace = null;")
- ctx.addMutableState(classNameDict, termDict, s"$termDict = null;")
+ val termLastMatching = ctx.addMutableState("UTF8String", "lastMatching")
+ val termLastReplace = ctx.addMutableState("UTF8String", "lastReplace")
+ val termDict = ctx.addMutableState(classNameDict, "dict")
nullSafeCodeGen(ctx, ev, (src, matching, replace) => {
val check = if (matchingExpr.foldable && replaceExpr.foldable) {
@@ -2065,15 +2061,12 @@ case class FormatNumber(x: Expression, d: Expression)
// SPARK-13515: US Locale configures the DecimalFormat object to use a dot ('.')
// as a decimal separator.
val usLocale = "US"
- val lastDValue = ctx.freshName("lastDValue")
- val pattern = ctx.freshName("pattern")
- val numberFormat = ctx.freshName("numberFormat")
val i = ctx.freshName("i")
val dFormat = ctx.freshName("dFormat")
- ctx.addMutableState(ctx.JAVA_INT, lastDValue, s"$lastDValue = -100;")
- ctx.addMutableState(sb, pattern, s"$pattern = new $sb();")
- ctx.addMutableState(df, numberFormat,
- s"""$numberFormat = new $df("", new $dfs($l.$usLocale));""")
+ val lastDValue = ctx.addMutableState(ctx.JAVA_INT, "lastDValue", v => s"$v = -100;")
+ val pattern = ctx.addMutableState(sb, "pattern", v => s"$v = new $sb();")
+ val numberFormat = ctx.addMutableState(df, "numberFormat",
+ v => s"""$v = new $df("", new $dfs($l.$usLocale));""")
s"""
if ($d >= 0) {
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 5acadf8cf330e..6a4d1e997c3c5 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
@@ -47,7 +47,62 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations)
def batches: Seq[Batch] = {
- Batch("Eliminate Distinct", Once, EliminateDistinct) ::
+ val operatorOptimizationRuleSet =
+ Seq(
+ // Operator push down
+ PushProjectionThroughUnion,
+ ReorderJoin,
+ EliminateOuterJoin,
+ PushPredicateThroughJoin,
+ PushDownPredicate,
+ LimitPushDown,
+ ColumnPruning,
+ InferFiltersFromConstraints,
+ // Operator combine
+ CollapseRepartition,
+ CollapseProject,
+ CollapseWindow,
+ CombineFilters,
+ CombineLimits,
+ CombineUnions,
+ // Constant folding and strength reduction
+ NullPropagation,
+ ConstantPropagation,
+ FoldablePropagation,
+ OptimizeIn,
+ ConstantFolding,
+ ReorderAssociativeOperator,
+ LikeSimplification,
+ BooleanSimplification,
+ SimplifyConditionals,
+ RemoveDispensableExpressions,
+ SimplifyBinaryComparison,
+ PruneFilters,
+ EliminateSorts,
+ SimplifyCasts,
+ SimplifyCaseConversionExpressions,
+ RewriteCorrelatedScalarSubquery,
+ EliminateSerialization,
+ RemoveRedundantAliases,
+ RemoveRedundantProject,
+ SimplifyCreateStructOps,
+ SimplifyCreateArrayOps,
+ SimplifyCreateMapOps,
+ CombineConcats) ++
+ extendedOperatorOptimizationRules
+
+ val operatorOptimizationBatch: Seq[Batch] = {
+ val rulesWithoutInferFiltersFromConstraints =
+ operatorOptimizationRuleSet.filterNot(_ == InferFiltersFromConstraints)
+ Batch("Operator Optimization before Inferring Filters", fixedPoint,
+ rulesWithoutInferFiltersFromConstraints: _*) ::
+ Batch("Infer Filters", Once,
+ InferFiltersFromConstraints) ::
+ Batch("Operator Optimization after Inferring Filters", fixedPoint,
+ rulesWithoutInferFiltersFromConstraints: _*) :: Nil
+ }
+
+ (Batch("Eliminate Distinct", Once, EliminateDistinct) ::
// Technically some of the rules in Finish Analysis are not optimizer rules and belong more
// in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime).
// However, because we also use the analyzer to canonicalized queries (for view definition),
@@ -81,68 +136,26 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
ReplaceDistinctWithAggregate) ::
Batch("Aggregate", fixedPoint,
RemoveLiteralFromGroupExpressions,
- RemoveRepetitionFromGroupExpressions) ::
- Batch("Operator Optimizations", fixedPoint, Seq(
- // Operator push down
- PushProjectionThroughUnion,
- ReorderJoin,
- EliminateOuterJoin,
- InferFiltersFromConstraints,
- BooleanSimplification,
- PushPredicateThroughJoin,
- PushDownPredicate,
- LimitPushDown,
- ColumnPruning,
- // Operator combine
- CollapseRepartition,
- CollapseProject,
- CollapseWindow,
- CombineFilters,
- CombineLimits,
- CombineUnions,
- // Constant folding and strength reduction
- NullPropagation,
- ConstantPropagation,
- FoldablePropagation,
- OptimizeIn,
- ConstantFolding,
- ReorderAssociativeOperator,
- LikeSimplification,
- BooleanSimplification,
- SimplifyConditionals,
- RemoveDispensableExpressions,
- SimplifyBinaryComparison,
- PruneFilters,
- EliminateSorts,
- SimplifyCasts,
- SimplifyCaseConversionExpressions,
- RewriteCorrelatedScalarSubquery,
- EliminateSerialization,
- RemoveRedundantAliases,
- RemoveRedundantProject,
- SimplifyCreateStructOps,
- SimplifyCreateArrayOps,
- SimplifyCreateMapOps,
- CombineConcats) ++
- extendedOperatorOptimizationRules: _*) ::
+ RemoveRepetitionFromGroupExpressions) :: Nil ++
+ operatorOptimizationBatch) :+
Batch("Join Reorder", Once,
- CostBasedJoinReorder) ::
+ CostBasedJoinReorder) :+
Batch("Decimal Optimizations", fixedPoint,
- DecimalAggregates) ::
+ DecimalAggregates) :+
Batch("Object Expressions Optimization", fixedPoint,
EliminateMapObjects,
- CombineTypedFilters) ::
+ CombineTypedFilters) :+
Batch("LocalRelation", fixedPoint,
ConvertToLocalRelation,
- PropagateEmptyRelation) ::
+ PropagateEmptyRelation) :+
// The following batch should be executed after batch "Join Reorder" and "LocalRelation".
Batch("Check Cartesian Products", Once,
- CheckCartesianProducts) ::
+ CheckCartesianProducts) :+
Batch("RewriteSubquery", Once,
RewritePredicateSubquery,
ColumnPruning,
CollapseProject,
- RemoveRedundantProject) :: Nil
+ RemoveRedundantProject)
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 6305b6c84bae3..85295aff19808 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -614,7 +614,6 @@ object SimplifyCasts extends Rule[LogicalPlan] {
object RemoveDispensableExpressions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case UnaryPositive(child) => child
- case PromotePrecision(child) => child
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
index 6f868cbd072c8..71e852afe0659 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
+import scala.collection.mutable.ArrayBuffer
import scala.math.BigDecimal.RoundingMode
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
@@ -212,4 +213,172 @@ object EstimationUtils {
}
}
+ /**
+ * Returns overlapped ranges between two histograms, in the given value range
+ * [lowerBound, upperBound].
+ */
+ def getOverlappedRanges(
+ leftHistogram: Histogram,
+ rightHistogram: Histogram,
+ lowerBound: Double,
+ upperBound: Double): Seq[OverlappedRange] = {
+ val overlappedRanges = new ArrayBuffer[OverlappedRange]()
+ // Only bins whose range intersect [lowerBound, upperBound] have join possibility.
+ val leftBins = leftHistogram.bins
+ .filter(b => b.lo <= upperBound && b.hi >= lowerBound)
+ val rightBins = rightHistogram.bins
+ .filter(b => b.lo <= upperBound && b.hi >= lowerBound)
+
+ leftBins.foreach { lb =>
+ rightBins.foreach { rb =>
+ val (left, leftHeight) = trimBin(lb, leftHistogram.height, lowerBound, upperBound)
+ val (right, rightHeight) = trimBin(rb, rightHistogram.height, lowerBound, upperBound)
+ // Only collect overlapped ranges.
+ if (left.lo <= right.hi && left.hi >= right.lo) {
+ // Collect overlapped ranges.
+ val range = if (right.lo >= left.lo && right.hi >= left.hi) {
+ // Case1: the left bin is "smaller" than the right bin
+ // left.lo right.lo left.hi right.hi
+ // --------+------------------+------------+----------------+------->
+ if (left.hi == right.lo) {
+ // The overlapped range has only one value.
+ OverlappedRange(
+ lo = right.lo,
+ hi = right.lo,
+ leftNdv = 1,
+ rightNdv = 1,
+ leftNumRows = leftHeight / left.ndv,
+ rightNumRows = rightHeight / right.ndv
+ )
+ } else {
+ val leftRatio = (left.hi - right.lo) / (left.hi - left.lo)
+ val rightRatio = (left.hi - right.lo) / (right.hi - right.lo)
+ OverlappedRange(
+ lo = right.lo,
+ hi = left.hi,
+ leftNdv = left.ndv * leftRatio,
+ rightNdv = right.ndv * rightRatio,
+ leftNumRows = leftHeight * leftRatio,
+ rightNumRows = rightHeight * rightRatio
+ )
+ }
+ } else if (right.lo <= left.lo && right.hi <= left.hi) {
+ // Case2: the left bin is "larger" than the right bin
+ // right.lo left.lo right.hi left.hi
+ // --------+------------------+------------+----------------+------->
+ if (right.hi == left.lo) {
+ // The overlapped range has only one value.
+ OverlappedRange(
+ lo = right.hi,
+ hi = right.hi,
+ leftNdv = 1,
+ rightNdv = 1,
+ leftNumRows = leftHeight / left.ndv,
+ rightNumRows = rightHeight / right.ndv
+ )
+ } else {
+ val leftRatio = (right.hi - left.lo) / (left.hi - left.lo)
+ val rightRatio = (right.hi - left.lo) / (right.hi - right.lo)
+ OverlappedRange(
+ lo = left.lo,
+ hi = right.hi,
+ leftNdv = left.ndv * leftRatio,
+ rightNdv = right.ndv * rightRatio,
+ leftNumRows = leftHeight * leftRatio,
+ rightNumRows = rightHeight * rightRatio
+ )
+ }
+ } else if (right.lo >= left.lo && right.hi <= left.hi) {
+ // Case3: the left bin contains the right bin
+ // left.lo right.lo right.hi left.hi
+ // --------+------------------+------------+----------------+------->
+ val leftRatio = (right.hi - right.lo) / (left.hi - left.lo)
+ OverlappedRange(
+ lo = right.lo,
+ hi = right.hi,
+ leftNdv = left.ndv * leftRatio,
+ rightNdv = right.ndv,
+ leftNumRows = leftHeight * leftRatio,
+ rightNumRows = rightHeight
+ )
+ } else {
+ assert(right.lo <= left.lo && right.hi >= left.hi)
+ // Case4: the right bin contains the left bin
+ // right.lo left.lo left.hi right.hi
+ // --------+------------------+------------+----------------+------->
+ val rightRatio = (left.hi - left.lo) / (right.hi - right.lo)
+ OverlappedRange(
+ lo = left.lo,
+ hi = left.hi,
+ leftNdv = left.ndv,
+ rightNdv = right.ndv * rightRatio,
+ leftNumRows = leftHeight,
+ rightNumRows = rightHeight * rightRatio
+ )
+ }
+ overlappedRanges += range
+ }
+ }
+ }
+ overlappedRanges
+ }
+
+ /**
+ * Given an original bin and a value range [lowerBound, upperBound], returns the trimmed part
+ * of the bin in that range and its number of rows.
+ * @param bin the input histogram bin.
+ * @param height the number of rows of the given histogram bin inside an equi-height histogram.
+ * @param lowerBound lower bound of the given range.
+ * @param upperBound upper bound of the given range.
+ * @return trimmed part of the given bin and its number of rows.
+ */
+ def trimBin(bin: HistogramBin, height: Double, lowerBound: Double, upperBound: Double)
+ : (HistogramBin, Double) = {
+ val (lo, hi) = if (bin.lo <= lowerBound && bin.hi >= upperBound) {
+ // bin.lo lowerBound upperBound bin.hi
+ // --------+------------------+------------+-------------+------->
+ (lowerBound, upperBound)
+ } else if (bin.lo <= lowerBound && bin.hi >= lowerBound) {
+ // bin.lo lowerBound bin.hi upperBound
+ // --------+------------------+------------+-------------+------->
+ (lowerBound, bin.hi)
+ } else if (bin.lo <= upperBound && bin.hi >= upperBound) {
+ // lowerBound bin.lo upperBound bin.hi
+ // --------+------------------+------------+-------------+------->
+ (bin.lo, upperBound)
+ } else {
+ // lowerBound bin.lo bin.hi upperBound
+ // --------+------------------+------------+-------------+------->
+ assert(bin.lo >= lowerBound && bin.hi <= upperBound)
+ (bin.lo, bin.hi)
+ }
+
+ if (hi == lo) {
+ // Note that bin.hi == bin.lo also falls into this branch.
+ (HistogramBin(lo, hi, 1), height / bin.ndv)
+ } else {
+ assert(bin.hi != bin.lo)
+ val ratio = (hi - lo) / (bin.hi - bin.lo)
+ (HistogramBin(lo, hi, math.ceil(bin.ndv * ratio).toLong), height * ratio)
+ }
+ }
+
+ /**
+ * A join between two equi-height histograms may produce multiple overlapped ranges.
+ * Each overlapped range is produced by a part of one bin in the left histogram and a part of
+ * one bin in the right histogram.
+ * @param lo lower bound of this overlapped range.
+ * @param hi higher bound of this overlapped range.
+ * @param leftNdv ndv in the left part.
+ * @param rightNdv ndv in the right part.
+ * @param leftNumRows number of rows in the left part.
+ * @param rightNumRows number of rows in the right part.
+ */
+ case class OverlappedRange(
+ lo: Double,
+ hi: Double,
+ leftNdv: Double,
+ rightNdv: Double,
+ leftNumRows: Double,
+ rightNumRows: Double)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
index b073108c26ee5..f0294a4246703 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
@@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, Statistics}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Histogram, Join, Statistics}
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
@@ -191,8 +191,19 @@ case class JoinEstimation(join: Join) extends Logging {
val rInterval = ValueInterval(rightKeyStat.min, rightKeyStat.max, rightKey.dataType)
if (ValueInterval.isIntersected(lInterval, rInterval)) {
val (newMin, newMax) = ValueInterval.intersect(lInterval, rInterval, leftKey.dataType)
- val (card, joinStat) = computeByNdv(leftKey, rightKey, newMin, newMax)
- keyStatsAfterJoin += (leftKey -> joinStat, rightKey -> joinStat)
+ val (card, joinStat) = (leftKeyStat.histogram, rightKeyStat.histogram) match {
+ case (Some(l: Histogram), Some(r: Histogram)) =>
+ computeByHistogram(leftKey, rightKey, l, r, newMin, newMax)
+ case _ =>
+ computeByNdv(leftKey, rightKey, newMin, newMax)
+ }
+ keyStatsAfterJoin += (
+ // Histograms are propagated as unchanged. During future estimation, they should be
+ // truncated by the updated max/min. In this way, only pointers of the histograms are
+ // propagated and thus reduce memory consumption.
+ leftKey -> joinStat.copy(histogram = leftKeyStat.histogram),
+ rightKey -> joinStat.copy(histogram = rightKeyStat.histogram)
+ )
// Return cardinality estimated from the most selective join keys.
if (card < joinCard) joinCard = card
} else {
@@ -225,6 +236,43 @@ case class JoinEstimation(join: Join) extends Logging {
(ceil(card), newStats)
}
+ /** Compute join cardinality using equi-height histograms. */
+ private def computeByHistogram(
+ leftKey: AttributeReference,
+ rightKey: AttributeReference,
+ leftHistogram: Histogram,
+ rightHistogram: Histogram,
+ newMin: Option[Any],
+ newMax: Option[Any]): (BigInt, ColumnStat) = {
+ val overlappedRanges = getOverlappedRanges(
+ leftHistogram = leftHistogram,
+ rightHistogram = rightHistogram,
+ // Only numeric values have equi-height histograms.
+ lowerBound = newMin.get.toString.toDouble,
+ upperBound = newMax.get.toString.toDouble)
+
+ var card: BigDecimal = 0
+ var totalNdv: Double = 0
+ for (i <- overlappedRanges.indices) {
+ val range = overlappedRanges(i)
+ if (i == 0 || range.hi != overlappedRanges(i - 1).hi) {
+ // If range.hi == overlappedRanges(i - 1).hi, that means the current range has only one
+ // value, and this value is already counted in the previous range. So there is no need to
+ // count it in this range.
+ totalNdv += math.min(range.leftNdv, range.rightNdv)
+ }
+ // Apply the formula in this overlapped range.
+ card += range.leftNumRows * range.rightNumRows / math.max(range.leftNdv, range.rightNdv)
+ }
+
+ val leftKeyStat = leftStats.attributeStats(leftKey)
+ val rightKeyStat = rightStats.attributeStats(rightKey)
+ val newMaxLen = math.min(leftKeyStat.maxLen, rightKeyStat.maxLen)
+ val newAvgLen = (leftKeyStat.avgLen + rightKeyStat.avgLen) / 2
+ val newStats = ColumnStat(ceil(totalNdv), newMin, newMax, 0, newAvgLen, newMaxLen)
+ (ceil(card), newStats)
+ }
+
/**
* Propagate or update column stats for output attributes.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index b1ed25645b36c..fa69b8af62c85 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -45,7 +45,8 @@ object DateTimeUtils {
// it's 2440587.5, rounding up to compatible with Hive
final val JULIAN_DAY_OF_EPOCH = 2440588
final val SECONDS_PER_DAY = 60 * 60 * 24L
- final val MICROS_PER_SECOND = 1000L * 1000L
+ final val MICROS_PER_MILLIS = 1000L
+ final val MICROS_PER_SECOND = MICROS_PER_MILLIS * MILLIS_PER_SECOND
final val MILLIS_PER_SECOND = 1000L
final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L
final val MICROS_PER_DAY = MICROS_PER_SECOND * SECONDS_PER_DAY
@@ -909,6 +910,15 @@ object DateTimeUtils {
math.round(diff * 1e8) / 1e8
}
+ // Thursday = 0 since 1970/Jan/01 => Thursday
+ private val SUNDAY = 3
+ private val MONDAY = 4
+ private val TUESDAY = 5
+ private val WEDNESDAY = 6
+ private val THURSDAY = 0
+ private val FRIDAY = 1
+ private val SATURDAY = 2
+
/*
* Returns day of week from String. Starting from Thursday, marked as 0.
* (Because 1970-01-01 is Thursday).
@@ -916,13 +926,13 @@ object DateTimeUtils {
def getDayOfWeekFromString(string: UTF8String): Int = {
val dowString = string.toString.toUpperCase(Locale.ROOT)
dowString match {
- case "SU" | "SUN" | "SUNDAY" => 3
- case "MO" | "MON" | "MONDAY" => 4
- case "TU" | "TUE" | "TUESDAY" => 5
- case "WE" | "WED" | "WEDNESDAY" => 6
- case "TH" | "THU" | "THURSDAY" => 0
- case "FR" | "FRI" | "FRIDAY" => 1
- case "SA" | "SAT" | "SATURDAY" => 2
+ case "SU" | "SUN" | "SUNDAY" => SUNDAY
+ case "MO" | "MON" | "MONDAY" => MONDAY
+ case "TU" | "TUE" | "TUESDAY" => TUESDAY
+ case "WE" | "WED" | "WEDNESDAY" => WEDNESDAY
+ case "TH" | "THU" | "THURSDAY" => THURSDAY
+ case "FR" | "FRI" | "FRIDAY" => FRIDAY
+ case "SA" | "SAT" | "SATURDAY" => SATURDAY
case _ => -1
}
}
@@ -944,9 +954,16 @@ object DateTimeUtils {
date + daysToMonthEnd
}
- private val TRUNC_TO_YEAR = 1
- private val TRUNC_TO_MONTH = 2
- private val TRUNC_INVALID = -1
+ // Visible for testing.
+ private[sql] val TRUNC_TO_YEAR = 1
+ private[sql] val TRUNC_TO_MONTH = 2
+ private[sql] val TRUNC_TO_QUARTER = 3
+ private[sql] val TRUNC_TO_WEEK = 4
+ private[sql] val TRUNC_TO_DAY = 5
+ private[sql] val TRUNC_TO_HOUR = 6
+ private[sql] val TRUNC_TO_MINUTE = 7
+ private[sql] val TRUNC_TO_SECOND = 8
+ private[sql] val TRUNC_INVALID = -1
/**
* Returns the trunc date from original date and trunc level.
@@ -964,7 +981,62 @@ object DateTimeUtils {
}
/**
- * Returns the truncate level, could be TRUNC_YEAR, TRUNC_MONTH, or TRUNC_INVALID,
+ * Returns the trunc date time from original date time and trunc level.
+ * Trunc level should be generated using `parseTruncLevel()`, should be between 1 and 8
+ */
+ def truncTimestamp(t: SQLTimestamp, level: Int, timeZone: TimeZone): SQLTimestamp = {
+ var millis = t / MICROS_PER_MILLIS
+ val truncated = level match {
+ case TRUNC_TO_YEAR =>
+ val dDays = millisToDays(millis, timeZone)
+ daysToMillis(truncDate(dDays, level), timeZone)
+ case TRUNC_TO_MONTH =>
+ val dDays = millisToDays(millis, timeZone)
+ daysToMillis(truncDate(dDays, level), timeZone)
+ case TRUNC_TO_DAY =>
+ val offset = timeZone.getOffset(millis)
+ millis += offset
+ millis - millis % (MILLIS_PER_SECOND * SECONDS_PER_DAY) - offset
+ case TRUNC_TO_HOUR =>
+ val offset = timeZone.getOffset(millis)
+ millis += offset
+ millis - millis % (60 * 60 * MILLIS_PER_SECOND) - offset
+ case TRUNC_TO_MINUTE =>
+ millis - millis % (60 * MILLIS_PER_SECOND)
+ case TRUNC_TO_SECOND =>
+ millis - millis % MILLIS_PER_SECOND
+ case TRUNC_TO_WEEK =>
+ val dDays = millisToDays(millis, timeZone)
+ val prevMonday = getNextDateForDayOfWeek(dDays - 7, MONDAY)
+ daysToMillis(prevMonday, timeZone)
+ case TRUNC_TO_QUARTER =>
+ val dDays = millisToDays(millis, timeZone)
+ millis = daysToMillis(truncDate(dDays, TRUNC_TO_MONTH), timeZone)
+ val cal = Calendar.getInstance()
+ cal.setTimeInMillis(millis)
+ val quarter = getQuarter(dDays)
+ val month = quarter match {
+ case 1 => Calendar.JANUARY
+ case 2 => Calendar.APRIL
+ case 3 => Calendar.JULY
+ case 4 => Calendar.OCTOBER
+ }
+ cal.set(Calendar.MONTH, month)
+ cal.getTimeInMillis()
+ case _ =>
+ // caller make sure that this should never be reached
+ sys.error(s"Invalid trunc level: $level")
+ }
+ truncated * MICROS_PER_MILLIS
+ }
+
+ def truncTimestamp(d: SQLTimestamp, level: Int): SQLTimestamp = {
+ truncTimestamp(d, level, defaultTimeZone())
+ }
+
+ /**
+ * Returns the truncate level, could be TRUNC_YEAR, TRUNC_MONTH, TRUNC_TO_DAY, TRUNC_TO_HOUR,
+ * TRUNC_TO_MINUTE, TRUNC_TO_SECOND, TRUNC_TO_WEEK, TRUNC_TO_QUARTER or TRUNC_INVALID,
* TRUNC_INVALID means unsupported truncate level.
*/
def parseTruncLevel(format: UTF8String): Int = {
@@ -974,6 +1046,12 @@ object DateTimeUtils {
format.toString.toUpperCase(Locale.ROOT) match {
case "YEAR" | "YYYY" | "YY" => TRUNC_TO_YEAR
case "MON" | "MONTH" | "MM" => TRUNC_TO_MONTH
+ case "DAY" | "DD" => TRUNC_TO_DAY
+ case "HOUR" => TRUNC_TO_HOUR
+ case "MINUTE" => TRUNC_TO_MINUTE
+ case "SECOND" => TRUNC_TO_SECOND
+ case "WEEK" => TRUNC_TO_WEEK
+ case "QUARTER" => TRUNC_TO_QUARTER
case _ => TRUNC_INVALID
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index cf7e3ebce7411..bdc8d92e84079 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicReference
import scala.collection.JavaConverters._
import scala.collection.immutable
+import scala.util.matching.Regex
import org.apache.hadoop.fs.Path
@@ -1035,6 +1036,14 @@ object SQLConf {
.booleanConf
.createWithDefault(true)
+ val SQL_STRING_REDACTION_PATTERN =
+ ConfigBuilder("spark.sql.redaction.string.regex")
+ .doc("Regex to decide which parts of strings produced by Spark contain sensitive " +
+ "information. When this regex matches a string part, that string part is replaced by a " +
+ "dummy value. This is currently used to redact the output of SQL explain commands. " +
+ "When this conf is not set, the value from `spark.redaction.string.regex` is used.")
+ .fallbackConf(org.apache.spark.internal.config.STRING_REDACTION_PATTERN)
+
object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
@@ -1173,6 +1182,8 @@ class SQLConf extends Serializable with Logging {
def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS)
+ def stringRedationPattern: Option[Regex] = SQL_STRING_REDACTION_PATTERN.readFrom(reader)
+
/**
* Returns the [[Resolver]] for the current configuration, which can be used to determine if two
* identifiers are equal.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
index c018fc8a332fa..fe0ad39c29025 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala
@@ -95,4 +95,11 @@ object StaticSQLConf {
.stringConf
.toSequence
.createOptional
+
+ val UI_RETAINED_EXECUTIONS =
+ buildStaticConf("spark.sql.ui.retainedExecutions")
+ .doc("Number of executions to retain in the Spark UI.")
+ .intConf
+ .createWithDefault(1000)
+
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
index be638d80e45d8..6edb4348f8309 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
@@ -348,10 +348,10 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
test("SPARK-22704: Least and greatest use less global variables") {
val ctx1 = new CodegenContext()
Least(Seq(Literal(1), Literal(1))).genCode(ctx1)
- assert(ctx1.mutableStates.size == 1)
+ assert(ctx1.inlinedMutableStates.size == 1)
val ctx2 = new CodegenContext()
Greatest(Seq(Literal(1), Literal(1))).genCode(ctx2)
- assert(ctx2.mutableStates.size == 1)
+ assert(ctx2.inlinedMutableStates.size == 1)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
index 65617be05a434..1dd040e4696a1 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
@@ -851,6 +851,6 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
val ctx = new CodegenContext
cast("1", IntegerType).genCode(ctx)
cast("2", LongType).genCode(ctx)
- assert(ctx.mutableStates.length == 0)
+ assert(ctx.inlinedMutableStates.length == 0)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
index a969811019161..b1a44528e64d7 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
@@ -385,20 +385,43 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper {
val ctx = new CodegenContext
val schema = new StructType().add("a", IntegerType).add("b", StringType)
CreateExternalRow(Seq(Literal(1), Literal("x")), schema).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
test("SPARK-22696: InitializeJavaBean should not use global variables") {
val ctx = new CodegenContext
InitializeJavaBean(Literal.fromObject(new java.util.LinkedList[Int]),
Map("add" -> Literal(1))).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
test("SPARK-22716: addReferenceObj should not add mutable states") {
val ctx = new CodegenContext
val foo = new Object()
ctx.addReferenceObj("foo", foo)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
+ }
+
+ test("SPARK-18016: define mutable states by using an array") {
+ val ctx1 = new CodegenContext
+ for (i <- 1 to CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10) {
+ ctx1.addMutableState(ctx1.JAVA_INT, "i", v => s"$v = $i;")
+ }
+ assert(ctx1.inlinedMutableStates.size == CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD)
+ // When the number of primitive type mutable states is over the threshold, others are
+ // allocated into an array
+ assert(ctx1.arrayCompactedMutableStates.get(ctx1.JAVA_INT).get.arrayNames.size == 1)
+ assert(ctx1.mutableStateInitCode.size == CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10)
+
+ val ctx2 = new CodegenContext
+ for (i <- 1 to CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) {
+ ctx2.addMutableState("InternalRow[]", "r", v => s"$v = new InternalRow[$i];")
+ }
+ // When the number of non-primitive type mutable states is over the threshold, others are
+ // allocated into a new array
+ assert(ctx2.inlinedMutableStates.isEmpty)
+ assert(ctx2.arrayCompactedMutableStates.get("InternalRow[]").get.arrayNames.size == 2)
+ assert(ctx2.arrayCompactedMutableStates("InternalRow[]").getCurrentIndex == 10)
+ assert(ctx2.mutableStateInitCode.size == CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
index 6dfca7d73a3df..84190f0bd5f7d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
@@ -304,6 +304,6 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
test("SPARK-22693: CreateNamedStruct should not use global variables") {
val ctx = new CodegenContext
CreateNamedStruct(Seq("a", "x", "b", 2.0)).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala
index 60d84aae1fa3f..a099119732e25 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala
@@ -150,6 +150,6 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
test("SPARK-22705: case when should use less global variables") {
val ctx = new CodegenContext()
CaseWhen(Seq((Literal.create(false, BooleanType), Literal(1))), Literal(-1)).genCode(ctx)
- assert(ctx.mutableStates.size == 1)
+ assert(ctx.inlinedMutableStates.size == 1)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index 89d99f9678cda..63f6ceeb21b96 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -527,7 +527,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
NextDay(Literal(Date.valueOf("2015-07-23")), Literal.create(null, StringType)), null)
}
- test("function trunc") {
+ test("TruncDate") {
def testTrunc(input: Date, fmt: String, expected: Date): Unit = {
checkEvaluation(TruncDate(Literal.create(input, DateType), Literal.create(fmt, StringType)),
expected)
@@ -543,11 +543,82 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
testTrunc(date, fmt, Date.valueOf("2015-07-01"))
}
testTrunc(date, "DD", null)
+ testTrunc(date, "SECOND", null)
+ testTrunc(date, "HOUR", null)
testTrunc(date, null, null)
testTrunc(null, "MON", null)
testTrunc(null, null, null)
}
+ test("TruncTimestamp") {
+ def testTrunc(input: Timestamp, fmt: String, expected: Timestamp): Unit = {
+ checkEvaluation(
+ TruncTimestamp(Literal.create(fmt, StringType), Literal.create(input, TimestampType)),
+ expected)
+ checkEvaluation(
+ TruncTimestamp(
+ NonFoldableLiteral.create(fmt, StringType), Literal.create(input, TimestampType)),
+ expected)
+ }
+
+ withDefaultTimeZone(TimeZoneGMT) {
+ val inputDate = Timestamp.valueOf("2015-07-22 05:30:06")
+
+ Seq("yyyy", "YYYY", "year", "YEAR", "yy", "YY").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-01-01 00:00:00"))
+ }
+
+ Seq("month", "MONTH", "mon", "MON", "mm", "MM").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-01 00:00:00"))
+ }
+
+ Seq("DAY", "day", "DD", "dd").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-22 00:00:00"))
+ }
+
+ Seq("HOUR", "hour").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-22 05:00:00"))
+ }
+
+ Seq("MINUTE", "minute").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-22 05:30:00"))
+ }
+
+ Seq("SECOND", "second").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-22 05:30:06"))
+ }
+
+ Seq("WEEK", "week").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-20 00:00:00"))
+ }
+
+ Seq("QUARTER", "quarter").foreach { fmt =>
+ testTrunc(
+ inputDate, fmt,
+ Timestamp.valueOf("2015-07-01 00:00:00"))
+ }
+
+ testTrunc(inputDate, "INVALID", null)
+ testTrunc(inputDate, null, null)
+ testTrunc(null, "MON", null)
+ testTrunc(null, null, null)
+ }
+ }
+
test("from_unixtime") {
val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS"
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala
index a23cd95632770..cc6c15cb2c909 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala
@@ -159,7 +159,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
test("SPARK-22705: Coalesce should use less global variables") {
val ctx = new CodegenContext()
Coalesce(Seq(Literal("a"), Literal("b"))).genCode(ctx)
- assert(ctx.mutableStates.size == 1)
+ assert(ctx.inlinedMutableStates.size == 1)
}
test("AtLeastNNonNulls should not throw 64kb exception") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala
index 15cb0bea08f17..8a8f8e10225fa 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala
@@ -249,7 +249,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper {
test("SPARK-22705: In should use less global variables") {
val ctx = new CodegenContext()
In(Literal(1.0D), Seq(Literal(1.0D), Literal(2.0D))).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
test("INSET") {
@@ -440,6 +440,6 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper {
test("SPARK-22693: InSet should not use global variables") {
val ctx = new CodegenContext
InSet(Literal(1), Set(1, 2, 3, 4)).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
index 4fa61fbaf66c2..2a0a42c65b086 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala
@@ -183,8 +183,9 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
val ctx = new CodegenContext
RegExpReplace(Literal("100"), Literal("(\\d+)"), Literal("num")).genCode(ctx)
// four global variables (lastRegex, pattern, lastReplacement, and lastReplacementInUTF8)
- // are always required
- assert(ctx.mutableStates.length == 4)
+ // are always required, which are allocated in type-based global array
+ assert(ctx.inlinedMutableStates.length == 0)
+ assert(ctx.mutableStateInitCode.length == 4)
}
test("RegexExtract") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala
index 70dea4b39d55d..10e3ffd0dff97 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala
@@ -51,6 +51,6 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper {
test("SPARK-22695: ScalaUDF should not use global variables") {
val ctx = new CodegenContext
ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil).genCode(ctx)
- assert(ctx.mutableStates.isEmpty)
+ assert(ctx.inlinedMutableStates.isEmpty)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
index 6031bdf19e957..2c45b3b0c73d1 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala
@@ -219,4 +219,31 @@ class GeneratedProjectionSuite extends SparkFunSuite {
// - one is the mutableRow
assert(globalVariables.length == 3)
}
+
+ test("SPARK-18016: generated projections on wider table requiring state compaction") {
+ val N = 6000
+ val wideRow1 = new GenericInternalRow(new Array[Any](N))
+ val schema1 = StructType((1 to N).map(i => StructField("", IntegerType)))
+ val wideRow2 = new GenericInternalRow(
+ Array.tabulate[Any](N)(i => UTF8String.fromString(i.toString)))
+ val schema2 = StructType((1 to N).map(i => StructField("", StringType)))
+ val joined = new JoinedRow(wideRow1, wideRow2)
+ val joinedSchema = StructType(schema1 ++ schema2)
+ val nested = new JoinedRow(InternalRow(joined, joined), joined)
+ val nestedSchema = StructType(
+ Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ joinedSchema)
+
+ val safeProj = FromUnsafeProjection(nestedSchema)
+ val result = safeProj(nested)
+
+ // test generated MutableProjection
+ val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) =>
+ BoundReference(i, f.dataType, true)
+ }
+ val mutableProj = GenerateMutableProjection.generate(exprs)
+ val row1 = mutableProj(result)
+ assert(result === row1)
+ val row2 = mutableProj(result)
+ assert(result === row2)
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
index e3675367d78e4..0d11958876ce9 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
@@ -168,7 +168,7 @@ class ComplexTypesSuite extends PlanTest{
test("SPARK-22570: CreateArray should not create a lot of global variables") {
val ctx = new CodegenContext
CreateArray(Seq(Literal(1))).genCode(ctx)
- assert(ctx.mutableStates.length == 0)
+ assert(ctx.inlinedMutableStates.length == 0)
}
test("simplify map ops") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala
index 097c78eb27fca..26139d85d25fb 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala
@@ -23,7 +23,7 @@ import scala.collection.mutable
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeReference, EqualTo}
import org.apache.spark.sql.catalyst.plans._
-import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, Project, Statistics}
+import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types.{DateType, TimestampType, _}
@@ -67,6 +67,213 @@ class JoinEstimationSuite extends StatsEstimationTestBase {
rowCount = 2,
attributeStats = AttributeMap(Seq("key-1-2", "key-2-3").map(nameToColInfo)))
+ private def estimateByHistogram(
+ leftHistogram: Histogram,
+ rightHistogram: Histogram,
+ expectedMin: Double,
+ expectedMax: Double,
+ expectedNdv: Long,
+ expectedRows: Long): Unit = {
+ val col1 = attr("key1")
+ val col2 = attr("key2")
+ val c1 = generateJoinChild(col1, leftHistogram, expectedMin, expectedMax)
+ val c2 = generateJoinChild(col2, rightHistogram, expectedMin, expectedMax)
+
+ val c1JoinC2 = Join(c1, c2, Inner, Some(EqualTo(col1, col2)))
+ val c2JoinC1 = Join(c2, c1, Inner, Some(EqualTo(col2, col1)))
+ val expectedStatsAfterJoin = Statistics(
+ sizeInBytes = expectedRows * (8 + 2 * 4),
+ rowCount = Some(expectedRows),
+ attributeStats = AttributeMap(Seq(
+ col1 -> c1.stats.attributeStats(col1).copy(
+ distinctCount = expectedNdv, min = Some(expectedMin), max = Some(expectedMax)),
+ col2 -> c2.stats.attributeStats(col2).copy(
+ distinctCount = expectedNdv, min = Some(expectedMin), max = Some(expectedMax))))
+ )
+
+ // Join order should not affect estimation result.
+ Seq(c1JoinC2, c2JoinC1).foreach { join =>
+ assert(join.stats == expectedStatsAfterJoin)
+ }
+ }
+
+ private def generateJoinChild(
+ col: Attribute,
+ histogram: Histogram,
+ expectedMin: Double,
+ expectedMax: Double): LogicalPlan = {
+ val colStat = inferColumnStat(histogram)
+ StatsTestPlan(
+ outputList = Seq(col),
+ rowCount = (histogram.height * histogram.bins.length).toLong,
+ attributeStats = AttributeMap(Seq(col -> colStat)))
+ }
+
+ /** Column statistics should be consistent with histograms in tests. */
+ private def inferColumnStat(histogram: Histogram): ColumnStat = {
+ var ndv = 0L
+ for (i <- histogram.bins.indices) {
+ val bin = histogram.bins(i)
+ if (i == 0 || bin.hi != histogram.bins(i - 1).hi) {
+ ndv += bin.ndv
+ }
+ }
+ ColumnStat(distinctCount = ndv, min = Some(histogram.bins.head.lo),
+ max = Some(histogram.bins.last.hi), nullCount = 0, avgLen = 4, maxLen = 4,
+ histogram = Some(histogram))
+ }
+
+ test("equi-height histograms: a bin is contained by another one") {
+ val histogram1 = Histogram(height = 300, Array(
+ HistogramBin(lo = 10, hi = 30, ndv = 10), HistogramBin(lo = 30, hi = 60, ndv = 30)))
+ val histogram2 = Histogram(height = 100, Array(
+ HistogramBin(lo = 0, hi = 50, ndv = 50), HistogramBin(lo = 50, hi = 100, ndv = 40)))
+ // test bin trimming
+ val (t0, h0) = trimBin(histogram2.bins(0), height = 100, lowerBound = 10, upperBound = 60)
+ assert(t0 == HistogramBin(lo = 10, hi = 50, ndv = 40) && h0 == 80)
+ val (t1, h1) = trimBin(histogram2.bins(1), height = 100, lowerBound = 10, upperBound = 60)
+ assert(t1 == HistogramBin(lo = 50, hi = 60, ndv = 8) && h1 == 20)
+
+ val expectedRanges = Seq(
+ // histogram1.bins(0) overlaps t0
+ OverlappedRange(10, 30, 10, 40 * 1 / 2, 300, 80 * 1 / 2),
+ // histogram1.bins(1) overlaps t0
+ OverlappedRange(30, 50, 30 * 2 / 3, 40 * 1 / 2, 300 * 2 / 3, 80 * 1 / 2),
+ // histogram1.bins(1) overlaps t1
+ OverlappedRange(50, 60, 30 * 1 / 3, 8, 300 * 1 / 3, 20)
+ )
+ assert(expectedRanges.equals(
+ getOverlappedRanges(histogram1, histogram2, lowerBound = 10, upperBound = 60)))
+
+ estimateByHistogram(
+ leftHistogram = histogram1,
+ rightHistogram = histogram2,
+ expectedMin = 10,
+ expectedMax = 60,
+ expectedNdv = 10 + 20 + 8,
+ expectedRows = 300 * 40 / 20 + 200 * 40 / 20 + 100 * 20 / 10)
+ }
+
+ test("equi-height histograms: a bin has only one value after trimming") {
+ val histogram1 = Histogram(height = 300, Array(
+ HistogramBin(lo = 50, hi = 60, ndv = 10), HistogramBin(lo = 60, hi = 75, ndv = 3)))
+ val histogram2 = Histogram(height = 100, Array(
+ HistogramBin(lo = 0, hi = 50, ndv = 50), HistogramBin(lo = 50, hi = 100, ndv = 40)))
+ // test bin trimming
+ val (t0, h0) = trimBin(histogram2.bins(0), height = 100, lowerBound = 50, upperBound = 75)
+ assert(t0 == HistogramBin(lo = 50, hi = 50, ndv = 1) && h0 == 2)
+ val (t1, h1) = trimBin(histogram2.bins(1), height = 100, lowerBound = 50, upperBound = 75)
+ assert(t1 == HistogramBin(lo = 50, hi = 75, ndv = 20) && h1 == 50)
+
+ val expectedRanges = Seq(
+ // histogram1.bins(0) overlaps t0
+ OverlappedRange(50, 50, 1, 1, 300 / 10, 2),
+ // histogram1.bins(0) overlaps t1
+ OverlappedRange(50, 60, 10, 20 * 10 / 25, 300, 50 * 10 / 25),
+ // histogram1.bins(1) overlaps t1
+ OverlappedRange(60, 75, 3, 20 * 15 / 25, 300, 50 * 15 / 25)
+ )
+ assert(expectedRanges.equals(
+ getOverlappedRanges(histogram1, histogram2, lowerBound = 50, upperBound = 75)))
+
+ estimateByHistogram(
+ leftHistogram = histogram1,
+ rightHistogram = histogram2,
+ expectedMin = 50,
+ expectedMax = 75,
+ expectedNdv = 1 + 8 + 3,
+ expectedRows = 30 * 2 / 1 + 300 * 20 / 10 + 300 * 30 / 12)
+ }
+
+ test("equi-height histograms: skew distribution (some bins have only one value)") {
+ val histogram1 = Histogram(height = 300, Array(
+ HistogramBin(lo = 30, hi = 30, ndv = 1),
+ HistogramBin(lo = 30, hi = 30, ndv = 1),
+ HistogramBin(lo = 30, hi = 60, ndv = 30)))
+ val histogram2 = Histogram(height = 100, Array(
+ HistogramBin(lo = 0, hi = 50, ndv = 50), HistogramBin(lo = 50, hi = 100, ndv = 40)))
+ // test bin trimming
+ val (t0, h0) = trimBin(histogram2.bins(0), height = 100, lowerBound = 30, upperBound = 60)
+ assert(t0 == HistogramBin(lo = 30, hi = 50, ndv = 20) && h0 == 40)
+ val (t1, h1) = trimBin(histogram2.bins(1), height = 100, lowerBound = 30, upperBound = 60)
+ assert(t1 ==HistogramBin(lo = 50, hi = 60, ndv = 8) && h1 == 20)
+
+ val expectedRanges = Seq(
+ OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
+ OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
+ OverlappedRange(30, 50, 30 * 2 / 3, 20, 300 * 2 / 3, 40),
+ OverlappedRange(50, 60, 30 * 1 / 3, 8, 300 * 1 / 3, 20)
+ )
+ assert(expectedRanges.equals(
+ getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 60)))
+
+ estimateByHistogram(
+ leftHistogram = histogram1,
+ rightHistogram = histogram2,
+ expectedMin = 30,
+ expectedMax = 60,
+ expectedNdv = 1 + 20 + 8,
+ expectedRows = 300 * 2 / 1 + 300 * 2 / 1 + 200 * 40 / 20 + 100 * 20 / 10)
+ }
+
+ test("equi-height histograms: skew distribution (histograms have different skewed values") {
+ val histogram1 = Histogram(height = 300, Array(
+ HistogramBin(lo = 30, hi = 30, ndv = 1), HistogramBin(lo = 30, hi = 60, ndv = 30)))
+ val histogram2 = Histogram(height = 100, Array(
+ HistogramBin(lo = 0, hi = 50, ndv = 50), HistogramBin(lo = 50, hi = 50, ndv = 1)))
+ // test bin trimming
+ val (t0, h0) = trimBin(histogram1.bins(1), height = 300, lowerBound = 30, upperBound = 50)
+ assert(t0 == HistogramBin(lo = 30, hi = 50, ndv = 20) && h0 == 200)
+ val (t1, h1) = trimBin(histogram2.bins(0), height = 100, lowerBound = 30, upperBound = 50)
+ assert(t1 == HistogramBin(lo = 30, hi = 50, ndv = 20) && h1 == 40)
+
+ val expectedRanges = Seq(
+ OverlappedRange(30, 30, 1, 1, 300, 40 / 20),
+ OverlappedRange(30, 50, 20, 20, 200, 40),
+ OverlappedRange(50, 50, 1, 1, 200 / 20, 100)
+ )
+ assert(expectedRanges.equals(
+ getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 50)))
+
+ estimateByHistogram(
+ leftHistogram = histogram1,
+ rightHistogram = histogram2,
+ expectedMin = 30,
+ expectedMax = 50,
+ expectedNdv = 1 + 20,
+ expectedRows = 300 * 2 / 1 + 200 * 40 / 20 + 10 * 100 / 1)
+ }
+
+ test("equi-height histograms: skew distribution (both histograms have the same skewed value") {
+ val histogram1 = Histogram(height = 300, Array(
+ HistogramBin(lo = 30, hi = 30, ndv = 1), HistogramBin(lo = 30, hi = 60, ndv = 30)))
+ val histogram2 = Histogram(height = 150, Array(
+ HistogramBin(lo = 0, hi = 30, ndv = 30), HistogramBin(lo = 30, hi = 30, ndv = 1)))
+ // test bin trimming
+ val (t0, h0) = trimBin(histogram1.bins(1), height = 300, lowerBound = 30, upperBound = 30)
+ assert(t0 == HistogramBin(lo = 30, hi = 30, ndv = 1) && h0 == 10)
+ val (t1, h1) = trimBin(histogram2.bins(0), height = 150, lowerBound = 30, upperBound = 30)
+ assert(t1 == HistogramBin(lo = 30, hi = 30, ndv = 1) && h1 == 5)
+
+ val expectedRanges = Seq(
+ OverlappedRange(30, 30, 1, 1, 300, 5),
+ OverlappedRange(30, 30, 1, 1, 300, 150),
+ OverlappedRange(30, 30, 1, 1, 10, 5),
+ OverlappedRange(30, 30, 1, 1, 10, 150)
+ )
+ assert(expectedRanges.equals(
+ getOverlappedRanges(histogram1, histogram2, lowerBound = 30, upperBound = 30)))
+
+ estimateByHistogram(
+ leftHistogram = histogram1,
+ rightHistogram = histogram2,
+ expectedMin = 30,
+ expectedMax = 30,
+ // only one value: 30
+ expectedNdv = 1,
+ expectedRows = 300 * 5 / 1 + 300 * 150 / 1 + 10 * 5 / 1 + 10 * 150 / 1)
+ }
+
test("cross join") {
// table1 (key-1-5 int, key-5-9 int): (1, 9), (2, 8), (3, 7), (4, 6), (5, 5)
// table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index c8cf16d937352..625ff38943fa3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -563,6 +563,76 @@ class DateTimeUtilsSuite extends SparkFunSuite {
}
}
+ test("truncTimestamp") {
+ def testTrunc(
+ level: Int,
+ expected: String,
+ inputTS: SQLTimestamp,
+ timezone: TimeZone = DateTimeUtils.defaultTimeZone()): Unit = {
+ val truncated =
+ DateTimeUtils.truncTimestamp(inputTS, level, timezone)
+ val expectedTS =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected))
+ assert(truncated === expectedTS.get)
+ }
+
+ val defaultInputTS =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359"))
+ val defaultInputTS1 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359"))
+ val defaultInputTS2 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359"))
+ val defaultInputTS3 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359"))
+ val defaultInputTS4 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359"))
+
+ testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_DAY, "2015-03-05T00:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T09:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:32:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:05", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-02T00:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", defaultInputTS1.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", defaultInputTS2.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", defaultInputTS3.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-23T00:00:00", defaultInputTS4.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", defaultInputTS.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", defaultInputTS1.get)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS2.get)
+
+ for (tz <- DateTimeTestUtils.ALL_TIMEZONES) {
+ DateTimeTestUtils.withDefaultTimeZone(tz) {
+ val inputTS =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359"))
+ val inputTS1 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359"))
+ val inputTS2 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359"))
+ val inputTS3 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359"))
+ val inputTS4 =
+ DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359"))
+
+ testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_DAY, "2015-03-05T00:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T09:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:32:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:05", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-02T00:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS1.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS2.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS3.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-23T00:00:00", inputTS4.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS1.get, tz)
+ testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", inputTS2.get, tz)
+ }
+ }
+ }
+
test("daysToMillis and millisToDays") {
val c = Calendar.getInstance(TimeZonePST)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index c34cf0a7a7718..ef00562672a7e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -527,7 +527,7 @@ class Dataset[T] private[sql](
*/
@Experimental
@InterfaceStability.Evolving
- def checkpoint(): Dataset[T] = checkpoint(eager = true)
+ def checkpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = true)
/**
* Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the
@@ -540,9 +540,52 @@ class Dataset[T] private[sql](
*/
@Experimental
@InterfaceStability.Evolving
- def checkpoint(eager: Boolean): Dataset[T] = {
+ def checkpoint(eager: Boolean): Dataset[T] = checkpoint(eager = eager, reliableCheckpoint = true)
+
+ /**
+ * Eagerly locally checkpoints a Dataset and return the new Dataset. Checkpointing can be
+ * used to truncate the logical plan of this Dataset, which is especially useful in iterative
+ * algorithms where the plan may grow exponentially. Local checkpoints are written to executor
+ * storage and despite potentially faster they are unreliable and may compromise job completion.
+ *
+ * @group basic
+ * @since 2.3.0
+ */
+ @Experimental
+ @InterfaceStability.Evolving
+ def localCheckpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = false)
+
+ /**
+ * Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to truncate
+ * the logical plan of this Dataset, which is especially useful in iterative algorithms where the
+ * plan may grow exponentially. Local checkpoints are written to executor storage and despite
+ * potentially faster they are unreliable and may compromise job completion.
+ *
+ * @group basic
+ * @since 2.3.0
+ */
+ @Experimental
+ @InterfaceStability.Evolving
+ def localCheckpoint(eager: Boolean): Dataset[T] = checkpoint(
+ eager = eager,
+ reliableCheckpoint = false
+ )
+
+ /**
+ * Returns a checkpointed version of this Dataset.
+ *
+ * @param eager Whether to checkpoint this dataframe immediately
+ * @param reliableCheckpoint Whether to create a reliable checkpoint saved to files inside the
+ * checkpoint directory. If false creates a local checkpoint using
+ * the caching subsystem
+ */
+ private def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): Dataset[T] = {
val internalRdd = queryExecution.toRdd.map(_.copy())
- internalRdd.checkpoint()
+ if (reliableCheckpoint) {
+ internalRdd.checkpoint()
+ } else {
+ internalRdd.localCheckpoint()
+ }
if (eager) {
internalRdd.count()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
index 5a1d680c99f66..b05fe49a6ac3b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Dataset, SparkSession}
import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
import org.apache.spark.storage.StorageLevel
@@ -94,14 +94,13 @@ class CacheManager extends Logging {
logWarning("Asked to cache already cached data.")
} else {
val sparkSession = query.sparkSession
- cachedData.add(CachedData(
- planToCache,
- InMemoryRelation(
- sparkSession.sessionState.conf.useCompression,
- sparkSession.sessionState.conf.columnBatchSize,
- storageLevel,
- sparkSession.sessionState.executePlan(planToCache).executedPlan,
- tableName)))
+ val inMemoryRelation = InMemoryRelation(
+ sparkSession.sessionState.conf.useCompression,
+ sparkSession.sessionState.conf.columnBatchSize, storageLevel,
+ sparkSession.sessionState.executePlan(planToCache).executedPlan,
+ tableName,
+ planToCache.stats)
+ cachedData.add(CachedData(planToCache, inMemoryRelation))
}
}
@@ -148,7 +147,8 @@ class CacheManager extends Logging {
batchSize = cd.cachedRepresentation.batchSize,
storageLevel = cd.cachedRepresentation.storageLevel,
child = spark.sessionState.executePlan(cd.plan).executedPlan,
- tableName = cd.cachedRepresentation.tableName)
+ tableName = cd.cachedRepresentation.tableName,
+ statsOfPlanToCache = cd.plan.stats)
needToRecache += cd.copy(cachedRepresentation = newCache)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
index a9bfb634fbdea..782cec5e292ba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala
@@ -68,30 +68,26 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport {
*/
// TODO: return ColumnarBatch.Rows instead
override protected def doProduce(ctx: CodegenContext): String = {
- val input = ctx.freshName("input")
// PhysicalRDD always just has one input
- ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
+ val input = ctx.addMutableState("scala.collection.Iterator", "input",
+ v => s"$v = inputs[0];")
// metrics
val numOutputRows = metricTerm(ctx, "numOutputRows")
val scanTimeMetric = metricTerm(ctx, "scanTime")
- val scanTimeTotalNs = ctx.freshName("scanTime")
- ctx.addMutableState(ctx.JAVA_LONG, scanTimeTotalNs, s"$scanTimeTotalNs = 0;")
+ val scanTimeTotalNs = ctx.addMutableState(ctx.JAVA_LONG, "scanTime") // init as scanTime = 0
val columnarBatchClz = classOf[ColumnarBatch].getName
- val batch = ctx.freshName("batch")
- ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;")
+ val batch = ctx.addMutableState(columnarBatchClz, "batch")
- val idx = ctx.freshName("batchIdx")
- ctx.addMutableState(ctx.JAVA_INT, idx, s"$idx = 0;")
- val colVars = output.indices.map(i => ctx.freshName("colInstance" + i))
+ val idx = ctx.addMutableState(ctx.JAVA_INT, "batchIdx") // init as batchIdx = 0
val columnVectorClzs = vectorTypes.getOrElse(
- Seq.fill(colVars.size)(classOf[ColumnVector].getName))
- val columnAssigns = colVars.zip(columnVectorClzs).zipWithIndex.map {
- case ((name, columnVectorClz), i) =>
- ctx.addMutableState(columnVectorClz, name, s"$name = null;")
- s"$name = ($columnVectorClz) $batch.column($i);"
- }
+ Seq.fill(output.indices.size)(classOf[ColumnVector].getName))
+ val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map {
+ case (columnVectorClz, i) =>
+ val name = ctx.addMutableState(columnVectorClz, s"colInstance$i")
+ (name, s"$name = ($columnVectorClz) $batch.column($i);")
+ }.unzip
val nextBatch = ctx.freshName("nextBatch")
val nextBatchFuncName = ctx.addNewFunction(nextBatch,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
index 747749bc72e66..d1ff82c7c06bc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
@@ -69,7 +69,7 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport {
* Shorthand for calling redactString() without specifying redacting rules
*/
private def redact(text: String): String = {
- Utils.redact(SparkSession.getActiveSession.map(_.sparkContext.conf).orNull, text)
+ Utils.redact(sqlContext.sessionState.conf.stringRedationPattern, text)
}
}
@@ -110,8 +110,7 @@ case class RowDataSourceScanExec(
override protected def doProduce(ctx: CodegenContext): String = {
val numOutputRows = metricTerm(ctx, "numOutputRows")
// PhysicalRDD always just has one input
- val input = ctx.freshName("input")
- ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
+ val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];")
val exprRows = output.zipWithIndex.map{ case (a, i) =>
BoundReference(i, a.dataType, a.nullable)
}
@@ -353,8 +352,7 @@ case class FileSourceScanExec(
}
val numOutputRows = metricTerm(ctx, "numOutputRows")
// PhysicalRDD always just has one input
- val input = ctx.freshName("input")
- ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
+ val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];")
val row = ctx.freshName("row")
ctx.INPUT_ROW = row
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index 946475a1e9751..8bfe3eff0c3b3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -194,13 +194,13 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
}
}
- def simpleString: String = {
+ def simpleString: String = withRedaction {
s"""== Physical Plan ==
|${stringOrError(executedPlan.treeString(verbose = false))}
""".stripMargin.trim
}
- override def toString: String = {
+ override def toString: String = withRedaction {
def output = Utils.truncatedString(
analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ")
val analyzedPlan = Seq(
@@ -219,7 +219,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
""".stripMargin.trim
}
- def stringWithStats: String = {
+ def stringWithStats: String = withRedaction {
// trigger to compute stats for logical plans
optimizedPlan.stats
@@ -231,6 +231,13 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
""".stripMargin.trim
}
+ /**
+ * Redact the sensitive information in the given string.
+ */
+ private def withRedaction(message: String): String = {
+ Utils.redact(sparkSession.sessionState.conf.stringRedationPattern, message)
+ }
+
/** A special namespace for commands that can be used to debug query execution. */
// scalastyle:off
object debug {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
index c0e21343ae623..daff3c49e7517 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala
@@ -133,20 +133,18 @@ case class SortExec(
override def needStopCheck: Boolean = false
override protected def doProduce(ctx: CodegenContext): String = {
- val needToSort = ctx.freshName("needToSort")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, needToSort, s"$needToSort = true;")
+ val needToSort = ctx.addMutableState(ctx.JAVA_BOOLEAN, "needToSort", v => s"$v = true;")
// Initialize the class member variables. This includes the instance of the Sorter and
// the iterator to return sorted rows.
val thisPlan = ctx.addReferenceObj("plan", this)
- sorterVariable = ctx.freshName("sorter")
- ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, sorterVariable,
- s"$sorterVariable = $thisPlan.createSorter();")
- val metrics = ctx.freshName("metrics")
- ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
- s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
- val sortedIterator = ctx.freshName("sortedIter")
- ctx.addMutableState("scala.collection.Iterator", sortedIterator, "")
+ // inline mutable state since not many Sort operations in a task
+ sorterVariable = ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, "sorter",
+ v => s"$v = $thisPlan.createSorter();", forceInline = true)
+ val metrics = ctx.addMutableState(classOf[TaskMetrics].getName, "metrics",
+ v => s"$v = org.apache.spark.TaskContext.get().taskMetrics();", forceInline = true)
+ val sortedIterator = ctx.addMutableState("scala.collection.Iterator", "sortedIter",
+ forceInline = true)
val addToSorter = ctx.freshName("addToSorter")
val addToSorterFuncName = ctx.addNewFunction(addToSorter,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
index 7166b7771e4db..9e7008d1e0c31 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
@@ -282,9 +282,10 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp
}
override def doProduce(ctx: CodegenContext): String = {
- val input = ctx.freshName("input")
// Right now, InputAdapter is only used when there is one input RDD.
- ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
+ // inline mutable state since an inputAdaptor in a task
+ val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];",
+ forceInline = true)
val row = ctx.freshName("row")
s"""
| while ($input.hasNext() && !stopEarly()) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
index 9cadd13999e72..b1af360d85095 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
@@ -178,8 +178,7 @@ case class HashAggregateExec(
private var bufVars: Seq[ExprCode] = _
private def doProduceWithoutKeys(ctx: CodegenContext): String = {
- val initAgg = ctx.freshName("initAgg")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, initAgg, s"$initAgg = false;")
+ val initAgg = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initAgg")
// The generated function doesn't have input row in the code context.
ctx.INPUT_ROW = null
@@ -187,10 +186,8 @@ case class HashAggregateExec(
val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
val initExpr = functions.flatMap(f => f.initialValues)
bufVars = initExpr.map { e =>
- val isNull = ctx.freshName("bufIsNull")
- val value = ctx.freshName("bufValue")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, isNull)
- ctx.addMutableState(ctx.javaType(e.dataType), value)
+ val isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "bufIsNull")
+ val value = ctx.addMutableState(ctx.javaType(e.dataType), "bufValue")
// The initial expression should not access any column
val ev = e.genCode(ctx)
val initVars = s"""
@@ -568,8 +565,7 @@ case class HashAggregateExec(
}
private def doProduceWithKeys(ctx: CodegenContext): String = {
- val initAgg = ctx.freshName("initAgg")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, initAgg, s"$initAgg = false;")
+ val initAgg = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initAgg")
if (sqlContext.conf.enableTwoLevelAggMap) {
enableTwoLevelHashMap(ctx)
} else {
@@ -583,42 +579,41 @@ case class HashAggregateExec(
val thisPlan = ctx.addReferenceObj("plan", this)
// Create a name for the iterator from the fast hash map.
- val iterTermForFastHashMap = ctx.freshName("fastHashMapIter")
- if (isFastHashMapEnabled) {
+ val iterTermForFastHashMap = if (isFastHashMapEnabled) {
// Generates the fast hash map class and creates the fash hash map term.
- fastHashMapTerm = ctx.freshName("fastHashMap")
val fastHashMapClassName = ctx.freshName("FastHashMap")
if (isVectorizedHashMapEnabled) {
val generatedMap = new VectorizedHashMapGenerator(ctx, aggregateExpressions,
fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
ctx.addInnerClass(generatedMap)
- ctx.addMutableState(fastHashMapClassName, fastHashMapTerm,
- s"$fastHashMapTerm = new $fastHashMapClassName();")
- ctx.addMutableState(s"java.util.Iterator", iterTermForFastHashMap)
+ fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, "vectorizedHastHashMap",
+ v => s"$v = new $fastHashMapClassName();")
+ ctx.addMutableState(s"java.util.Iterator", "vectorizedFastHashMapIter")
} else {
val generatedMap = new RowBasedHashMapGenerator(ctx, aggregateExpressions,
fastHashMapClassName, groupingKeySchema, bufferSchema).generate()
ctx.addInnerClass(generatedMap)
- ctx.addMutableState(fastHashMapClassName, fastHashMapTerm,
- s"$fastHashMapTerm = new $fastHashMapClassName(" +
+ fastHashMapTerm = ctx.addMutableState(fastHashMapClassName, "fastHashMap",
+ v => s"$v = new $fastHashMapClassName(" +
s"$thisPlan.getTaskMemoryManager(), $thisPlan.getEmptyAggregationBuffer());")
ctx.addMutableState(
"org.apache.spark.unsafe.KVIterator",
- iterTermForFastHashMap)
+ "fastHashMapIter")
}
}
// Create a name for the iterator from the regular hash map.
- val iterTerm = ctx.freshName("mapIter")
- ctx.addMutableState(classOf[KVIterator[UnsafeRow, UnsafeRow]].getName, iterTerm)
+ // inline mutable state since not many aggregation operations in a task
+ val iterTerm = ctx.addMutableState(classOf[KVIterator[UnsafeRow, UnsafeRow]].getName,
+ "mapIter", forceInline = true)
// create hashMap
- hashMapTerm = ctx.freshName("hashMap")
val hashMapClassName = classOf[UnsafeFixedWidthAggregationMap].getName
- ctx.addMutableState(hashMapClassName, hashMapTerm, s"$hashMapTerm = $thisPlan.createHashMap();")
- sorterTerm = ctx.freshName("sorter")
- ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, sorterTerm)
+ hashMapTerm = ctx.addMutableState(hashMapClassName, "hashMap",
+ v => s"$v = $thisPlan.createHashMap();")
+ sorterTerm = ctx.addMutableState(classOf[UnsafeKVExternalSorter].getName, "sorter",
+ forceInline = true)
val doAgg = ctx.freshName("doAggregateWithKeys")
val peakMemory = metricTerm(ctx, "peakMemory")
@@ -758,8 +753,7 @@ case class HashAggregateExec(
val (checkFallbackForGeneratedHashMap, checkFallbackForBytesToBytesMap, resetCounter,
incCounter) = if (testFallbackStartsAt.isDefined) {
- val countTerm = ctx.freshName("fallbackCounter")
- ctx.addMutableState(ctx.JAVA_INT, countTerm, s"$countTerm = 0;")
+ val countTerm = ctx.addMutableState(ctx.JAVA_INT, "fallbackCounter")
(s"$countTerm < ${testFallbackStartsAt.get._1}",
s"$countTerm < ${testFallbackStartsAt.get._2}", s"$countTerm = 0;", s"$countTerm += 1;")
} else {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index 85b4529501ea8..1c613b19c4ab1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
@@ -46,10 +46,8 @@ abstract class HashMapGenerator(
val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate])
val initExpr = functions.flatMap(f => f.initialValues)
initExpr.map { e =>
- val isNull = ctx.freshName("bufIsNull")
- val value = ctx.freshName("bufValue")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, isNull)
- ctx.addMutableState(ctx.javaType(e.dataType), value)
+ val isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "bufIsNull")
+ val value = ctx.addMutableState(ctx.javaType(e.dataType), "bufValue")
val ev = e.genCode(ctx)
val initVars =
s"""
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
index 756eeb642e2d0..9dc334c1ead3c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.aggregate
import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
+import org.apache.spark.memory.SparkOutOfMemoryError
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -205,7 +206,7 @@ class TungstenAggregationIterator(
buffer = hashMap.getAggregationBufferFromUnsafeRow(groupingKey)
if (buffer == null) {
// failed to allocate the first page
- throw new OutOfMemoryError("No enough memory for aggregation")
+ throw new SparkOutOfMemoryError("No enough memory for aggregation")
}
}
processRow(buffer, newInput)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
index c9a15147e30d0..78137d3f97cfc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala
@@ -279,29 +279,30 @@ case class SampleExec(
override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
val numOutput = metricTerm(ctx, "numOutputRows")
- val sampler = ctx.freshName("sampler")
if (withReplacement) {
val samplerClass = classOf[PoissonSampler[UnsafeRow]].getName
val initSampler = ctx.freshName("initSampler")
- val initSamplerFuncName = ctx.addNewFunction(initSampler,
- s"""
- | private void $initSampler() {
- | $sampler = new $samplerClass($upperBound - $lowerBound, false);
- | java.util.Random random = new java.util.Random(${seed}L);
- | long randomSeed = random.nextLong();
- | int loopCount = 0;
- | while (loopCount < partitionIndex) {
- | randomSeed = random.nextLong();
- | loopCount += 1;
- | }
- | $sampler.setSeed(randomSeed);
- | }
- """.stripMargin.trim)
-
- ctx.addMutableState(s"$samplerClass", sampler,
- s"$initSamplerFuncName();")
+ // inline mutable state since not many Sample operations in a task
+ val sampler = ctx.addMutableState(s"$samplerClass", "sampleReplace",
+ v => {
+ val initSamplerFuncName = ctx.addNewFunction(initSampler,
+ s"""
+ | private void $initSampler() {
+ | $v = new $samplerClass($upperBound - $lowerBound, false);
+ | java.util.Random random = new java.util.Random(${seed}L);
+ | long randomSeed = random.nextLong();
+ | int loopCount = 0;
+ | while (loopCount < partitionIndex) {
+ | randomSeed = random.nextLong();
+ | loopCount += 1;
+ | }
+ | $v.setSeed(randomSeed);
+ | }
+ """.stripMargin.trim)
+ s"$initSamplerFuncName();"
+ }, forceInline = true)
val samplingCount = ctx.freshName("samplingCount")
s"""
@@ -313,10 +314,10 @@ case class SampleExec(
""".stripMargin.trim
} else {
val samplerClass = classOf[BernoulliCellSampler[UnsafeRow]].getName
- ctx.addMutableState(s"$samplerClass", sampler,
- s"""
- | $sampler = new $samplerClass($lowerBound, $upperBound, false);
- | $sampler.setSeed(${seed}L + partitionIndex);
+ val sampler = ctx.addMutableState(s"$samplerClass", "sampler",
+ v => s"""
+ | $v = new $samplerClass($lowerBound, $upperBound, false);
+ | $v.setSeed(${seed}L + partitionIndex);
""".stripMargin.trim)
s"""
@@ -363,20 +364,18 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range)
protected override def doProduce(ctx: CodegenContext): String = {
val numOutput = metricTerm(ctx, "numOutputRows")
- val initTerm = ctx.freshName("initRange")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, initTerm, s"$initTerm = false;")
- val number = ctx.freshName("number")
- ctx.addMutableState(ctx.JAVA_LONG, number, s"$number = 0L;")
+ val initTerm = ctx.addMutableState(ctx.JAVA_BOOLEAN, "initRange")
+ val number = ctx.addMutableState(ctx.JAVA_LONG, "number")
val value = ctx.freshName("value")
val ev = ExprCode("", "false", value)
val BigInt = classOf[java.math.BigInteger].getName
- val taskContext = ctx.freshName("taskContext")
- ctx.addMutableState("TaskContext", taskContext, s"$taskContext = TaskContext.get();")
- val inputMetrics = ctx.freshName("inputMetrics")
- ctx.addMutableState("InputMetrics", inputMetrics,
- s"$inputMetrics = $taskContext.taskMetrics().inputMetrics();")
+ // inline mutable state since not many Range operations in a task
+ val taskContext = ctx.addMutableState("TaskContext", "taskContext",
+ v => s"$v = TaskContext.get();", forceInline = true)
+ val inputMetrics = ctx.addMutableState("InputMetrics", "inputMetrics",
+ v => s"$v = $taskContext.taskMetrics().inputMetrics();", forceInline = true)
// In order to periodically update the metrics without inflicting performance penalty, this
// operator produces elements in batches. After a batch is complete, the metrics are updated
@@ -386,12 +385,10 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range)
// the metrics.
// Once number == batchEnd, it's time to progress to the next batch.
- val batchEnd = ctx.freshName("batchEnd")
- ctx.addMutableState(ctx.JAVA_LONG, batchEnd, s"$batchEnd = 0;")
+ val batchEnd = ctx.addMutableState(ctx.JAVA_LONG, "batchEnd")
// How many values should still be generated by this range operator.
- val numElementsTodo = ctx.freshName("numElementsTodo")
- ctx.addMutableState(ctx.JAVA_LONG, numElementsTodo, s"$numElementsTodo = 0L;")
+ val numElementsTodo = ctx.addMutableState(ctx.JAVA_LONG, "numElementsTodo")
// How many values should be generated in the next batch.
val nextBatchTodo = ctx.freshName("nextBatchTodo")
@@ -440,10 +437,6 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range)
| }
""".stripMargin)
- val input = ctx.freshName("input")
- // Right now, Range is only used when there is one upstream.
- ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];")
-
val localIdx = ctx.freshName("localIdx")
val localEnd = ctx.freshName("localEnd")
val range = ctx.freshName("range")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
index ff5dd707f0b38..4f28eeb725cbb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
@@ -70,7 +70,6 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera
val ctx = newCodeGenContext()
val numFields = columnTypes.size
val (initializeAccessors, extractors) = columnTypes.zipWithIndex.map { case (dt, index) =>
- val accessorName = ctx.freshName("accessor")
val accessorCls = dt match {
case NullType => classOf[NullColumnAccessor].getName
case BooleanType => classOf[BooleanColumnAccessor].getName
@@ -89,7 +88,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera
case array: ArrayType => classOf[ArrayColumnAccessor].getName
case t: MapType => classOf[MapColumnAccessor].getName
}
- ctx.addMutableState(accessorCls, accessorName)
+ val accessorName = ctx.addMutableState(accessorCls, "accessor")
val createCode = dt match {
case t if ctx.isPrimitiveType(dt) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
index a1c62a729900e..51928d914841e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
@@ -37,8 +37,10 @@ object InMemoryRelation {
batchSize: Int,
storageLevel: StorageLevel,
child: SparkPlan,
- tableName: Option[String]): InMemoryRelation =
- new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)()
+ tableName: Option[String],
+ statsOfPlanToCache: Statistics): InMemoryRelation =
+ new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)(
+ statsOfPlanToCache = statsOfPlanToCache)
}
@@ -60,7 +62,8 @@ case class InMemoryRelation(
@transient child: SparkPlan,
tableName: Option[String])(
@transient var _cachedColumnBuffers: RDD[CachedBatch] = null,
- val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator)
+ val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator,
+ statsOfPlanToCache: Statistics = null)
extends logical.LeafNode with MultiInstanceRelation {
override protected def innerChildren: Seq[SparkPlan] = Seq(child)
@@ -71,9 +74,8 @@ case class InMemoryRelation(
override def computeStats(): Statistics = {
if (batchStats.value == 0L) {
- // Underlying columnar RDD hasn't been materialized, no useful statistics information
- // available, return the default statistics.
- Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes)
+ // Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache
+ statsOfPlanToCache
} else {
Statistics(sizeInBytes = batchStats.value.longValue)
}
@@ -142,7 +144,7 @@ case class InMemoryRelation(
def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = {
InMemoryRelation(
newOutput, useCompression, batchSize, storageLevel, child, tableName)(
- _cachedColumnBuffers, batchStats)
+ _cachedColumnBuffers, batchStats, statsOfPlanToCache)
}
override def newInstance(): this.type = {
@@ -154,11 +156,12 @@ case class InMemoryRelation(
child,
tableName)(
_cachedColumnBuffers,
- batchStats).asInstanceOf[this.type]
+ batchStats,
+ statsOfPlanToCache).asInstanceOf[this.type]
}
def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers
override protected def otherCopyArgs: Seq[AnyRef] =
- Seq(_cachedColumnBuffers, batchStats)
+ Seq(_cachedColumnBuffers, batchStats, statsOfPlanToCache)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
index c96ed6ef41016..ee763e23415cf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala
@@ -134,19 +134,18 @@ case class BroadcastHashJoinExec(
// create a name for HashedRelation
val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]()
val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation)
- val relationTerm = ctx.freshName("relation")
val clsName = broadcastRelation.value.getClass.getName
// At the end of the task, we update the avg hash probe.
val avgHashProbe = metricTerm(ctx, "avgHashProbe")
- val addTaskListener = genTaskListener(avgHashProbe, relationTerm)
- ctx.addMutableState(clsName, relationTerm,
- s"""
- | $relationTerm = (($clsName) $broadcast.value()).asReadOnlyCopy();
- | incPeakExecutionMemory($relationTerm.estimatedSize());
- | $addTaskListener
- """.stripMargin)
+ // inline mutable state since not many join operations in a task
+ val relationTerm = ctx.addMutableState(clsName, "relation",
+ v => s"""
+ | $v = (($clsName) $broadcast.value()).asReadOnlyCopy();
+ | incPeakExecutionMemory($v.estimatedSize());
+ | ${genTaskListener(avgHashProbe, v)}
+ """.stripMargin, forceInline = true)
(broadcastRelation, relationTerm)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
index 554b73181116c..073730462a75f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala
@@ -422,10 +422,9 @@ case class SortMergeJoinExec(
*/
private def genScanner(ctx: CodegenContext): (String, String) = {
// Create class member for next row from both sides.
- val leftRow = ctx.freshName("leftRow")
- ctx.addMutableState("InternalRow", leftRow)
- val rightRow = ctx.freshName("rightRow")
- ctx.addMutableState("InternalRow", rightRow, s"$rightRow = null;")
+ // inline mutable state since not many join operations in a task
+ val leftRow = ctx.addMutableState("InternalRow", "leftRow", forceInline = true)
+ val rightRow = ctx.addMutableState("InternalRow", "rightRow", forceInline = true)
// Create variables for join keys from both sides.
val leftKeyVars = createJoinKey(ctx, leftRow, leftKeys, left.output)
@@ -436,14 +435,13 @@ case class SortMergeJoinExec(
val rightKeyVars = copyKeys(ctx, rightKeyTmpVars)
// A list to hold all matched rows from right side.
- val matches = ctx.freshName("matches")
val clsName = classOf[ExternalAppendOnlyUnsafeRowArray].getName
val spillThreshold = getSpillThreshold
val inMemoryThreshold = getInMemoryThreshold
- ctx.addMutableState(clsName, matches,
- s"$matches = new $clsName($inMemoryThreshold, $spillThreshold);")
+ val matches = ctx.addMutableState(clsName, "matches",
+ v => s"$v = new $clsName($inMemoryThreshold, $spillThreshold);")
// Copy the left keys as class members so they could be used in next function call.
val matchedKeyVars = copyKeys(ctx, leftKeyVars)
@@ -578,10 +576,11 @@ case class SortMergeJoinExec(
override def needCopyResult: Boolean = true
override def doProduce(ctx: CodegenContext): String = {
- val leftInput = ctx.freshName("leftInput")
- ctx.addMutableState("scala.collection.Iterator", leftInput, s"$leftInput = inputs[0];")
- val rightInput = ctx.freshName("rightInput")
- ctx.addMutableState("scala.collection.Iterator", rightInput, s"$rightInput = inputs[1];")
+ // inline mutable state since not many join operations in a task
+ val leftInput = ctx.addMutableState("scala.collection.Iterator", "leftInput",
+ v => s"$v = inputs[0];", forceInline = true)
+ val rightInput = ctx.addMutableState("scala.collection.Iterator", "rightInput",
+ v => s"$v = inputs[1];", forceInline = true)
val (leftRow, matches) = genScanner(ctx)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
index a8556f6ba107a..cccee63bc0680 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
@@ -71,8 +71,7 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport {
}
override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
- val stopEarly = ctx.freshName("stopEarly")
- ctx.addMutableState(ctx.JAVA_BOOLEAN, stopEarly, s"$stopEarly = false;")
+ val stopEarly = ctx.addMutableState(ctx.JAVA_BOOLEAN, "stopEarly") // init as stopEarly = false
ctx.addNewFunction("stopEarly", s"""
@Override
@@ -80,8 +79,7 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport {
return $stopEarly;
}
""", inlineToOuterClass = true)
- val countTerm = ctx.freshName("count")
- ctx.addMutableState(ctx.JAVA_INT, countTerm, s"$countTerm = 0;")
+ val countTerm = ctx.addMutableState(ctx.JAVA_INT, "count") // init as count = 0
s"""
| if ($countTerm < $limit) {
| $countTerm += 1;
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
index 43cec4807ae4d..cf0000c6393a3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
@@ -27,14 +27,15 @@ import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.metric._
-import org.apache.spark.status.LiveEntity
+import org.apache.spark.sql.internal.StaticSQLConf._
+import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity}
import org.apache.spark.status.config._
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.kvstore.KVStore
private[sql] class SQLAppStatusListener(
conf: SparkConf,
- kvstore: KVStore,
+ kvstore: ElementTrackingStore,
live: Boolean,
ui: Option[SparkUI] = None)
extends SparkListener with Logging {
@@ -51,6 +52,23 @@ private[sql] class SQLAppStatusListener(
private var uiInitialized = false
+ kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get(UI_RETAINED_EXECUTIONS)) { count =>
+ cleanupExecutions(count)
+ }
+
+ kvstore.onFlush {
+ if (!live) {
+ val now = System.nanoTime()
+ liveExecutions.values.asScala.foreach { exec =>
+ // This saves the partial aggregated metrics to the store; this works currently because
+ // when the SHS sees an updated event log, all old data for the application is thrown
+ // away.
+ exec.metricsValues = aggregateMetrics(exec)
+ exec.write(kvstore, now)
+ }
+ }
+ }
+
override def onJobStart(event: SparkListenerJobStart): Unit = {
val executionIdString = event.properties.getProperty(SQLExecution.EXECUTION_ID_KEY)
if (executionIdString == null) {
@@ -317,6 +335,17 @@ private[sql] class SQLAppStatusListener(
}
}
+ private def cleanupExecutions(count: Long): Unit = {
+ val countToDelete = count - conf.get(UI_RETAINED_EXECUTIONS)
+ if (countToDelete <= 0) {
+ return
+ }
+
+ val toDelete = KVUtils.viewToSeq(kvstore.view(classOf[SQLExecutionUIData]),
+ countToDelete.toInt) { e => e.completionTime.isDefined }
+ toDelete.foreach { e => kvstore.delete(e.getClass(), e.executionId) }
+ }
+
}
private class LiveExecutionData(val executionId: Long) extends LiveEntity {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala
index 586d3ae411c74..7fd5f7395cdf3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala
@@ -27,7 +27,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize
import org.apache.spark.{JobExecutionStatus, SparkConf}
import org.apache.spark.scheduler.SparkListener
-import org.apache.spark.status.AppStatusPlugin
+import org.apache.spark.status.{AppStatusPlugin, ElementTrackingStore}
import org.apache.spark.status.KVUtils.KVIndexParam
import org.apache.spark.ui.SparkUI
import org.apache.spark.util.Utils
@@ -84,7 +84,7 @@ private[sql] class SQLAppStatusPlugin extends AppStatusPlugin {
override def setupListeners(
conf: SparkConf,
- store: KVStore,
+ store: ElementTrackingStore,
addListenerFn: SparkListener => Unit,
live: Boolean): Unit = {
// For live applications, the listener is installed in [[setupUI]]. This also avoids adding
@@ -100,7 +100,8 @@ private[sql] class SQLAppStatusPlugin extends AppStatusPlugin {
case Some(sc) =>
// If this is a live application, then install a listener that will enable the SQL
// tab as soon as there's a SQL event posted to the bus.
- val listener = new SQLAppStatusListener(sc.conf, ui.store.store, true, Some(ui))
+ val listener = new SQLAppStatusListener(sc.conf,
+ ui.store.store.asInstanceOf[ElementTrackingStore], true, Some(ui))
sc.listenerBus.addToStatusQueue(listener)
case _ =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 3e4659b9eae60..052a3f533da5a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -2797,6 +2797,21 @@ object functions {
TruncDate(date.expr, Literal(format))
}
+ /**
+ * Returns timestamp truncated to the unit specified by the format.
+ *
+ * @param format: 'year', 'yyyy', 'yy' for truncate by year,
+ * 'month', 'mon', 'mm' for truncate by month,
+ * 'day', 'dd' for truncate by day,
+ * Other options are: 'second', 'minute', 'hour', 'week', 'month', 'quarter'
+ *
+ * @group datetime_funcs
+ * @since 2.3.0
+ */
+ def date_trunc(format: String, timestamp: Column): Column = withExpr {
+ TruncTimestamp(Literal(format), timestamp.expr)
+ }
+
/**
* Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders
* that time as a timestamp in the given time zone. For example, 'GMT+1' would yield
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
index a42e28053a96a..41aa02c2b5e35 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala
@@ -298,6 +298,21 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo
*/
def csv(path: String): DataFrame = format("csv").load(path)
+ /**
+ * Loads a ORC file stream, returning the result as a `DataFrame`.
+ *
+ * You can set the following ORC-specific option(s) for reading ORC files:
+ *
+ * - `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be
+ * considered in every trigger.
+ *
+ *
+ * @since 2.3.0
+ */
+ def orc(path: String): DataFrame = {
+ format("orc").load(path)
+ }
+
/**
* Loads a Parquet file stream, returning the result as a `DataFrame`.
*
diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
new file mode 100644
index 0000000000000..442f2355f8e3a
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/booleanEquality.sql
@@ -0,0 +1,122 @@
+--
+-- 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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT true = cast(1 as tinyint) FROM t;
+SELECT true = cast(1 as smallint) FROM t;
+SELECT true = cast(1 as int) FROM t;
+SELECT true = cast(1 as bigint) FROM t;
+SELECT true = cast(1 as float) FROM t;
+SELECT true = cast(1 as double) FROM t;
+SELECT true = cast(1 as decimal(10, 0)) FROM t;
+SELECT true = cast(1 as string) FROM t;
+SELECT true = cast('1' as binary) FROM t;
+SELECT true = cast(1 as boolean) FROM t;
+SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT true = cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT true <=> cast(1 as tinyint) FROM t;
+SELECT true <=> cast(1 as smallint) FROM t;
+SELECT true <=> cast(1 as int) FROM t;
+SELECT true <=> cast(1 as bigint) FROM t;
+SELECT true <=> cast(1 as float) FROM t;
+SELECT true <=> cast(1 as double) FROM t;
+SELECT true <=> cast(1 as decimal(10, 0)) FROM t;
+SELECT true <=> cast(1 as string) FROM t;
+SELECT true <=> cast('1' as binary) FROM t;
+SELECT true <=> cast(1 as boolean) FROM t;
+SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as tinyint) = true FROM t;
+SELECT cast(1 as smallint) = true FROM t;
+SELECT cast(1 as int) = true FROM t;
+SELECT cast(1 as bigint) = true FROM t;
+SELECT cast(1 as float) = true FROM t;
+SELECT cast(1 as double) = true FROM t;
+SELECT cast(1 as decimal(10, 0)) = true FROM t;
+SELECT cast(1 as string) = true FROM t;
+SELECT cast('1' as binary) = true FROM t;
+SELECT cast(1 as boolean) = true FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) = true FROM t;
+
+SELECT cast(1 as tinyint) <=> true FROM t;
+SELECT cast(1 as smallint) <=> true FROM t;
+SELECT cast(1 as int) <=> true FROM t;
+SELECT cast(1 as bigint) <=> true FROM t;
+SELECT cast(1 as float) <=> true FROM t;
+SELECT cast(1 as double) <=> true FROM t;
+SELECT cast(1 as decimal(10, 0)) <=> true FROM t;
+SELECT cast(1 as string) <=> true FROM t;
+SELECT cast('1' as binary) <=> true FROM t;
+SELECT cast(1 as boolean) <=> true FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t;
+
+SELECT false = cast(0 as tinyint) FROM t;
+SELECT false = cast(0 as smallint) FROM t;
+SELECT false = cast(0 as int) FROM t;
+SELECT false = cast(0 as bigint) FROM t;
+SELECT false = cast(0 as float) FROM t;
+SELECT false = cast(0 as double) FROM t;
+SELECT false = cast(0 as decimal(10, 0)) FROM t;
+SELECT false = cast(0 as string) FROM t;
+SELECT false = cast('0' as binary) FROM t;
+SELECT false = cast(0 as boolean) FROM t;
+SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT false = cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT false <=> cast(0 as tinyint) FROM t;
+SELECT false <=> cast(0 as smallint) FROM t;
+SELECT false <=> cast(0 as int) FROM t;
+SELECT false <=> cast(0 as bigint) FROM t;
+SELECT false <=> cast(0 as float) FROM t;
+SELECT false <=> cast(0 as double) FROM t;
+SELECT false <=> cast(0 as decimal(10, 0)) FROM t;
+SELECT false <=> cast(0 as string) FROM t;
+SELECT false <=> cast('0' as binary) FROM t;
+SELECT false <=> cast(0 as boolean) FROM t;
+SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(0 as tinyint) = false FROM t;
+SELECT cast(0 as smallint) = false FROM t;
+SELECT cast(0 as int) = false FROM t;
+SELECT cast(0 as bigint) = false FROM t;
+SELECT cast(0 as float) = false FROM t;
+SELECT cast(0 as double) = false FROM t;
+SELECT cast(0 as decimal(10, 0)) = false FROM t;
+SELECT cast(0 as string) = false FROM t;
+SELECT cast('0' as binary) = false FROM t;
+SELECT cast(0 as boolean) = false FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) = false FROM t;
+
+SELECT cast(0 as tinyint) <=> false FROM t;
+SELECT cast(0 as smallint) <=> false FROM t;
+SELECT cast(0 as int) <=> false FROM t;
+SELECT cast(0 as bigint) <=> false FROM t;
+SELECT cast(0 as float) <=> false FROM t;
+SELECT cast(0 as double) <=> false FROM t;
+SELECT cast(0 as decimal(10, 0)) <=> false FROM t;
+SELECT cast(0 as string) <=> false FROM t;
+SELECT cast('0' as binary) <=> false FROM t;
+SELECT cast(0 as boolean) <=> false FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/division.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/division.sql
new file mode 100644
index 0000000000000..d669740ddd9ca
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/division.sql
@@ -0,0 +1,174 @@
+--
+-- 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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as int) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as float) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as double) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as string) FROM t;
+SELECT cast(1 as tinyint) / cast('1' as binary) FROM t;
+SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t;
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as smallint) / cast(1 as smallint) FROM t;
+SELECT cast(1 as smallint) / cast(1 as int) FROM t;
+SELECT cast(1 as smallint) / cast(1 as bigint) FROM t;
+SELECT cast(1 as smallint) / cast(1 as float) FROM t;
+SELECT cast(1 as smallint) / cast(1 as double) FROM t;
+SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as smallint) / cast(1 as string) FROM t;
+SELECT cast(1 as smallint) / cast('1' as binary) FROM t;
+SELECT cast(1 as smallint) / cast(1 as boolean) FROM t;
+SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as int) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as int) / cast(1 as smallint) FROM t;
+SELECT cast(1 as int) / cast(1 as int) FROM t;
+SELECT cast(1 as int) / cast(1 as bigint) FROM t;
+SELECT cast(1 as int) / cast(1 as float) FROM t;
+SELECT cast(1 as int) / cast(1 as double) FROM t;
+SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as int) / cast(1 as string) FROM t;
+SELECT cast(1 as int) / cast('1' as binary) FROM t;
+SELECT cast(1 as int) / cast(1 as boolean) FROM t;
+SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as bigint) / cast(1 as smallint) FROM t;
+SELECT cast(1 as bigint) / cast(1 as int) FROM t;
+SELECT cast(1 as bigint) / cast(1 as bigint) FROM t;
+SELECT cast(1 as bigint) / cast(1 as float) FROM t;
+SELECT cast(1 as bigint) / cast(1 as double) FROM t;
+SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as bigint) / cast(1 as string) FROM t;
+SELECT cast(1 as bigint) / cast('1' as binary) FROM t;
+SELECT cast(1 as bigint) / cast(1 as boolean) FROM t;
+SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as float) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as float) / cast(1 as smallint) FROM t;
+SELECT cast(1 as float) / cast(1 as int) FROM t;
+SELECT cast(1 as float) / cast(1 as bigint) FROM t;
+SELECT cast(1 as float) / cast(1 as float) FROM t;
+SELECT cast(1 as float) / cast(1 as double) FROM t;
+SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as float) / cast(1 as string) FROM t;
+SELECT cast(1 as float) / cast('1' as binary) FROM t;
+SELECT cast(1 as float) / cast(1 as boolean) FROM t;
+SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as double) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as double) / cast(1 as smallint) FROM t;
+SELECT cast(1 as double) / cast(1 as int) FROM t;
+SELECT cast(1 as double) / cast(1 as bigint) FROM t;
+SELECT cast(1 as double) / cast(1 as float) FROM t;
+SELECT cast(1 as double) / cast(1 as double) FROM t;
+SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as double) / cast(1 as string) FROM t;
+SELECT cast(1 as double) / cast('1' as binary) FROM t;
+SELECT cast(1 as double) / cast(1 as boolean) FROM t;
+SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as string) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as string) / cast(1 as smallint) FROM t;
+SELECT cast(1 as string) / cast(1 as int) FROM t;
+SELECT cast(1 as string) / cast(1 as bigint) FROM t;
+SELECT cast(1 as string) / cast(1 as float) FROM t;
+SELECT cast(1 as string) / cast(1 as double) FROM t;
+SELECT cast(1 as string) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as string) / cast(1 as string) FROM t;
+SELECT cast(1 as string) / cast('1' as binary) FROM t;
+SELECT cast(1 as string) / cast(1 as boolean) FROM t;
+SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('1' as binary) / cast(1 as tinyint) FROM t;
+SELECT cast('1' as binary) / cast(1 as smallint) FROM t;
+SELECT cast('1' as binary) / cast(1 as int) FROM t;
+SELECT cast('1' as binary) / cast(1 as bigint) FROM t;
+SELECT cast('1' as binary) / cast(1 as float) FROM t;
+SELECT cast('1' as binary) / cast(1 as double) FROM t;
+SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast('1' as binary) / cast(1 as string) FROM t;
+SELECT cast('1' as binary) / cast('1' as binary) FROM t;
+SELECT cast('1' as binary) / cast(1 as boolean) FROM t;
+SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t;
+SELECT cast(1 as boolean) / cast(1 as smallint) FROM t;
+SELECT cast(1 as boolean) / cast(1 as int) FROM t;
+SELECT cast(1 as boolean) / cast(1 as bigint) FROM t;
+SELECT cast(1 as boolean) / cast(1 as float) FROM t;
+SELECT cast(1 as boolean) / cast(1 as double) FROM t;
+SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast(1 as boolean) / cast(1 as string) FROM t;
+SELECT cast(1 as boolean) / cast('1' as binary) FROM t;
+SELECT cast(1 as boolean) / cast(1 as boolean) FROM t;
+SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql
new file mode 100644
index 0000000000000..66e9689850d93
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql
@@ -0,0 +1,175 @@
+--
+-- 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.
+--
+
+CREATE TEMPORARY VIEW t AS SELECT 1;
+
+-- UNION
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
+
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t;
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t;
diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out
new file mode 100644
index 0000000000000..46775d79ff4a2
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out
@@ -0,0 +1,802 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 97
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT true = cast(1 as tinyint) FROM t
+-- !query 1 schema
+struct<(CAST(true AS TINYINT) = CAST(1 AS TINYINT)):boolean>
+-- !query 1 output
+true
+
+
+-- !query 2
+SELECT true = cast(1 as smallint) FROM t
+-- !query 2 schema
+struct<(CAST(true AS SMALLINT) = CAST(1 AS SMALLINT)):boolean>
+-- !query 2 output
+true
+
+
+-- !query 3
+SELECT true = cast(1 as int) FROM t
+-- !query 3 schema
+struct<(CAST(true AS INT) = CAST(1 AS INT)):boolean>
+-- !query 3 output
+true
+
+
+-- !query 4
+SELECT true = cast(1 as bigint) FROM t
+-- !query 4 schema
+struct<(CAST(true AS BIGINT) = CAST(1 AS BIGINT)):boolean>
+-- !query 4 output
+true
+
+
+-- !query 5
+SELECT true = cast(1 as float) FROM t
+-- !query 5 schema
+struct<(CAST(true AS FLOAT) = CAST(1 AS FLOAT)):boolean>
+-- !query 5 output
+true
+
+
+-- !query 6
+SELECT true = cast(1 as double) FROM t
+-- !query 6 schema
+struct<(CAST(true AS DOUBLE) = CAST(1 AS DOUBLE)):boolean>
+-- !query 6 output
+true
+
+
+-- !query 7
+SELECT true = cast(1 as decimal(10, 0)) FROM t
+-- !query 7 schema
+struct<(CAST(true AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean>
+-- !query 7 output
+true
+
+
+-- !query 8
+SELECT true = cast(1 as string) FROM t
+-- !query 8 schema
+struct<(true = CAST(CAST(1 AS STRING) AS BOOLEAN)):boolean>
+-- !query 8 output
+true
+
+
+-- !query 9
+SELECT true = cast('1' as binary) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(true = CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7
+
+
+-- !query 10
+SELECT true = cast(1 as boolean) FROM t
+-- !query 10 schema
+struct<(true = CAST(1 AS BOOLEAN)):boolean>
+-- !query 10 output
+true
+
+
+-- !query 11
+SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(true = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7
+
+
+-- !query 12
+SELECT true = cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(true = CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7
+
+
+-- !query 13
+SELECT true <=> cast(1 as tinyint) FROM t
+-- !query 13 schema
+struct<(CAST(true AS TINYINT) <=> CAST(1 AS TINYINT)):boolean>
+-- !query 13 output
+true
+
+
+-- !query 14
+SELECT true <=> cast(1 as smallint) FROM t
+-- !query 14 schema
+struct<(CAST(true AS SMALLINT) <=> CAST(1 AS SMALLINT)):boolean>
+-- !query 14 output
+true
+
+
+-- !query 15
+SELECT true <=> cast(1 as int) FROM t
+-- !query 15 schema
+struct<(CAST(true AS INT) <=> CAST(1 AS INT)):boolean>
+-- !query 15 output
+true
+
+
+-- !query 16
+SELECT true <=> cast(1 as bigint) FROM t
+-- !query 16 schema
+struct<(CAST(true AS BIGINT) <=> CAST(1 AS BIGINT)):boolean>
+-- !query 16 output
+true
+
+
+-- !query 17
+SELECT true <=> cast(1 as float) FROM t
+-- !query 17 schema
+struct<(CAST(true AS FLOAT) <=> CAST(1 AS FLOAT)):boolean>
+-- !query 17 output
+true
+
+
+-- !query 18
+SELECT true <=> cast(1 as double) FROM t
+-- !query 18 schema
+struct<(CAST(true AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean>
+-- !query 18 output
+true
+
+
+-- !query 19
+SELECT true <=> cast(1 as decimal(10, 0)) FROM t
+-- !query 19 schema
+struct<(CAST(true AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean>
+-- !query 19 output
+true
+
+
+-- !query 20
+SELECT true <=> cast(1 as string) FROM t
+-- !query 20 schema
+struct<(true <=> CAST(CAST(1 AS STRING) AS BOOLEAN)):boolean>
+-- !query 20 output
+true
+
+
+-- !query 21
+SELECT true <=> cast('1' as binary) FROM t
+-- !query 21 schema
+struct<>
+-- !query 21 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(true <=> CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7
+
+
+-- !query 22
+SELECT true <=> cast(1 as boolean) FROM t
+-- !query 22 schema
+struct<(true <=> CAST(1 AS BOOLEAN)):boolean>
+-- !query 22 output
+true
+
+
+-- !query 23
+SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 23 schema
+struct<>
+-- !query 23 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(true <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7
+
+
+-- !query 24
+SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 24 schema
+struct<>
+-- !query 24 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(true <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(true <=> CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7
+
+
+-- !query 25
+SELECT cast(1 as tinyint) = true FROM t
+-- !query 25 schema
+struct<(CAST(1 AS TINYINT) = CAST(true AS TINYINT)):boolean>
+-- !query 25 output
+true
+
+
+-- !query 26
+SELECT cast(1 as smallint) = true FROM t
+-- !query 26 schema
+struct<(CAST(1 AS SMALLINT) = CAST(true AS SMALLINT)):boolean>
+-- !query 26 output
+true
+
+
+-- !query 27
+SELECT cast(1 as int) = true FROM t
+-- !query 27 schema
+struct<(CAST(1 AS INT) = CAST(true AS INT)):boolean>
+-- !query 27 output
+true
+
+
+-- !query 28
+SELECT cast(1 as bigint) = true FROM t
+-- !query 28 schema
+struct<(CAST(1 AS BIGINT) = CAST(true AS BIGINT)):boolean>
+-- !query 28 output
+true
+
+
+-- !query 29
+SELECT cast(1 as float) = true FROM t
+-- !query 29 schema
+struct<(CAST(1 AS FLOAT) = CAST(true AS FLOAT)):boolean>
+-- !query 29 output
+true
+
+
+-- !query 30
+SELECT cast(1 as double) = true FROM t
+-- !query 30 schema
+struct<(CAST(1 AS DOUBLE) = CAST(true AS DOUBLE)):boolean>
+-- !query 30 output
+true
+
+
+-- !query 31
+SELECT cast(1 as decimal(10, 0)) = true FROM t
+-- !query 31 schema
+struct<(CAST(1 AS DECIMAL(10,0)) = CAST(true AS DECIMAL(10,0))):boolean>
+-- !query 31 output
+true
+
+
+-- !query 32
+SELECT cast(1 as string) = true FROM t
+-- !query 32 schema
+struct<(CAST(CAST(1 AS STRING) AS BOOLEAN) = true):boolean>
+-- !query 32 output
+true
+
+
+-- !query 33
+SELECT cast('1' as binary) = true FROM t
+-- !query 33 schema
+struct<>
+-- !query 33 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) = true)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = true)' (binary and boolean).; line 1 pos 7
+
+
+-- !query 34
+SELECT cast(1 as boolean) = true FROM t
+-- !query 34 schema
+struct<(CAST(1 AS BOOLEAN) = true):boolean>
+-- !query 34 output
+true
+
+
+-- !query 35
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t
+-- !query 35 schema
+struct<>
+-- !query 35 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = true)' (timestamp and boolean).; line 1 pos 7
+
+
+-- !query 36
+SELECT cast('2017-12-11 09:30:00' as date) = true FROM t
+-- !query 36 schema
+struct<>
+-- !query 36 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = true)' (date and boolean).; line 1 pos 7
+
+
+-- !query 37
+SELECT cast(1 as tinyint) <=> true FROM t
+-- !query 37 schema
+struct<(CAST(1 AS TINYINT) <=> CAST(true AS TINYINT)):boolean>
+-- !query 37 output
+true
+
+
+-- !query 38
+SELECT cast(1 as smallint) <=> true FROM t
+-- !query 38 schema
+struct<(CAST(1 AS SMALLINT) <=> CAST(true AS SMALLINT)):boolean>
+-- !query 38 output
+true
+
+
+-- !query 39
+SELECT cast(1 as int) <=> true FROM t
+-- !query 39 schema
+struct<(CAST(1 AS INT) <=> CAST(true AS INT)):boolean>
+-- !query 39 output
+true
+
+
+-- !query 40
+SELECT cast(1 as bigint) <=> true FROM t
+-- !query 40 schema
+struct<(CAST(1 AS BIGINT) <=> CAST(true AS BIGINT)):boolean>
+-- !query 40 output
+true
+
+
+-- !query 41
+SELECT cast(1 as float) <=> true FROM t
+-- !query 41 schema
+struct<(CAST(1 AS FLOAT) <=> CAST(true AS FLOAT)):boolean>
+-- !query 41 output
+true
+
+
+-- !query 42
+SELECT cast(1 as double) <=> true FROM t
+-- !query 42 schema
+struct<(CAST(1 AS DOUBLE) <=> CAST(true AS DOUBLE)):boolean>
+-- !query 42 output
+true
+
+
+-- !query 43
+SELECT cast(1 as decimal(10, 0)) <=> true FROM t
+-- !query 43 schema
+struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(true AS DECIMAL(10,0))):boolean>
+-- !query 43 output
+true
+
+
+-- !query 44
+SELECT cast(1 as string) <=> true FROM t
+-- !query 44 schema
+struct<(CAST(CAST(1 AS STRING) AS BOOLEAN) <=> true):boolean>
+-- !query 44 output
+true
+
+
+-- !query 45
+SELECT cast('1' as binary) <=> true FROM t
+-- !query 45 schema
+struct<>
+-- !query 45 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) <=> true)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> true)' (binary and boolean).; line 1 pos 7
+
+
+-- !query 46
+SELECT cast(1 as boolean) <=> true FROM t
+-- !query 46 schema
+struct<(CAST(1 AS BOOLEAN) <=> true):boolean>
+-- !query 46 output
+true
+
+
+-- !query 47
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t
+-- !query 47 schema
+struct<>
+-- !query 47 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> true)' (timestamp and boolean).; line 1 pos 7
+
+
+-- !query 48
+SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t
+-- !query 48 schema
+struct<>
+-- !query 48 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> true)' (date and boolean).; line 1 pos 7
+
+
+-- !query 49
+SELECT false = cast(0 as tinyint) FROM t
+-- !query 49 schema
+struct<(CAST(false AS TINYINT) = CAST(0 AS TINYINT)):boolean>
+-- !query 49 output
+true
+
+
+-- !query 50
+SELECT false = cast(0 as smallint) FROM t
+-- !query 50 schema
+struct<(CAST(false AS SMALLINT) = CAST(0 AS SMALLINT)):boolean>
+-- !query 50 output
+true
+
+
+-- !query 51
+SELECT false = cast(0 as int) FROM t
+-- !query 51 schema
+struct<(CAST(false AS INT) = CAST(0 AS INT)):boolean>
+-- !query 51 output
+true
+
+
+-- !query 52
+SELECT false = cast(0 as bigint) FROM t
+-- !query 52 schema
+struct<(CAST(false AS BIGINT) = CAST(0 AS BIGINT)):boolean>
+-- !query 52 output
+true
+
+
+-- !query 53
+SELECT false = cast(0 as float) FROM t
+-- !query 53 schema
+struct<(CAST(false AS FLOAT) = CAST(0 AS FLOAT)):boolean>
+-- !query 53 output
+true
+
+
+-- !query 54
+SELECT false = cast(0 as double) FROM t
+-- !query 54 schema
+struct<(CAST(false AS DOUBLE) = CAST(0 AS DOUBLE)):boolean>
+-- !query 54 output
+true
+
+
+-- !query 55
+SELECT false = cast(0 as decimal(10, 0)) FROM t
+-- !query 55 schema
+struct<(CAST(false AS DECIMAL(10,0)) = CAST(0 AS DECIMAL(10,0))):boolean>
+-- !query 55 output
+true
+
+
+-- !query 56
+SELECT false = cast(0 as string) FROM t
+-- !query 56 schema
+struct<(false = CAST(CAST(0 AS STRING) AS BOOLEAN)):boolean>
+-- !query 56 output
+true
+
+
+-- !query 57
+SELECT false = cast('0' as binary) FROM t
+-- !query 57 schema
+struct<>
+-- !query 57 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false = CAST('0' AS BINARY))' due to data type mismatch: differing types in '(false = CAST('0' AS BINARY))' (boolean and binary).; line 1 pos 7
+
+
+-- !query 58
+SELECT false = cast(0 as boolean) FROM t
+-- !query 58 schema
+struct<(false = CAST(0 AS BOOLEAN)):boolean>
+-- !query 58 output
+true
+
+
+-- !query 59
+SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 59 schema
+struct<>
+-- !query 59 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(false = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7
+
+
+-- !query 60
+SELECT false = cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 60 schema
+struct<>
+-- !query 60 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(false = CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7
+
+
+-- !query 61
+SELECT false <=> cast(0 as tinyint) FROM t
+-- !query 61 schema
+struct<(CAST(false AS TINYINT) <=> CAST(0 AS TINYINT)):boolean>
+-- !query 61 output
+true
+
+
+-- !query 62
+SELECT false <=> cast(0 as smallint) FROM t
+-- !query 62 schema
+struct<(CAST(false AS SMALLINT) <=> CAST(0 AS SMALLINT)):boolean>
+-- !query 62 output
+true
+
+
+-- !query 63
+SELECT false <=> cast(0 as int) FROM t
+-- !query 63 schema
+struct<(CAST(false AS INT) <=> CAST(0 AS INT)):boolean>
+-- !query 63 output
+true
+
+
+-- !query 64
+SELECT false <=> cast(0 as bigint) FROM t
+-- !query 64 schema
+struct<(CAST(false AS BIGINT) <=> CAST(0 AS BIGINT)):boolean>
+-- !query 64 output
+true
+
+
+-- !query 65
+SELECT false <=> cast(0 as float) FROM t
+-- !query 65 schema
+struct<(CAST(false AS FLOAT) <=> CAST(0 AS FLOAT)):boolean>
+-- !query 65 output
+true
+
+
+-- !query 66
+SELECT false <=> cast(0 as double) FROM t
+-- !query 66 schema
+struct<(CAST(false AS DOUBLE) <=> CAST(0 AS DOUBLE)):boolean>
+-- !query 66 output
+true
+
+
+-- !query 67
+SELECT false <=> cast(0 as decimal(10, 0)) FROM t
+-- !query 67 schema
+struct<(CAST(false AS DECIMAL(10,0)) <=> CAST(0 AS DECIMAL(10,0))):boolean>
+-- !query 67 output
+true
+
+
+-- !query 68
+SELECT false <=> cast(0 as string) FROM t
+-- !query 68 schema
+struct<(false <=> CAST(CAST(0 AS STRING) AS BOOLEAN)):boolean>
+-- !query 68 output
+true
+
+
+-- !query 69
+SELECT false <=> cast('0' as binary) FROM t
+-- !query 69 schema
+struct<>
+-- !query 69 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false <=> CAST('0' AS BINARY))' due to data type mismatch: differing types in '(false <=> CAST('0' AS BINARY))' (boolean and binary).; line 1 pos 7
+
+
+-- !query 70
+SELECT false <=> cast(0 as boolean) FROM t
+-- !query 70 schema
+struct<(false <=> CAST(0 AS BOOLEAN)):boolean>
+-- !query 70 output
+true
+
+
+-- !query 71
+SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 71 schema
+struct<>
+-- !query 71 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(false <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7
+
+
+-- !query 72
+SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 72 schema
+struct<>
+-- !query 72 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(false <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(false <=> CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7
+
+
+-- !query 73
+SELECT cast(0 as tinyint) = false FROM t
+-- !query 73 schema
+struct<(CAST(0 AS TINYINT) = CAST(false AS TINYINT)):boolean>
+-- !query 73 output
+true
+
+
+-- !query 74
+SELECT cast(0 as smallint) = false FROM t
+-- !query 74 schema
+struct<(CAST(0 AS SMALLINT) = CAST(false AS SMALLINT)):boolean>
+-- !query 74 output
+true
+
+
+-- !query 75
+SELECT cast(0 as int) = false FROM t
+-- !query 75 schema
+struct<(CAST(0 AS INT) = CAST(false AS INT)):boolean>
+-- !query 75 output
+true
+
+
+-- !query 76
+SELECT cast(0 as bigint) = false FROM t
+-- !query 76 schema
+struct<(CAST(0 AS BIGINT) = CAST(false AS BIGINT)):boolean>
+-- !query 76 output
+true
+
+
+-- !query 77
+SELECT cast(0 as float) = false FROM t
+-- !query 77 schema
+struct<(CAST(0 AS FLOAT) = CAST(false AS FLOAT)):boolean>
+-- !query 77 output
+true
+
+
+-- !query 78
+SELECT cast(0 as double) = false FROM t
+-- !query 78 schema
+struct<(CAST(0 AS DOUBLE) = CAST(false AS DOUBLE)):boolean>
+-- !query 78 output
+true
+
+
+-- !query 79
+SELECT cast(0 as decimal(10, 0)) = false FROM t
+-- !query 79 schema
+struct<(CAST(0 AS DECIMAL(10,0)) = CAST(false AS DECIMAL(10,0))):boolean>
+-- !query 79 output
+true
+
+
+-- !query 80
+SELECT cast(0 as string) = false FROM t
+-- !query 80 schema
+struct<(CAST(CAST(0 AS STRING) AS BOOLEAN) = false):boolean>
+-- !query 80 output
+true
+
+
+-- !query 81
+SELECT cast('0' as binary) = false FROM t
+-- !query 81 schema
+struct<>
+-- !query 81 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('0' AS BINARY) = false)' due to data type mismatch: differing types in '(CAST('0' AS BINARY) = false)' (binary and boolean).; line 1 pos 7
+
+
+-- !query 82
+SELECT cast(0 as boolean) = false FROM t
+-- !query 82 schema
+struct<(CAST(0 AS BOOLEAN) = false):boolean>
+-- !query 82 output
+true
+
+
+-- !query 83
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t
+-- !query 83 schema
+struct<>
+-- !query 83 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = false)' (timestamp and boolean).; line 1 pos 7
+
+
+-- !query 84
+SELECT cast('2017-12-11 09:30:00' as date) = false FROM t
+-- !query 84 schema
+struct<>
+-- !query 84 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = false)' (date and boolean).; line 1 pos 7
+
+
+-- !query 85
+SELECT cast(0 as tinyint) <=> false FROM t
+-- !query 85 schema
+struct<(CAST(0 AS TINYINT) <=> CAST(false AS TINYINT)):boolean>
+-- !query 85 output
+true
+
+
+-- !query 86
+SELECT cast(0 as smallint) <=> false FROM t
+-- !query 86 schema
+struct<(CAST(0 AS SMALLINT) <=> CAST(false AS SMALLINT)):boolean>
+-- !query 86 output
+true
+
+
+-- !query 87
+SELECT cast(0 as int) <=> false FROM t
+-- !query 87 schema
+struct<(CAST(0 AS INT) <=> CAST(false AS INT)):boolean>
+-- !query 87 output
+true
+
+
+-- !query 88
+SELECT cast(0 as bigint) <=> false FROM t
+-- !query 88 schema
+struct<(CAST(0 AS BIGINT) <=> CAST(false AS BIGINT)):boolean>
+-- !query 88 output
+true
+
+
+-- !query 89
+SELECT cast(0 as float) <=> false FROM t
+-- !query 89 schema
+struct<(CAST(0 AS FLOAT) <=> CAST(false AS FLOAT)):boolean>
+-- !query 89 output
+true
+
+
+-- !query 90
+SELECT cast(0 as double) <=> false FROM t
+-- !query 90 schema
+struct<(CAST(0 AS DOUBLE) <=> CAST(false AS DOUBLE)):boolean>
+-- !query 90 output
+true
+
+
+-- !query 91
+SELECT cast(0 as decimal(10, 0)) <=> false FROM t
+-- !query 91 schema
+struct<(CAST(0 AS DECIMAL(10,0)) <=> CAST(false AS DECIMAL(10,0))):boolean>
+-- !query 91 output
+true
+
+
+-- !query 92
+SELECT cast(0 as string) <=> false FROM t
+-- !query 92 schema
+struct<(CAST(CAST(0 AS STRING) AS BOOLEAN) <=> false):boolean>
+-- !query 92 output
+true
+
+
+-- !query 93
+SELECT cast('0' as binary) <=> false FROM t
+-- !query 93 schema
+struct<>
+-- !query 93 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('0' AS BINARY) <=> false)' due to data type mismatch: differing types in '(CAST('0' AS BINARY) <=> false)' (binary and boolean).; line 1 pos 7
+
+
+-- !query 94
+SELECT cast(0 as boolean) <=> false FROM t
+-- !query 94 schema
+struct<(CAST(0 AS BOOLEAN) <=> false):boolean>
+-- !query 94 output
+true
+
+
+-- !query 95
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t
+-- !query 95 schema
+struct<>
+-- !query 95 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> false)' (timestamp and boolean).; line 1 pos 7
+
+
+-- !query 96
+SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t
+-- !query 96 schema
+struct<>
+-- !query 96 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> false)' (date and boolean).; line 1 pos 7
diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
new file mode 100644
index 0000000000000..017e0fea30e90
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out
@@ -0,0 +1,1242 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 145
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t
+-- !query 1 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 1 output
+1.0
+
+
+-- !query 2
+SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t
+-- !query 2 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 2 output
+1.0
+
+
+-- !query 3
+SELECT cast(1 as tinyint) / cast(1 as int) FROM t
+-- !query 3 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 3 output
+1.0
+
+
+-- !query 4
+SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t
+-- !query 4 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 4 output
+1.0
+
+
+-- !query 5
+SELECT cast(1 as tinyint) / cast(1 as float) FROM t
+-- !query 5 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 5 output
+1.0
+
+
+-- !query 6
+SELECT cast(1 as tinyint) / cast(1 as double) FROM t
+-- !query 6 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double>
+-- !query 6 output
+1.0
+
+
+-- !query 7
+SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t
+-- !query 7 schema
+struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)>
+-- !query 7 output
+1
+
+
+-- !query 8
+SELECT cast(1 as tinyint) / cast(1 as string) FROM t
+-- !query 8 schema
+struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double>
+-- !query 8 output
+1.0
+
+
+-- !query 9
+SELECT cast(1 as tinyint) / cast('1' as binary) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' (tinyint and binary).; line 1 pos 7
+
+
+-- !query 10
+SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' (tinyint and boolean).; line 1 pos 7
+
+
+-- !query 11
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (tinyint and timestamp).; line 1 pos 7
+
+
+-- !query 12
+SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00' AS DATE))' (tinyint and date).; line 1 pos 7
+
+
+-- !query 13
+SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t
+-- !query 13 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 13 output
+1.0
+
+
+-- !query 14
+SELECT cast(1 as smallint) / cast(1 as smallint) FROM t
+-- !query 14 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 14 output
+1.0
+
+
+-- !query 15
+SELECT cast(1 as smallint) / cast(1 as int) FROM t
+-- !query 15 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 15 output
+1.0
+
+
+-- !query 16
+SELECT cast(1 as smallint) / cast(1 as bigint) FROM t
+-- !query 16 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 16 output
+1.0
+
+
+-- !query 17
+SELECT cast(1 as smallint) / cast(1 as float) FROM t
+-- !query 17 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 17 output
+1.0
+
+
+-- !query 18
+SELECT cast(1 as smallint) / cast(1 as double) FROM t
+-- !query 18 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double>
+-- !query 18 output
+1.0
+
+
+-- !query 19
+SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t
+-- !query 19 schema
+struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)>
+-- !query 19 output
+1
+
+
+-- !query 20
+SELECT cast(1 as smallint) / cast(1 as string) FROM t
+-- !query 20 schema
+struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double>
+-- !query 20 output
+1.0
+
+
+-- !query 21
+SELECT cast(1 as smallint) / cast('1' as binary) FROM t
+-- !query 21 schema
+struct<>
+-- !query 21 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS SMALLINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('1' AS BINARY))' (smallint and binary).; line 1 pos 7
+
+
+-- !query 22
+SELECT cast(1 as smallint) / cast(1 as boolean) FROM t
+-- !query 22 schema
+struct<>
+-- !query 22 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))' (smallint and boolean).; line 1 pos 7
+
+
+-- !query 23
+SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 23 schema
+struct<>
+-- !query 23 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (smallint and timestamp).; line 1 pos 7
+
+
+-- !query 24
+SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 24 schema
+struct<>
+-- !query 24 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00' AS DATE))' (smallint and date).; line 1 pos 7
+
+
+-- !query 25
+SELECT cast(1 as int) / cast(1 as tinyint) FROM t
+-- !query 25 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 25 output
+1.0
+
+
+-- !query 26
+SELECT cast(1 as int) / cast(1 as smallint) FROM t
+-- !query 26 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 26 output
+1.0
+
+
+-- !query 27
+SELECT cast(1 as int) / cast(1 as int) FROM t
+-- !query 27 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 27 output
+1.0
+
+
+-- !query 28
+SELECT cast(1 as int) / cast(1 as bigint) FROM t
+-- !query 28 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 28 output
+1.0
+
+
+-- !query 29
+SELECT cast(1 as int) / cast(1 as float) FROM t
+-- !query 29 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 29 output
+1.0
+
+
+-- !query 30
+SELECT cast(1 as int) / cast(1 as double) FROM t
+-- !query 30 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double>
+-- !query 30 output
+1.0
+
+
+-- !query 31
+SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t
+-- !query 31 schema
+struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)>
+-- !query 31 output
+1
+
+
+-- !query 32
+SELECT cast(1 as int) / cast(1 as string) FROM t
+-- !query 32 schema
+struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double>
+-- !query 32 output
+1.0
+
+
+-- !query 33
+SELECT cast(1 as int) / cast('1' as binary) FROM t
+-- !query 33 schema
+struct<>
+-- !query 33 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS INT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('1' AS BINARY))' (int and binary).; line 1 pos 7
+
+
+-- !query 34
+SELECT cast(1 as int) / cast(1 as boolean) FROM t
+-- !query 34 schema
+struct<>
+-- !query 34 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS INT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST(1 AS BOOLEAN))' (int and boolean).; line 1 pos 7
+
+
+-- !query 35
+SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 35 schema
+struct<>
+-- !query 35 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (int and timestamp).; line 1 pos 7
+
+
+-- !query 36
+SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 36 schema
+struct<>
+-- !query 36 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00' AS DATE))' (int and date).; line 1 pos 7
+
+
+-- !query 37
+SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t
+-- !query 37 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 37 output
+1.0
+
+
+-- !query 38
+SELECT cast(1 as bigint) / cast(1 as smallint) FROM t
+-- !query 38 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 38 output
+1.0
+
+
+-- !query 39
+SELECT cast(1 as bigint) / cast(1 as int) FROM t
+-- !query 39 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 39 output
+1.0
+
+
+-- !query 40
+SELECT cast(1 as bigint) / cast(1 as bigint) FROM t
+-- !query 40 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 40 output
+1.0
+
+
+-- !query 41
+SELECT cast(1 as bigint) / cast(1 as float) FROM t
+-- !query 41 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 41 output
+1.0
+
+
+-- !query 42
+SELECT cast(1 as bigint) / cast(1 as double) FROM t
+-- !query 42 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double>
+-- !query 42 output
+1.0
+
+
+-- !query 43
+SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t
+-- !query 43 schema
+struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)>
+-- !query 43 output
+1
+
+
+-- !query 44
+SELECT cast(1 as bigint) / cast(1 as string) FROM t
+-- !query 44 schema
+struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double>
+-- !query 44 output
+1.0
+
+
+-- !query 45
+SELECT cast(1 as bigint) / cast('1' as binary) FROM t
+-- !query 45 schema
+struct<>
+-- !query 45 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BIGINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('1' AS BINARY))' (bigint and binary).; line 1 pos 7
+
+
+-- !query 46
+SELECT cast(1 as bigint) / cast(1 as boolean) FROM t
+-- !query 46 schema
+struct<>
+-- !query 46 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))' (bigint and boolean).; line 1 pos 7
+
+
+-- !query 47
+SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 47 schema
+struct<>
+-- !query 47 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (bigint and timestamp).; line 1 pos 7
+
+
+-- !query 48
+SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 48 schema
+struct<>
+-- !query 48 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00' AS DATE))' (bigint and date).; line 1 pos 7
+
+
+-- !query 49
+SELECT cast(1 as float) / cast(1 as tinyint) FROM t
+-- !query 49 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 49 output
+1.0
+
+
+-- !query 50
+SELECT cast(1 as float) / cast(1 as smallint) FROM t
+-- !query 50 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 50 output
+1.0
+
+
+-- !query 51
+SELECT cast(1 as float) / cast(1 as int) FROM t
+-- !query 51 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 51 output
+1.0
+
+
+-- !query 52
+SELECT cast(1 as float) / cast(1 as bigint) FROM t
+-- !query 52 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 52 output
+1.0
+
+
+-- !query 53
+SELECT cast(1 as float) / cast(1 as float) FROM t
+-- !query 53 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 53 output
+1.0
+
+
+-- !query 54
+SELECT cast(1 as float) / cast(1 as double) FROM t
+-- !query 54 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double>
+-- !query 54 output
+1.0
+
+
+-- !query 55
+SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t
+-- !query 55 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) AS DOUBLE)):double>
+-- !query 55 output
+1.0
+
+
+-- !query 56
+SELECT cast(1 as float) / cast(1 as string) FROM t
+-- !query 56 schema
+struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double>
+-- !query 56 output
+1.0
+
+
+-- !query 57
+SELECT cast(1 as float) / cast('1' as binary) FROM t
+-- !query 57 schema
+struct<>
+-- !query 57 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS FLOAT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('1' AS BINARY))' (float and binary).; line 1 pos 7
+
+
+-- !query 58
+SELECT cast(1 as float) / cast(1 as boolean) FROM t
+-- !query 58 schema
+struct<>
+-- !query 58 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))' (float and boolean).; line 1 pos 7
+
+
+-- !query 59
+SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 59 schema
+struct<>
+-- !query 59 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (float and timestamp).; line 1 pos 7
+
+
+-- !query 60
+SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 60 schema
+struct<>
+-- !query 60 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00' AS DATE))' (float and date).; line 1 pos 7
+
+
+-- !query 61
+SELECT cast(1 as double) / cast(1 as tinyint) FROM t
+-- !query 61 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 61 output
+1.0
+
+
+-- !query 62
+SELECT cast(1 as double) / cast(1 as smallint) FROM t
+-- !query 62 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 62 output
+1.0
+
+
+-- !query 63
+SELECT cast(1 as double) / cast(1 as int) FROM t
+-- !query 63 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 63 output
+1.0
+
+
+-- !query 64
+SELECT cast(1 as double) / cast(1 as bigint) FROM t
+-- !query 64 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 64 output
+1.0
+
+
+-- !query 65
+SELECT cast(1 as double) / cast(1 as float) FROM t
+-- !query 65 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 65 output
+1.0
+
+
+-- !query 66
+SELECT cast(1 as double) / cast(1 as double) FROM t
+-- !query 66 schema
+struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double>
+-- !query 66 output
+1.0
+
+
+-- !query 67
+SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t
+-- !query 67 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double>
+-- !query 67 output
+1.0
+
+
+-- !query 68
+SELECT cast(1 as double) / cast(1 as string) FROM t
+-- !query 68 schema
+struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double>
+-- !query 68 output
+1.0
+
+
+-- !query 69
+SELECT cast(1 as double) / cast('1' as binary) FROM t
+-- !query 69 schema
+struct<>
+-- !query 69 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DOUBLE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('1' AS BINARY))' (double and binary).; line 1 pos 7
+
+
+-- !query 70
+SELECT cast(1 as double) / cast(1 as boolean) FROM t
+-- !query 70 schema
+struct<>
+-- !query 70 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7
+
+
+-- !query 71
+SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 71 schema
+struct<>
+-- !query 71 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7
+
+
+-- !query 72
+SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 72 schema
+struct<>
+-- !query 72 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7
+
+
+-- !query 73
+SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t
+-- !query 73 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)>
+-- !query 73 output
+1
+
+
+-- !query 74
+SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t
+-- !query 74 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)>
+-- !query 74 output
+1
+
+
+-- !query 75
+SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t
+-- !query 75 schema
+struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)>
+-- !query 75 output
+1
+
+
+-- !query 76
+SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t
+-- !query 76 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)>
+-- !query 76 output
+1
+
+
+-- !query 77
+SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t
+-- !query 77 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 77 output
+1.0
+
+
+-- !query 78
+SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t
+-- !query 78 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double>
+-- !query 78 output
+1.0
+
+
+-- !query 79
+SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t
+-- !query 79 schema
+struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)>
+-- !query 79 output
+1
+
+
+-- !query 80
+SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t
+-- !query 80 schema
+struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double>
+-- !query 80 output
+1.0
+
+
+-- !query 81
+SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t
+-- !query 81 schema
+struct<>
+-- !query 81 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7
+
+
+-- !query 82
+SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t
+-- !query 82 schema
+struct<>
+-- !query 82 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7
+
+
+-- !query 83
+SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 83 schema
+struct<>
+-- !query 83 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7
+
+
+-- !query 84
+SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 84 schema
+struct<>
+-- !query 84 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7
+
+
+-- !query 85
+SELECT cast(1 as string) / cast(1 as tinyint) FROM t
+-- !query 85 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double>
+-- !query 85 output
+1.0
+
+
+-- !query 86
+SELECT cast(1 as string) / cast(1 as smallint) FROM t
+-- !query 86 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double>
+-- !query 86 output
+1.0
+
+
+-- !query 87
+SELECT cast(1 as string) / cast(1 as int) FROM t
+-- !query 87 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double>
+-- !query 87 output
+1.0
+
+
+-- !query 88
+SELECT cast(1 as string) / cast(1 as bigint) FROM t
+-- !query 88 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double>
+-- !query 88 output
+1.0
+
+
+-- !query 89
+SELECT cast(1 as string) / cast(1 as float) FROM t
+-- !query 89 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double>
+-- !query 89 output
+1.0
+
+
+-- !query 90
+SELECT cast(1 as string) / cast(1 as double) FROM t
+-- !query 90 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS DOUBLE)):double>
+-- !query 90 output
+1.0
+
+
+-- !query 91
+SELECT cast(1 as string) / cast(1 as decimal(10, 0)) FROM t
+-- !query 91 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double>
+-- !query 91 output
+1.0
+
+
+-- !query 92
+SELECT cast(1 as string) / cast(1 as string) FROM t
+-- !query 92 schema
+struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double>
+-- !query 92 output
+1.0
+
+
+-- !query 93
+SELECT cast(1 as string) / cast('1' as binary) FROM t
+-- !query 93 schema
+struct<>
+-- !query 93 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('1' AS BINARY))' (double and binary).; line 1 pos 7
+
+
+-- !query 94
+SELECT cast(1 as string) / cast(1 as boolean) FROM t
+-- !query 94 schema
+struct<>
+-- !query 94 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7
+
+
+-- !query 95
+SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 95 schema
+struct<>
+-- !query 95 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7
+
+
+-- !query 96
+SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 96 schema
+struct<>
+-- !query 96 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7
+
+
+-- !query 97
+SELECT cast('1' as binary) / cast(1 as tinyint) FROM t
+-- !query 97 schema
+struct<>
+-- !query 97 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS TINYINT))' (binary and tinyint).; line 1 pos 7
+
+
+-- !query 98
+SELECT cast('1' as binary) / cast(1 as smallint) FROM t
+-- !query 98 schema
+struct<>
+-- !query 98 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS SMALLINT))' (binary and smallint).; line 1 pos 7
+
+
+-- !query 99
+SELECT cast('1' as binary) / cast(1 as int) FROM t
+-- !query 99 schema
+struct<>
+-- !query 99 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS INT))' (binary and int).; line 1 pos 7
+
+
+-- !query 100
+SELECT cast('1' as binary) / cast(1 as bigint) FROM t
+-- !query 100 schema
+struct<>
+-- !query 100 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS BIGINT))' (binary and bigint).; line 1 pos 7
+
+
+-- !query 101
+SELECT cast('1' as binary) / cast(1 as float) FROM t
+-- !query 101 schema
+struct<>
+-- !query 101 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS FLOAT))' (binary and float).; line 1 pos 7
+
+
+-- !query 102
+SELECT cast('1' as binary) / cast(1 as double) FROM t
+-- !query 102 schema
+struct<>
+-- !query 102 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DOUBLE))' (binary and double).; line 1 pos 7
+
+
+-- !query 103
+SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t
+-- !query 103 schema
+struct<>
+-- !query 103 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7
+
+
+-- !query 104
+SELECT cast('1' as binary) / cast(1 as string) FROM t
+-- !query 104 schema
+struct<>
+-- !query 104 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(CAST(1 AS STRING) AS DOUBLE))' (binary and double).; line 1 pos 7
+
+
+-- !query 105
+SELECT cast('1' as binary) / cast('1' as binary) FROM t
+-- !query 105 schema
+struct<>
+-- !query 105 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST('1' AS BINARY))' due to data type mismatch: '(CAST('1' AS BINARY) / CAST('1' AS BINARY))' requires (double or decimal) type, not binary; line 1 pos 7
+
+
+-- !query 106
+SELECT cast('1' as binary) / cast(1 as boolean) FROM t
+-- !query 106 schema
+struct<>
+-- !query 106 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS BOOLEAN))' (binary and boolean).; line 1 pos 7
+
+
+-- !query 107
+SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 107 schema
+struct<>
+-- !query 107 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (binary and timestamp).; line 1 pos 7
+
+
+-- !query 108
+SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 108 schema
+struct<>
+-- !query 108 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00' AS DATE))' (binary and date).; line 1 pos 7
+
+
+-- !query 109
+SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t
+-- !query 109 schema
+struct<>
+-- !query 109 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))' (boolean and tinyint).; line 1 pos 7
+
+
+-- !query 110
+SELECT cast(1 as boolean) / cast(1 as smallint) FROM t
+-- !query 110 schema
+struct<>
+-- !query 110 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))' (boolean and smallint).; line 1 pos 7
+
+
+-- !query 111
+SELECT cast(1 as boolean) / cast(1 as int) FROM t
+-- !query 111 schema
+struct<>
+-- !query 111 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS INT))' (boolean and int).; line 1 pos 7
+
+
+-- !query 112
+SELECT cast(1 as boolean) / cast(1 as bigint) FROM t
+-- !query 112 schema
+struct<>
+-- !query 112 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))' (boolean and bigint).; line 1 pos 7
+
+
+-- !query 113
+SELECT cast(1 as boolean) / cast(1 as float) FROM t
+-- !query 113 schema
+struct<>
+-- !query 113 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))' (boolean and float).; line 1 pos 7
+
+
+-- !query 114
+SELECT cast(1 as boolean) / cast(1 as double) FROM t
+-- !query 114 schema
+struct<>
+-- !query 114 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))' (boolean and double).; line 1 pos 7
+
+
+-- !query 115
+SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t
+-- !query 115 schema
+struct<>
+-- !query 115 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))' (boolean and decimal(10,0)).; line 1 pos 7
+
+
+-- !query 116
+SELECT cast(1 as boolean) / cast(1 as string) FROM t
+-- !query 116 schema
+struct<>
+-- !query 116 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(CAST(1 AS STRING) AS DOUBLE))' (boolean and double).; line 1 pos 7
+
+
+-- !query 117
+SELECT cast(1 as boolean) / cast('1' as binary) FROM t
+-- !query 117 schema
+struct<>
+-- !query 117 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7
+
+
+-- !query 118
+SELECT cast(1 as boolean) / cast(1 as boolean) FROM t
+-- !query 118 schema
+struct<>
+-- !query 118 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))' due to data type mismatch: '(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))' requires (double or decimal) type, not boolean; line 1 pos 7
+
+
+-- !query 119
+SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 119 schema
+struct<>
+-- !query 119 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7
+
+
+-- !query 120
+SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 120 schema
+struct<>
+-- !query 120 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7
+
+
+-- !query 121
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t
+-- !query 121 schema
+struct<>
+-- !query 121 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS TINYINT))' (timestamp and tinyint).; line 1 pos 7
+
+
+-- !query 122
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t
+-- !query 122 schema
+struct<>
+-- !query 122 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS SMALLINT))' (timestamp and smallint).; line 1 pos 7
+
+
+-- !query 123
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t
+-- !query 123 schema
+struct<>
+-- !query 123 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS INT))' (timestamp and int).; line 1 pos 7
+
+
+-- !query 124
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t
+-- !query 124 schema
+struct<>
+-- !query 124 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BIGINT))' (timestamp and bigint).; line 1 pos 7
+
+
+-- !query 125
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t
+-- !query 125 schema
+struct<>
+-- !query 125 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS FLOAT))' (timestamp and float).; line 1 pos 7
+
+
+-- !query 126
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t
+-- !query 126 schema
+struct<>
+-- !query 126 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DOUBLE))' (timestamp and double).; line 1 pos 7
+
+
+-- !query 127
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t
+-- !query 127 schema
+struct<>
+-- !query 127 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7
+
+
+-- !query 128
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t
+-- !query 128 schema
+struct<>
+-- !query 128 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' (timestamp and double).; line 1 pos 7
+
+
+-- !query 129
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t
+-- !query 129 schema
+struct<>
+-- !query 129 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS BINARY))' (timestamp and binary).; line 1 pos 7
+
+
+-- !query 130
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t
+-- !query 130 schema
+struct<>
+-- !query 130 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BOOLEAN))' (timestamp and boolean).; line 1 pos 7
+
+
+-- !query 131
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 131 schema
+struct<>
+-- !query 131 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' requires (double or decimal) type, not timestamp; line 1 pos 7
+
+
+-- !query 132
+SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 132 schema
+struct<>
+-- !query 132 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00' AS DATE))' (timestamp and date).; line 1 pos 7
+
+
+-- !query 133
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t
+-- !query 133 schema
+struct<>
+-- !query 133 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS TINYINT))' (date and tinyint).; line 1 pos 7
+
+
+-- !query 134
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t
+-- !query 134 schema
+struct<>
+-- !query 134 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS SMALLINT))' (date and smallint).; line 1 pos 7
+
+
+-- !query 135
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t
+-- !query 135 schema
+struct<>
+-- !query 135 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS INT))' (date and int).; line 1 pos 7
+
+
+-- !query 136
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t
+-- !query 136 schema
+struct<>
+-- !query 136 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BIGINT))' (date and bigint).; line 1 pos 7
+
+
+-- !query 137
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t
+-- !query 137 schema
+struct<>
+-- !query 137 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS FLOAT))' (date and float).; line 1 pos 7
+
+
+-- !query 138
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t
+-- !query 138 schema
+struct<>
+-- !query 138 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DOUBLE))' (date and double).; line 1 pos 7
+
+
+-- !query 139
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t
+-- !query 139 schema
+struct<>
+-- !query 139 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7
+
+
+-- !query 140
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t
+-- !query 140 schema
+struct<>
+-- !query 140 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(CAST(1 AS STRING) AS DOUBLE))' (date and double).; line 1 pos 7
+
+
+-- !query 141
+SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t
+-- !query 141 schema
+struct<>
+-- !query 141 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS BINARY))' (date and binary).; line 1 pos 7
+
+
+-- !query 142
+SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t
+-- !query 142 schema
+struct<>
+-- !query 142 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BOOLEAN))' (date and boolean).; line 1 pos 7
+
+
+-- !query 143
+SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 143 schema
+struct<>
+-- !query 143 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (date and timestamp).; line 1 pos 7
+
+
+-- !query 144
+SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 144 schema
+struct<>
+-- !query 144 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00' AS DATE))' requires (double or decimal) type, not date; line 1 pos 7
diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
new file mode 100644
index 0000000000000..20a9e47217238
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
@@ -0,0 +1,1305 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 145
+
+
+-- !query 0
+CREATE TEMPORARY VIEW t AS SELECT 1
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 1 schema
+struct
+-- !query 1 output
+1
+2
+
+
+-- !query 2
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 2 schema
+struct
+-- !query 2 output
+1
+2
+
+
+-- !query 3
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 3 schema
+struct
+-- !query 3 output
+1
+2
+
+
+-- !query 4
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 4 schema
+struct
+-- !query 4 output
+1
+2
+
+
+-- !query 5
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 5 schema
+struct
+-- !query 5 output
+1.0
+2.0
+
+
+-- !query 6
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 6 schema
+struct
+-- !query 6 output
+1.0
+2.0
+
+
+-- !query 7
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 7 schema
+struct
+-- !query 7 output
+1
+2
+
+
+-- !query 8
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 8 schema
+struct
+-- !query 8 output
+1
+2
+
+
+-- !query 9
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 9 schema
+struct<>
+-- !query 9 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> tinyint at the first column of the second table;
+
+
+-- !query 10
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> tinyint at the first column of the second table;
+
+
+-- !query 11
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> tinyint at the first column of the second table;
+
+
+-- !query 12
+SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> tinyint at the first column of the second table;
+
+
+-- !query 13
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 13 schema
+struct
+-- !query 13 output
+1
+2
+
+
+-- !query 14
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 14 schema
+struct
+-- !query 14 output
+1
+2
+
+
+-- !query 15
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 15 schema
+struct
+-- !query 15 output
+1
+2
+
+
+-- !query 16
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 16 schema
+struct
+-- !query 16 output
+1
+2
+
+
+-- !query 17
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 17 schema
+struct
+-- !query 17 output
+1.0
+2.0
+
+
+-- !query 18
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 18 schema
+struct
+-- !query 18 output
+1.0
+2.0
+
+
+-- !query 19
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 19 schema
+struct
+-- !query 19 output
+1
+2
+
+
+-- !query 20
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 20 schema
+struct
+-- !query 20 output
+1
+2
+
+
+-- !query 21
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 21 schema
+struct<>
+-- !query 21 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> smallint at the first column of the second table;
+
+
+-- !query 22
+SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 22 schema
+struct<>
+-- !query 22 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> smallint at the first column of the second table;
+
+
+-- !query 23
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 23 schema
+struct<>
+-- !query 23 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> smallint at the first column of the second table;
+
+
+-- !query 24
+SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 24 schema
+struct<>
+-- !query 24 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> smallint at the first column of the second table;
+
+
+-- !query 25
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 25 schema
+struct
+-- !query 25 output
+1
+2
+
+
+-- !query 26
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 26 schema
+struct
+-- !query 26 output
+1
+2
+
+
+-- !query 27
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 27 schema
+struct
+-- !query 27 output
+1
+2
+
+
+-- !query 28
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 28 schema
+struct
+-- !query 28 output
+1
+2
+
+
+-- !query 29
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 29 schema
+struct
+-- !query 29 output
+1.0
+2.0
+
+
+-- !query 30
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 30 schema
+struct
+-- !query 30 output
+1.0
+2.0
+
+
+-- !query 31
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 31 schema
+struct
+-- !query 31 output
+1
+2
+
+
+-- !query 32
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 32 schema
+struct
+-- !query 32 output
+1
+2
+
+
+-- !query 33
+SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 33 schema
+struct<>
+-- !query 33 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> int at the first column of the second table;
+
+
+-- !query 34
+SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 34 schema
+struct<>
+-- !query 34 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> int at the first column of the second table;
+
+
+-- !query 35
+SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 35 schema
+struct<>
+-- !query 35 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> int at the first column of the second table;
+
+
+-- !query 36
+SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 36 schema
+struct<>
+-- !query 36 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> int at the first column of the second table;
+
+
+-- !query 37
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 37 schema
+struct
+-- !query 37 output
+1
+2
+
+
+-- !query 38
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 38 schema
+struct
+-- !query 38 output
+1
+2
+
+
+-- !query 39
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 39 schema
+struct
+-- !query 39 output
+1
+2
+
+
+-- !query 40
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 40 schema
+struct
+-- !query 40 output
+1
+2
+
+
+-- !query 41
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 41 schema
+struct
+-- !query 41 output
+1.0
+2.0
+
+
+-- !query 42
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 42 schema
+struct
+-- !query 42 output
+1.0
+2.0
+
+
+-- !query 43
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 43 schema
+struct
+-- !query 43 output
+1
+2
+
+
+-- !query 44
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 44 schema
+struct
+-- !query 44 output
+1
+2
+
+
+-- !query 45
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 45 schema
+struct<>
+-- !query 45 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> bigint at the first column of the second table;
+
+
+-- !query 46
+SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 46 schema
+struct<>
+-- !query 46 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> bigint at the first column of the second table;
+
+
+-- !query 47
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 47 schema
+struct<>
+-- !query 47 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> bigint at the first column of the second table;
+
+
+-- !query 48
+SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 48 schema
+struct<>
+-- !query 48 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> bigint at the first column of the second table;
+
+
+-- !query 49
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 49 schema
+struct
+-- !query 49 output
+1.0
+2.0
+
+
+-- !query 50
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 50 schema
+struct
+-- !query 50 output
+1.0
+2.0
+
+
+-- !query 51
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 51 schema
+struct
+-- !query 51 output
+1.0
+2.0
+
+
+-- !query 52
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 52 schema
+struct
+-- !query 52 output
+1.0
+2.0
+
+
+-- !query 53
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 53 schema
+struct
+-- !query 53 output
+1.0
+2.0
+
+
+-- !query 54
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 54 schema
+struct
+-- !query 54 output
+1.0
+2.0
+
+
+-- !query 55
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 55 schema
+struct
+-- !query 55 output
+1.0
+2.0
+
+
+-- !query 56
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 56 schema
+struct
+-- !query 56 output
+1.0
+2
+
+
+-- !query 57
+SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 57 schema
+struct<>
+-- !query 57 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> float at the first column of the second table;
+
+
+-- !query 58
+SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 58 schema
+struct<>
+-- !query 58 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> float at the first column of the second table;
+
+
+-- !query 59
+SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 59 schema
+struct<>
+-- !query 59 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> float at the first column of the second table;
+
+
+-- !query 60
+SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 60 schema
+struct<>
+-- !query 60 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> float at the first column of the second table;
+
+
+-- !query 61
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 61 schema
+struct
+-- !query 61 output
+1.0
+2.0
+
+
+-- !query 62
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 62 schema
+struct
+-- !query 62 output
+1.0
+2.0
+
+
+-- !query 63
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 63 schema
+struct
+-- !query 63 output
+1.0
+2.0
+
+
+-- !query 64
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 64 schema
+struct
+-- !query 64 output
+1.0
+2.0
+
+
+-- !query 65
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 65 schema
+struct
+-- !query 65 output
+1.0
+2.0
+
+
+-- !query 66
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 66 schema
+struct
+-- !query 66 output
+1.0
+2.0
+
+
+-- !query 67
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 67 schema
+struct
+-- !query 67 output
+1.0
+2.0
+
+
+-- !query 68
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 68 schema
+struct
+-- !query 68 output
+1.0
+2
+
+
+-- !query 69
+SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 69 schema
+struct<>
+-- !query 69 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> double at the first column of the second table;
+
+
+-- !query 70
+SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 70 schema
+struct<>
+-- !query 70 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> double at the first column of the second table;
+
+
+-- !query 71
+SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 71 schema
+struct<>
+-- !query 71 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> double at the first column of the second table;
+
+
+-- !query 72
+SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 72 schema
+struct<>
+-- !query 72 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> double at the first column of the second table;
+
+
+-- !query 73
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 73 schema
+struct
+-- !query 73 output
+1
+2
+
+
+-- !query 74
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 74 schema
+struct
+-- !query 74 output
+1
+2
+
+
+-- !query 75
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 75 schema
+struct
+-- !query 75 output
+1
+2
+
+
+-- !query 76
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 76 schema
+struct
+-- !query 76 output
+1
+2
+
+
+-- !query 77
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 77 schema
+struct
+-- !query 77 output
+1.0
+2.0
+
+
+-- !query 78
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 78 schema
+struct
+-- !query 78 output
+1.0
+2.0
+
+
+-- !query 79
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 79 schema
+struct
+-- !query 79 output
+1
+2
+
+
+-- !query 80
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 80 schema
+struct
+-- !query 80 output
+1
+2
+
+
+-- !query 81
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 81 schema
+struct<>
+-- !query 81 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> decimal(10,0) at the first column of the second table;
+
+
+-- !query 82
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 82 schema
+struct<>
+-- !query 82 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> decimal(10,0) at the first column of the second table;
+
+
+-- !query 83
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 83 schema
+struct<>
+-- !query 83 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> decimal(10,0) at the first column of the second table;
+
+
+-- !query 84
+SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 84 schema
+struct<>
+-- !query 84 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> decimal(10,0) at the first column of the second table;
+
+
+-- !query 85
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 85 schema
+struct
+-- !query 85 output
+1
+2
+
+
+-- !query 86
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 86 schema
+struct
+-- !query 86 output
+1
+2
+
+
+-- !query 87
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 87 schema
+struct
+-- !query 87 output
+1
+2
+
+
+-- !query 88
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 88 schema
+struct
+-- !query 88 output
+1
+2
+
+
+-- !query 89
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 89 schema
+struct
+-- !query 89 output
+1
+2.0
+
+
+-- !query 90
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 90 schema
+struct
+-- !query 90 output
+1
+2.0
+
+
+-- !query 91
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 91 schema
+struct
+-- !query 91 output
+1
+2
+
+
+-- !query 92
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 92 schema
+struct
+-- !query 92 output
+1
+2
+
+
+-- !query 93
+SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 93 schema
+struct<>
+-- !query 93 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> string at the first column of the second table;
+
+
+-- !query 94
+SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 94 schema
+struct<>
+-- !query 94 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> string at the first column of the second table;
+
+
+-- !query 95
+SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 95 schema
+struct
+-- !query 95 output
+1
+2017-12-11 09:30:00
+
+
+-- !query 96
+SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 96 schema
+struct
+-- !query 96 output
+1
+2017-12-11
+
+
+-- !query 97
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 97 schema
+struct<>
+-- !query 97 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. tinyint <> binary at the first column of the second table;
+
+
+-- !query 98
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 98 schema
+struct<>
+-- !query 98 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. smallint <> binary at the first column of the second table;
+
+
+-- !query 99
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 99 schema
+struct<>
+-- !query 99 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. int <> binary at the first column of the second table;
+
+
+-- !query 100
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 100 schema
+struct<>
+-- !query 100 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. bigint <> binary at the first column of the second table;
+
+
+-- !query 101
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 101 schema
+struct<>
+-- !query 101 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. float <> binary at the first column of the second table;
+
+
+-- !query 102
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 102 schema
+struct<>
+-- !query 102 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. double <> binary at the first column of the second table;
+
+
+-- !query 103
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 103 schema
+struct<>
+-- !query 103 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. decimal(10,0) <> binary at the first column of the second table;
+
+
+-- !query 104
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 104 schema
+struct<>
+-- !query 104 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. string <> binary at the first column of the second table;
+
+
+-- !query 105
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 105 schema
+struct
+-- !query 105 output
+1
+2
+
+
+-- !query 106
+SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 106 schema
+struct<>
+-- !query 106 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> binary at the first column of the second table;
+
+
+-- !query 107
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 107 schema
+struct<>
+-- !query 107 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> binary at the first column of the second table;
+
+
+-- !query 108
+SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 108 schema
+struct<>
+-- !query 108 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> binary at the first column of the second table;
+
+
+-- !query 109
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 109 schema
+struct<>
+-- !query 109 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. tinyint <> boolean at the first column of the second table;
+
+
+-- !query 110
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 110 schema
+struct<>
+-- !query 110 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. smallint <> boolean at the first column of the second table;
+
+
+-- !query 111
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 111 schema
+struct<>
+-- !query 111 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. int <> boolean at the first column of the second table;
+
+
+-- !query 112
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 112 schema
+struct<>
+-- !query 112 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. bigint <> boolean at the first column of the second table;
+
+
+-- !query 113
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 113 schema
+struct<>
+-- !query 113 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. float <> boolean at the first column of the second table;
+
+
+-- !query 114
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 114 schema
+struct<>
+-- !query 114 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. double <> boolean at the first column of the second table;
+
+
+-- !query 115
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 115 schema
+struct<>
+-- !query 115 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. decimal(10,0) <> boolean at the first column of the second table;
+
+
+-- !query 116
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 116 schema
+struct<>
+-- !query 116 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. string <> boolean at the first column of the second table;
+
+
+-- !query 117
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 117 schema
+struct<>
+-- !query 117 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> boolean at the first column of the second table;
+
+
+-- !query 118
+SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 118 schema
+struct
+-- !query 118 output
+true
+
+
+-- !query 119
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 119 schema
+struct<>
+-- !query 119 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. timestamp <> boolean at the first column of the second table;
+
+
+-- !query 120
+SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 120 schema
+struct<>
+-- !query 120 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. date <> boolean at the first column of the second table;
+
+
+-- !query 121
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 121 schema
+struct<>
+-- !query 121 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. tinyint <> timestamp at the first column of the second table;
+
+
+-- !query 122
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 122 schema
+struct<>
+-- !query 122 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. smallint <> timestamp at the first column of the second table;
+
+
+-- !query 123
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 123 schema
+struct<>
+-- !query 123 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. int <> timestamp at the first column of the second table;
+
+
+-- !query 124
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 124 schema
+struct<>
+-- !query 124 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. bigint <> timestamp at the first column of the second table;
+
+
+-- !query 125
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 125 schema
+struct<>
+-- !query 125 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. float <> timestamp at the first column of the second table;
+
+
+-- !query 126
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 126 schema
+struct<>
+-- !query 126 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. double <> timestamp at the first column of the second table;
+
+
+-- !query 127
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 127 schema
+struct<>
+-- !query 127 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. decimal(10,0) <> timestamp at the first column of the second table;
+
+
+-- !query 128
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 128 schema
+struct
+-- !query 128 output
+2
+2017-12-12 09:30:00
+
+
+-- !query 129
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 129 schema
+struct<>
+-- !query 129 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> timestamp at the first column of the second table;
+
+
+-- !query 130
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 130 schema
+struct<>
+-- !query 130 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> timestamp at the first column of the second table;
+
+
+-- !query 131
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 131 schema
+struct
+-- !query 131 output
+2017-12-11 09:30:00
+2017-12-12 09:30:00
+
+
+-- !query 132
+SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 132 schema
+struct
+-- !query 132 output
+2017-12-11 00:00:00
+2017-12-12 09:30:00
+
+
+-- !query 133
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t
+-- !query 133 schema
+struct<>
+-- !query 133 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. tinyint <> date at the first column of the second table;
+
+
+-- !query 134
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t
+-- !query 134 schema
+struct<>
+-- !query 134 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. smallint <> date at the first column of the second table;
+
+
+-- !query 135
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t
+-- !query 135 schema
+struct<>
+-- !query 135 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. int <> date at the first column of the second table;
+
+
+-- !query 136
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t
+-- !query 136 schema
+struct<>
+-- !query 136 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. bigint <> date at the first column of the second table;
+
+
+-- !query 137
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t
+-- !query 137 schema
+struct<>
+-- !query 137 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. float <> date at the first column of the second table;
+
+
+-- !query 138
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t
+-- !query 138 schema
+struct<>
+-- !query 138 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. double <> date at the first column of the second table;
+
+
+-- !query 139
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t
+-- !query 139 schema
+struct<>
+-- !query 139 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. decimal(10,0) <> date at the first column of the second table;
+
+
+-- !query 140
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t
+-- !query 140 schema
+struct
+-- !query 140 output
+2
+2017-12-12
+
+
+-- !query 141
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t
+-- !query 141 schema
+struct<>
+-- !query 141 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. binary <> date at the first column of the second table;
+
+
+-- !query 142
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t
+-- !query 142 schema
+struct<>
+-- !query 142 output
+org.apache.spark.sql.AnalysisException
+Union can only be performed on tables with the compatible column types. boolean <> date at the first column of the second table;
+
+
+-- !query 143
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t
+-- !query 143 schema
+struct
+-- !query 143 output
+2017-12-11 09:30:00
+2017-12-12 00:00:00
+
+
+-- !query 144
+SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t
+-- !query 144 schema
+struct
+-- !query 144 output
+2017-12-11
+2017-12-12
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index b02db7721aa7f..bd1e7adefc7a9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -1156,67 +1156,82 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
}
Seq(true, false).foreach { eager =>
- def testCheckpointing(testName: String)(f: => Unit): Unit = {
- test(s"Dataset.checkpoint() - $testName (eager = $eager)") {
- withTempDir { dir =>
- val originalCheckpointDir = spark.sparkContext.checkpointDir
-
- try {
- spark.sparkContext.setCheckpointDir(dir.getCanonicalPath)
+ Seq(true, false).foreach { reliable =>
+ def testCheckpointing(testName: String)(f: => Unit): Unit = {
+ test(s"Dataset.checkpoint() - $testName (eager = $eager, reliable = $reliable)") {
+ if (reliable) {
+ withTempDir { dir =>
+ val originalCheckpointDir = spark.sparkContext.checkpointDir
+
+ try {
+ spark.sparkContext.setCheckpointDir(dir.getCanonicalPath)
+ f
+ } finally {
+ // Since the original checkpointDir can be None, we need
+ // to set the variable directly.
+ spark.sparkContext.checkpointDir = originalCheckpointDir
+ }
+ }
+ } else {
+ // Local checkpoints dont require checkpoint_dir
f
- } finally {
- // Since the original checkpointDir can be None, we need
- // to set the variable directly.
- spark.sparkContext.checkpointDir = originalCheckpointDir
}
}
}
- }
- testCheckpointing("basic") {
- val ds = spark.range(10).repartition('id % 2).filter('id > 5).orderBy('id.desc)
- val cp = ds.checkpoint(eager)
+ testCheckpointing("basic") {
+ val ds = spark.range(10).repartition('id % 2).filter('id > 5).orderBy('id.desc)
+ val cp = if (reliable) ds.checkpoint(eager) else ds.localCheckpoint(eager)
- val logicalRDD = cp.logicalPlan match {
- case plan: LogicalRDD => plan
- case _ =>
- val treeString = cp.logicalPlan.treeString(verbose = true)
- fail(s"Expecting a LogicalRDD, but got\n$treeString")
- }
+ val logicalRDD = cp.logicalPlan match {
+ case plan: LogicalRDD => plan
+ case _ =>
+ val treeString = cp.logicalPlan.treeString(verbose = true)
+ fail(s"Expecting a LogicalRDD, but got\n$treeString")
+ }
- val dsPhysicalPlan = ds.queryExecution.executedPlan
- val cpPhysicalPlan = cp.queryExecution.executedPlan
+ val dsPhysicalPlan = ds.queryExecution.executedPlan
+ val cpPhysicalPlan = cp.queryExecution.executedPlan
- assertResult(dsPhysicalPlan.outputPartitioning) { logicalRDD.outputPartitioning }
- assertResult(dsPhysicalPlan.outputOrdering) { logicalRDD.outputOrdering }
+ assertResult(dsPhysicalPlan.outputPartitioning) {
+ logicalRDD.outputPartitioning
+ }
+ assertResult(dsPhysicalPlan.outputOrdering) {
+ logicalRDD.outputOrdering
+ }
- assertResult(dsPhysicalPlan.outputPartitioning) { cpPhysicalPlan.outputPartitioning }
- assertResult(dsPhysicalPlan.outputOrdering) { cpPhysicalPlan.outputOrdering }
+ assertResult(dsPhysicalPlan.outputPartitioning) {
+ cpPhysicalPlan.outputPartitioning
+ }
+ assertResult(dsPhysicalPlan.outputOrdering) {
+ cpPhysicalPlan.outputOrdering
+ }
- // For a lazy checkpoint() call, the first check also materializes the checkpoint.
- checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*)
+ // For a lazy checkpoint() call, the first check also materializes the checkpoint.
+ checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*)
- // Reads back from checkpointed data and check again.
- checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*)
- }
+ // Reads back from checkpointed data and check again.
+ checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*)
+ }
- testCheckpointing("should preserve partitioning information") {
- val ds = spark.range(10).repartition('id % 2)
- val cp = ds.checkpoint(eager)
+ testCheckpointing("should preserve partitioning information") {
+ val ds = spark.range(10).repartition('id % 2)
+ val cp = if (reliable) ds.checkpoint(eager) else ds.localCheckpoint(eager)
- val agg = cp.groupBy('id % 2).agg(count('id))
+ val agg = cp.groupBy('id % 2).agg(count('id))
- agg.queryExecution.executedPlan.collectFirst {
- case ShuffleExchangeExec(_, _: RDDScanExec, _) =>
- case BroadcastExchangeExec(_, _: RDDScanExec) =>
- }.foreach { _ =>
- fail(
- "No Exchange should be inserted above RDDScanExec since the checkpointed Dataset " +
- "preserves partitioning information:\n\n" + agg.queryExecution
- )
- }
+ agg.queryExecution.executedPlan.collectFirst {
+ case ShuffleExchangeExec(_, _: RDDScanExec, _) =>
+ case BroadcastExchangeExec(_, _: RDDScanExec) =>
+ }.foreach { _ =>
+ fail(
+ "No Exchange should be inserted above RDDScanExec since the checkpointed Dataset " +
+ "preserves partitioning information:\n\n" + agg.queryExecution
+ )
+ }
- checkAnswer(agg, ds.groupBy('id % 2).agg(count('id)))
+ checkAnswer(agg, ds.groupBy('id % 2).agg(count('id)))
+ }
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
index 3a8694839bb24..6bbf38516cdf6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
@@ -435,6 +435,52 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
Seq(Row(Date.valueOf("2015-07-01")), Row(Date.valueOf("2014-12-01"))))
}
+ test("function date_trunc") {
+ val df = Seq(
+ (1, Timestamp.valueOf("2015-07-22 10:01:40.523")),
+ (2, Timestamp.valueOf("2014-12-31 05:29:06.876"))).toDF("i", "t")
+
+ checkAnswer(
+ df.select(date_trunc("YY", col("t"))),
+ Seq(Row(Timestamp.valueOf("2015-01-01 00:00:00")),
+ Row(Timestamp.valueOf("2014-01-01 00:00:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('MONTH', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-01 00:00:00")),
+ Row(Timestamp.valueOf("2014-12-01 00:00:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('DAY', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-22 00:00:00")),
+ Row(Timestamp.valueOf("2014-12-31 00:00:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('HOUR', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-22 10:00:00")),
+ Row(Timestamp.valueOf("2014-12-31 05:00:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('MINUTE', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-22 10:01:00")),
+ Row(Timestamp.valueOf("2014-12-31 05:29:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('SECOND', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-22 10:01:40")),
+ Row(Timestamp.valueOf("2014-12-31 05:29:06"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('WEEK', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-20 00:00:00")),
+ Row(Timestamp.valueOf("2014-12-29 00:00:00"))))
+
+ checkAnswer(
+ df.selectExpr("date_trunc('QUARTER', t)"),
+ Seq(Row(Timestamp.valueOf("2015-07-01 00:00:00")),
+ Row(Timestamp.valueOf("2014-10-01 00:00:00"))))
+ }
+
test("from_unixtime") {
val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US)
val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
index 423e1288e8dcb..c8d045a32d73c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
@@ -20,6 +20,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.SparkConf
import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
/**
@@ -52,4 +53,34 @@ class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext {
assert(df.queryExecution.simpleString.contains(replacement))
}
}
+
+ private def isIncluded(queryExecution: QueryExecution, msg: String): Boolean = {
+ queryExecution.toString.contains(msg) ||
+ queryExecution.simpleString.contains(msg) ||
+ queryExecution.stringWithStats.contains(msg)
+ }
+
+ test("explain is redacted using SQLConf") {
+ withTempDir { dir =>
+ val basePath = dir.getCanonicalPath
+ spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString)
+ val df = spark.read.parquet(basePath)
+ val replacement = "*********"
+
+ // Respect SparkConf and replace file:/
+ assert(isIncluded(df.queryExecution, replacement))
+
+ assert(isIncluded(df.queryExecution, "FileScan"))
+ assert(!isIncluded(df.queryExecution, "file:/"))
+
+ withSQLConf(SQLConf.SQL_STRING_REDACTION_PATTERN.key -> "(?i)FileScan") {
+ // Respect SQLConf and replace FileScan
+ assert(isIncluded(df.queryExecution, replacement))
+
+ assert(!isIncluded(df.queryExecution, "FileScan"))
+ assert(isIncluded(df.queryExecution, "file:/"))
+ }
+ }
+ }
+
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
index e662e294228db..ff7c5e58e9863 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._
import org.apache.spark.storage.StorageLevel._
+import org.apache.spark.util.Utils
class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
import testImplicits._
@@ -40,7 +41,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
data.createOrReplaceTempView(s"testData$dataType")
val storageLevel = MEMORY_ONLY
val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
- val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None)
+ val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None,
+ data.logicalPlan.stats)
assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel)
inMemoryRelation.cachedColumnBuffers.collect().head match {
@@ -116,7 +118,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
test("simple columnar query") {
val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan
- val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None)
+ val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None,
+ testData.logicalPlan.stats)
checkAnswer(scan, testData.collect().toSeq)
}
@@ -132,8 +135,10 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
}
test("projection") {
- val plan = spark.sessionState.executePlan(testData.select('value, 'key).logicalPlan).sparkPlan
- val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None)
+ val logicalPlan = testData.select('value, 'key).logicalPlan
+ val plan = spark.sessionState.executePlan(logicalPlan).sparkPlan
+ val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None,
+ logicalPlan.stats)
checkAnswer(scan, testData.collect().map {
case Row(key: Int, value: String) => value -> key
@@ -149,7 +154,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") {
val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan
- val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None)
+ val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None,
+ testData.logicalPlan.stats)
checkAnswer(scan, testData.collect().toSeq)
checkAnswer(scan, testData.collect().toSeq)
@@ -323,7 +329,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
test("SPARK-17549: cached table size should be correctly calculated") {
val data = spark.sparkContext.parallelize(1 to 10, 5).toDF()
val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
- val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None)
+ val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan.stats)
// Materialize the data.
val expectedAnswer = data.collect()
@@ -448,8 +454,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") {
val attribute = AttributeReference("a", IntegerType)()
- val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY,
- LocalTableScanExec(Seq(attribute), Nil), None)
+ val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil)
+ val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None, null)
val tableScanExec = InMemoryTableScanExec(Seq(attribute),
Seq(In(attribute, Nil)), testRelation)
assert(tableScanExec.partitionFilters.isEmpty)
@@ -479,4 +485,43 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
}
}
}
+
+ test("SPARK-22673: InMemoryRelation should utilize existing stats of the plan to be cached") {
+ withSQLConf("spark.sql.cbo.enabled" -> "true") {
+ withTempPath { workDir =>
+ withTable("table1") {
+ val workDirPath = workDir.getAbsolutePath
+ val data = Seq(100, 200, 300, 400).toDF("count")
+ data.write.parquet(workDirPath)
+ val dfFromFile = spark.read.parquet(workDirPath).cache()
+ val inMemoryRelation = dfFromFile.queryExecution.optimizedPlan.collect {
+ case plan: InMemoryRelation => plan
+ }.head
+ // InMemoryRelation's stats is file size before the underlying RDD is materialized
+ assert(inMemoryRelation.computeStats().sizeInBytes === 740)
+
+ // InMemoryRelation's stats is updated after materializing RDD
+ dfFromFile.collect()
+ assert(inMemoryRelation.computeStats().sizeInBytes === 16)
+
+ // test of catalog table
+ val dfFromTable = spark.catalog.createTable("table1", workDirPath).cache()
+ val inMemoryRelation2 = dfFromTable.queryExecution.optimizedPlan.
+ collect { case plan: InMemoryRelation => plan }.head
+
+ // Even CBO enabled, InMemoryRelation's stats keeps as the file size before table's stats
+ // is calculated
+ assert(inMemoryRelation2.computeStats().sizeInBytes === 740)
+
+ // InMemoryRelation's stats should be updated after calculating stats of the table
+ // clear cache to simulate a fresh environment
+ dfFromTable.unpersist(blocking = true)
+ spark.sql("ANALYZE TABLE table1 COMPUTE STATISTICS")
+ val inMemoryRelation3 = spark.read.table("table1").cache().queryExecution.optimizedPlan.
+ collect { case plan: InMemoryRelation => plan }.head
+ assert(inMemoryRelation3.computeStats().sizeInBytes === 48)
+ }
+ }
+ }
+ }
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala
index eba8d55daad58..932950687942c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala
@@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.util.quietly
import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.status.ElementTrackingStore
import org.apache.spark.status.config._
import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator}
import org.apache.spark.util.kvstore.InMemoryStore
@@ -43,7 +44,9 @@ import org.apache.spark.util.kvstore.InMemoryStore
class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils {
import testImplicits._
- override protected def sparkConf = super.sparkConf.set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
+ override protected def sparkConf = {
+ super.sparkConf.set(LIVE_ENTITY_UPDATE_PERIOD, 0L).set(ASYNC_TRACKING_ENABLED, false)
+ }
private def createTestDataFrame: DataFrame = {
Seq(
@@ -107,10 +110,12 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest
private def sqlStoreTest(name: String)
(fn: (SQLAppStatusStore, SparkListenerBus) => Unit): Unit = {
test(name) {
- val store = new InMemoryStore()
+ val conf = sparkConf
+ val store = new ElementTrackingStore(new InMemoryStore(), conf)
val bus = new ReplayListenerBus()
- val listener = new SQLAppStatusListener(sparkConf, store, true)
+ val listener = new SQLAppStatusListener(conf, store, true)
bus.addListener(listener)
+ store.close(false)
val sqlStore = new SQLAppStatusStore(store, Some(listener))
fn(sqlStore, bus)
}
@@ -491,15 +496,15 @@ private case class MyPlan(sc: SparkContext, expectedValue: Long) extends LeafExe
class SQLListenerMemoryLeakSuite extends SparkFunSuite {
- // TODO: this feature is not yet available in SQLAppStatusStore.
- ignore("no memory leak") {
- quietly {
- val conf = new SparkConf()
- .setMaster("local")
- .setAppName("test")
- .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly
- .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly
- withSpark(new SparkContext(conf)) { sc =>
+ test("no memory leak") {
+ val conf = new SparkConf()
+ .setMaster("local")
+ .setAppName("test")
+ .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly
+ .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly
+ .set(ASYNC_TRACKING_ENABLED, false)
+ withSpark(new SparkContext(conf)) { sc =>
+ quietly {
val spark = new SparkSession(sc)
import spark.implicits._
// Run 100 successful executions and 100 failed executions.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
index 08db06b94904b..2a2552211857a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
@@ -305,6 +305,10 @@ class FileStreamSinkSuite extends StreamTest {
testFormat(Some("parquet"))
}
+ test("orc") {
+ testFormat(Some("orc"))
+ }
+
test("text") {
testFormat(Some("text"))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index c5b57bca18313..f4fa7fa7954d6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -87,6 +87,28 @@ abstract class FileStreamSourceTest
}
}
+ case class AddOrcFileData(data: DataFrame, src: File, tmp: File) extends AddFileData {
+ override def addData(source: FileStreamSource): Unit = {
+ AddOrcFileData.writeToFile(data, src, tmp)
+ }
+ }
+
+ object AddOrcFileData {
+ def apply(seq: Seq[String], src: File, tmp: File): AddOrcFileData = {
+ AddOrcFileData(seq.toDS().toDF(), src, tmp)
+ }
+
+ /** Write orc files in a temp dir, and move the individual files to the 'src' dir */
+ def writeToFile(df: DataFrame, src: File, tmp: File): Unit = {
+ val tmpDir = Utils.tempFileWith(new File(tmp, "orc"))
+ df.write.orc(tmpDir.getCanonicalPath)
+ src.mkdirs()
+ tmpDir.listFiles().foreach { f =>
+ f.renameTo(new File(src, s"${f.getName}"))
+ }
+ }
+ }
+
case class AddParquetFileData(data: DataFrame, src: File, tmp: File) extends AddFileData {
override def addData(source: FileStreamSource): Unit = {
AddParquetFileData.writeToFile(data, src, tmp)
@@ -249,6 +271,42 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
}
}
+ // =============== ORC file stream schema tests ================
+
+ test("FileStreamSource schema: orc, existing files, no schema") {
+ withTempDir { src =>
+ Seq("a", "b", "c").toDS().as("userColumn").toDF().write
+ .mode(org.apache.spark.sql.SaveMode.Overwrite)
+ .orc(src.getCanonicalPath)
+
+ // Without schema inference, should throw error
+ withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") {
+ intercept[IllegalArgumentException] {
+ createFileStreamSourceAndGetSchema(
+ format = Some("orc"), path = Some(src.getCanonicalPath), schema = None)
+ }
+ }
+
+ // With schema inference, should infer correct schema
+ withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
+ val schema = createFileStreamSourceAndGetSchema(
+ format = Some("orc"), path = Some(src.getCanonicalPath), schema = None)
+ assert(schema === new StructType().add("value", StringType))
+ }
+ }
+ }
+
+ test("FileStreamSource schema: orc, existing files, schema") {
+ withTempPath { src =>
+ Seq("a", "b", "c").toDS().as("oldUserColumn").toDF()
+ .write.orc(new File(src, "1").getCanonicalPath)
+ val userSchema = new StructType().add("userColumn", StringType)
+ val schema = createFileStreamSourceAndGetSchema(
+ format = Some("orc"), path = Some(src.getCanonicalPath), schema = Some(userSchema))
+ assert(schema === userSchema)
+ }
+ }
+
// =============== Parquet file stream schema tests ================
test("FileStreamSource schema: parquet, existing files, no schema") {
@@ -508,6 +566,59 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
}
}
+ // =============== ORC file stream tests ================
+
+ test("read from orc files") {
+ withTempDirs { case (src, tmp) =>
+ val fileStream = createFileStream("orc", src.getCanonicalPath, Some(valueSchema))
+ val filtered = fileStream.filter($"value" contains "keep")
+
+ testStream(filtered)(
+ AddOrcFileData(Seq("drop1", "keep2", "keep3"), src, tmp),
+ CheckAnswer("keep2", "keep3"),
+ StopStream,
+ AddOrcFileData(Seq("drop4", "keep5", "keep6"), src, tmp),
+ StartStream(),
+ CheckAnswer("keep2", "keep3", "keep5", "keep6"),
+ AddOrcFileData(Seq("drop7", "keep8", "keep9"), src, tmp),
+ CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
+ )
+ }
+ }
+
+ test("read from orc files with changing schema") {
+ withTempDirs { case (src, tmp) =>
+ withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
+
+ // Add a file so that we can infer its schema
+ AddOrcFileData.writeToFile(Seq("value0").toDF("k"), src, tmp)
+
+ val fileStream = createFileStream("orc", src.getCanonicalPath)
+
+ // FileStreamSource should infer the column "k"
+ assert(fileStream.schema === StructType(Seq(StructField("k", StringType))))
+
+ // After creating DF and before starting stream, add data with different schema
+ // Should not affect the inferred schema any more
+ AddOrcFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp)
+
+ testStream(fileStream)(
+ // Should not pick up column v in the file added before start
+ AddOrcFileData(Seq("value2").toDF("k"), src, tmp),
+ CheckAnswer("value0", "value1", "value2"),
+
+ // Should read data in column k, and ignore v
+ AddOrcFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp),
+ CheckAnswer("value0", "value1", "value2", "value3"),
+
+ // Should ignore rows that do not have the necessary k column
+ AddOrcFileData(Seq("value5").toDF("v"), src, tmp),
+ CheckAnswer("value0", "value1", "value2", "value3", null)
+ )
+ }
+ }
+ }
+
// =============== Parquet file stream tests ================
test("read from parquet files") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
index 9e696b2236b68..fa4b2dd6a6c9b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala
@@ -28,7 +28,7 @@ import com.google.common.util.concurrent.UncheckedExecutionException
import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.plans.logical.Range
@@ -418,6 +418,37 @@ class StreamSuite extends StreamTest {
assert(OutputMode.Update === InternalOutputModes.Update)
}
+ override protected def sparkConf: SparkConf = super.sparkConf
+ .set("spark.redaction.string.regex", "file:/[\\w_]+")
+
+ test("explain - redaction") {
+ val replacement = "*********"
+
+ val inputData = MemoryStream[String]
+ val df = inputData.toDS().map(_ + "foo").groupBy("value").agg(count("*"))
+ // Test StreamingQuery.display
+ val q = df.writeStream.queryName("memory_explain").outputMode("complete").format("memory")
+ .start()
+ .asInstanceOf[StreamingQueryWrapper]
+ .streamingQuery
+ try {
+ inputData.addData("abc")
+ q.processAllAvailable()
+
+ val explainWithoutExtended = q.explainInternal(false)
+ assert(explainWithoutExtended.contains(replacement))
+ assert(explainWithoutExtended.contains("StateStoreRestore"))
+ assert(!explainWithoutExtended.contains("file:/"))
+
+ val explainWithExtended = q.explainInternal(true)
+ assert(explainWithExtended.contains(replacement))
+ assert(explainWithExtended.contains("StateStoreRestore"))
+ assert(!explainWithoutExtended.contains("file:/"))
+ } finally {
+ q.stop()
+ }
+ }
+
test("explain") {
val inputData = MemoryStream[String]
val df = inputData.toDS().map(_ + "foo").groupBy("value").agg(count("*"))
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
index 3f2de108f069a..6740d3bb59dc3 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.hive.ql.processors.CommandProcessor;
import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
+import org.apache.hadoop.hive.ql.session.OperationLog;
import org.apache.hive.service.cli.HiveSQLException;
import org.apache.hive.service.cli.OperationType;
import org.apache.hive.service.cli.session.HiveSession;
@@ -67,4 +68,16 @@ protected void setConfOverlay(Map confOverlay) {
this.confOverlay = confOverlay;
}
}
+
+ protected void registerCurrentOperationLog() {
+ if (isOperationLogEnabled) {
+ if (operationLog == null) {
+ LOG.warn("Failed to get current OperationLog object of Operation: " +
+ getHandle().getHandleIdentifier());
+ isOperationLogEnabled = false;
+ return;
+ }
+ OperationLog.setCurrentOperationLog(operationLog);
+ }
+ }
}
diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 5014cedd870b6..fd9108eb53ca9 100644
--- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -274,18 +274,6 @@ private Hive getSessionHive() throws HiveSQLException {
}
}
- private void registerCurrentOperationLog() {
- if (isOperationLogEnabled) {
- if (operationLog == null) {
- LOG.warn("Failed to get current OperationLog object of Operation: " +
- getHandle().getHandleIdentifier());
- isOperationLogEnabled = false;
- return;
- }
- OperationLog.setCurrentOperationLog(operationLog);
- }
- }
-
private void cleanup(OperationState state) throws HiveSQLException {
setState(state);
if (shouldRunAsync()) {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
index f5191fa9132bd..664bc20601eaa 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala
@@ -170,6 +170,7 @@ private[hive] class SparkExecuteStatementOperation(
override def run(): Unit = {
val doAsAction = new PrivilegedExceptionAction[Unit]() {
override def run(): Unit = {
+ registerCurrentOperationLog()
try {
execute()
} catch {