diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE
index e7ed23dea0f9c..be57f007abbf0 100644
--- a/.github/PULL_REQUEST_TEMPLATE
+++ b/.github/PULL_REQUEST_TEMPLATE
@@ -1,10 +1,42 @@
-## What changes were proposed in this pull request?
+
-(Please fill in changes proposed in this fix)
+### What changes were proposed in this pull request?
+
-## How was this patch tested?
-(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
-(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
+### Why are the changes needed?
+
-Please review https://spark.apache.org/contributing.html before opening a pull request.
+
+### Does this PR introduce any user-facing change?
+
+
+
+### How was this patch tested?
+
diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index fed6b826fd25f..ec499e7bf9cbd 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -22,5 +22,6 @@ jobs:
version: ${{ matrix.java }}
- name: Build with Maven
run: |
- export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m"
- ./build/mvn -DskipTests package
+ export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
+ export MAVEN_CLI_OPTS="--no-transfer-progress"
+ ./build/mvn $MAVEN_CLI_OPTS -DskipTests package
diff --git a/LICENSE-binary b/LICENSE-binary
index 0d3dcd78fcbd8..65b1d61374ec7 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -464,10 +464,8 @@ javax.xml.stream:stax-api https://jcp.org/en/jsr/detail?id=173
Common Development and Distribution License (CDDL) 1.1
------------------------------------------------------
-javax.annotation:javax.annotation-api https://jcp.org/en/jsr/detail?id=250
javax.servlet:javax.servlet-api https://javaee.github.io/servlet-spec/
javax.transaction:jta http://www.oracle.com/technetwork/java/index.html
-javax.ws.rs:javax.ws.rs-api https://github.com/jax-rs
javax.xml.bind:jaxb-api https://github.com/javaee/jaxb-v2
org.glassfish.hk2:hk2-api https://github.com/javaee/glassfish
org.glassfish.hk2:hk2-locator (same)
@@ -492,6 +490,12 @@ jakarta.xml.bind:jakarta.xml.bind-api
com.sun.istack:istack-commons-runtime
+Eclipse Public License (EPL) 2.0
+--------------------------------
+
+jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca
+jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api
+
Mozilla Public License (MPL) 1.1
--------------------------------
diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 3d31be809be61..f4780862099d3 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -13,7 +13,7 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"),
License: Apache License (== 2.0)
URL: https://www.apache.org/ https://spark.apache.org/
BugReports: https://spark.apache.org/contributing.html
-SystemRequirements: Java (== 8)
+SystemRequirements: Java (>= 8, < 12)
Depends:
R (>= 3.1),
methods
diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R
index 660f0864403e0..2ff68ab7b9d77 100644
--- a/R/pkg/R/client.R
+++ b/R/pkg/R/client.R
@@ -64,7 +64,9 @@ checkJavaVersion <- function() {
javaBin <- "java"
javaHome <- Sys.getenv("JAVA_HOME")
javaReqs <- utils::packageDescription(utils::packageName(), fields = c("SystemRequirements"))
- sparkJavaVersion <- as.numeric(tail(strsplit(javaReqs, "[(=)]")[[1]], n = 1L))
+ sparkJavaVersions <- strsplit(javaReqs, "[(,)]")[[1]]
+ minJavaVersion <- as.numeric(strsplit(sparkJavaVersions[[2]], ">= ")[[1]][[2]])
+ maxJavaVersion <- as.numeric(strsplit(sparkJavaVersions[[3]], "< ")[[1]][[2]])
if (javaHome != "") {
javaBin <- file.path(javaHome, "bin", javaBin)
}
@@ -91,12 +93,19 @@ checkJavaVersion <- function() {
}, javaVersionOut)
javaVersionStr <- strsplit(javaVersionFilter[[1]], "[\"]")[[1L]][2]
- # javaVersionStr is of the form 1.8.0_92.
- # Extract 8 from it to compare to sparkJavaVersion
- javaVersionNum <- as.integer(strsplit(javaVersionStr, "[.]")[[1L]][2])
- if (javaVersionNum != sparkJavaVersion) {
- stop(paste("Java version", sparkJavaVersion, "is required for this package; found version:",
- javaVersionStr))
+ # javaVersionStr is of the form 1.8.0_92/9.0.x/11.0.x.
+ # We are using 8, 9, 10, 11 for sparkJavaVersion.
+ versions <- strsplit(javaVersionStr, "[.]")[[1L]]
+ if ("1" == versions[1]) {
+ javaVersionNum <- as.integer(versions[2])
+ } else {
+ javaVersionNum <- as.integer(versions[1])
+ }
+ if (javaVersionNum < minJavaVersion || javaVersionNum >= maxJavaVersion) {
+ stop(paste0("Java version, greater than or equal to ", minJavaVersion,
+ " and less than ", maxJavaVersion,
+ ", is required for this package; found version: ",
+ javaVersionStr))
}
return(javaVersionNum)
}
diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R
index 619153645d925..51ae2d2954a9a 100644
--- a/R/pkg/R/context.R
+++ b/R/pkg/R/context.R
@@ -325,7 +325,8 @@ setCheckpointDirSC <- function(sc, dirName) {
#'
#' A directory can be given if the recursive option is set to true.
#' Currently directories are only supported for Hadoop-supported filesystems.
-#' Refer Hadoop-supported filesystems at \url{https://wiki.apache.org/hadoop/HCFS}.
+#' Refer Hadoop-supported filesystems at
+#' \url{https://cwiki.apache.org/confluence/display/HADOOP2/HCFS}.
#'
#' Note: A path can be added only once. Subsequent additions of the same path are ignored.
#'
diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R
index 1f1b187aef567..2da3a022f8941 100644
--- a/R/pkg/tests/fulltests/test_mllib_classification.R
+++ b/R/pkg/tests/fulltests/test_mllib_classification.R
@@ -308,7 +308,7 @@ test_that("spark.mlp", {
expect_equal(summary$layers, c(4, 5, 4, 3))
expect_equal(length(summary$weights), 64)
expect_equal(head(summary$weights, 5), list(-24.28415, 107.8701, 16.86376, 1.103736, 9.244488),
- tolerance = 1e-6)
+ tolerance = 1e-1)
# Test predict method
mlpTestDF <- df
diff --git a/core/pom.xml b/core/pom.xml
index b340c044a4cc4..b190ee213f38b 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -260,6 +260,15 @@
org.glassfish.jersey.containersjersey-container-servlet-core
+
+ org.glassfish.jersey.inject
+ jersey-hk2
+
+
+ org.glassfish.jersey.test-framework.providers
+ jersey-test-framework-provider-simple
+ test
+ io.nettynetty-all
diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
index c393df8f02226..a841508578a98 100644
--- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
@@ -19,6 +19,7 @@ package org.apache.spark
import java.util.{Properties, Timer, TimerTask}
+import scala.concurrent.TimeoutException
import scala.concurrent.duration._
import org.apache.spark.annotation.{Experimental, Since}
@@ -117,12 +118,30 @@ class BarrierTaskContext private[spark] (
timer.schedule(timerTask, 60000, 60000)
try {
- barrierCoordinator.askSync[Unit](
+ val abortableRpcFuture = barrierCoordinator.askAbortable[Unit](
message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId,
barrierEpoch),
// Set a fixed timeout for RPC here, so users shall get a SparkException thrown by
// BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework.
timeout = new RpcTimeout(365.days, "barrierTimeout"))
+
+ // Wait the RPC future to be completed, but every 1 second it will jump out waiting
+ // and check whether current spark task is killed. If killed, then throw
+ // a `TaskKilledException`, otherwise continue wait RPC until it completes.
+ while(!abortableRpcFuture.toFuture.isCompleted) {
+ if (taskContext.isInterrupted()) {
+ val reason = taskContext.getKillReason().get
+ abortableRpcFuture.abort(reason)
+ throw new TaskKilledException(reason)
+ }
+ // wait RPC future for at most 1 second
+ try {
+ ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second)
+ } catch {
+ case _: TimeoutException => Unit // await future time reach 1 second.
+ }
+ }
+
barrierEpoch += 1
logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) finished " +
"global sync successfully, waited for " +
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
index a4817b3cf770d..5d0f1dcc88097 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala
@@ -156,7 +156,7 @@ private[python] object PythonHadoopUtil {
* Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]]
*/
def mapToConf(map: java.util.Map[String, String]): Configuration = {
- val conf = new Configuration()
+ val conf = new Configuration(false)
map.asScala.foreach { case (k, v) => conf.set(k, v) }
conf
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 080dcca035928..4d76ff76e6752 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -335,7 +335,7 @@ private[spark] object PythonRDD extends Logging {
valueConverterClass: String,
confAsMap: java.util.HashMap[String, String],
batchSize: Int): JavaRDD[Array[Byte]] = {
- val conf = PythonHadoopUtil.mapToConf(confAsMap)
+ val conf = getMergedConf(confAsMap, sc.hadoopConfiguration())
val rdd =
newAPIHadoopRDDFromClassNames[K, V, F](sc,
None, inputFormatClass, keyClass, valueClass, conf)
@@ -404,7 +404,7 @@ private[spark] object PythonRDD extends Logging {
valueConverterClass: String,
confAsMap: java.util.HashMap[String, String],
batchSize: Int): JavaRDD[Array[Byte]] = {
- val conf = PythonHadoopUtil.mapToConf(confAsMap)
+ val conf = getMergedConf(confAsMap, sc.hadoopConfiguration())
val rdd =
hadoopRDDFromClassNames[K, V, F](sc,
None, inputFormatClass, keyClass, valueClass, conf)
@@ -620,7 +620,7 @@ private[spark] object PythonRDD extends Logging {
keyConverterClass: String,
valueConverterClass: String,
useNewAPI: Boolean): Unit = {
- val conf = PythonHadoopUtil.mapToConf(confAsMap)
+ val conf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration)
val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized),
keyConverterClass, valueConverterClass, new JavaToWritableConverter)
if (useNewAPI) {
diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
index 854093851f5d0..0746e43babf9a 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala
@@ -42,7 +42,7 @@ import org.apache.spark.util.Utils
* 3. When all necessary tasks completed successfully, the driver calls commitJob. If the job
* failed to execute (e.g. too many failed tasks), the job should call abortJob.
*/
-abstract class FileCommitProtocol {
+abstract class FileCommitProtocol extends Logging {
import FileCommitProtocol._
/**
@@ -129,7 +129,9 @@ abstract class FileCommitProtocol {
* before the job has finished. These same task commit messages will be passed to commitJob()
* if the entire job succeeds.
*/
- def onTaskCommit(taskCommit: TaskCommitMessage): Unit = {}
+ def onTaskCommit(taskCommit: TaskCommitMessage): Unit = {
+ logDebug(s"onTaskCommit($taskCommit)")
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index 7477e03bfaa76..11ce608f52ee2 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ -17,6 +17,7 @@
package org.apache.spark.internal.io
+import java.io.IOException
import java.util.{Date, UUID}
import scala.collection.mutable
@@ -136,7 +137,7 @@ class HadoopMapReduceCommitProtocol(
tmpOutputPath
}
- private def getFilename(taskContext: TaskAttemptContext, ext: String): String = {
+ protected def getFilename(taskContext: TaskAttemptContext, ext: String): String = {
// The file name looks like part-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003-c000.parquet
// Note that %05d does not truncate the split number, so if we have more than 100000 tasks,
// the file name is fine and won't overflow.
@@ -205,11 +206,28 @@ class HadoopMapReduceCommitProtocol(
}
}
+ /**
+ * Abort the job; log and ignore any IO exception thrown.
+ * This is invariably invoked in an exception handler; raising
+ * an exception here will lose the root cause of the failure.
+ *
+ * @param jobContext job context
+ */
override def abortJob(jobContext: JobContext): Unit = {
- committer.abortJob(jobContext, JobStatus.State.FAILED)
- if (hasValidPath) {
- val fs = stagingDir.getFileSystem(jobContext.getConfiguration)
- fs.delete(stagingDir, true)
+ try {
+ committer.abortJob(jobContext, JobStatus.State.FAILED)
+ } catch {
+ case e: IOException =>
+ logWarning(s"Exception while aborting ${jobContext.getJobID}", e)
+ }
+ try {
+ if (hasValidPath) {
+ val fs = stagingDir.getFileSystem(jobContext.getConfiguration)
+ fs.delete(stagingDir, true)
+ }
+ } catch {
+ case e: IOException =>
+ logWarning(s"Exception while aborting ${jobContext.getJobID}", e)
}
}
@@ -222,17 +240,35 @@ class HadoopMapReduceCommitProtocol(
override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = {
val attemptId = taskContext.getTaskAttemptID
+ logTrace(s"Commit task ${attemptId}")
SparkHadoopMapRedUtil.commitTask(
committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId)
new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet)
}
+ /**
+ * Abort the task; log and ignore any failure thrown.
+ * This is invariably invoked in an exception handler; raising
+ * an exception here will lose the root cause of the failure.
+ *
+ * @param taskContext context
+ */
override def abortTask(taskContext: TaskAttemptContext): Unit = {
- committer.abortTask(taskContext)
+ try {
+ committer.abortTask(taskContext)
+ } catch {
+ case e: IOException =>
+ logWarning(s"Exception while aborting ${taskContext.getTaskAttemptID}", e)
+ }
// best effort cleanup of other staged files
- for ((src, _) <- addedAbsPathFiles) {
- val tmp = new Path(src)
- tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false)
+ try {
+ for ((src, _) <- addedAbsPathFiles) {
+ val tmp = new Path(src)
+ tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false)
+ }
+ } catch {
+ case e: IOException =>
+ logWarning(s"Exception while aborting ${taskContext.getTaskAttemptID}", e)
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
index b12cd4254f19e..1d27fe7db193f 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
@@ -117,8 +117,8 @@ private[spark] class NettyBlockTransferService(
try {
val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter {
override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) {
- val client = clientFactory.createClient(host, port)
try {
+ val client = clientFactory.createClient(host, port)
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener,
transportConf, tempFileManager).start()
} catch {
diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala
index 6c4c0383b3c7f..49d58929a97a4 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala
@@ -46,6 +46,17 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf)
*/
def send(message: Any): Unit
+ /**
+ * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a
+ * [[AbortableRpcFuture]] to receive the reply within the specified timeout.
+ * The [[AbortableRpcFuture]] instance wraps [[Future]] with additional `abort` method.
+ *
+ * This method only sends the message once and never retries.
+ */
+ def askAbortable[T: ClassTag](message: Any, timeout: RpcTimeout): AbortableRpcFuture[T] = {
+ throw new UnsupportedOperationException()
+ }
+
/**
* Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to
* receive the reply within the specified timeout.
@@ -93,3 +104,21 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf)
}
}
+
+/**
+ * An exception thrown if the RPC is aborted.
+ */
+class RpcAbortException(message: String) extends Exception(message)
+
+/**
+ * A wrapper for [[Future]] but add abort method.
+ * This is used in long run RPC and provide an approach to abort the RPC.
+ */
+private[spark] class AbortableRpcFuture[T: ClassTag](
+ future: Future[T],
+ onAbort: String => Unit) {
+
+ def abort(reason: String): Unit = onAbort(reason)
+
+ def toFuture: Future[T] = future
+}
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
index 5dce43b7523d9..265e158d7c5e3 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
@@ -204,7 +204,8 @@ private[netty] class NettyRpcEnv(
clientFactory.createClient(address.host, address.port)
}
- private[netty] def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = {
+ private[netty] def askAbortable[T: ClassTag](
+ message: RequestMessage, timeout: RpcTimeout): AbortableRpcFuture[T] = {
val promise = Promise[Any]()
val remoteAddr = message.receiver.address
@@ -225,6 +226,10 @@ private[netty] class NettyRpcEnv(
}
}
+ def onAbort(reason: String): Unit = {
+ onFailure(new RpcAbortException(reason))
+ }
+
try {
if (remoteAddr == address) {
val p = Promise[Any]()
@@ -240,6 +245,7 @@ private[netty] class NettyRpcEnv(
postToOutbox(message.receiver, rpcMessage)
promise.future.failed.foreach {
case _: TimeoutException => rpcMessage.onTimeout()
+ case _: RpcAbortException => rpcMessage.onAbort()
case _ =>
}(ThreadUtils.sameThread)
}
@@ -257,7 +263,14 @@ private[netty] class NettyRpcEnv(
case NonFatal(e) =>
onFailure(e)
}
- promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread)
+
+ new AbortableRpcFuture[T](
+ promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread),
+ onAbort)
+ }
+
+ private[netty] def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = {
+ askAbortable(message, timeout).toFuture
}
private[netty] def serialize(content: Any): ByteBuffer = {
@@ -528,8 +541,13 @@ private[netty] class NettyRpcEndpointRef(
override def name: String = endpointAddress.name
+ override def askAbortable[T: ClassTag](
+ message: Any, timeout: RpcTimeout): AbortableRpcFuture[T] = {
+ nettyEnv.askAbortable(new RequestMessage(nettyEnv.address, this, message), timeout)
+ }
+
override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = {
- nettyEnv.ask(new RequestMessage(nettyEnv.address, this, message), timeout)
+ askAbortable(message, timeout).toFuture
}
override def send(message: Any): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala
index 3db63934813af..205e6e966866f 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala
@@ -66,14 +66,22 @@ private[netty] case class RpcOutboxMessage(
this.requestId = client.sendRpc(content, this)
}
- def onTimeout(): Unit = {
+ private[netty] def removeRpcRequest(): Unit = {
if (client != null) {
client.removeRpcRequest(requestId)
} else {
- logError("Ask timeout before connecting successfully")
+ logError("Ask terminated before connecting successfully")
}
}
+ def onTimeout(): Unit = {
+ removeRpcRequest()
+ }
+
+ def onAbort(): Unit = {
+ removeRpcRequest()
+ }
+
override def onFailure(e: Throwable): Unit = {
_onFailure(e)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 482691c94f87e..c03e3e0bbaf59 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1571,13 +1571,13 @@ private[spark] class DAGScheduler(
// guaranteed to be determinate, so the input data of the reducers will not change
// even if the map tasks are re-tried.
if (mapStage.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) {
- // It's a little tricky to find all the succeeding stages of `failedStage`, because
+ // It's a little tricky to find all the succeeding stages of `mapStage`, because
// each stage only know its parents not children. Here we traverse the stages from
// the leaf nodes (the result stages of active jobs), and rollback all the stages
- // in the stage chains that connect to the `failedStage`. To speed up the stage
+ // in the stage chains that connect to the `mapStage`. To speed up the stage
// traversing, we collect the stages to rollback first. If a stage needs to
// rollback, all its succeeding stages need to rollback to.
- val stagesToRollback = HashSet(failedStage)
+ val stagesToRollback = HashSet[Stage](mapStage)
def collectStagesToRollback(stageChain: List[Stage]): Unit = {
if (stagesToRollback.contains(stageChain.head)) {
diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
index 04b0b4c37df9e..8df331251c749 100644
--- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
@@ -29,6 +29,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration}
import scala.util.control.NonFatal
import org.apache.spark.SparkException
+import org.apache.spark.rpc.RpcAbortException
private[spark] object ThreadUtils {
@@ -220,8 +221,10 @@ private[spark] object ThreadUtils {
} catch {
case e: SparkFatalException =>
throw e.throwable
- // TimeoutException is thrown in the current thread, so not need to warp the exception.
- case NonFatal(t) if !t.isInstanceOf[TimeoutException] =>
+ // TimeoutException and RpcAbortException is thrown in the current thread, so not need to warp
+ // the exception.
+ case NonFatal(t)
+ if !t.isInstanceOf[TimeoutException] && !t.isInstanceOf[RpcAbortException] =>
throw new SparkException("Exception thrown in awaitResult: ", t)
}
}
diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
index e2ec50fb1f172..aae5fb002e1e8 100644
--- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala
@@ -17,16 +17,42 @@
package org.apache.spark.api.python
-import java.io.{ByteArrayOutputStream, DataOutputStream}
+import java.io.{ByteArrayOutputStream, DataOutputStream, File}
import java.net.{InetAddress, Socket}
import java.nio.charset.StandardCharsets
+import java.util
+import scala.annotation.tailrec
+import scala.collection.JavaConverters._
import scala.concurrent.duration.Duration
-import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.{LongWritable, Text}
+import org.apache.hadoop.mapred.TextInputFormat
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.rdd.{HadoopRDD, RDD}
import org.apache.spark.security.{SocketAuthHelper, SocketAuthServer}
+import org.apache.spark.util.Utils
+
+class PythonRDDSuite extends SparkFunSuite with LocalSparkContext {
-class PythonRDDSuite extends SparkFunSuite {
+ var tempDir: File = _
+
+ override def beforeAll(): Unit = {
+ super.beforeAll()
+ tempDir = Utils.createTempDir()
+ }
+
+ override def afterAll(): Unit = {
+ try {
+ Utils.deleteRecursively(tempDir)
+ } finally {
+ super.afterAll()
+ }
+ }
test("Writing large strings to the worker") {
val input: List[String] = List("a"*100000)
@@ -65,4 +91,59 @@ class PythonRDDSuite extends SparkFunSuite {
throw new Exception("exception within handleConnection")
}
}
+
+ test("mapToConf should not load defaults") {
+ val map = Map("key" -> "value")
+ val conf = PythonHadoopUtil.mapToConf(map.asJava)
+ assert(conf.size() === map.size)
+ assert(conf.get("key") === map("key"))
+ }
+
+ test("SparkContext's hadoop configuration should be respected in PythonRDD") {
+ // hadoop conf with default configurations
+ val hadoopConf = new Configuration()
+ assert(hadoopConf.size() > 0)
+ val headEntry = hadoopConf.asScala.head
+ val (firstKey, firstValue) = (headEntry.getKey, headEntry.getValue)
+
+ // passed to spark conf with a different value(prefixed by spark.)
+ val conf = new SparkConf().setAppName("test").setMaster("local")
+ conf.set("spark.hadoop." + firstKey, "spark." + firstValue)
+
+ sc = new SparkContext(conf)
+ val outDir = new File(tempDir, "output").getAbsolutePath
+ // write output as HadoopRDD's input
+ sc.makeRDD(1 to 1000, 10).saveAsTextFile(outDir)
+
+ val javaSparkContext = new JavaSparkContext(sc)
+ val confMap = new util.HashMap[String, String]()
+ // set input path in job conf
+ confMap.put(FileInputFormat.INPUT_DIR, outDir)
+
+ val pythonRDD = PythonRDD.hadoopRDD(
+ javaSparkContext,
+ classOf[TextInputFormat].getCanonicalName,
+ classOf[LongWritable].getCanonicalName,
+ classOf[Text].getCanonicalName,
+ null,
+ null,
+ confMap,
+ 0
+ )
+
+ @tailrec
+ def getRootRDD(rdd: RDD[_]): RDD[_] = {
+ rdd.dependencies match {
+ case Nil => rdd
+ case dependency :: _ => getRootRDD(dependency.rdd)
+ }
+ }
+
+ // retrieve hadoopRDD as it's a root RDD
+ val hadoopRDD = getRootRDD(pythonRDD).asInstanceOf[HadoopRDD[_, _]]
+ val jobConf = hadoopRDD.getConf
+ // the jobConf passed to HadoopRDD should contain SparkContext's hadoop items rather the default
+ // configs in client's Configuration
+ assert(jobConf.get(firstKey) === "spark." + firstValue)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
index 99b4e8fe8280c..5bdf71be35b3b 100644
--- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
@@ -191,6 +191,48 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
}
}
+ test("ask a message abort") {
+ env.setupEndpoint("ask-abort", new RpcEndpoint {
+ override val rpcEnv = env
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ case msg: String =>
+ Thread.sleep(10000)
+ context.reply(msg)
+ }
+ })
+
+ val conf = new SparkConf()
+ val shortProp = "spark.rpc.short.timeout"
+ conf.set(Network.RPC_RETRY_WAIT, 0L)
+ conf.set(Network.RPC_NUM_RETRIES, 1)
+ val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true)
+ // Use anotherEnv to find out the RpcEndpointRef
+ val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-abort")
+ try {
+ val e = intercept[RpcAbortException] {
+ val timeout = new RpcTimeout(10.seconds, shortProp)
+ val abortableRpcFuture = rpcEndpointRef.askAbortable[String](
+ "hello", timeout)
+
+ new Thread {
+ override def run: Unit = {
+ Thread.sleep(100)
+ abortableRpcFuture.abort("TestAbort")
+ }
+ }.start()
+
+ timeout.awaitResult(abortableRpcFuture.toFuture)
+ }
+ // The SparkException cause should be a RpcAbortException with "TestAbort" message
+ assert(e.isInstanceOf[RpcAbortException])
+ assert(e.getMessage.contains("TestAbort"))
+ } finally {
+ anotherEnv.shutdown()
+ anotherEnv.awaitTermination()
+ }
+ }
+
test("onStart and onStop") {
val stopLatch = new CountDownLatch(1)
val calledMethods = mutable.ArrayBuffer[String]()
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
index 112fd31a060e6..101d83314855b 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.scheduler
+import java.io.File
+
import scala.util.Random
import org.apache.spark._
@@ -153,4 +155,53 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext {
assert(error.contains("The coordinator didn't get all barrier sync requests"))
assert(error.contains("within 1 second(s)"))
}
+
+ test("barrier task killed") {
+ val conf = new SparkConf()
+ .set("spark.barrier.sync.timeout", "1")
+ .set(TEST_NO_STAGE_RETRY, true)
+ .setMaster("local-cluster[4, 1, 1024]")
+ .setAppName("test-cluster")
+ sc = new SparkContext(conf)
+
+ withTempDir { dir =>
+ val killedFlagFile = "barrier.task.killed"
+ val rdd = sc.makeRDD(Seq(0, 1), 2)
+ val rdd2 = rdd.barrier().mapPartitions { it =>
+ val context = BarrierTaskContext.get()
+ if (context.partitionId() == 0) {
+ try {
+ context.barrier()
+ } catch {
+ case _: TaskKilledException =>
+ new File(dir, killedFlagFile).createNewFile()
+ }
+ } else {
+ Thread.sleep(5000)
+ context.barrier()
+ }
+ it
+ }
+
+ val listener = new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
+ new Thread {
+ override def run: Unit = {
+ Thread.sleep(1000)
+ sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = false)
+ }
+ }.start()
+ }
+ }
+ sc.addSparkListener(listener)
+
+ intercept[SparkException] {
+ rdd2.collect()
+ }
+
+ sc.removeSparkListener(listener)
+
+ assert(new File(dir, killedFlagFile).exists(), "Expect barrier task being killed.")
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index cff3ebf2fb7e0..2b3423f9a4d40 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -2741,27 +2741,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"),
null))
- val failedStages = scheduler.failedStages.toSeq
- assert(failedStages.length == 2)
- // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry.
- assert(failedStages.collect {
- case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage
- }.head.findMissingPartitions() == Seq(0))
- // The result stage is still waiting for its 2 tasks to complete
- assert(failedStages.collect {
- case stage: ResultStage => stage
- }.head.findMissingPartitions() == Seq(0, 1))
-
- scheduler.resubmitFailedStages()
-
- // The first task of the `shuffleMapRdd2` failed with fetch failure
- runEvent(makeCompletionEvent(
- taskSets(3).tasks(0),
- FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"),
- null))
-
- // The job should fail because Spark can't rollback the shuffle map stage.
- assert(failure != null && failure.getMessage.contains("Spark cannot rollback"))
+ // The second shuffle map stage need to rerun, the job will abort for the indeterminate
+ // stage rerun.
+ // TODO: After we support re-generate shuffle file(SPARK-25341), this test will be extended.
+ assert(failure != null && failure.getMessage
+ .contains("Spark cannot rollback the ShuffleMapStage 1"))
}
private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = {
@@ -2872,6 +2856,33 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
assert(latch.await(10, TimeUnit.SECONDS))
}
+ test("SPARK-28699: abort stage if parent stage is indeterminate stage") {
+ val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true)
+
+ val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2))
+ val shuffleId = shuffleDep.shuffleId
+ val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker)
+
+ submit(finalRdd, Array(0, 1))
+
+ // Finish the first shuffle map stage.
+ complete(taskSets(0), Seq(
+ (Success, makeMapStatus("hostA", 2)),
+ (Success, makeMapStatus("hostB", 2))))
+ assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty))
+
+ runEvent(makeCompletionEvent(
+ taskSets(1).tasks(0),
+ FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
+ null))
+
+ // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd` needs to retry.
+ // The result stage is still waiting for its 2 tasks to complete.
+ // Because of shuffleMapRdd is indeterminate, this job will be abort.
+ assert(failure != null && failure.getMessage
+ .contains("Spark cannot rollback the ShuffleMapStage 0"))
+ }
+
test("Completions in zombie tasksets update status of non-zombie taskset") {
val parts = 4
val shuffleMapRdd = new MyRDD(sc, parts, Nil)
diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7
index 82389ea6aa58a..8638139d966d0 100644
--- a/dev/deps/spark-deps-hadoop-2.7
+++ b/dev/deps/spark-deps-hadoop-2.7
@@ -7,7 +7,7 @@ antlr-2.7.7.jar
antlr-runtime-3.4.jar
antlr4-runtime-4.7.1.jar
aopalliance-1.0.jar
-aopalliance-repackaged-2.4.0-b34.jar
+aopalliance-repackaged-2.5.0.jar
apache-log4j-extras-1.2.17.jar
apacheds-i18n-2.0.0-M15.jar
apacheds-kerberos-codec-2.0.0-M15.jar
@@ -30,7 +30,7 @@ commons-beanutils-1.9.3.jar
commons-cli-1.2.jar
commons-codec-1.10.jar
commons-collections-3.2.2.jar
-commons-compiler-3.0.13.jar
+commons-compiler-3.0.15.jar
commons-compress-1.8.1.jar
commons-configuration-1.6.jar
commons-crypto-1.0.0.jar
@@ -75,9 +75,9 @@ hadoop-yarn-client-2.7.4.jar
hadoop-yarn-common-2.7.4.jar
hadoop-yarn-server-common-2.7.4.jar
hadoop-yarn-server-web-proxy-2.7.4.jar
-hk2-api-2.4.0-b34.jar
-hk2-locator-2.4.0-b34.jar
-hk2-utils-2.4.0-b34.jar
+hk2-api-2.5.0.jar
+hk2-locator-2.5.0.jar
+hk2-utils-2.5.0.jar
hppc-0.7.2.jar
htrace-core-3.1.0-incubating.jar
httpclient-4.5.6.jar
@@ -87,7 +87,7 @@ ivy-2.4.0.jar
jackson-annotations-2.9.9.jar
jackson-core-2.9.9.jar
jackson-core-asl-1.9.13.jar
-jackson-databind-2.9.9.jar
+jackson-databind-2.9.9.3.jar
jackson-dataformat-yaml-2.9.9.jar
jackson-jaxrs-1.9.13.jar
jackson-mapper-asl-1.9.13.jar
@@ -95,26 +95,26 @@ jackson-module-jaxb-annotations-2.9.9.jar
jackson-module-paranamer-2.9.9.jar
jackson-module-scala_2.12-2.9.9.jar
jackson-xc-1.9.13.jar
+jakarta.annotation-api-1.3.4.jar
+jakarta.inject-2.5.0.jar
+jakarta.ws.rs-api-2.1.5.jar
jakarta.xml.bind-api-2.3.2.jar
-janino-3.0.13.jar
-javassist-3.18.1-GA.jar
-javax.annotation-api-1.2.jar
+janino-3.0.15.jar
+javassist-3.22.0-CR2.jar
javax.inject-1.jar
-javax.inject-2.4.0-b34.jar
javax.servlet-api-3.1.0.jar
-javax.ws.rs-api-2.0.1.jar
javolution-5.5.1.jar
jaxb-api-2.2.2.jar
jaxb-runtime-2.3.2.jar
jcl-over-slf4j-1.7.16.jar
jdo-api-3.0.1.jar
-jersey-client-2.22.2.jar
-jersey-common-2.22.2.jar
-jersey-container-servlet-2.22.2.jar
-jersey-container-servlet-core-2.22.2.jar
-jersey-guava-2.22.2.jar
-jersey-media-jaxb-2.22.2.jar
-jersey-server-2.22.2.jar
+jersey-client-2.29.jar
+jersey-common-2.29.jar
+jersey-container-servlet-2.29.jar
+jersey-container-servlet-core-2.29.jar
+jersey-hk2-2.29.jar
+jersey-media-jaxb-2.29.jar
+jersey-server-2.29.jar
jetty-6.1.26.jar
jetty-sslengine-6.1.26.jar
jetty-util-6.1.26.jar
@@ -160,7 +160,7 @@ orc-core-1.5.5-nohive.jar
orc-mapreduce-1.5.5-nohive.jar
orc-shims-1.5.5.jar
oro-2.0.8.jar
-osgi-resource-locator-1.0.1.jar
+osgi-resource-locator-1.0.3.jar
paranamer-2.8.jar
parquet-column-1.10.1.jar
parquet-common-1.10.1.jar
@@ -192,7 +192,7 @@ stream-2.9.6.jar
stringtemplate-3.2.1.jar
super-csv-2.2.0.jar
univocity-parsers-2.7.3.jar
-validation-api-1.1.0.Final.jar
+validation-api-2.0.1.Final.jar
xbean-asm7-shaded-4.14.jar
xercesImpl-2.9.1.jar
xmlenc-0.52.jar
diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2
index a02735d2be893..af93dd167b4d0 100644
--- a/dev/deps/spark-deps-hadoop-3.2
+++ b/dev/deps/spark-deps-hadoop-3.2
@@ -8,7 +8,7 @@ antlr-2.7.7.jar
antlr-runtime-3.4.jar
antlr4-runtime-4.7.1.jar
aopalliance-1.0.jar
-aopalliance-repackaged-2.4.0-b34.jar
+aopalliance-repackaged-2.5.0.jar
apache-log4j-extras-1.2.17.jar
arpack_combined_all-0.1.jar
arrow-format-0.12.0.jar
@@ -28,7 +28,7 @@ commons-beanutils-1.9.3.jar
commons-cli-1.2.jar
commons-codec-1.10.jar
commons-collections-3.2.2.jar
-commons-compiler-3.0.13.jar
+commons-compiler-3.0.15.jar
commons-compress-1.8.1.jar
commons-configuration2-2.1.1.jar
commons-crypto-1.0.0.jar
@@ -76,9 +76,9 @@ hadoop-yarn-registry-3.2.0.jar
hadoop-yarn-server-common-3.2.0.jar
hadoop-yarn-server-web-proxy-3.2.0.jar
hive-storage-api-2.6.0.jar
-hk2-api-2.4.0-b34.jar
-hk2-locator-2.4.0-b34.jar
-hk2-utils-2.4.0-b34.jar
+hk2-api-2.5.0.jar
+hk2-locator-2.5.0.jar
+hk2-utils-2.5.0.jar
hppc-0.7.2.jar
htrace-core4-4.1.0-incubating.jar
httpclient-4.5.6.jar
@@ -88,7 +88,7 @@ ivy-2.4.0.jar
jackson-annotations-2.9.9.jar
jackson-core-2.9.9.jar
jackson-core-asl-1.9.13.jar
-jackson-databind-2.9.9.jar
+jackson-databind-2.9.9.3.jar
jackson-dataformat-yaml-2.9.9.jar
jackson-jaxrs-base-2.9.5.jar
jackson-jaxrs-json-provider-2.9.5.jar
@@ -96,27 +96,27 @@ jackson-mapper-asl-1.9.13.jar
jackson-module-jaxb-annotations-2.9.9.jar
jackson-module-paranamer-2.9.9.jar
jackson-module-scala_2.12-2.9.9.jar
+jakarta.annotation-api-1.3.4.jar
+jakarta.inject-2.5.0.jar
+jakarta.ws.rs-api-2.1.5.jar
jakarta.xml.bind-api-2.3.2.jar
-janino-3.0.13.jar
-javassist-3.18.1-GA.jar
-javax.annotation-api-1.2.jar
+janino-3.0.15.jar
+javassist-3.22.0-CR2.jar
javax.inject-1.jar
-javax.inject-2.4.0-b34.jar
javax.servlet-api-3.1.0.jar
-javax.ws.rs-api-2.0.1.jar
javolution-5.5.1.jar
jaxb-api-2.2.11.jar
jaxb-runtime-2.3.2.jar
jcip-annotations-1.0-1.jar
jcl-over-slf4j-1.7.16.jar
jdo-api-3.0.1.jar
-jersey-client-2.22.2.jar
-jersey-common-2.22.2.jar
-jersey-container-servlet-2.22.2.jar
-jersey-container-servlet-core-2.22.2.jar
-jersey-guava-2.22.2.jar
-jersey-media-jaxb-2.22.2.jar
-jersey-server-2.22.2.jar
+jersey-client-2.29.jar
+jersey-common-2.29.jar
+jersey-container-servlet-2.29.jar
+jersey-container-servlet-core-2.29.jar
+jersey-hk2-2.29.jar
+jersey-media-jaxb-2.29.jar
+jersey-server-2.29.jar
jetty-webapp-9.4.18.v20190429.jar
jetty-xml-9.4.18.v20190429.jar
jline-2.14.6.jar
@@ -179,7 +179,7 @@ orc-core-1.5.5-nohive.jar
orc-mapreduce-1.5.5-nohive.jar
orc-shims-1.5.5.jar
oro-2.0.8.jar
-osgi-resource-locator-1.0.1.jar
+osgi-resource-locator-1.0.3.jar
paranamer-2.8.jar
parquet-column-1.10.1.jar
parquet-common-1.10.1.jar
@@ -212,7 +212,7 @@ stringtemplate-3.2.1.jar
super-csv-2.2.0.jar
token-provider-1.0.1.jar
univocity-parsers-2.7.3.jar
-validation-api-1.1.0.Final.jar
+validation-api-2.0.1.Final.jar
woodstox-core-5.0.3.jar
xbean-asm7-shaded-4.14.jar
xz-1.5.jar
diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml
index 9bbb115bcdda5..717911b5a4645 100644
--- a/docs/_data/menu-sql.yaml
+++ b/docs/_data/menu-sql.yaml
@@ -72,12 +72,164 @@
url: sql-migration-guide-hive-compatibility.html
- text: SQL Reserved/Non-Reserved Keywords
url: sql-reserved-and-non-reserved-keywords.html
-- text: Reference
- url: sql-reference.html
+
+- text: SQL Reference
+ url: sql-ref.html
subitems:
- text: Data Types
- url: sql-reference.html#data-types
+ url: sql-ref-datatypes.html
- text: NaN Semantics
- url: sql-reference.html#nan-semantics
- - text: Arithmetic operations
- url: sql-reference.html#arithmetic-operations
+ url: sql-ref-nan-semantics.html
+ - text: SQL Syntax
+ url: sql-ref-syntax.html
+ subitems:
+ - text: Data Definition Statements
+ url: sql-ref-syntax-ddl.html
+ subitems:
+ - text: ALTER DATABASE
+ url: sql-ref-syntax-ddl-alter-database.html
+ - text: ALTER TABLE
+ url: sql-ref-syntax-ddl-alter-table.html
+ - text: ALTER VIEW
+ url: sql-ref-syntax-ddl-alter-view.html
+ - text: CREATE DATABASE
+ url: sql-ref-syntax-ddl-create-database.html
+ - text: CREATE FUNCTION
+ url: sql-ref-syntax-ddl-create-function.html
+ - text: CREATE TABLE
+ url: sql-ref-syntax-ddl-create-table.html
+ - text: CREATE VIEW
+ url: sql-ref-syntax-ddl-create-view.html
+ - text: DROP DATABASE
+ url: sql-ref-syntax-ddl-drop-database.html
+ - text: DROP FUNCTION
+ url: sql-ref-syntax-ddl-drop-function.html
+ - text: DROP TABLE
+ url: sql-ref-syntax-ddl-drop-table.html
+ - text: DROP VIEW
+ url: sql-ref-syntax-ddl-drop-view.html
+ - text: TRUNCATE TABLE
+ url: sql-ref-syntax-ddl-truncate-table.html
+ - text: REPAIR TABLE
+ url: sql-ref-syntax-ddl-repair-table.html
+ - text: Data Manipulation Statements
+ url: sql-ref-syntax-dml.html
+ subitems:
+ - text: INSERT
+ url: sql-ref-syntax-dml-insert.html
+ - text: LOAD
+ url: sql-ref-syntax-dml-load.html
+ - text: Data Retrieval(Queries)
+ url: sql-ref-syntax-qry.html
+ subitems:
+ - text: SELECT
+ url: sql-ref-syntax-qry-select.html
+ subitems:
+ - text: DISTINCT Clause
+ url: sql-ref-syntax-qry-select-distinct.html
+ - text: Joins
+ url: sql-ref-syntax-qry-select-join.html
+ - text: ORDER BY Clause
+ url: sql-ref-syntax-qry-select-orderby.html
+ - text: GROUP BY Clause
+ url: sql-ref-syntax-qry-select-groupby.html
+ - text: HAVING Clause
+ url: sql-ref-syntax-qry-select-having.html
+ - text: LIMIT Clause
+ url: sql-ref-syntax-qry-select-limit.html
+ - text: Set operations
+ url: sql-ref-syntax-qry-select-setops.html
+ - text: Common Table Expression(CTE)
+ url: sql-ref-syntax-qry-select-cte.html
+ - text: Subqueries
+ url: sql-ref-syntax-qry-select-subqueries.html
+ - text: Query hints
+ url: sql-ref-syntax-qry-select-hints.html
+ - text: SAMPLING
+ url: sql-ref-syntax-qry-sampling.html
+ - text: WINDOWING ANALYTIC FUNCTIONS
+ url: sql-ref-syntax-qry-window.html
+ - text: AGGREGATION (CUBE/ROLLUP/GROUPING)
+ url: sql-ref-syntax-qry-aggregation.html
+ - text: EXPLAIN
+ url: sql-ref-syntax-qry-explain.html
+ - text: Auxilarry Statements
+ url: sql-ref-syntax-aux.html
+ subitems:
+ - text: Analyze statement
+ url: sql-ref-syntax-aux-analyze.html
+ subitems:
+ - text: ANALYZE TABLE
+ url: sql-ref-syntax-aux-analyze-table.html
+ - text: Caching statements
+ url: sql-ref-syntax-aux-cache.html
+ subitems:
+ - text: CACHE TABLE
+ url: sql-ref-syntax-aux-cache-cache-table.html
+ - text: UNCACHE TABLE
+ url: sql-ref-syntax-aux-cache-uncache-table.html
+ - text: CLEAR CACHE
+ url: sql-ref-syntax-aux-cache-clear-cache.html
+ - text: Describe Commands
+ url: sql-ref-syntax-aux-describe.html
+ subitems:
+ - text: DESCRIBE DATABASE
+ url: sql-ref-syntax-aux-describe-database.html
+ - text: DESCRIBE TABLE
+ url: sql-ref-syntax-aux-describe-table.html
+ - text: DESCRIBE FUNCTION
+ url: sql-ref-syntax-aux-describe-function.html
+ - text: DESCRIBE QUERY
+ url: sql-ref-syntax-aux-describe-query.html
+ - text: Show commands
+ url: sql-ref-syntax-aux-show.html
+ subitems:
+ - text: SHOW COLUMNS
+ url: sql-ref-syntax-aux-show-columns.html
+ - text: SHOW DATABASES
+ url: sql-ref-syntax-aux-show-databases.html
+ - text: SHOW FUNCTIONS
+ url: sql-ref-syntax-aux-show-functions.html
+ - text: SHOW TABLE
+ url: sql-ref-syntax-aux-show-table.html
+ - text: SHOW TABLES
+ url: sql-ref-syntax-aux-show-tables.html
+ - text: SHOW TBLPROPERTIES
+ url: sql-ref-syntax-aux-show-tblproperties.html
+ - text: SHOW PARTITIONS
+ url: sql-ref-syntax-aux-show-partitions.html
+ - text: SHOW CREATE TABLE
+ url: sql-ref-syntax-aux-show-create-table.html
+ - text: Configuration Management Commands
+ url: sql-ref-syntax-aux-conf-mgmt.html
+ subitems:
+ - text: SET
+ url: sql-ref-syntax-aux-conf-mgmt-set.html
+ - text: RESET
+ url: sql-ref-syntax-aux-conf-mgmt-reset.html
+ - text: Resource Management Commands
+ url: sql-ref-syntax-aux-resource-mgmt.html
+ subitems:
+ - text: ADD FILE
+ url: sql-ref-syntax-aux-resource-mgmt-add-file.html
+ - text: ADD JAR
+ url: sql-ref-syntax-aux-resource-mgmt-add-jar.html
+ - text: Functions
+ url: sql-ref-functions.html
+ subitems:
+ - text: Builtin Functions
+ url: sql-ref-functions-builtin.html
+ subitems:
+ - text: Scalar functions
+ url: sql-ref-functions-builtin-scalar.html
+ - text: Aggregate functions
+ url: sql-ref-functions-builtin-aggregate.html
+ - text: User defined Functions
+ url: sql-ref-functions-udf.html
+ subitems:
+ - text: Scalar functions
+ url: sql-ref-functions-udf-scalar.html
+ - text: Aggregate functions
+ url: sql-ref-functions-udf-aggregate.html
+ - text: Arthmetic operations
+ url: sql-ref-arithmetic-ops.html
diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md
index b64ffe55d8359..a8d40fe7456e4 100644
--- a/docs/cloud-integration.md
+++ b/docs/cloud-integration.md
@@ -125,7 +125,7 @@ consult the relevant documentation.
### Recommended settings for writing to object stores
For object stores whose consistency model means that rename-based commits are safe
-use the `FileOutputCommitter` v2 algorithm for performance:
+use the `FileOutputCommitter` v2 algorithm for performance; v1 for safety.
```
spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2
@@ -143,8 +143,30 @@ job failure:
spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true
```
+The original v1 commit algorithm renames the output of successful tasks
+to a job attempt directory, and then renames all the files in that directory
+into the final destination during the job commit phase:
+
+```
+spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 1
+```
+
+The slow performance of mimicked renames on Amazon S3 makes this algorithm
+very, very slow. The recommended solution to this is switch to an S3 "Zero Rename"
+committer (see below).
+
+For reference, here are the performance and safety characteristics of
+different stores and connectors when renaming directories:
+
+| Store | Connector | Directory Rename Safety | Rename Performance |
+|---------------|-----------|-------------------------|--------------------|
+| Amazon S3 | s3a | Unsafe | O(data) |
+| Azure Storage | wasb | Safe | O(files) |
+| Azure Datalake Gen 2 | abfs | Safe | O(1) |
+| Google GCS | gs | Safe | O(1) |
+
As storing temporary files can run up charges; delete
-directories called `"_temporary"` on a regular basis to avoid this.
+directories called `"_temporary"` on a regular basis.
### Parquet I/O Settings
@@ -190,15 +212,49 @@ while they are still being written. Applications can write straight to the monit
atomic `rename()` operation.
Otherwise the checkpointing may be slow and potentially unreliable.
+## Committing work into cloud storage safely and fast.
+
+As covered earlier, commit-by-rename is dangerous on any object store which
+exhibits eventual consistency (example: S3), and often slower than classic
+filesystem renames.
+
+Some object store connectors provide custom committers to commit tasks and
+jobs without using rename. In versions of Spark built with Hadoop 3.1 or later,
+the S3A connector for AWS S3 is such a committer.
+
+Instead of writing data to a temporary directory on the store for renaming,
+these committers write the files to the final destination, but do not issue
+the final POST command to make a large "multi-part" upload visible. Those
+operations are postponed until the job commit itself. As a result, task and
+job commit are much faster, and task failures do not affect the result.
+
+To switch to the S3A committers, use a version of Spark was built with Hadoop
+3.1 or later, and switch the committers through the following options.
+
+```
+spark.hadoop.fs.s3a.committer.name directory
+spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
+spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
+```
+
+It has been tested with the most common formats supported by Spark.
+
+```python
+mydataframe.write.format("parquet").save("s3a://bucket/destination")
+```
+
+More details on these committers can be found in the latest Hadoop documentation.
+
## Further Reading
Here is the documentation on the standard connectors both from Apache and the cloud providers.
-* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+
-* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7
-* [Azure Data Lake](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8
-* [Amazon S3 via S3A and S3N](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+
+* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html).
+* [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html).
+* [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html).
+* [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html).
+* [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html).
* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon
* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google
-
+* [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver)
diff --git a/docs/css/main.css b/docs/css/main.css
index 7f1e99bf67224..fdfe4bda852a0 100755
--- a/docs/css/main.css
+++ b/docs/css/main.css
@@ -196,11 +196,14 @@ a.anchorjs-link:hover { text-decoration: none; }
width: 210px;
float: left;
position: fixed;
+ overflow-y: scroll;
+ top: 0;
+ bottom: 0;
}
.left-menu {
padding: 0px;
- width: 199px;
+ width: 399px;
}
.left-menu h3 {
diff --git a/docs/img/AllJobsPageDetail1.png b/docs/img/AllJobsPageDetail1.png
new file mode 100644
index 0000000000000..c76d2fdefb1aa
Binary files /dev/null and b/docs/img/AllJobsPageDetail1.png differ
diff --git a/docs/img/AllJobsPageDetail2.png b/docs/img/AllJobsPageDetail2.png
new file mode 100644
index 0000000000000..b7203b2e66586
Binary files /dev/null and b/docs/img/AllJobsPageDetail2.png differ
diff --git a/docs/img/AllJobsPageDetail3.png b/docs/img/AllJobsPageDetail3.png
new file mode 100644
index 0000000000000..75b7caec119b2
Binary files /dev/null and b/docs/img/AllJobsPageDetail3.png differ
diff --git a/docs/img/JobPageDetail1.png b/docs/img/JobPageDetail1.png
new file mode 100644
index 0000000000000..1ee741d1f09d0
Binary files /dev/null and b/docs/img/JobPageDetail1.png differ
diff --git a/docs/img/JobPageDetail2.png b/docs/img/JobPageDetail2.png
new file mode 100644
index 0000000000000..ab6d7bdf15ec2
Binary files /dev/null and b/docs/img/JobPageDetail2.png differ
diff --git a/docs/img/JobPageDetail3.png b/docs/img/JobPageDetail3.png
new file mode 100644
index 0000000000000..9f691e4ed2b6b
Binary files /dev/null and b/docs/img/JobPageDetail3.png differ
diff --git a/docs/index.md b/docs/index.md
index ddc792521e7e0..4217918a87462 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -46,7 +46,7 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy
locally on one machine --- all you need is to have `java` installed on your system `PATH`,
or the `JAVA_HOME` environment variable pointing to a Java installation.
-Spark runs on Java 8+, Scala 2.12, Python 2.7+/3.4+ and R 3.1+.
+Spark runs on Java 8, Scala 2.12, Python 2.7+/3.4+ and R 3.1+.
Python 2 support is deprecated as of Spark 3.0.0.
R prior to version 3.4 support is deprecated as of Spark 3.0.0.
For the Scala API, Spark {{site.SPARK_VERSION}}
diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md
index fc849d3912b98..13be6d51a7ece 100644
--- a/docs/sql-distributed-sql-engine.md
+++ b/docs/sql-distributed-sql-engine.md
@@ -85,6 +85,8 @@ To test, use beeline to connect to the JDBC/ODBC server in http mode with:
beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path=
+If you closed a session and do CTAS, you must set `fs.%s.impl.disable.cache` to true in `hive-site.xml`.
+See more details in [[SPARK-21067]](https://issues.apache.org/jira/browse/SPARK-21067).
## Running the Spark SQL CLI
diff --git a/docs/sql-ref-arithmetic-ops.md b/docs/sql-ref-arithmetic-ops.md
new file mode 100644
index 0000000000000..7bc8ffe31c990
--- /dev/null
+++ b/docs/sql-ref-arithmetic-ops.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Arithmetic Operations
+displayTitle: Arithmetic Operations
+license: |
+ 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.
+---
+
+Operations performed on numeric types (with the exception of decimal) are not checked for overflow. This means that in case an operation causes an overflow, the result is the same that the same operation returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, the result is a negative number).
diff --git a/docs/sql-reference.md b/docs/sql-ref-datatypes.md
similarity index 94%
rename from docs/sql-reference.md
rename to docs/sql-ref-datatypes.md
index 2ec26ecc2e1e9..0add62b10ed6b 100644
--- a/docs/sql-reference.md
+++ b/docs/sql-ref-datatypes.md
@@ -1,7 +1,7 @@
---
layout: global
-title: Reference
-displayTitle: Reference
+title: Data Types
+displayTitle: Data Types
license: |
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
@@ -19,10 +19,6 @@ license: |
limitations under the License.
---
-* Table of contents
-{:toc}
-
-## Data Types
Spark SQL and DataFrames support the following data types:
@@ -634,25 +630,5 @@ from pyspark.sql.types import *
-
-
-
-## NaN Semantics
-
-There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that
-does not exactly match standard floating point semantics.
-Specifically:
-
- - NaN = NaN returns true.
- - In aggregations, all NaN values are grouped together.
- - NaN is treated as a normal value in join keys.
- - NaN values go last when in ascending order, larger than any other numeric value.
-
-## Arithmetic operations
-
-Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow.
-This means that in case an operation causes an overflow, the result is the same that the same operation
-returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable,
-the result is a negative number).
diff --git a/docs/sql-ref-functions-builtin-aggregate.md b/docs/sql-ref-functions-builtin-aggregate.md
new file mode 100644
index 0000000000000..3fcd78245ca04
--- /dev/null
+++ b/docs/sql-ref-functions-builtin-aggregate.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Builtin Aggregate Functions
+displayTitle: Builtin Aggregate Functions
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-functions-builtin-scalar.md b/docs/sql-ref-functions-builtin-scalar.md
new file mode 100644
index 0000000000000..1d818a25c4ac1
--- /dev/null
+++ b/docs/sql-ref-functions-builtin-scalar.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Builtin Scalar Functions
+displayTitle: Builtin Scalar Functions
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-functions-builtin.md b/docs/sql-ref-functions-builtin.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-functions-builtin.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-functions-udf-aggregate.md b/docs/sql-ref-functions-udf-aggregate.md
new file mode 100644
index 0000000000000..49c7b5824dfd6
--- /dev/null
+++ b/docs/sql-ref-functions-udf-aggregate.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: User defined Aggregate Functions (UDAF)
+displayTitle: User defined Aggregate Functions (UDAF)
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-functions-udf-scalar.md b/docs/sql-ref-functions-udf-scalar.md
new file mode 100644
index 0000000000000..cee135b572aea
--- /dev/null
+++ b/docs/sql-ref-functions-udf-scalar.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: User defined Scalar Functions (UDF)
+displayTitle: User defined Scalar Functions (UDF)
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-functions-udf.md b/docs/sql-ref-functions-udf.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-functions-udf.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-functions.md b/docs/sql-ref-functions.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-functions.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-nan-semantics.md b/docs/sql-ref-nan-semantics.md
new file mode 100644
index 0000000000000..f6a85728263a7
--- /dev/null
+++ b/docs/sql-ref-nan-semantics.md
@@ -0,0 +1,29 @@
+---
+layout: global
+title: Nan Semantics
+displayTitle: NaN Semantics
+license: |
+ 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.
+---
+
+There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that
+does not exactly match standard floating point semantics.
+Specifically:
+
+ - NaN = NaN returns true.
+ - In aggregations, all NaN values are grouped together.
+ - NaN is treated as a normal value in join keys.
+ - NaN values go last when in ascending order, larger than any other numeric value.
diff --git a/docs/sql-ref-syntax-aux-analyze-table.md b/docs/sql-ref-syntax-aux-analyze-table.md
new file mode 100644
index 0000000000000..e23098903e886
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-analyze-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ANALYZE TABLE
+displayTitle: ANALYZE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-analyze.md b/docs/sql-ref-syntax-aux-analyze.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-analyze.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md
new file mode 100644
index 0000000000000..16f26929280ff
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-cache-cache-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CACHE TABLE
+displayTitle: CACHE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md
new file mode 100644
index 0000000000000..88d126f0f528e
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CLEAR CACHE
+displayTitle: CLEAR CACHE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md
new file mode 100644
index 0000000000000..15081ff7dfad1
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: UNCACHE TABLE
+displayTitle: UNCACHE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-cache.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md
new file mode 100644
index 0000000000000..ad2d7f9a83316
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: RESET
+displayTitle: RESET
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set.md b/docs/sql-ref-syntax-aux-conf-mgmt-set.md
new file mode 100644
index 0000000000000..c38d68dbb4f1d
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-conf-mgmt-set.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SET
+displayTitle: SET
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-conf-mgmt.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux-describe-database.md b/docs/sql-ref-syntax-aux-describe-database.md
new file mode 100644
index 0000000000000..5d1c9deb13a7a
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-describe-database.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DESCRIBE DATABASE
+displayTitle: DESCRIBE DATABASE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-describe-function.md b/docs/sql-ref-syntax-aux-describe-function.md
new file mode 100644
index 0000000000000..55e8fc8e1dad3
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-describe-function.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DESCRIBE FUNCTION
+displayTitle: DESCRIBE FUNCTION
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-describe-query.md b/docs/sql-ref-syntax-aux-describe-query.md
new file mode 100644
index 0000000000000..e1c5c54a5bfbb
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-describe-query.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DESCRIBE QUERY
+displayTitle: DESCRIBE QUERY
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md
new file mode 100644
index 0000000000000..110a5e4c78171
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-describe-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DESCRIBE TABLE
+displayTitle: DESCRIBE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-describe.md b/docs/sql-ref-syntax-aux-describe.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-describe.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md
new file mode 100644
index 0000000000000..f57b81ead6f49
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ADD FILE
+displayTitle: ADD FILE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
new file mode 100644
index 0000000000000..cd91119c8c2ba
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ADD JAR
+displayTitle: ADD JAR
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-resource-mgmt.md b/docs/sql-ref-syntax-aux-resource-mgmt.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-resource-mgmt.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux-show-columns.md b/docs/sql-ref-syntax-aux-show-columns.md
new file mode 100644
index 0000000000000..521ec0002c785
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-columns.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW COLUMNS
+displayTitle: SHOW COLUMNS
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-create-table.md b/docs/sql-ref-syntax-aux-show-create-table.md
new file mode 100644
index 0000000000000..2cf40915774c4
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-create-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW CREATE TABLE
+displayTitle: SHOW CREATE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md
new file mode 100644
index 0000000000000..e7aedf8cc79a6
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-databases.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW DATABASE
+displayTitle: SHOW DATABASE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-functions.md b/docs/sql-ref-syntax-aux-show-functions.md
new file mode 100644
index 0000000000000..ae689fdf227e9
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-functions.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW FUNCTIONS
+displayTitle: SHOW FUNCTIONS
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md
new file mode 100644
index 0000000000000..c6499de9cbb9e
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-partitions.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW PARTITIONS
+displayTitle: SHOW PARTITIONS
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-table.md b/docs/sql-ref-syntax-aux-show-table.md
new file mode 100644
index 0000000000000..ad549b6b11ecb
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW TABLE
+displayTitle: SHOW TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md
new file mode 100644
index 0000000000000..e4340d608bf5e
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-tables.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW TABLES
+displayTitle: SHOW TABLES
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md
new file mode 100644
index 0000000000000..586993eeb5cdd
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show-tblproperties.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: SHOW TBLPROPERTIES
+displayTitle: SHOW TBLPROPERTIES
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-aux-show.md b/docs/sql-ref-syntax-aux-show.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux-show.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-aux.md b/docs/sql-ref-syntax-aux.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-aux.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md
new file mode 100644
index 0000000000000..29e7ffb940107
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-alter-database.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ALTER DATABASE
+displayTitle: ALTER DATABASE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md
new file mode 100644
index 0000000000000..7fcd397915825
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-alter-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ALTER TABLE
+displayTitle: ALTER TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md
new file mode 100644
index 0000000000000..7a7947a3e0ac4
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-alter-view.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ALTER VIEW
+displayTitle: ALTER VIEW
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md
new file mode 100644
index 0000000000000..bbcd34a6d6853
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-create-database.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CREATE DATABASE
+displayTitle: CREATE DATABASE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md
new file mode 100644
index 0000000000000..f95a9eba42c2f
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-create-function.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CREATE FUNCTION
+displayTitle: CREATE FUNCTION
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-create-table.md b/docs/sql-ref-syntax-ddl-create-table.md
new file mode 100644
index 0000000000000..4ce8ef697c2c2
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-create-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CREATE TABLE
+displayTitle: CREATE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md
new file mode 100644
index 0000000000000..eff7df91f59c5
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-create-view.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: CREATE VIEW
+displayTitle: CREATE VIEW
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-drop-database.md b/docs/sql-ref-syntax-ddl-drop-database.md
new file mode 100644
index 0000000000000..cd900a7e393db
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-drop-database.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DROP DATABASE
+displayTitle: DROP DATABASE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-drop-function.md b/docs/sql-ref-syntax-ddl-drop-function.md
new file mode 100644
index 0000000000000..9cf1f445e91f4
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-drop-function.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DROP FUNCTION
+displayTitle: DROP FUNCTION
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-drop-table.md b/docs/sql-ref-syntax-ddl-drop-table.md
new file mode 100644
index 0000000000000..a036e66c3906d
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-drop-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DROP TABLE
+displayTitle: DROP TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md
new file mode 100644
index 0000000000000..9ad22500fd9e4
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-drop-view.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: DROP VIEW
+displayTitle: DROP VIEW
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-repair-table.md b/docs/sql-ref-syntax-ddl-repair-table.md
new file mode 100644
index 0000000000000..f21de558b8a07
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-repair-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: REPAIR TABLE
+displayTitle: REPAIR TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md
new file mode 100644
index 0000000000000..2704259391e94
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl-truncate-table.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: TRUNCATE TABLE
+displayTitle: TRUNCATE TABLE
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-ddl.md b/docs/sql-ref-syntax-ddl.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-ddl.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-dml-insert.md b/docs/sql-ref-syntax-dml-insert.md
new file mode 100644
index 0000000000000..200be07a33095
--- /dev/null
+++ b/docs/sql-ref-syntax-dml-insert.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: INSERT
+displayTitle: INSERT
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md
new file mode 100644
index 0000000000000..fd25ba314e0b6
--- /dev/null
+++ b/docs/sql-ref-syntax-dml-load.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: LOAD
+displayTitle: LOAD
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-dml.md b/docs/sql-ref-syntax-dml.md
new file mode 100644
index 0000000000000..7fd537e0a0f5c
--- /dev/null
+++ b/docs/sql-ref-syntax-dml.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Data Manipulation Statements
+displayTitle: Data Manipulation Statements
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-qry-aggregation.md b/docs/sql-ref-syntax-qry-aggregation.md
new file mode 100644
index 0000000000000..fbe6e61c84fd1
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-aggregation.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Aggregation (CUBE/ROLLUP/GROUPING)
+displayTitle: Aggregation (CUBE/ROLLUP/GROUPING)
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-explain.md b/docs/sql-ref-syntax-qry-explain.md
new file mode 100644
index 0000000000000..501f2083f0f8c
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-explain.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: EXPLAIN
+displayTitle: EXPLAIN
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-sampling.md b/docs/sql-ref-syntax-qry-sampling.md
new file mode 100644
index 0000000000000..a5efb361ee847
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-sampling.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Sampling Queries
+displayTitle: Sampling Queries
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-cte.md b/docs/sql-ref-syntax-qry-select-cte.md
new file mode 100644
index 0000000000000..2bd7748291a9f
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-cte.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Common Table Expression (CTE)
+displayTitle: Common Table Expression (CTE)
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-distinct.md b/docs/sql-ref-syntax-qry-select-distinct.md
new file mode 100644
index 0000000000000..2ed7931c202f3
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-distinct.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Distinct operator
+displayTitle: Distinct operator
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md
new file mode 100644
index 0000000000000..8ba7d155f3716
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-groupby.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: GROUPBY Clause
+displayTitle: GROUPBY Clause
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md
new file mode 100644
index 0000000000000..ca92eb0d4daf0
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-having.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: HAVING Clause
+displayTitle: HAVING Clause
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-hints.md b/docs/sql-ref-syntax-qry-select-hints.md
new file mode 100644
index 0000000000000..eb9655fc0d731
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-hints.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Query Hints
+displayTitle: Query Hints
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-join.md b/docs/sql-ref-syntax-qry-select-join.md
new file mode 100644
index 0000000000000..ee66dbdb8c823
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-join.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: JOIN
+displayTitle: JOIN
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md
new file mode 100644
index 0000000000000..d7fac3bb98929
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-limit.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: LIMIT operator
+displayTitle: LIMIT operator
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md
new file mode 100644
index 0000000000000..1f7c031e4aa88
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-orderby.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: ORDER BY Clause
+displayTitle: ORDER BY Clause
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-setops.md b/docs/sql-ref-syntax-qry-select-setops.md
new file mode 100644
index 0000000000000..526f857520aa2
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-setops.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Set Operations
+displayTitle: Set Operations
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select-subqueries.md b/docs/sql-ref-syntax-qry-select-subqueries.md
new file mode 100644
index 0000000000000..976e201f2714b
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select-subqueries.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Subqueries
+displayTitle: Subqueries
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md
new file mode 100644
index 0000000000000..41972ef070831
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-select.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: SELECT
+displayTitle: SELECT
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax-qry-window.md b/docs/sql-ref-syntax-qry-window.md
new file mode 100644
index 0000000000000..767f47715e89c
--- /dev/null
+++ b/docs/sql-ref-syntax-qry-window.md
@@ -0,0 +1,22 @@
+---
+layout: global
+title: Windowing Analytic Functions
+displayTitle: Windowing Analytic Functions
+license: |
+ 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 page is under construction**
diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax-qry.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref-syntax.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/sql-ref.md b/docs/sql-ref.md
new file mode 100644
index 0000000000000..eb0e73d00e848
--- /dev/null
+++ b/docs/sql-ref.md
@@ -0,0 +1,25 @@
+---
+layout: global
+title: Reference
+displayTitle: Reference
+license: |
+ 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.
+---
+
+Spark SQL is a Apache Spark's module for working with structured data.
+This guide is a reference for Structured Query Language (SQL) for Apache
+Spark. This document describes the SQL constructs supported by Spark in detail
+along with usage examples when applicable.
diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md
index e07a0e5f33e57..b0d3e162c28c7 100644
--- a/docs/structured-streaming-programming-guide.md
+++ b/docs/structured-streaming-programming-guide.md
@@ -1843,7 +1843,7 @@ Here are the details of all the sinks in Spark.
@@ -2251,13 +2251,11 @@ When the streaming query is started, Spark calls the function or the object’s
- The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM or Python process crashes in the middle.
-- **Note:** The partitionId and epochId in the open() method can be used to deduplicate generated data
- when failures cause reprocessing of some input data. This depends on the execution mode of the query.
- If the streaming query is being executed in the micro-batch mode, then every partition represented
- by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data.
- Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit
- data and achieve exactly-once guarantees. However, if the streaming query is being executed
- in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication.
+- **Note:** Spark does not guarantee same output for (partitionId, epochId), so deduplication
+ cannot be achieved with (partitionId, epochId). e.g. source provides different number of
+ partitions for some reasons, Spark optimization changes number of partitions, etc.
+ See [SPARK-28650](https://issues.apache.org/jira/browse/SPARK-28650) for more details.
+ If you need deduplication on output, try out `foreachBatch` instead.
#### Triggers
The trigger settings of a streaming query define the timing of streaming data processing, whether
diff --git a/docs/web-ui.md b/docs/web-ui.md
index a15c114efc502..d6ee8eb707007 100644
--- a/docs/web-ui.md
+++ b/docs/web-ui.md
@@ -35,6 +35,64 @@ progress of all jobs and the overall event timeline. When you click on a job on
page, you see the details page for that job. The details page further shows the event timeline,
DAG visualization, and all stages of the job.
+The information that is displayed in this section is
+* User: Current Spark user
+* Total uptime: Time since Spark application started
+* Scheduling mode: See [job scheduling](job-scheduling.html#configuring-pool-properties)
+* Number of jobs per status: Active, Completed, Failed
+
+
+
+
+
+* Event timeline: Displays in chronological order the events related to the executors (added, removed) and the jobs
+
+
+
+
+
+* Details of jobs grouped by status: Displays detailed information of the jobs including Job ID, description (with a link to detailed job page), submitted time, duration, stages summary and tasks progress bar
+
+
+
+
+
+
+When you click on a specific job, you can see the detailed information of this job.
+
+### Jobs detail
+
+This page displays the details of a specific job identified by its job ID.
+* Job Status: (running, succeeded, failed)
+* Number of stages per status (active, pending, completed, skipped, failed)
+* Associated SQL Query: Link to the sql tab for this job
+* Event timeline: Displays in chronological order the events related to the executors (added, removed) and the stages of the job
+
+
+
+
+
+* DAG visualization: Visual representation of the directed acyclic graph of this job where vertices represent the RDDs or DataFrames and the edges represent an operation to be applied on RDD.
+
+
+
+
+
+* List of stages (grouped by state active, pending, completed, skipped, and failed)
+ * Stage ID
+ * Description of the stage
+ * Submitted timestamp
+ * Duration of the stage
+ * Tasks progress bar
+ * Input: Bytes read from storage in this stage
+ * Output: Bytes written in storage in this stage
+ * Shuffle read: Total shuffle bytes and records read, includes both data read locally and data read from remote executors
+ * Shuffle write: Bytes and records written to disk in order to be read by a shuffle in a future stage
+
+
+
+
+
## Stages Tab
The Stages tab displays a summary page that shows the current state of all stages of all jobs in
the Spark application, and, when you click on a stage, a details page for that stage. The details
@@ -117,3 +175,4 @@ illustrate how Spark parses, analyzes, optimizes and performs the query.
The web UI includes a Streaming tab if the application uses Spark streaming. This tab displays
scheduling delay and processing time for each micro-batch in the data stream, which can be useful
for troubleshooting the streaming application.
+
diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala
index 27915562fded0..4b39e711aa287 100644
--- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala
+++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala
@@ -24,11 +24,11 @@ import org.apache.spark.sql.{RandomDataGenerator, Row}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal}
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
class AvroCatalystDataConversionSuite extends SparkFunSuite
- with SharedSQLContext
+ with SharedSparkSession
with ExpressionEvalHelper {
private def roundTripTest(data: Literal): Unit = {
diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala
index ffd77c5ff6101..c4f995015dfca 100644
--- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala
+++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala
@@ -30,9 +30,9 @@ import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.execution.LocalTableScanExec
import org.apache.spark.sql.functions.{col, struct}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
-class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("roundtrip in to_avro and from_avro - int and string") {
diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
index 96382764b053c..d4649912f1e7a 100644
--- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
+++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala
@@ -28,10 +28,10 @@ import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{StructField, StructType, TimestampType}
-abstract class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
import testImplicits._
val dateSchema = s"""
diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
index a7c9e3fb7d329..366d1d233bc33 100644
--- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
+++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala
@@ -38,11 +38,11 @@ import org.apache.spark.sql._
import org.apache.spark.sql.TestingUDT.{IntervalData, NullData, NullUDT}
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+abstract class AvroSuite extends QueryTest with SharedSparkSession {
import testImplicits._
val episodesAvro = testFile("episodes.avro")
diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
index e9e547eb9a263..519a5cddb3639 100644
--- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
+++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala
@@ -26,12 +26,10 @@ import scala.util.control.NonFatal
import com.spotify.docker.client._
import com.spotify.docker.client.exceptions.ImageNotFoundException
import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding}
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.Eventually
import org.scalatest.time.SpanSugar._
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.DockerUtils
abstract class DatabaseOnDocker {
@@ -66,11 +64,7 @@ abstract class DatabaseOnDocker {
def getStartupProcessName: Option[String]
}
-abstract class DockerJDBCIntegrationSuite
- extends SparkFunSuite
- with BeforeAndAfterAll
- with Eventually
- with SharedSQLContext {
+abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventually {
val db: DatabaseOnDocker
diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
index 8cdc4a1806b2b..6faa888cf18ed 100644
--- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
+++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.{RowDataSourceScanExec, WholeStageCodegenE
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCRelation}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.tags.DockerTest
@@ -54,7 +54,7 @@ import org.apache.spark.tags.DockerTest
* repository.
*/
@DockerTest
-class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext {
+class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSparkSession {
import testImplicits._
override val db = new DatabaseOnDocker {
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
index 2858ff1162b58..e4ed84552b964 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala
@@ -31,7 +31,7 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT
import org.apache.spark.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.kafka010.KafkaSource._
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
@@ -76,7 +76,7 @@ private[kafka010] class KafkaSource(
sqlContext: SQLContext,
kafkaReader: KafkaOffsetReader,
executorKafkaParams: ju.Map[String, Object],
- sourceOptions: Map[String, String],
+ sourceOptions: CaseInsensitiveMap[String],
metadataPath: String,
startingOffsets: KafkaOffsetRangeLimit,
failOnDataLoss: Boolean)
diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
index 372bcab1cab30..c3f0be4be96e2 100644
--- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
+++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
@@ -67,7 +67,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
schema: Option[StructType],
providerName: String,
parameters: Map[String, String]): (String, StructType) = {
- validateStreamOptions(parameters)
+ val caseInsensitiveParameters = CaseInsensitiveMap(parameters)
+ validateStreamOptions(caseInsensitiveParameters)
require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one")
(shortName(), KafkaOffsetReader.kafkaSchema)
}
@@ -85,7 +86,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
// id. Hence, we should generate a unique id for each query.
val uniqueGroupId = streamingUniqueGroupId(caseInsensitiveParameters, metadataPath)
- val specifiedKafkaParams = convertToSpecifiedParams(parameters)
+ val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters)
val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit)
@@ -121,7 +122,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
parameters: Map[String, String]): BaseRelation = {
val caseInsensitiveParameters = CaseInsensitiveMap(parameters)
validateBatchOptions(caseInsensitiveParameters)
- val specifiedKafkaParams = convertToSpecifiedParams(parameters)
+ val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters)
val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(
caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit)
@@ -146,8 +147,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
parameters: Map[String, String],
partitionColumns: Seq[String],
outputMode: OutputMode): Sink = {
- val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim)
- val specifiedKafkaParams = kafkaParamsForProducer(parameters)
+ val caseInsensitiveParameters = CaseInsensitiveMap(parameters)
+ val defaultTopic = caseInsensitiveParameters.get(TOPIC_OPTION_KEY).map(_.trim)
+ val specifiedKafkaParams = kafkaParamsForProducer(caseInsensitiveParameters)
new KafkaSink(sqlContext, specifiedKafkaParams, defaultTopic)
}
@@ -163,8 +165,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
s"${SaveMode.ErrorIfExists} (default).")
case _ => // good
}
- val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim)
- val specifiedKafkaParams = kafkaParamsForProducer(parameters)
+ val caseInsensitiveParameters = CaseInsensitiveMap(parameters)
+ val topic = caseInsensitiveParameters.get(TOPIC_OPTION_KEY).map(_.trim)
+ val specifiedKafkaParams = kafkaParamsForProducer(caseInsensitiveParameters)
KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, specifiedKafkaParams,
topic)
@@ -184,28 +187,31 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
}
- private def strategy(caseInsensitiveParams: Map[String, String]) =
- caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
- case (ASSIGN, value) =>
- AssignStrategy(JsonUtils.partitions(value))
- case (SUBSCRIBE, value) =>
- SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty))
- case (SUBSCRIBE_PATTERN, value) =>
- SubscribePatternStrategy(value.trim())
- case _ =>
- // Should never reach here as we are already matching on
- // matched strategy names
- throw new IllegalArgumentException("Unknown option")
+ private def strategy(params: CaseInsensitiveMap[String]) = {
+ val lowercaseParams = params.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
+
+ lowercaseParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
+ case (ASSIGN, value) =>
+ AssignStrategy(JsonUtils.partitions(value))
+ case (SUBSCRIBE, value) =>
+ SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty))
+ case (SUBSCRIBE_PATTERN, value) =>
+ SubscribePatternStrategy(value.trim())
+ case _ =>
+ // Should never reach here as we are already matching on
+ // matched strategy names
+ throw new IllegalArgumentException("Unknown option")
+ }
}
- private def failOnDataLoss(caseInsensitiveParams: Map[String, String]) =
- caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean
+ private def failOnDataLoss(params: CaseInsensitiveMap[String]) =
+ params.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean
- private def validateGeneralOptions(parameters: Map[String, String]): Unit = {
+ private def validateGeneralOptions(params: CaseInsensitiveMap[String]): Unit = {
// Validate source options
- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
+ val lowercaseParams = params.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
val specifiedStrategies =
- caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq
+ lowercaseParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq
if (specifiedStrategies.isEmpty) {
throw new IllegalArgumentException(
@@ -217,7 +223,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
+ STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.")
}
- caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
+ lowercaseParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match {
case (ASSIGN, value) =>
if (!value.trim.startsWith("{")) {
throw new IllegalArgumentException(
@@ -233,7 +239,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
s"'subscribe' is '$value'")
}
case (SUBSCRIBE_PATTERN, value) =>
- val pattern = caseInsensitiveParams(SUBSCRIBE_PATTERN).trim()
+ val pattern = params(SUBSCRIBE_PATTERN).trim()
if (pattern.isEmpty) {
throw new IllegalArgumentException(
"Pattern to subscribe is empty as specified value for option " +
@@ -246,22 +252,22 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
// Validate minPartitions value if present
- if (caseInsensitiveParams.contains(MIN_PARTITIONS_OPTION_KEY)) {
- val p = caseInsensitiveParams(MIN_PARTITIONS_OPTION_KEY).toInt
+ if (params.contains(MIN_PARTITIONS_OPTION_KEY)) {
+ val p = params(MIN_PARTITIONS_OPTION_KEY).toInt
if (p <= 0) throw new IllegalArgumentException("minPartitions must be positive")
}
// Validate user-specified Kafka options
- if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
+ if (params.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) {
logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE)
- if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) {
+ if (params.contains(GROUP_ID_PREFIX)) {
logWarning("Option 'groupIdPrefix' will be ignored as " +
s"option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set.")
}
}
- if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) {
+ if (params.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) {
throw new IllegalArgumentException(
s"""
|Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported.
@@ -275,14 +281,14 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
""".stripMargin)
}
- if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) {
+ if (params.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) {
throw new IllegalArgumentException(
s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys "
+ "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations "
+ "to explicitly deserialize the keys.")
}
- if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}"))
+ if (params.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}"))
{
throw new IllegalArgumentException(
s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as "
@@ -295,29 +301,29 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe
otherUnsupportedConfigs.foreach { c =>
- if (caseInsensitiveParams.contains(s"kafka.$c")) {
+ if (params.contains(s"kafka.$c")) {
throw new IllegalArgumentException(s"Kafka option '$c' is not supported")
}
}
- if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) {
+ if (!params.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) {
throw new IllegalArgumentException(
s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " +
s"configuring Kafka consumer")
}
}
- private def validateStreamOptions(caseInsensitiveParams: Map[String, String]) = {
+ private def validateStreamOptions(params: CaseInsensitiveMap[String]) = {
// Stream specific options
- caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ =>
+ params.get(ENDING_OFFSETS_OPTION_KEY).map(_ =>
throw new IllegalArgumentException("ending offset not valid in streaming queries"))
- validateGeneralOptions(caseInsensitiveParams)
+ validateGeneralOptions(params)
}
- private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = {
+ private def validateBatchOptions(params: CaseInsensitiveMap[String]) = {
// Batch specific options
KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match {
+ params, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match {
case EarliestOffsetRangeLimit => // good to go
case LatestOffsetRangeLimit =>
throw new IllegalArgumentException("starting offset can't be latest " +
@@ -332,7 +338,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
KafkaSourceProvider.getKafkaOffsetRangeLimit(
- caseInsensitiveParams, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match {
+ params, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match {
case EarliestOffsetRangeLimit =>
throw new IllegalArgumentException("ending offset can't be earliest " +
"for batch queries on Kafka")
@@ -346,10 +352,10 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
}
}
- validateGeneralOptions(caseInsensitiveParams)
+ validateGeneralOptions(params)
// Don't want to throw an error, but at least log a warning.
- if (caseInsensitiveParams.get(MAX_OFFSET_PER_TRIGGER.toLowerCase(Locale.ROOT)).isDefined) {
+ if (params.contains(MAX_OFFSET_PER_TRIGGER)) {
logWarning("maxOffsetsPerTrigger option ignored in batch queries")
}
}
@@ -375,7 +381,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
new WriteBuilder {
private var inputSchema: StructType = _
private val topic = Option(options.get(TOPIC_OPTION_KEY)).map(_.trim)
- private val producerParams = kafkaParamsForProducer(options.asScala.toMap)
+ private val producerParams =
+ kafkaParamsForProducer(CaseInsensitiveMap(options.asScala.toMap))
override def withInputDataSchema(schema: StructType): WriteBuilder = {
this.inputSchema = schema
@@ -486,10 +493,10 @@ private[kafka010] object KafkaSourceProvider extends Logging {
private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets"
private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss"
private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions"
- private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxOffsetsPerTrigger"
- private[kafka010] val FETCH_OFFSET_NUM_RETRY = "fetchOffset.numRetries"
- private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchOffset.retryIntervalMs"
- private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaConsumer.pollTimeoutMs"
+ private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxoffsetspertrigger"
+ private[kafka010] val FETCH_OFFSET_NUM_RETRY = "fetchoffset.numretries"
+ private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchoffset.retryintervalms"
+ private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaconsumer.polltimeoutms"
private val GROUP_ID_PREFIX = "groupidprefix"
val TOPIC_OPTION_KEY = "topic"
@@ -525,7 +532,7 @@ private[kafka010] object KafkaSourceProvider extends Logging {
private val deserClassName = classOf[ByteArrayDeserializer].getName
def getKafkaOffsetRangeLimit(
- params: Map[String, String],
+ params: CaseInsensitiveMap[String],
offsetOptionKey: String,
defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = {
params.get(offsetOptionKey).map(_.trim) match {
@@ -583,9 +590,8 @@ private[kafka010] object KafkaSourceProvider extends Logging {
* Returns a unique batch consumer group (group.id), allowing the user to set the prefix of
* the consumer group
*/
- private[kafka010] def batchUniqueGroupId(parameters: Map[String, String]): String = {
- val groupIdPrefix = parameters
- .getOrElse(GROUP_ID_PREFIX, "spark-kafka-relation")
+ private[kafka010] def batchUniqueGroupId(params: CaseInsensitiveMap[String]): String = {
+ val groupIdPrefix = params.getOrElse(GROUP_ID_PREFIX, "spark-kafka-relation")
s"${groupIdPrefix}-${UUID.randomUUID}"
}
@@ -594,29 +600,27 @@ private[kafka010] object KafkaSourceProvider extends Logging {
* the consumer group
*/
private def streamingUniqueGroupId(
- parameters: Map[String, String],
+ params: CaseInsensitiveMap[String],
metadataPath: String): String = {
- val groupIdPrefix = parameters
- .getOrElse(GROUP_ID_PREFIX, "spark-kafka-source")
+ val groupIdPrefix = params.getOrElse(GROUP_ID_PREFIX, "spark-kafka-source")
s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}"
}
private[kafka010] def kafkaParamsForProducer(
- parameters: Map[String, String]): ju.Map[String, Object] = {
- val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) }
- if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) {
+ params: CaseInsensitiveMap[String]): ju.Map[String, Object] = {
+ if (params.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) {
throw new IllegalArgumentException(
s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys "
+ "are serialized with ByteArraySerializer.")
}
- if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) {
+ if (params.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) {
throw new IllegalArgumentException(
s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as "
+ "value are serialized with ByteArraySerializer.")
}
- val specifiedKafkaParams = convertToSpecifiedParams(parameters)
+ val specifiedKafkaParams = convertToSpecifiedParams(params)
KafkaConfigUpdater("executor", specifiedKafkaParams)
.set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName)
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala
index 0b3355426df10..35c1379de160b 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala
@@ -24,9 +24,9 @@ import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.common.serialization.ByteArraySerializer
import org.scalatest.PrivateMethodTester
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class CachedKafkaProducerSuite extends SharedSQLContext with PrivateMethodTester with KafkaTest {
+class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTester with KafkaTest {
type KP = KafkaProducer[Array[Byte], Array[Byte]]
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala
index 0d0fb9c3ab5af..2aa869c02bc5d 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala
@@ -30,10 +30,10 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer
import org.scalatest.PrivateMethodTester
import org.apache.spark.{TaskContext, TaskContextImpl}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.ThreadUtils
-class KafkaDataConsumerSuite extends SharedSQLContext with PrivateMethodTester {
+class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester {
protected var testUtils: KafkaTestUtils = _
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
index ba8340ea59c14..3e02580b53961 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.{StreamTest, Trigger}
-import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
+import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession}
/**
* This is a basic test trait which will set up a Kafka cluster that keeps only several records in
@@ -43,7 +43,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
* does see missing offsets, you can check the earliest offset in `eventually` and make sure it's
* not 0 rather than sleeping a hard-code duration.
*/
-trait KafkaMissingOffsetsTest extends SharedSQLContext {
+trait KafkaMissingOffsetsTest extends SharedSparkSession {
protected var testUtils: KafkaTestUtils = _
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index bb9b3696fe8f6..8663a5d8d26cb 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -34,6 +34,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.time.SpanSugar._
import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession}
+import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
import org.apache.spark.sql.execution.streaming._
@@ -44,10 +45,10 @@ import org.apache.spark.sql.kafka010.KafkaSourceProvider._
import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream
import org.apache.spark.sql.streaming.{StreamTest, Trigger}
import org.apache.spark.sql.streaming.util.StreamManualClock
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest {
+abstract class KafkaSourceTest extends StreamTest with SharedSparkSession with KafkaTest {
protected var testUtils: KafkaTestUtils = _
@@ -1336,14 +1337,16 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest {
(ENDING_OFFSETS_OPTION_KEY, "laTest", LatestOffsetRangeLimit),
(STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""",
SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) {
- val offset = getKafkaOffsetRangeLimit(Map(optionKey -> optionValue), optionKey, answer)
+ val offset = getKafkaOffsetRangeLimit(
+ CaseInsensitiveMap[String](Map(optionKey -> optionValue)), optionKey, answer)
assert(offset === answer)
}
for ((optionKey, answer) <- Seq(
(STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit),
(ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) {
- val offset = getKafkaOffsetRangeLimit(Map.empty, optionKey, answer)
+ val offset = getKafkaOffsetRangeLimit(
+ CaseInsensitiveMap[String](Map.empty), optionKey, answer)
assert(offset === answer)
}
}
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
index 84d1ab6f75c3e..ed41080eab459 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
@@ -32,10 +32,10 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.BaseRelation
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
-abstract class KafkaRelationSuiteBase extends QueryTest with SharedSQLContext with KafkaTest {
+abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession with KafkaTest {
import testImplicits._
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala
index 032adc70466fa..2746cf96c9cdb 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala
@@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BinaryType, DataType}
-abstract class KafkaSinkSuiteBase extends QueryTest with SharedSQLContext with KafkaTest {
+abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest {
protected var testUtils: KafkaTestUtils = _
override def beforeAll(): Unit = {
diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
index efec51d09745f..ef902fcab3b50 100644
--- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
+++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
@@ -21,9 +21,9 @@ import java.io.File
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.streaming.OffsetSuite
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class KafkaSourceOffsetSuite extends OffsetSuite with SharedSQLContext {
+class KafkaSourceOffsetSuite extends OffsetSuite with SharedSparkSession {
compare(
one = KafkaSourceOffset(("t", 0, 1L)),
diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml
index dbf4b98d5fadb..31c729c501733 100644
--- a/hadoop-cloud/pom.xml
+++ b/hadoop-cloud/pom.xml
@@ -198,6 +198,45 @@
-->
hadoop-3.2
+
+ src/hadoop-3/main/scala
+ src/hadoop-3/test/scala
+
+
+
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-scala-sources
+ generate-sources
+
+ add-source
+
+
+
+ ${extra.source.dir}
+
+
+
+
+ add-scala-test-sources
+ generate-test-sources
+
+ add-test-source
+
+
+
+ ${extra.testsource.dir}
+
+
+
+
+
+
+ 3.8.13.2.10
- 3.0.13
- 2.22.2
+ 3.0.15
+ 2.292.9.33.5.23.0.0
@@ -700,7 +701,7 @@
com.fasterxml.jackson.corejackson-databind
- ${fasterxml.jackson.version}
+ ${fasterxml.jackson.databind.version}com.fasterxml.jackson.core
@@ -729,11 +730,25 @@
org.glassfish.jersey.corejersey-server${jersey.version}
+
+
+
+ jakarta.xml.bind
+ jakarta.xml.bind-api
+
+ org.glassfish.jersey.corejersey-common${jersey.version}
+
+
+
+ com.sun.activation
+ jakarta.activation
+
+ org.glassfish.jersey.core
@@ -750,6 +765,17 @@
jersey-container-servlet-core${jersey.version}
+
+ org.glassfish.jersey.inject
+ jersey-hk2
+ ${jersey.version}
+
+
+ org.glassfish.jersey.test-framework.providers
+ jersey-test-framework-provider-simple
+ ${jersey.version}
+ test
+ org.glassfish.jerseyjersey-client
diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py
index ce6543952bf6d..5a1202b0ba13c 100644
--- a/python/pyspark/ml/classification.py
+++ b/python/pyspark/ml/classification.py
@@ -22,8 +22,9 @@
from pyspark import since, keyword_only
from pyspark.ml import Estimator, Model
from pyspark.ml.param.shared import *
-from pyspark.ml.regression import DecisionTreeModel, DecisionTreeRegressionModel, \
- GBTParams, HasVarianceImpurity, RandomForestParams, TreeEnsembleModel
+from pyspark.ml.regression import DecisionTreeModel, DecisionTreeParams, \
+ DecisionTreeRegressionModel, GBTParams, HasVarianceImpurity, RandomForestParams, \
+ TreeEnsembleModel
from pyspark.ml.util import *
from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams
from pyspark.ml.wrapper import JavaWrapper
diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py
index 1b0c8c5d28b78..ca2e4a0f5bde1 100644
--- a/python/pyspark/ml/param/_shared_params_code_gen.py
+++ b/python/pyspark/ml/param/_shared_params_code_gen.py
@@ -174,45 +174,4 @@ def get$Name(self):
param_code = _gen_param_header(name, doc, defaultValueStr, typeConverter)
code.append(param_code + "\n" + _gen_param_code(name, doc, defaultValueStr))
- decisionTreeParams = [
- ("maxDepth", "Maximum depth of the tree. (>= 0) E.g., depth 0 means 1 leaf node; " +
- "depth 1 means 1 internal node + 2 leaf nodes.", "TypeConverters.toInt"),
- ("maxBins", "Max number of bins for" +
- " discretizing continuous features. Must be >=2 and >= number of categories for any" +
- " categorical feature.", "TypeConverters.toInt"),
- ("minInstancesPerNode", "Minimum number of instances each child must have after split. " +
- "If a split causes the left or right child to have fewer than minInstancesPerNode, the " +
- "split will be discarded as invalid. Should be >= 1.", "TypeConverters.toInt"),
- ("minInfoGain", "Minimum information gain for a split to be considered at a tree node.",
- "TypeConverters.toFloat"),
- ("maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation. If too small," +
- " then 1 node will be split per iteration, and its aggregates may exceed this size.",
- "TypeConverters.toInt"),
- ("cacheNodeIds", "If false, the algorithm will pass trees to executors to match " +
- "instances with nodes. If true, the algorithm will cache node IDs for each instance. " +
- "Caching can speed up training of deeper trees. Users can set how often should the " +
- "cache be checkpointed or disable it by setting checkpointInterval.",
- "TypeConverters.toBoolean")]
-
- decisionTreeCode = '''class DecisionTreeParams(Params):
- """
- Mixin for Decision Tree parameters.
- """
-
- $dummyPlaceHolders
-
- def __init__(self):
- super(DecisionTreeParams, self).__init__()'''
- dtParamMethods = ""
- dummyPlaceholders = ""
- paramTemplate = """$name = Param($owner, "$name", "$doc", typeConverter=$typeConverterStr)"""
- for name, doc, typeConverterStr in decisionTreeParams:
- if typeConverterStr is None:
- typeConverterStr = str(None)
- variable = paramTemplate.replace("$name", name).replace("$doc", doc) \
- .replace("$typeConverterStr", typeConverterStr)
- dummyPlaceholders += variable.replace("$owner", "Params._dummy()") + "\n "
- dtParamMethods += _gen_param_code(name, doc, None) + "\n"
- code.append(decisionTreeCode.replace("$dummyPlaceHolders", dummyPlaceholders) + "\n" +
- dtParamMethods)
print("\n\n\n".join(code))
diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py
index 56d6190723161..9527ef6f576b2 100644
--- a/python/pyspark/ml/param/shared.py
+++ b/python/pyspark/ml/param/shared.py
@@ -747,57 +747,3 @@ def getValidationIndicatorCol(self):
Gets the value of validationIndicatorCol or its default value.
"""
return self.getOrDefault(self.validationIndicatorCol)
-
-
-class DecisionTreeParams(Params):
- """
- Mixin for Decision Tree parameters.
- """
-
- maxDepth = Param(Params._dummy(), "maxDepth", "Maximum depth of the tree. (>= 0) E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.", typeConverter=TypeConverters.toInt)
- maxBins = Param(Params._dummy(), "maxBins", "Max number of bins for discretizing continuous features. Must be >=2 and >= number of categories for any categorical feature.", typeConverter=TypeConverters.toInt)
- minInstancesPerNode = Param(Params._dummy(), "minInstancesPerNode", "Minimum number of instances each child must have after split. If a split causes the left or right child to have fewer than minInstancesPerNode, the split will be discarded as invalid. Should be >= 1.", typeConverter=TypeConverters.toInt)
- minInfoGain = Param(Params._dummy(), "minInfoGain", "Minimum information gain for a split to be considered at a tree node.", typeConverter=TypeConverters.toFloat)
- maxMemoryInMB = Param(Params._dummy(), "maxMemoryInMB", "Maximum memory in MB allocated to histogram aggregation. If too small, then 1 node will be split per iteration, and its aggregates may exceed this size.", typeConverter=TypeConverters.toInt)
- cacheNodeIds = Param(Params._dummy(), "cacheNodeIds", "If false, the algorithm will pass trees to executors to match instances with nodes. If true, the algorithm will cache node IDs for each instance. Caching can speed up training of deeper trees. Users can set how often should the cache be checkpointed or disable it by setting checkpointInterval.", typeConverter=TypeConverters.toBoolean)
-
-
- def __init__(self):
- super(DecisionTreeParams, self).__init__()
-
- def getMaxDepth(self):
- """
- Gets the value of maxDepth or its default value.
- """
- return self.getOrDefault(self.maxDepth)
-
- def getMaxBins(self):
- """
- Gets the value of maxBins or its default value.
- """
- return self.getOrDefault(self.maxBins)
-
- def getMinInstancesPerNode(self):
- """
- Gets the value of minInstancesPerNode or its default value.
- """
- return self.getOrDefault(self.minInstancesPerNode)
-
- def getMinInfoGain(self):
- """
- Gets the value of minInfoGain or its default value.
- """
- return self.getOrDefault(self.minInfoGain)
-
- def getMaxMemoryInMB(self):
- """
- Gets the value of maxMemoryInMB or its default value.
- """
- return self.getOrDefault(self.maxMemoryInMB)
-
- def getCacheNodeIds(self):
- """
- Gets the value of cacheNodeIds or its default value.
- """
- return self.getOrDefault(self.cacheNodeIds)
-
diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py
index 349130f22fade..aca9e6f53cd59 100644
--- a/python/pyspark/ml/regression.py
+++ b/python/pyspark/ml/regression.py
@@ -584,6 +584,80 @@ def predictions(self):
return self._call_java("predictions")
+class DecisionTreeParams(Params):
+ """
+ Mixin for Decision Tree parameters.
+ """
+
+ maxDepth = Param(Params._dummy(), "maxDepth", "Maximum depth of the tree. (>= 0) E.g., " +
+ "depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.",
+ typeConverter=TypeConverters.toInt)
+
+ maxBins = Param(Params._dummy(), "maxBins", "Max number of bins for discretizing continuous " +
+ "features. Must be >=2 and >= number of categories for any categorical " +
+ "feature.", typeConverter=TypeConverters.toInt)
+
+ minInstancesPerNode = Param(Params._dummy(), "minInstancesPerNode", "Minimum number of " +
+ "instances each child must have after split. If a split causes " +
+ "the left or right child to have fewer than " +
+ "minInstancesPerNode, the split will be discarded as invalid. " +
+ "Should be >= 1.", typeConverter=TypeConverters.toInt)
+
+ minInfoGain = Param(Params._dummy(), "minInfoGain", "Minimum information gain for a split " +
+ "to be considered at a tree node.", typeConverter=TypeConverters.toFloat)
+
+ maxMemoryInMB = Param(Params._dummy(), "maxMemoryInMB", "Maximum memory in MB allocated to " +
+ "histogram aggregation. If too small, then 1 node will be split per " +
+ "iteration, and its aggregates may exceed this size.",
+ typeConverter=TypeConverters.toInt)
+
+ cacheNodeIds = Param(Params._dummy(), "cacheNodeIds", "If false, the algorithm will pass " +
+ "trees to executors to match instances with nodes. If true, the " +
+ "algorithm will cache node IDs for each instance. Caching can speed " +
+ "up training of deeper trees. Users can set how often should the cache " +
+ "be checkpointed or disable it by setting checkpointInterval.",
+ typeConverter=TypeConverters.toBoolean)
+
+ def __init__(self):
+ super(DecisionTreeParams, self).__init__()
+
+ def getMaxDepth(self):
+ """
+ Gets the value of maxDepth or its default value.
+ """
+ return self.getOrDefault(self.maxDepth)
+
+ def getMaxBins(self):
+ """
+ Gets the value of maxBins or its default value.
+ """
+ return self.getOrDefault(self.maxBins)
+
+ def getMinInstancesPerNode(self):
+ """
+ Gets the value of minInstancesPerNode or its default value.
+ """
+ return self.getOrDefault(self.minInstancesPerNode)
+
+ def getMinInfoGain(self):
+ """
+ Gets the value of minInfoGain or its default value.
+ """
+ return self.getOrDefault(self.minInfoGain)
+
+ def getMaxMemoryInMB(self):
+ """
+ Gets the value of maxMemoryInMB or its default value.
+ """
+ return self.getOrDefault(self.maxMemoryInMB)
+
+ def getCacheNodeIds(self):
+ """
+ Gets the value of cacheNodeIds or its default value.
+ """
+ return self.getOrDefault(self.cacheNodeIds)
+
+
class TreeEnsembleParams(DecisionTreeParams):
"""
Mixin for Decision Tree-based ensemble algorithms parameters.
diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py
index 4061fda3b19c9..1864a66487a28 100644
--- a/python/pyspark/ml/tests/test_algorithms.py
+++ b/python/pyspark/ml/tests/test_algorithms.py
@@ -86,7 +86,7 @@ def test_raw_and_probability_prediction(self):
expected_rawPrediction = [-11.6081922998, -8.15827998691, 22.17757045]
self.assertTrue(result.prediction, expected_prediction)
self.assertTrue(np.allclose(result.probability, expected_probability, atol=1E-4))
- self.assertTrue(np.allclose(result.rawPrediction, expected_rawPrediction, atol=1E-4))
+ self.assertTrue(np.allclose(result.rawPrediction, expected_rawPrediction, atol=1))
class OneVsRestTests(SparkSessionTestCase):
diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
index 3524fcfeb7958..f220911993096 100644
--- a/python/pyspark/mllib/clustering.py
+++ b/python/pyspark/mllib/clustering.py
@@ -383,11 +383,11 @@ class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader):
>>> model.predict([-0.1,-0.05])
0
>>> softPredicted = model.predictSoft([-0.1,-0.05])
- >>> abs(softPredicted[0] - 1.0) < 0.001
+ >>> abs(softPredicted[0] - 1.0) < 0.03
True
- >>> abs(softPredicted[1] - 0.0) < 0.001
+ >>> abs(softPredicted[1] - 0.0) < 0.03
True
- >>> abs(softPredicted[2] - 0.0) < 0.001
+ >>> abs(softPredicted[2] - 0.0) < 0.03
True
>>> path = tempfile.mkdtemp()
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 09d20fc7673a2..9e5b61f9e00b4 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -1569,8 +1569,9 @@ def format_string(format, *cols):
"""
Formats the arguments in printf-style and returns the result as a string column.
- :param col: the column name of the numeric value to be formatted
- :param d: the N decimal places
+ :param format: string that can contain embedded format tags and used as result column's value
+ :param cols: list of column names (string) or list of :class:`Column` expressions to
+ be used in formatting
>>> df = spark.createDataFrame([(5, "hello")], ['a', 'b'])
>>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect()
diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml
index 23106cb7ec68f..b350b050118fd 100644
--- a/resource-managers/kubernetes/core/pom.xml
+++ b/resource-managers/kubernetes/core/pom.xml
@@ -66,6 +66,11 @@
com.fasterxml.jackson.dataformatjackson-dataformat-yaml
+
+
+ javax.annotation
+ javax.annotation-api
+
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 5a67caf8e2a60..ae9486d7e44cf 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -851,7 +851,9 @@ object ApplicationMaster extends Logging {
master = new ApplicationMaster(amArgs, sparkConf, yarnConf)
val ugi = sparkConf.get(PRINCIPAL) match {
- case Some(principal) =>
+ // We only need to log in with the keytab in cluster mode. In client mode, the driver
+ // handles the user keytab.
+ case Some(principal) if amArgs.userClass != null =>
val originalCreds = UserGroupInformation.getCurrentUser().getCredentials()
SparkHadoopUtil.get.loginUserFromKeytab(principal, sparkConf.get(KEYTAB).orNull)
val newUGI = UserGroupInformation.getCurrentUser()
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 651e706021fcb..5fc6894f89098 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -767,15 +767,8 @@ private[spark] class Client(
val props = confToProperties(sparkConf)
// If propagating the keytab to the AM, override the keytab name with the name of the
- // distributed file. Otherwise remove princpal/keytab from the conf, so they're not seen
- // by the AM at all.
- amKeytabFileName match {
- case Some(kt) =>
- props.setProperty(KEYTAB.key, kt)
- case None =>
- props.remove(PRINCIPAL.key)
- props.remove(KEYTAB.key)
- }
+ // distributed file.
+ amKeytabFileName.foreach { kt => props.setProperty(KEYTAB.key, kt) }
writePropertiesToArchive(props, SPARK_CONF_FILE, confStream)
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala
index cd9bcc0f44f74..d5079202c8fee 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala
@@ -24,7 +24,7 @@ import scala.collection.JavaConverters._
import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, NamespaceChange, TableChange}
import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType}
-import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException}
import org.apache.spark.sql.sources.v2.Table
import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType}
@@ -219,5 +219,7 @@ object CatalogV2Util {
Option(catalog.asTableCatalog.loadTable(ident))
} catch {
case _: NoSuchTableException => None
+ case _: NoSuchDatabaseException => None
+ case _: NoSuchNamespaceException => None
}
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala
new file mode 100644
index 0000000000000..8813d0ab840d0
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/UnresolvedTable.scala
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.sources.v2.internal
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalog.v2.expressions.{LogicalExpressions, Transform}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.sources.v2.{Table, TableCapability}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * An implementation of catalog v2 `Table` to expose v1 table metadata.
+ */
+case class UnresolvedTable(v1Table: CatalogTable) extends Table {
+ implicit class IdentifierHelper(identifier: TableIdentifier) {
+ def quoted: String = {
+ identifier.database match {
+ case Some(db) =>
+ Seq(db, identifier.table).map(quote).mkString(".")
+ case _ =>
+ quote(identifier.table)
+
+ }
+ }
+
+ private def quote(part: String): String = {
+ if (part.contains(".") || part.contains("`")) {
+ s"`${part.replace("`", "``")}`"
+ } else {
+ part
+ }
+ }
+ }
+
+ def catalogTable: CatalogTable = v1Table
+
+ lazy val options: Map[String, String] = {
+ v1Table.storage.locationUri match {
+ case Some(uri) =>
+ v1Table.storage.properties + ("path" -> uri.toString)
+ case _ =>
+ v1Table.storage.properties
+ }
+ }
+
+ override lazy val properties: util.Map[String, String] = v1Table.properties.asJava
+
+ override lazy val schema: StructType = v1Table.schema
+
+ override lazy val partitioning: Array[Transform] = {
+ val partitions = new mutable.ArrayBuffer[Transform]()
+
+ v1Table.partitionColumnNames.foreach { col =>
+ partitions += LogicalExpressions.identity(col)
+ }
+
+ v1Table.bucketSpec.foreach { spec =>
+ partitions += LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*)
+ }
+
+ partitions.toArray
+ }
+
+ override def name: String = v1Table.identifier.quoted
+
+ override def capabilities: util.Set[TableCapability] = new util.HashSet[TableCapability]()
+
+ override def toString: String = s"UnresolvedTable($name)"
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala
new file mode 100644
index 0000000000000..c91a73a47e110
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalog.v2
+
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow
+ * the caller to look up a catalog by name.
+ */
+class CatalogManager(conf: SQLConf) extends Logging {
+
+ private val catalogs = mutable.HashMap.empty[String, CatalogPlugin]
+
+ def catalog(name: String): CatalogPlugin = synchronized {
+ catalogs.getOrElseUpdate(name, Catalogs.load(name, conf))
+ }
+
+ def defaultCatalog: Option[CatalogPlugin] = {
+ conf.defaultV2Catalog.flatMap { catalogName =>
+ try {
+ Some(catalog(catalogName))
+ } catch {
+ case NonFatal(e) =>
+ logError(s"Cannot load default v2 catalog: $catalogName", e)
+ None
+ }
+ }
+ }
+
+ def v2SessionCatalog: Option[CatalogPlugin] = {
+ try {
+ Some(catalog(CatalogManager.SESSION_CATALOG_NAME))
+ } catch {
+ case NonFatal(e) =>
+ logError("Cannot load v2 session catalog", e)
+ None
+ }
+ }
+
+ private def getDefaultNamespace(c: CatalogPlugin) = c match {
+ case c: SupportsNamespaces => c.defaultNamespace()
+ case _ => Array.empty[String]
+ }
+
+ private var _currentNamespace = {
+ // The builtin catalog use "default" as the default database.
+ defaultCatalog.map(getDefaultNamespace).getOrElse(Array("default"))
+ }
+
+ def currentNamespace: Array[String] = synchronized {
+ _currentNamespace
+ }
+
+ def setCurrentNamespace(namespace: Array[String]): Unit = synchronized {
+ _currentNamespace = namespace
+ }
+
+ private var _currentCatalog = conf.defaultV2Catalog
+
+ // Returns the name of current catalog. None means the current catalog is the builtin catalog.
+ def currentCatalog: Option[String] = synchronized {
+ _currentCatalog
+ }
+
+ def setCurrentCatalog(catalogName: String): Unit = synchronized {
+ _currentCatalog = Some(catalogName)
+ _currentNamespace = getDefaultNamespace(catalog(catalogName))
+ }
+
+ // Clear all the registered catalogs. Only used in tests.
+ private[sql] def reset(): Unit = synchronized {
+ catalogs.clear()
+ _currentNamespace = defaultCatalog.map(getDefaultNamespace).getOrElse(Array("default"))
+ _currentCatalog = conf.defaultV2Catalog
+ }
+}
+
+object CatalogManager {
+ val SESSION_CATALOG_NAME: String = "session"
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala
index 5f7ee30cdab79..fe2b320e5d46c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql.catalog.v2
-import scala.util.control.NonFatal
-
import org.apache.spark.annotation.Experimental
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -29,10 +27,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
@Experimental
trait LookupCatalog extends Logging {
- import LookupCatalog._
-
- protected def defaultCatalogName: Option[String] = None
- protected def lookupCatalog(name: String): CatalogPlugin
+ protected val catalogManager: CatalogManager
/**
* Returns the default catalog. When set, this catalog is used for all identifiers that do not
@@ -42,15 +37,7 @@ trait LookupCatalog extends Logging {
* If this is None and a table's provider (source) is a v2 provider, the v2 session catalog will
* be used.
*/
- def defaultCatalog: Option[CatalogPlugin] = {
- try {
- defaultCatalogName.map(lookupCatalog)
- } catch {
- case NonFatal(e) =>
- logError(s"Cannot load default v2 catalog: ${defaultCatalogName.get}", e)
- None
- }
- }
+ def defaultCatalog: Option[CatalogPlugin] = catalogManager.defaultCatalog
/**
* This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the
@@ -58,15 +45,7 @@ trait LookupCatalog extends Logging {
* This happens when the source implementation extends the v2 TableProvider API and is not listed
* in the fallback configuration, spark.sql.sources.write.useV1SourceList
*/
- def sessionCatalog: Option[CatalogPlugin] = {
- try {
- Some(lookupCatalog(SESSION_CATALOG_NAME))
- } catch {
- case NonFatal(e) =>
- logError("Cannot load v2 session catalog", e)
- None
- }
- }
+ def sessionCatalog: Option[CatalogPlugin] = catalogManager.v2SessionCatalog
/**
* Extract catalog plugin and remaining identifier names.
@@ -79,7 +58,7 @@ trait LookupCatalog extends Logging {
Some((None, parts))
case Seq(catalogName, tail @ _*) =>
try {
- Some((Some(lookupCatalog(catalogName)), tail))
+ Some((Some(catalogManager.catalog(catalogName)), tail))
} catch {
case _: CatalogNotFoundException =>
Some((None, parts))
@@ -137,7 +116,3 @@ trait LookupCatalog extends Logging {
}
}
}
-
-object LookupCatalog {
- val SESSION_CATALOG_NAME: String = "session"
-}
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 7267ad8d37c82..0846b9ab127ec 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
@@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.util.Random
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange}
+import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange}
import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform}
import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util.loadTable
import org.apache.spark.sql.catalyst._
@@ -44,6 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
import org.apache.spark.sql.sources.v2.Table
+import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.types._
/**
@@ -109,10 +110,7 @@ class Analyzer(
this(catalog, conf, conf.optimizerMaxIterations)
}
- override protected def defaultCatalogName: Option[String] = conf.defaultV2Catalog
-
- override protected def lookupCatalog(name: String): CatalogPlugin =
- throw new CatalogNotFoundException("No catalog lookup function")
+ override val catalogManager: CatalogManager = new CatalogManager(conf)
def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = {
AnalysisHelper.markInAnalyzer {
@@ -650,8 +648,14 @@ class Analyzer(
if catalog.isTemporaryTable(ident) =>
u // temporary views take precedence over catalog table names
- case u @ UnresolvedRelation(CatalogObjectIdentifier(Some(catalogPlugin), ident)) =>
- loadTable(catalogPlugin, ident).map(DataSourceV2Relation.create).getOrElse(u)
+ case u @ UnresolvedRelation(CatalogObjectIdentifier(maybeCatalog, ident)) =>
+ maybeCatalog.orElse(sessionCatalog)
+ .flatMap(loadTable(_, ident))
+ .map {
+ case unresolved: UnresolvedTable => u
+ case resolved => DataSourceV2Relation.create(resolved)
+ }
+ .getOrElse(u)
}
}
@@ -1640,7 +1644,8 @@ class Analyzer(
// AggregateExpression.
case wf: AggregateWindowFunction =>
if (isDistinct) {
- failAnalysis(s"${wf.prettyName} does not support the modifier DISTINCT")
+ failAnalysis(
+ s"DISTINCT specified, but ${wf.prettyName} is not an aggregate function")
} else {
wf
}
@@ -1649,7 +1654,8 @@ class Analyzer(
// This function is not an aggregate function, just return the resolved one.
case other =>
if (isDistinct) {
- failAnalysis(s"${other.prettyName} does not support the modifier DISTINCT")
+ failAnalysis(
+ s"DISTINCT specified, but ${other.prettyName} is not an aggregate function")
} else {
other
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 74559f5d88796..2a6124a4079a1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -1114,7 +1114,7 @@ class SessionCatalog(
}
externalCatalog.dropFunction(db, name.funcName)
} else if (!ignoreIfNotExists) {
- throw new NoSuchFunctionException(db = db, func = identifier.toString)
+ throw new NoSuchPermanentFunctionException(db = db, func = identifier.toString)
}
}
@@ -1137,7 +1137,7 @@ class SessionCatalog(
}
externalCatalog.alterFunction(db, newFuncDefinition)
} else {
- throw new NoSuchFunctionException(db = db, func = identifier.toString)
+ throw new NoSuchPermanentFunctionException(db = db, func = identifier.toString)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
index ef204ec82c527..d04fe9249d064 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
@@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.TypeUtils
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
@ExpressionDescription(
@@ -89,5 +90,9 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast
)
}
- override lazy val evaluateExpression: Expression = sum
+ override lazy val evaluateExpression: Expression = resultType match {
+ case d: DecimalType => CheckOverflow(sum, d, SQLConf.get.decimalOperationsNullOnOverflow)
+ case _ => sum
+ }
+
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
index 87b35b10d29fa..ed26bb375de25 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala
@@ -445,6 +445,8 @@ case class ArrayExists(
false
}
}
+
+ override def prettyName: String = "exists"
}
/**
@@ -512,6 +514,8 @@ case class ArrayForAll(
forall
}
}
+
+ override def prettyName: String = "forall"
}
/**
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
index 1cc5dd8ce1d54..f0356f5a42d67 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
@@ -165,12 +165,12 @@ class AnalysisErrorSuite extends AnalysisTest {
errorTest(
"distinct function",
CatalystSqlParser.parsePlan("SELECT hex(DISTINCT a) FROM TaBlE"),
- "hex does not support the modifier DISTINCT" :: Nil)
+ "DISTINCT specified, but hex is not an aggregate function" :: Nil)
errorTest(
"distinct window function",
CatalystSqlParser.parsePlan("SELECT percent_rank(DISTINCT a) over () FROM TaBlE"),
- "percent_rank does not support the modifier DISTINCT" :: Nil)
+ "DISTINCT specified, but percent_rank is not an aggregate function" :: Nil)
errorTest(
"nested aggregate functions",
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index bce85534ce7e4..59fb941f41f7e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -1429,7 +1429,7 @@ abstract class SessionCatalogSuite extends AnalysisTest {
catalog.dropFunction(
FunctionIdentifier("something", Some("unknown_db")), ignoreIfNotExists = false)
}
- intercept[NoSuchFunctionException] {
+ intercept[NoSuchPermanentFunctionException] {
catalog.dropFunction(FunctionIdentifier("does_not_exist"), ignoreIfNotExists = false)
}
catalog.dropFunction(FunctionIdentifier("does_not_exist"), ignoreIfNotExists = true)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala
index 52543d16d4815..229ed22b507a3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala
@@ -16,11 +16,14 @@
*/
package org.apache.spark.sql.catalyst.catalog.v2
+import org.mockito.ArgumentMatchers.any
+import org.mockito.Mockito.{mock, when}
+import org.mockito.invocation.InvocationOnMock
import org.scalatest.Inside
import org.scalatest.Matchers._
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog}
+import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -33,10 +36,17 @@ private case class TestCatalogPlugin(override val name: String) extends CatalogP
class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside {
import CatalystSqlParser._
- private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap
+ private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap
- override def lookupCatalog(name: String): CatalogPlugin =
- catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found"))
+ override val catalogManager: CatalogManager = {
+ val manager = mock(classOf[CatalogManager])
+ when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => {
+ val name = invocation.getArgument[String](0)
+ catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found"))
+ })
+ when(manager.defaultCatalog).thenReturn(None)
+ manager
+ }
test("catalog object identifier") {
Seq(
@@ -118,12 +128,17 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside {
class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside {
import CatalystSqlParser._
- private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap
-
- override def defaultCatalogName: Option[String] = Some("prod")
+ private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap
- override def lookupCatalog(name: String): CatalogPlugin =
- catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found"))
+ override val catalogManager: CatalogManager = {
+ val manager = mock(classOf[CatalogManager])
+ when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => {
+ val name = invocation.getArgument[String](0)
+ catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found"))
+ })
+ when(manager.defaultCatalog).thenReturn(catalogs.get("prod"))
+ manager
+ }
test("catalog object identifier") {
Seq(
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 195690f88cdf2..056337205ae7e 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
@@ -542,12 +542,12 @@ class DateTimeUtilsSuite extends SparkFunSuite {
// There are some days are skipped entirely in some timezone, skip them here.
val skipped_days = Map[String, Set[Int]](
- "Kwajalein" -> Set(8632),
+ "Kwajalein" -> Set(8632, 8633),
"Pacific/Apia" -> Set(15338),
"Pacific/Enderbury" -> Set(9130, 9131),
"Pacific/Fakaofo" -> Set(15338),
"Pacific/Kiritimati" -> Set(9130, 9131),
- "Pacific/Kwajalein" -> Set(8632),
+ "Pacific/Kwajalein" -> Set(8632, 8633),
"MIT" -> Set(15338))
for (tz <- ALL_TIMEZONES) {
val skipped = skipped_days.getOrElse(tz.getID, Set.empty)
diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt
new file mode 100644
index 0000000000000..7ee185e9adc59
--- /dev/null
+++ b/sql/core/benchmarks/ExtractBenchmark-results.txt
@@ -0,0 +1,145 @@
+================================================================================================
+Extract
+================================================================================================
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+cast to timestamp wholestage off 407 432 36 24.6 40.7 1.0X
+cast to timestamp wholestage on 348 396 80 28.7 34.8 1.2X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+MILLENNIUM of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+MILLENNIUM of timestamp wholestage off 1407 1408 2 7.1 140.7 1.0X
+MILLENNIUM of timestamp wholestage on 1334 1380 81 7.5 133.4 1.1X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+CENTURY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+CENTURY of timestamp wholestage off 1362 1364 3 7.3 136.2 1.0X
+CENTURY of timestamp wholestage on 1334 1342 8 7.5 133.4 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+DECADE of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+DECADE of timestamp wholestage off 1226 1229 4 8.2 122.6 1.0X
+DECADE of timestamp wholestage on 1218 1225 8 8.2 121.8 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+YEAR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+YEAR of timestamp wholestage off 1207 1210 4 8.3 120.7 1.0X
+YEAR of timestamp wholestage on 1201 1216 17 8.3 120.1 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+ISOYEAR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+ISOYEAR of timestamp wholestage off 1442 1446 6 6.9 144.2 1.0X
+ISOYEAR of timestamp wholestage on 1315 1336 18 7.6 131.5 1.1X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+QUARTER of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+QUARTER of timestamp wholestage off 1443 1454 16 6.9 144.3 1.0X
+QUARTER of timestamp wholestage on 1429 1442 9 7.0 142.9 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+MONTH of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+MONTH of timestamp wholestage off 1196 1200 5 8.4 119.6 1.0X
+MONTH of timestamp wholestage on 1192 1204 10 8.4 119.2 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+WEEK of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+WEEK of timestamp wholestage off 2103 2104 2 4.8 210.3 1.0X
+WEEK of timestamp wholestage on 1798 1804 8 5.6 179.8 1.2X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+DAY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+DAY of timestamp wholestage off 1211 1228 23 8.3 121.1 1.0X
+DAY of timestamp wholestage on 1204 1212 6 8.3 120.4 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+DAYOFWEEK of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+DAYOFWEEK of timestamp wholestage off 1387 1389 3 7.2 138.7 1.0X
+DAYOFWEEK of timestamp wholestage on 1353 1360 8 7.4 135.3 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+DOW of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+DOW of timestamp wholestage off 1373 1373 0 7.3 137.3 1.0X
+DOW of timestamp wholestage on 1361 1372 15 7.3 136.1 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+ISODOW of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+ISODOW of timestamp wholestage off 1311 1366 77 7.6 131.1 1.0X
+ISODOW of timestamp wholestage on 1307 1314 6 7.7 130.7 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+DOY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+DOY of timestamp wholestage off 1241 1243 2 8.1 124.1 1.0X
+DOY of timestamp wholestage on 1229 1239 9 8.1 122.9 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+HOUR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+HOUR of timestamp wholestage off 353 358 8 28.3 35.3 1.0X
+HOUR of timestamp wholestage on 358 365 5 27.9 35.8 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+MINUTE of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+MINUTE of timestamp wholestage off 353 354 2 28.3 35.3 1.0X
+MINUTE of timestamp wholestage on 362 368 9 27.6 36.2 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+SECOND of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+SECOND of timestamp wholestage off 341 350 13 29.3 34.1 1.0X
+SECOND of timestamp wholestage on 362 368 7 27.6 36.2 0.9X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+MILLISECONDS of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+MILLISECONDS of timestamp wholestage off 36785 36808 32 0.3 3678.5 1.0X
+MILLISECONDS of timestamp wholestage on 36644 36760 72 0.3 3664.4 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+MICROSECONDS of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+MICROSECONDS of timestamp wholestage off 446 447 0 22.4 44.6 1.0X
+MICROSECONDS of timestamp wholestage on 458 463 4 21.8 45.8 1.0X
+
+OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64
+Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz
+EPOCH of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative
+------------------------------------------------------------------------------------------------------------------------
+EPOCH of timestamp wholestage off 29807 29811 5 0.3 2980.7 1.0X
+EPOCH of timestamp wholestage on 29843 29930 64 0.3 2984.3 1.0X
+
+
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index af7ddd756ae89..0b49cf24e6c7f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -37,6 +37,7 @@ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister}
import org.apache.spark.sql.sources.v2._
import org.apache.spark.sql.sources.v2.TableCapability._
+import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.types.{IntegerType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -251,19 +252,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
assertNotBucketed("save")
val session = df.sparkSession
- val useV1Sources =
- session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",")
val cls = DataSource.lookupDataSource(source, session.sessionState.conf)
- val shouldUseV1Source = cls.newInstance() match {
- case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true
- case _ => useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT))
- }
+ val canUseV2 = canUseV2Source(session, cls) && partitioningColumns.isEmpty
// In Data Source V2 project, partitioning is still under development.
// Here we fallback to V1 if partitioning columns are specified.
// TODO(SPARK-26778): use V2 implementations when partitioning feature is supported.
- if (!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(cls) &&
- partitioningColumns.isEmpty) {
+ if (canUseV2) {
val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider]
val sessionOptions = DataSourceV2Utils.extractSessionConfigs(
provider, session.sessionState.conf)
@@ -493,13 +488,20 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier}
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
- import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
val session = df.sparkSession
+ val provider = DataSource.lookupDataSource(source, session.sessionState.conf)
+ val canUseV2 = canUseV2Source(session, provider)
+ val sessionCatalogOpt = session.sessionState.analyzer.sessionCatalog
session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match {
case CatalogObjectIdentifier(Some(catalog), ident) =>
saveAsTable(catalog.asTableCatalog, ident, modeForDSV2)
- // TODO(SPARK-28666): This should go through V2SessionCatalog
+
+ case CatalogObjectIdentifier(None, ident)
+ if canUseV2 && sessionCatalogOpt.isDefined && ident.namespace().length <= 1 =>
+ // We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility
+ // for now.
+ saveAsTable(sessionCatalogOpt.get.asTableCatalog, ident, modeForDSV1)
case AsTableIdentifier(tableIdentifier) =>
saveAsTable(tableIdentifier)
@@ -525,6 +527,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
}
val command = (mode, tableOpt) match {
+ case (_, Some(table: UnresolvedTable)) =>
+ return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption))
+
case (SaveMode.Append, Some(table)) =>
AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan)
@@ -830,6 +835,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
private def modeForDSV2 = mode.getOrElse(SaveMode.Append)
+ private def canUseV2Source(session: SparkSession, providerClass: Class[_]): Boolean = {
+ val useV1Sources =
+ session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",")
+ val shouldUseV1Source = providerClass.newInstance() match {
+ case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true
+ case _ => useV1Sources.contains(providerClass.getCanonicalName.toLowerCase(Locale.ROOT))
+ }
+ !shouldUseV1Source && classOf[TableProvider].isAssignableFrom(providerClass)
+ }
+
///////////////////////////////////////////////////////////////////////////////////////
// Builder pattern config options
///////////////////////////////////////////////////////////////////////////////////////
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
index 5c0fe798b1044..a0b0a34a01073 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala
@@ -50,14 +50,11 @@ import org.apache.spark.annotation.Evolving
*
* Important points to note:
*
- *
The `partitionId` and `epochId` can be used to deduplicate generated data when failures
- * cause reprocessing of some input data. This depends on the execution mode of the query. If
- * the streaming query is being executed in the micro-batch mode, then every partition
- * represented by a unique tuple (partitionId, epochId) is guaranteed to have the same data.
- * Hence, (partitionId, epochId) can be used to deduplicate and/or transactionally commit data
- * and achieve exactly-once guarantees. However, if the streaming query is being executed in the
- * continuous mode, then this guarantee does not hold and therefore should not be used for
- * deduplication.
+ *
Spark doesn't guarantee same output for (partitionId, epochId), so deduplication
+ * cannot be achieved with (partitionId, epochId). e.g. source provides different number of
+ * partitions for some reason, Spark optimization changes number of partitions, etc.
+ * Refer SPARK-28650 for more details. If you need deduplication on output, try out
+ * `foreachBatch` instead.
*
*
The `close()` method will be called if `open()` method returns successfully (irrespective
* of the return value), except if the JVM crashes in the middle.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index 90d1b9205787b..e0d0062e976c1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -607,12 +607,6 @@ class SparkSession private(
*/
@transient lazy val catalog: Catalog = new CatalogImpl(self)
- @transient private lazy val catalogs = new mutable.HashMap[String, CatalogPlugin]()
-
- private[sql] def catalog(name: String): CatalogPlugin = synchronized {
- catalogs.getOrElseUpdate(name, Catalogs.load(name, sessionState.conf))
- }
-
/**
* Returns the specified table/view as a `DataFrame`.
*
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index c4edadba278fe..4514675c56f39 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -985,7 +985,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
} else {
CreateTable(tableDescWithPartitionColNames, mode, Some(q))
}
- case None => CreateTable(tableDesc, mode, None)
+ case None =>
+ // When creating partitioned table, we must specify data type for the partition columns.
+ if (Option(ctx.partitionColumnNames).isDefined) {
+ val errorMessage = "Must specify a data type for each partition column while creating " +
+ "Hive partitioned table."
+ operationNotAllowed(errorMessage, ctx)
+ }
+
+ CreateTable(tableDesc, mode, None)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
index e7bbbd7bf4642..4f13568d84b37 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
@@ -60,7 +60,8 @@ import org.apache.spark.util.ThreadUtils
case class AdaptiveSparkPlanExec(
initialPlan: SparkPlan,
@transient session: SparkSession,
- @transient subqueryMap: Map[Long, ExecSubqueryExpression],
+ @transient preprocessingRules: Seq[Rule[SparkPlan]],
+ @transient subqueryCache: TrieMap[SparkPlan, BaseSubqueryExec],
@transient stageCache: TrieMap[SparkPlan, QueryStageExec],
@transient queryExecution: QueryExecution)
extends LeafExecNode {
@@ -73,24 +74,27 @@ case class AdaptiveSparkPlanExec(
override protected def batches: Seq[Batch] = Seq()
}
+ @transient private val ensureRequirements = EnsureRequirements(conf)
+
// A list of physical plan rules to be applied before creation of query stages. The physical
// plan should reach a final status of query stages (i.e., no more addition or removal of
// Exchange nodes) after running these rules.
- @transient private val queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq(
- PlanAdaptiveSubqueries(subqueryMap),
- EnsureRequirements(conf)
+ private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq(
+ ensureRequirements
)
// A list of physical optimizer rules to be applied to a new stage before its execution. These
// optimizations should be stage-independent.
@transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq(
+ ReuseAdaptiveSubquery(conf, subqueryCache),
ReduceNumShufflePartitions(conf),
ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf,
session.sessionState.columnarRules),
CollapseCodegenStages(conf)
)
- @volatile private var currentPhysicalPlan = initialPlan
+ @volatile private var currentPhysicalPlan =
+ applyPhysicalRules(initialPlan, queryStagePreparationRules)
private var isFinalPlan = false
@@ -205,6 +209,16 @@ case class AdaptiveSparkPlanExec(
depth + 1, lastChildren :+ true, append, verbose, "", addSuffix = false, maxFields)
}
+ override def hashCode(): Int = initialPlan.hashCode()
+
+ override def equals(obj: Any): Boolean = {
+ if (!obj.isInstanceOf[AdaptiveSparkPlanExec]) {
+ return false
+ }
+
+ this.initialPlan == obj.asInstanceOf[AdaptiveSparkPlanExec].initialPlan
+ }
+
/**
* This method is called recursively to traverse the plan tree bottom-up and create a new query
* stage or try reusing an existing stage if the current node is an [[Exchange]] node and all of
@@ -356,7 +370,7 @@ case class AdaptiveSparkPlanExec(
val optimized = optimizer.execute(logicalPlan)
SparkSession.setActiveSession(session)
val sparkPlan = session.sessionState.planner.plan(ReturnAnswer(optimized)).next()
- val newPlan = applyPhysicalRules(sparkPlan, queryStagePreparationRules)
+ val newPlan = applyPhysicalRules(sparkPlan, preprocessingRules ++ queryStagePreparationRules)
(newPlan, optimized)
}
@@ -403,17 +417,6 @@ object AdaptiveSparkPlanExec {
private val executionContext = ExecutionContext.fromExecutorService(
ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16))
- /**
- * Creates the list of physical plan rules to be applied before creation of query stages.
- */
- def createQueryStagePreparationRules(
- conf: SQLConf,
- subqueryMap: Map[Long, ExecSubqueryExpression]): Seq[Rule[SparkPlan]] = {
- Seq(
- PlanAdaptiveSubqueries(subqueryMap),
- EnsureRequirements(conf))
- }
-
/**
* Apply a list of physical operator rules on a [[SparkPlan]].
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
index 50c10a154a3c5..5d95f2c77be0c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala
@@ -41,6 +41,9 @@ case class InsertAdaptiveSparkPlan(
private val conf = session.sessionState.conf
+ // Subquery-reuse is shared across the entire query.
+ private val subqueryCache = new TrieMap[SparkPlan, BaseSubqueryExec]()
+
// Exchange-reuse is shared across the entire query, including sub-queries.
private val stageCache = new TrieMap[SparkPlan, QueryStageExec]()
@@ -53,12 +56,13 @@ case class InsertAdaptiveSparkPlan(
// Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. Fall
// back to non-adaptive mode if adaptive execution is supported in any of the sub-queries.
val subqueryMap = buildSubqueryMap(plan)
- // Run preparation rules.
- val preparations = AdaptiveSparkPlanExec.createQueryStagePreparationRules(
- session.sessionState.conf, subqueryMap)
- val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preparations)
+ val planSubqueriesRule = PlanAdaptiveSubqueries(subqueryMap)
+ val preprocessingRules = Seq(
+ planSubqueriesRule)
+ // Run pre-processing rules.
+ val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preprocessingRules)
logDebug(s"Adaptive execution enabled for plan: $plan")
- AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache, qe)
+ AdaptiveSparkPlanExec(newPlan, session, preprocessingRules, subqueryCache, stageCache, qe)
} catch {
case SubqueryAdaptiveNotSupportedException(subquery) =>
logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " +
@@ -87,48 +91,33 @@ case class InsertAdaptiveSparkPlan(
* For each sub-query, generate the adaptive execution plan for each sub-query by applying this
* rule, or reuse the execution plan from another sub-query of the same semantics if possible.
*/
- private def buildSubqueryMap(plan: SparkPlan): Map[Long, ExecSubqueryExpression] = {
- val subqueryMapBuilder = mutable.HashMap.empty[Long, ExecSubqueryExpression]
+ private def buildSubqueryMap(plan: SparkPlan): mutable.HashMap[Long, ExecSubqueryExpression] = {
+ val subqueryMap = mutable.HashMap.empty[Long, ExecSubqueryExpression]
plan.foreach(_.expressions.foreach(_.foreach {
case expressions.ScalarSubquery(p, _, exprId)
- if !subqueryMapBuilder.contains(exprId.id) =>
- val executedPlan = getExecutedPlan(p)
+ if !subqueryMap.contains(exprId.id) =>
+ val executedPlan = compileSubquery(p)
+ verifyAdaptivePlan(executedPlan, p)
val scalarSubquery = execution.ScalarSubquery(
SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId)
- subqueryMapBuilder.put(exprId.id, scalarSubquery)
+ subqueryMap.put(exprId.id, scalarSubquery)
case _ =>
}))
- // Reuse subqueries
- if (session.sessionState.conf.subqueryReuseEnabled) {
- // Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls.
- val reuseMap = mutable.HashMap[StructType, mutable.ArrayBuffer[BaseSubqueryExec]]()
- subqueryMapBuilder.keySet.foreach { exprId =>
- val sub = subqueryMapBuilder(exprId)
- val sameSchema =
- reuseMap.getOrElseUpdate(sub.plan.schema, mutable.ArrayBuffer.empty)
- val sameResult = sameSchema.find(_.sameResult(sub.plan))
- if (sameResult.isDefined) {
- val newExpr = sub.withNewPlan(ReusedSubqueryExec(sameResult.get))
- subqueryMapBuilder.update(exprId, newExpr)
- } else {
- sameSchema += sub.plan
- }
- }
- }
-
- subqueryMapBuilder.toMap
+ subqueryMap
}
- private def getExecutedPlan(plan: LogicalPlan): SparkPlan = {
+ def compileSubquery(plan: LogicalPlan): SparkPlan = {
val queryExec = new QueryExecution(session, plan)
// Apply the same instance of this rule to sub-queries so that sub-queries all share the
// same `stageCache` for Exchange reuse.
- val adaptivePlan = this.applyInternal(queryExec.sparkPlan, queryExec)
- if (!adaptivePlan.isInstanceOf[AdaptiveSparkPlanExec]) {
- throw SubqueryAdaptiveNotSupportedException(plan)
+ this.applyInternal(queryExec.sparkPlan, queryExec)
+ }
+
+ private def verifyAdaptivePlan(plan: SparkPlan, logicalPlan: LogicalPlan): Unit = {
+ if (!plan.isInstanceOf[AdaptiveSparkPlanExec]) {
+ throw SubqueryAdaptiveNotSupportedException(logicalPlan)
}
- adaptivePlan
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala
index 4af7432d7bed0..91d4359224a6a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.{ExecSubqueryExpression, SparkPlan}
case class PlanAdaptiveSubqueries(
- subqueryMap: Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] {
+ subqueryMap: scala.collection.Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] {
def apply(plan: SparkPlan): SparkPlan = {
plan.transformAllExpressions {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala
index 78923433eaab9..1a85d5c02075b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala
@@ -82,7 +82,12 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] {
// `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions,
// we should skip it when calculating the `partitionStartIndices`.
val validMetrics = shuffleMetrics.filter(_ != null)
- if (validMetrics.nonEmpty) {
+ // We may have different pre-shuffle partition numbers, don't reduce shuffle partition number
+ // in that case. For example when we union fully aggregated data (data is arranged to a single
+ // partition) and a result of a SortMergeJoin (multiple partitions).
+ val distinctNumPreShufflePartitions =
+ validMetrics.map(stats => stats.bytesByPartitionId.length).distinct
+ if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) {
val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray)
// This transformation adds new nodes, so we must use `transformUp` here.
plan.transformUp {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala
new file mode 100644
index 0000000000000..432f7e204791b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReuseAdaptiveSubquery.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.adaptive
+
+import scala.collection.concurrent.TrieMap
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan}
+import org.apache.spark.sql.internal.SQLConf
+
+case class ReuseAdaptiveSubquery(
+ conf: SQLConf,
+ reuseMap: TrieMap[SparkPlan, BaseSubqueryExec]) extends Rule[SparkPlan] {
+
+ def apply(plan: SparkPlan): SparkPlan = {
+ if (!conf.subqueryReuseEnabled) {
+ return plan
+ }
+
+ plan.transformAllExpressions {
+ case sub: ExecSubqueryExpression =>
+ val newPlan = reuseMap.getOrElseUpdate(sub.plan.canonicalized, sub.plan)
+ if (newPlan.ne(sub.plan)) {
+ sub.withNewPlan(ReusedSubqueryExec(newPlan))
+ } else {
+ sub
+ }
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala
index 4791fe5fb5251..2744339fe4722 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala
@@ -22,31 +22,28 @@ import java.util.Locale
import scala.collection.mutable
import org.apache.spark.sql.{AnalysisException, SaveMode}
-import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog, TableCatalog}
+import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, TableCatalog}
import org.apache.spark.sql.catalog.v2.expressions.Transform
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedAttribute, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation}
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation}
import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, SubqueryAlias}
import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand}
-import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.TableProvider
import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType}
-import org.apache.spark.sql.util.SchemaUtils
case class DataSourceResolution(
conf: SQLConf,
- lookup: LookupCatalog)
- extends Rule[LogicalPlan] with CastSupport {
+ catalogManager: CatalogManager)
+ extends Rule[LogicalPlan] with CastSupport with LookupCatalog {
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
- import lookup._
- lazy val v2SessionCatalog: CatalogPlugin = lookup.sessionCatalog
- .getOrElse(throw new AnalysisException("No v2 session catalog implementation is available"))
+ def v2SessionCatalog: CatalogPlugin = sessionCatalog.getOrElse(
+ throw new AnalysisException("No v2 session catalog implementation is available"))
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case CreateTableStatement(
@@ -183,8 +180,6 @@ case class DataSourceResolution(
val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation)
DeleteFromTable(aliased, delete.condition)
- case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) =>
- UnresolvedCatalogRelation(catalogTable)
}
object V1WriteProvider {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
index bcb10ae5999fc..ac786bbaac6d7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
@@ -43,11 +43,10 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister {
protected def getPaths(map: CaseInsensitiveStringMap): Seq[String] = {
val objectMapper = new ObjectMapper()
- Option(map.get("paths")).map { pathStr =>
+ val paths = Option(map.get("paths")).map { pathStr =>
objectMapper.readValue(pathStr, classOf[Array[String]]).toSeq
- }.getOrElse {
- Option(map.get("path")).toSeq
- }
+ }.getOrElse(Seq.empty)
+ paths ++ Option(map.get("path")).toSeq
}
protected def getTableName(paths: Seq[String]): String = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
index 79ea8756721ed..6dcebe29537d4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
@@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchT
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog}
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SessionState
-import org.apache.spark.sql.sources.v2.{Table, TableCapability}
+import org.apache.spark.sql.sources.v2.Table
+import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -70,7 +71,7 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog {
throw new NoSuchTableException(ident)
}
- CatalogTableAsV2(catalogTable)
+ UnresolvedTable(catalogTable)
}
override def invalidateTable(ident: Identifier): Unit = {
@@ -179,66 +180,6 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog {
override def toString: String = s"V2SessionCatalog($name)"
}
-/**
- * An implementation of catalog v2 [[Table]] to expose v1 table metadata.
- */
-case class CatalogTableAsV2(v1Table: CatalogTable) extends Table {
- implicit class IdentifierHelper(identifier: TableIdentifier) {
- def quoted: String = {
- identifier.database match {
- case Some(db) =>
- Seq(db, identifier.table).map(quote).mkString(".")
- case _ =>
- quote(identifier.table)
-
- }
- }
-
- private def quote(part: String): String = {
- if (part.contains(".") || part.contains("`")) {
- s"`${part.replace("`", "``")}`"
- } else {
- part
- }
- }
- }
-
- def catalogTable: CatalogTable = v1Table
-
- lazy val options: Map[String, String] = {
- v1Table.storage.locationUri match {
- case Some(uri) =>
- v1Table.storage.properties + ("path" -> uri.toString)
- case _ =>
- v1Table.storage.properties
- }
- }
-
- override lazy val properties: util.Map[String, String] = v1Table.properties.asJava
-
- override lazy val schema: StructType = v1Table.schema
-
- override lazy val partitioning: Array[Transform] = {
- val partitions = new mutable.ArrayBuffer[Transform]()
-
- v1Table.partitionColumnNames.foreach { col =>
- partitions += LogicalExpressions.identity(col)
- }
-
- v1Table.bucketSpec.foreach { spec =>
- partitions += LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*)
- }
-
- partitions.toArray
- }
-
- override def name: String = v1Table.identifier.quoted
-
- override def capabilities: util.Set[TableCapability] = new util.HashSet[TableCapability]()
-
- override def toString: String = s"CatalogTableAsV2($name)"
-}
-
private[sql] object V2SessionCatalog {
/**
* Convert v2 Transforms to v1 partition columns and an optional bucket spec.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala
index 6da1b3a49c442..838c7d497e35b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala
@@ -21,6 +21,7 @@ import java.util
import scala.collection.JavaConverters._
+import org.apache.spark.SparkException
import org.apache.spark.sql.{ForeachWriter, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
@@ -133,6 +134,7 @@ class ForeachDataWriter[T](
// If open returns false, we should skip writing rows.
private val opened = writer.open(partitionId, epochId)
+ private var closeCalled: Boolean = false
override def write(record: InternalRow): Unit = {
if (!opened) return
@@ -141,17 +143,26 @@ class ForeachDataWriter[T](
writer.process(rowConverter(record))
} catch {
case t: Throwable =>
- writer.close(t)
+ closeWriter(t)
throw t
}
}
override def commit(): WriterCommitMessage = {
- writer.close(null)
+ closeWriter(null)
ForeachWriterCommitMessage
}
- override def abort(): Unit = {}
+ override def abort(): Unit = {
+ closeWriter(new SparkException("Foreach writer has been aborted due to a task failure"))
+ }
+
+ private def closeWriter(errorOrNull: Throwable): Unit = {
+ if (!closeCalled) {
+ closeCalled = true
+ writer.close(errorOrNull)
+ }
+ }
}
/**
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 afafde114a3ef..6b8127bab1cb4 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
@@ -2594,8 +2594,21 @@ object functions {
* @group datetime_funcs
* @since 1.5.0
*/
- def add_months(startDate: Column, numMonths: Int): Column = withExpr {
- AddMonths(startDate.expr, Literal(numMonths))
+ def add_months(startDate: Column, numMonths: Int): Column = add_months(startDate, lit(numMonths))
+
+ /**
+ * Returns the date that is `numMonths` after `startDate`.
+ *
+ * @param startDate A date, timestamp or string. If a string, the data must be in a format that
+ * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS`
+ * @param numMonths A column of the number of months to add to `startDate`, can be negative to
+ * subtract months
+ * @return A date, or null if `startDate` was a string that could not be cast to a date
+ * @group datetime_funcs
+ * @since 3.0.0
+ */
+ def add_months(startDate: Column, numMonths: Column): Column = withExpr {
+ AddMonths(startDate.expr, numMonths.expr)
}
/**
@@ -2644,7 +2657,19 @@ object functions {
* @group datetime_funcs
* @since 1.5.0
*/
- def date_add(start: Column, days: Int): Column = withExpr { DateAdd(start.expr, Literal(days)) }
+ def date_add(start: Column, days: Int): Column = date_add(start, lit(days))
+
+ /**
+ * Returns the date that is `days` days after `start`
+ *
+ * @param start A date, timestamp or string. If a string, the data must be in a format that
+ * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS`
+ * @param days A column of the number of days to add to `start`, can be negative to subtract days
+ * @return A date, or null if `start` was a string that could not be cast to a date
+ * @group datetime_funcs
+ * @since 3.0.0
+ */
+ def date_add(start: Column, days: Column): Column = withExpr { DateAdd(start.expr, days.expr) }
/**
* Returns the date that is `days` days before `start`
@@ -2656,7 +2681,20 @@ object functions {
* @group datetime_funcs
* @since 1.5.0
*/
- def date_sub(start: Column, days: Int): Column = withExpr { DateSub(start.expr, Literal(days)) }
+ def date_sub(start: Column, days: Int): Column = date_sub(start, lit(days))
+
+ /**
+ * Returns the date that is `days` days before `start`
+ *
+ * @param start A date, timestamp or string. If a string, the data must be in a format that
+ * can be cast to a date, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS`
+ * @param days A column of the number of days to subtract from `start`, can be negative to add
+ * days
+ * @return A date, or null if `start` was a string that could not be cast to a date
+ * @group datetime_funcs
+ * @since 3.0.0
+ */
+ def date_sub(start: Column, days: Column): Column = withExpr { DateSub(start.expr, days.expr) }
/**
* Returns the number of days from `start` to `end`.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
index 16a63793e931c..2369c341762cf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala
@@ -171,7 +171,7 @@ abstract class BaseSessionStateBuilder(
new FindDataSourceTable(session) +:
new ResolveSQLOnFile(session) +:
new FallBackFileSourceV2(session) +:
- DataSourceResolution(conf, this) +:
+ DataSourceResolution(conf, this.catalogManager) +:
customResolutionRules
override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
@@ -188,8 +188,6 @@ abstract class BaseSessionStateBuilder(
V2WriteSupportCheck +:
V2StreamingScanSupportCheck +:
customCheckRules
-
- override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
index b962ab6feabcb..f1cde4fbf090b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.SparkContext
import org.apache.spark.annotation.{Experimental, Unstable}
import org.apache.spark.sql._
+import org.apache.spark.sql.catalog.v2.CatalogManager
import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
@@ -83,6 +84,8 @@ private[sql] class SessionState(
lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder()
+ def catalogManager: CatalogManager = analyzer.catalogManager
+
def newHadoopConf(): Configuration = SessionState.newHadoopConf(
sharedState.sparkContext.hadoopConfiguration,
conf)
diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql
new file mode 100644
index 0000000000000..c05aa156a13bf
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-join.sql
@@ -0,0 +1,2081 @@
+--
+-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
+--
+--
+-- JOIN
+-- Test JOIN clauses
+-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql
+--
+-- This test file was converted from pgSQL/join.sql.
+
+CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM
+ (VALUES (0), (123456), (-123456), (2147483647), (-2147483647))
+ AS v(f1);
+CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
+ (VALUES
+ (123, 456),
+ (123, 4567890123456789),
+ (4567890123456789, 123),
+ (4567890123456789, 4567890123456789),
+ (4567890123456789, -4567890123456789))
+ AS v(q1, q2);
+CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM
+ (VALUES (0.0), (1004.30), (-34.84),
+ (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double)))
+ AS v(f1);
+CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM
+ (VALUES ('doh!'), ('hi de ho neighbor'))
+ AS v(f1);
+CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1;
+
+CREATE TABLE J1_TBL (
+ i integer,
+ j integer,
+ t string
+) USING parquet;
+
+CREATE TABLE J2_TBL (
+ i integer,
+ k integer
+) USING parquet;
+
+
+INSERT INTO J1_TBL VALUES (1, 4, 'one');
+INSERT INTO J1_TBL VALUES (2, 3, 'two');
+INSERT INTO J1_TBL VALUES (3, 2, 'three');
+INSERT INTO J1_TBL VALUES (4, 1, 'four');
+INSERT INTO J1_TBL VALUES (5, 0, 'five');
+INSERT INTO J1_TBL VALUES (6, 6, 'six');
+INSERT INTO J1_TBL VALUES (7, 7, 'seven');
+INSERT INTO J1_TBL VALUES (8, 8, 'eight');
+INSERT INTO J1_TBL VALUES (0, NULL, 'zero');
+INSERT INTO J1_TBL VALUES (NULL, NULL, 'null');
+INSERT INTO J1_TBL VALUES (NULL, 0, 'zero');
+
+INSERT INTO J2_TBL VALUES (1, -1);
+INSERT INTO J2_TBL VALUES (2, 2);
+INSERT INTO J2_TBL VALUES (3, -3);
+INSERT INTO J2_TBL VALUES (2, 4);
+INSERT INTO J2_TBL VALUES (5, -5);
+INSERT INTO J2_TBL VALUES (5, -5);
+INSERT INTO J2_TBL VALUES (0, NULL);
+INSERT INTO J2_TBL VALUES (NULL, NULL);
+INSERT INTO J2_TBL VALUES (NULL, 0);
+
+-- [SPARK-20856] Do not need onerow because it only used for test statement using nested joins
+-- useful in some tests below
+-- create temp table onerow();
+-- insert into onerow default values;
+-- analyze onerow;
+
+
+--
+-- CORRELATION NAMES
+-- Make sure that table/column aliases are supported
+-- before diving into more complex join syntax.
+--
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t)
+ FROM J1_TBL AS tx;
+
+SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t)
+ FROM J1_TBL tx;
+
+SELECT udf('') AS `xxx`, a, udf(udf(b)), c
+ FROM J1_TBL AS t1 (a, b, c);
+
+SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c))
+ FROM J1_TBL t1 (a, b, c);
+
+SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e
+ FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e);
+
+-- [SPARK-28377] Fully support correlation names in the FROM clause
+-- SELECT '' AS "xxx", t1.a, t2.e
+-- FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e)
+-- WHERE t1.a = t2.d;
+
+
+--
+-- CROSS JOIN
+-- Qualifications are not allowed on cross joins,
+-- which degenerate into a standard unqualified inner join.
+--
+
+SELECT udf('') AS `xxx`, *
+ FROM J1_TBL CROSS JOIN J2_TBL;
+
+-- ambiguous column
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t
+ FROM J1_TBL CROSS JOIN J2_TBL;
+
+-- resolve previous ambiguity by specifying the table name
+SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t)
+ FROM J1_TBL t1 CROSS JOIN J2_TBL t2;
+
+SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk))
+ FROM (J1_TBL CROSS JOIN J2_TBL)
+ AS tx (ii, jj, tt, ii2, kk);
+
+-- [SPARK-28377] Fully support correlation names in the FROM clause
+-- SELECT '' AS `xxx`, tx.ii, tx.jj, tx.kk
+-- FROM (J1_TBL t1 (a, b, c) CROSS JOIN J2_TBL t2 (d, e))
+-- AS tx (ii, jj, tt, ii2, kk);
+
+SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k)
+ FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b;
+
+
+--
+--
+-- Inner joins (equi-joins)
+--
+--
+
+--
+-- Inner joins (equi-joins) with USING clause
+-- The USING syntax changes the shape of the resulting table
+-- by including a column in the USING clause only once in the result.
+--
+
+-- Inner equi-join on specified column
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k)
+ FROM J1_TBL INNER JOIN J2_TBL USING (i);
+
+-- Same as above, slightly different syntax
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k
+ FROM J1_TBL JOIN J2_TBL USING (i);
+
+SELECT udf('') AS `xxx`, *
+ FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a)
+ ORDER BY udf(udf(a)), udf(d);
+
+-- [SPARK-28377] Fully support correlation names in the FROM clause
+-- SELECT '' AS `xxx`, *
+-- FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, b) USING (b)
+-- ORDER BY b, t1.a;
+
+
+--
+-- NATURAL JOIN
+-- Inner equi-join on all columns with the same name
+--
+
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL NATURAL JOIN J2_TBL;
+
+SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d)
+ FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d);
+
+SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d
+ FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a);
+
+-- [SPARK-28377] Fully support correlation names in the FROM clause
+-- mismatch number of columns
+-- currently, Postgres will fill in with underlying names
+-- SELECT '' AS `xxx`, *
+-- FROM J1_TBL t1 (a, b) NATURAL JOIN J2_TBL t2 (a);
+
+
+--
+-- Inner joins (equi-joins)
+--
+
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+ FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i);
+
+SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k
+ FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k));
+
+
+--
+-- Non-equi-joins
+--
+
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+ FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k)));
+
+
+--
+-- Outer joins
+-- Note that OUTER is a noise word
+--
+
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(t);
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i)
+ ORDER BY udf(i), udf(udf(k)), udf(t);
+
+SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k)
+ FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i);
+
+SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k)
+ FROM J1_TBL RIGHT JOIN J2_TBL USING (i);
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k)
+ FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(t);
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k))
+ FROM J1_TBL FULL JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(udf(t));
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k))
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1);
+
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1));
+
+--
+-- semijoin selectivity for <>
+--
+-- explain (costs off)
+-- select * from int4_tbl i4, tenk1 a
+-- where exists(select * from tenk1 b
+-- where a.twothousand = b.twothousand and a.fivethous <> b.fivethous)
+-- and i4.f1 = a.tenthous;
+
+
+--
+-- More complicated constructs
+--
+
+--
+-- Multiway full join
+--
+
+CREATE TABLE t1 (name STRING, n INTEGER) USING parquet;
+CREATE TABLE t2 (name STRING, n INTEGER) USING parquet;
+CREATE TABLE t3 (name STRING, n INTEGER) USING parquet;
+
+INSERT INTO t1 VALUES ( 'bb', 11 );
+INSERT INTO t2 VALUES ( 'bb', 12 );
+INSERT INTO t2 VALUES ( 'cc', 22 );
+INSERT INTO t2 VALUES ( 'ee', 42 );
+INSERT INTO t3 VALUES ( 'bb', 13 );
+INSERT INTO t3 VALUES ( 'cc', 23 );
+INSERT INTO t3 VALUES ( 'dd', 33 );
+
+SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name);
+
+--
+-- Test interactions of join syntax and subqueries
+--
+
+-- Basic cases (we expect planner to pull up the subquery here)
+SELECT * FROM
+(SELECT udf(name) as name, t2.n FROM t2) as s2
+INNER JOIN
+(SELECT udf(udf(name)) as name, t3.n FROM t3) s3
+USING (name);
+
+SELECT * FROM
+(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2
+LEFT JOIN
+(SELECT udf(name) as name, t3.n FROM t3) s3
+USING (name);
+
+SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM
+(SELECT * FROM t2) as s2
+FULL JOIN
+(SELECT * FROM t3) s3
+USING (name);
+
+-- Cases with non-nullable expressions in subquery results;
+-- make sure these go to null as expected
+SELECT * FROM
+(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2
+NATURAL INNER JOIN
+(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3;
+
+SELECT * FROM
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL LEFT JOIN
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3;
+
+SELECT * FROM
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL FULL JOIN
+(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3;
+
+SELECT * FROM
+(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1
+NATURAL INNER JOIN
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL INNER JOIN
+(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3;
+
+SELECT * FROM
+(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1
+NATURAL FULL JOIN
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2
+NATURAL FULL JOIN
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3;
+
+SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM
+(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1
+NATURAL FULL JOIN
+ (SELECT * FROM
+ (SELECT name, udf(n) as s2_n FROM t2) as s2
+ NATURAL FULL JOIN
+ (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3
+ ) ss2;
+
+SELECT * FROM
+(SELECT name, n as s1_n FROM t1) as s1
+NATURAL FULL JOIN
+ (SELECT * FROM
+ (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
+ NATURAL FULL JOIN
+ (SELECT name, udf(n) as s3_n FROM t3) as s3
+ ) ss2;
+
+-- Constants as join keys can also be problematic
+SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM
+ (SELECT name, udf(n) as s1_n FROM t1) as s1
+FULL JOIN
+ (SELECT name, 2 as s2_n FROM t2) as s2
+ON (udf(udf(s1_n)) = udf(s2_n));
+
+
+-- Test for propagation of nullability constraints into sub-joins
+
+create or replace temporary view x as select * from
+ (values (1,11), (2,22), (3,null), (4,44), (5,null))
+ as v(x1, x2);
+
+create or replace temporary view y as select * from
+ (values (1,111), (2,222), (3,333), (4,null))
+ as v(y1, y2);
+
+select udf(udf(x1)), udf(x2) from x;
+select udf(y1), udf(udf(y2)) from y;
+
+select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null);
+select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null);
+
+select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1));
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1 and udf(x2) is not null);
+select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null);
+select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null);
+-- these should NOT give the same answers as above
+select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(xx1)) where (udf(x2) is not null);
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1) where (udf(y2) is not null);
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (x1 = udf(xx1)) where (xx2 is not null);
+
+--
+-- regression test: check for bug with propagation of implied equality
+-- to outside an IN
+--
+select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in
+ (select udf(unique1) from tenk1 b join tenk1 c using (unique1)
+ where udf(udf(b.unique2)) = udf(42));
+
+--
+-- regression test: check for failure to generate a plan with multiple
+-- degenerate IN clauses
+--
+select udf(count(*)) from tenk1 x where
+ udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+ udf(x.unique1) = 0 and
+ udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1)));
+
+-- try that with GEQO too
+-- begin;
+-- set geqo = on;
+-- set geqo_threshold = 2;
+select udf(udf(count(*))) from tenk1 x where
+ udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+ udf(x.unique1) = 0 and
+ udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1)));
+-- rollback;
+
+-- Skip this test because table b inherits from table a and we do not support this feature, see inherits.sql
+--
+-- regression test: be sure we cope with proven-dummy append rels
+--
+-- explain (costs off)
+-- select aa, bb, unique1, unique1
+-- from tenk1 right join b on aa = unique1
+-- where bb < bb and bb is null;
+
+-- select aa, bb, unique1, unique1
+-- from tenk1 right join b on aa = unique1
+-- where bb < bb and bb is null;
+
+--
+-- regression test: check handling of empty-FROM subquery underneath outer join
+--
+-- explain (costs off)
+-- select * from int8_tbl i1 left join (int8_tbl i2 join
+-- (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2
+-- order by 1, 2;
+
+select * from int8_tbl i1 left join (int8_tbl i2 join
+ (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2))
+order by udf(udf(1)), 2;
+
+--
+-- regression test: check a case where join_clause_is_movable_into() gives
+-- an imprecise result, causing an assertion failure
+--
+select udf(count(*))
+from
+ (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2
+ from tenk1 t1
+ left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1)
+ join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss,
+ tenk1 t4,
+ tenk1 t5
+where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1));
+
+--
+-- regression test: check a case where we formerly missed including an EC
+-- enforcement clause because it was expected to be handled at scan level
+--
+-- explain (costs off)
+-- select a.f1, b.f1, t.thousand, t.tenthous from
+-- tenk1 t,
+-- (select sum(f1)+1 as f1 from int4_tbl i4a) a,
+-- (select sum(f1) as f1 from int4_tbl i4b) b
+-- where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous;
+
+select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from
+ tenk1 t,
+ (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a,
+ (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b
+where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous));
+
+--
+-- check a case where we formerly got confused by conflicting sort orders
+-- in redundant merge join path keys
+--
+-- explain (costs off)
+-- select * from
+-- j1_tbl full join
+-- (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl
+-- on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k;
+
+select * from
+ j1_tbl full join
+ (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl
+ on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k);
+
+--
+-- a different check for handling of redundant sort keys in merge joins
+--
+-- explain (costs off)
+-- select count(*) from
+-- (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x
+-- left join
+-- (select * from tenk1 y order by y.unique2) y
+-- on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2;
+
+select udf(count(*)) from
+ (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x
+ left join
+ (select * from tenk1 y order by udf(y.unique2)) y
+ on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2;
+
+
+--
+-- Clean up
+--
+
+DROP TABLE t1;
+DROP TABLE t2;
+DROP TABLE t3;
+
+DROP TABLE J1_TBL;
+DROP TABLE J2_TBL;
+
+-- Both DELETE and UPDATE allow the specification of additional tables
+-- to "join" against to determine which rows should be modified.
+
+-- CREATE TEMP TABLE t1 (a int, b int);
+-- CREATE TEMP TABLE t2 (a int, b int);
+-- CREATE TEMP TABLE t3 (x int, y int);
+
+-- INSERT INTO t1 VALUES (5, 10);
+-- INSERT INTO t1 VALUES (15, 20);
+-- INSERT INTO t1 VALUES (100, 100);
+-- INSERT INTO t1 VALUES (200, 1000);
+-- INSERT INTO t2 VALUES (200, 2000);
+-- INSERT INTO t3 VALUES (5, 20);
+-- INSERT INTO t3 VALUES (6, 7);
+-- INSERT INTO t3 VALUES (7, 8);
+-- INSERT INTO t3 VALUES (500, 100);
+
+-- DELETE FROM t3 USING t1 table1 WHERE t3.x = table1.a;
+-- SELECT * FROM t3;
+-- DELETE FROM t3 USING t1 JOIN t2 USING (a) WHERE t3.x > t1.a;
+-- SELECT * FROM t3;
+-- DELETE FROM t3 USING t3 t3_other WHERE t3.x = t3_other.x AND t3.y = t3_other.y;
+-- SELECT * FROM t3;
+
+-- Test join against inheritance tree
+
+-- create temp table t2a () inherits (t2);
+
+-- insert into t2a values (200, 2001);
+
+-- select * from t1 left join t2 on (t1.a = t2.a);
+
+-- Test matching of column name with wrong alias
+
+-- select t1.x from t1 join t3 on (t1.a = t3.x);
+
+--
+-- regression test for 8.1 merge right join bug
+--
+
+create or replace temporary view tt1 as select * from
+ (values (1, 11), (2, NULL))
+ as v(tt1_id, joincol);
+
+create or replace temporary view tt2 as select * from
+ (values (21, 11), (22, 11))
+ as v(tt2_id, joincol);
+
+-- set enable_hashjoin to off;
+-- set enable_nestloop to off;
+
+-- these should give the same results
+
+select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol);
+
+select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol));
+
+-- reset enable_hashjoin;
+-- reset enable_nestloop;
+
+--
+-- regression test for bug #13908 (hash join with skew tuples & nbatch increase)
+--
+
+-- set work_mem to '64kB';
+-- set enable_mergejoin to off;
+
+-- explain (costs off)
+-- select count(*) from tenk1 a, tenk1 b
+-- where a.hundred = b.thousand and (b.fivethous % 10) < 10;
+select udf(count(*)) from tenk1 a, tenk1 b
+ where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10);
+
+-- reset work_mem;
+-- reset enable_mergejoin;
+
+--
+-- regression test for 8.2 bug with improper re-ordering of left joins
+--
+
+DROP TABLE IF EXISTS tt3;
+CREATE TABLE tt3(f1 int, f2 string) USING parquet;
+INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x;
+-- create index tt3i on tt3(f1);
+-- analyze tt3;
+
+DROP TABLE IF EXISTS tt4;
+CREATE TABLE tt4(f1 int) USING parquet;
+INSERT INTO tt4 VALUES (0),(1),(9999);
+-- analyze tt4;
+
+SELECT udf(udf(a.f1)) as f1
+FROM tt4 a
+LEFT JOIN (
+ SELECT b.f1
+ FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1)
+ WHERE udf(c.f1) IS NULL
+) AS d ON udf(a.f1) = d.f1
+WHERE udf(udf(d.f1)) IS NULL;
+
+--
+-- regression test for proper handling of outer joins within antijoins
+--
+
+-- create temp table tt4x(c1 int, c2 int, c3 int);
+
+-- explain (costs off)
+-- select * from tt4x t1
+-- where not exists (
+-- select 1 from tt4x t2
+-- left join tt4x t3 on t2.c3 = t3.c1
+-- left join ( select t5.c1 as c1
+-- from tt4x t4 left join tt4x t5 on t4.c2 = t5.c1
+-- ) a1 on t3.c2 = a1.c1
+-- where t1.c1 = t2.c2
+-- );
+
+--
+-- regression test for problems of the sort depicted in bug #3494
+--
+
+create or replace temporary view tt5 as select * from
+ (values (1, 10), (1, 11))
+ as v(f1, f2);
+create or replace temporary view tt6 as select * from
+ (values (1, 9), (1, 2), (2, 9))
+ as v(f1, f2);
+
+select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2));
+
+--
+-- regression test for problems of the sort depicted in bug #3588
+--
+
+create or replace temporary view xx as select * from
+ (values (1), (2), (3))
+ as v(pkxx);
+create or replace temporary view yy as select * from
+ (values (101, 1), (201, 2), (301, NULL))
+ as v(pkyy, pkxx);
+
+select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy,
+ udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx
+from yy
+ left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy)
+ left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx))
+ left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx);
+
+--
+-- regression test for improper pushing of constants across outer-join clauses
+-- (as seen in early 8.2.x releases)
+--
+
+create or replace temporary view zt1 as select * from
+ (values (53))
+ as v(f1);
+create or replace temporary view zt2 as select * from
+ (values (53))
+ as v(f2);
+create or replace temporary view zt3(f3 int) using parquet;
+
+select * from
+ zt2 left join zt3 on (udf(f2) = udf(udf(f3)))
+ left join zt1 on (udf(udf(f3)) = udf(f1))
+where udf(f2) = 53;
+
+create temp view zv1 as select *,'dummy' AS junk from zt1;
+
+select * from
+ zt2 left join zt3 on (f2 = udf(f3))
+ left join zv1 on (udf(f3) = f1)
+where udf(udf(f2)) = 53;
+
+--
+-- regression test for improper extraction of OR indexqual conditions
+-- (as seen in early 8.3.x releases)
+--
+
+select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred)
+from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous)
+where udf(a.unique1) = 42 and
+ ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3)));
+
+--
+-- test proper positioning of one-time quals in EXISTS (8.4devel bug)
+--
+-- prepare foo(bool) as
+-- select count(*) from tenk1 a left join tenk1 b
+-- on (a.unique2 = b.unique1 and exists
+-- (select 1 from tenk1 c where c.thousand = b.unique2 and $1));
+-- execute foo(true);
+-- execute foo(false);
+
+--
+-- test for sane behavior with noncanonical merge clauses, per bug #4926
+--
+
+-- begin;
+
+-- set enable_mergejoin = 1;
+-- set enable_hashjoin = 0;
+-- set enable_nestloop = 0;
+
+create or replace temporary view a (i integer) using parquet;
+create or replace temporary view b (x integer, y integer) using parquet;
+
+select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i);
+
+-- rollback;
+
+--
+-- test handling of merge clauses using record_ops
+--
+-- begin;
+
+-- create type mycomptype as (id int, v bigint);
+
+-- create temp table tidv (idv mycomptype);
+-- create index on tidv (idv);
+
+-- explain (costs off)
+-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv;
+
+-- set enable_mergejoin = 0;
+
+-- explain (costs off)
+-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv;
+
+-- rollback;
+
+--
+-- test NULL behavior of whole-row Vars, per bug #5025
+--
+select udf(t1.q2), udf(count(t2.*))
+from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1)
+group by udf(t1.q2) order by 1;
+
+select udf(udf(t1.q2)), udf(count(t2.*))
+from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1))
+group by udf(udf(t1.q2)) order by 1;
+
+-- [SPARK-28330] Enhance query limit
+-- select t1.q2, count(t2.*)
+-- from int8_tbl t1 left join (select * from int8_tbl offset 0) t2 on (t1.q2 = t2.q1)
+-- group by t1.q2 order by 1;
+
+select udf(t1.q2) as q2, udf(udf(count(t2.*)))
+from int8_tbl t1 left join
+ (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2
+ on (udf(t1.q2) = udf(t2.q1))
+group by t1.q2 order by 1;
+
+--
+-- test incorrect failure to NULL pulled-up subexpressions
+--
+-- begin;
+create or replace temporary view a as select * from
+ (values ('p'), ('q'))
+ as v(code);
+create or replace temporary view b as select * from
+ (values ('p', 1), ('p', 2))
+ as v(a, num);
+create or replace temporary view c as select * from
+ (values ('A', 'p'), ('B', 'q'), ('C', null))
+ as v(name, a);
+
+select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const)
+from c left join
+ (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const
+ from a left join
+ (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp
+ on udf(a.code) = udf(udf(b_grp.a))
+ ) as ss
+ on (udf(udf(c.a)) = udf(ss.code))
+order by c.name;
+
+-- rollback;
+
+--
+-- test incorrect handling of placeholders that only appear in targetlists,
+-- per bug #6154
+--
+SELECT * FROM
+( SELECT 1 as key1 ) sub1
+LEFT JOIN
+( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM
+ ( SELECT 1 as key3 ) sub3
+ LEFT JOIN
+ ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM
+ ( SELECT 1 as key5 ) sub5
+ LEFT JOIN
+ ( SELECT 2 as key6, 42 as value1 ) sub6
+ ON sub5.key5 = udf(sub6.key6)
+ ) sub4
+ ON udf(sub4.key5) = sub3.key3
+) sub2
+ON udf(udf(sub1.key1)) = udf(udf(sub2.key3));
+
+-- test the path using join aliases, too
+SELECT * FROM
+( SELECT 1 as key1 ) sub1
+LEFT JOIN
+( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM
+ ( SELECT 1 as key3 ) sub3
+ LEFT JOIN
+ ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM
+ ( SELECT 1 as key5 ) sub5
+ LEFT JOIN
+ ( SELECT 2 as key6, 42 as value1 ) sub6
+ ON udf(udf(sub5.key5)) = sub6.key6
+ ) sub4
+ ON sub4.key5 = sub3.key3
+) sub2
+ON sub1.key1 = udf(udf(sub2.key3));
+
+--
+-- test case where a PlaceHolderVar is used as a nestloop parameter
+--
+
+-- EXPLAIN (COSTS OFF)
+-- SELECT qq, unique1
+-- FROM
+-- ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1
+-- FULL OUTER JOIN
+-- ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2
+-- USING (qq)
+-- INNER JOIN tenk1 c ON qq = unique2;
+
+SELECT udf(qq), udf(udf(unique1))
+ FROM
+ ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1
+ FULL OUTER JOIN
+ ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2
+ USING (qq)
+ INNER JOIN tenk1 c ON udf(qq) = udf(unique2);
+
+--
+-- nested nestloops can require nested PlaceHolderVars
+--
+
+create or replace temporary view nt1 as select * from
+ (values(1,true,true), (2,true,false), (3,false,false))
+ as v(id, a1, a2);
+create or replace temporary view nt2 as select * from
+ (values(1,1,true,true), (2,2,true,false), (3,3,false,false))
+ as v(id, nt1_id, b1, b2);
+create or replace temporary view nt3 as select * from
+ (values(1,1,true), (2,2,false), (3,3,true))
+ as v(id, nt2_id, c1);
+-- explain (costs off)
+-- select nt3.id
+-- from nt3 as nt3
+-- left join
+-- (select nt2.*, (nt2.b1 and ss1.a3) AS b3
+-- from nt2 as nt2
+-- left join
+-- (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1
+-- on ss1.id = nt2.nt1_id
+-- ) as ss2
+-- on ss2.id = nt3.nt2_id
+-- where nt3.id = 1 and ss2.b3;
+
+select udf(nt3.id)
+from nt3 as nt3
+ left join
+ (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3
+ from nt2 as nt2
+ left join
+ (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1
+ on ss1.id = udf(udf(nt2.nt1_id))
+ ) as ss2
+ on udf(ss2.id) = nt3.nt2_id
+where udf(nt3.id) = 1 and udf(ss2.b3);
+
+-- [SPARK-28379] Correlated scalar subqueries must be aggregated
+--
+-- test case where a PlaceHolderVar is propagated into a subquery
+--
+
+-- explain (costs off)
+-- select * from
+-- int8_tbl t1 left join
+-- (select q1 as x, 42 as y from int8_tbl t2) ss
+-- on t1.q2 = ss.x
+-- where
+-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1)
+-- order by 1,2;
+
+-- select * from
+-- int8_tbl t1 left join
+-- (select q1 as x, 42 as y from int8_tbl t2) ss
+-- on t1.q2 = ss.x
+-- where
+-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1)
+-- order by 1,2;
+
+--
+-- test the corner cases FULL JOIN ON TRUE and FULL JOIN ON FALSE
+--
+select * from int4_tbl a full join int4_tbl b on true;
+select * from int4_tbl a full join int4_tbl b on false;
+
+--
+-- test for ability to use a cartesian join when necessary
+--
+
+-- explain (costs off)
+-- select * from
+-- tenk1 join int4_tbl on f1 = twothousand,
+-- int4(sin(1)) q1,
+-- int4(sin(0)) q2
+-- where q1 = thousand or q2 = thousand;
+
+-- explain (costs off)
+-- select * from
+-- tenk1 join int4_tbl on f1 = twothousand,
+-- int4(sin(1)) q1,
+-- int4(sin(0)) q2
+-- where thousand = (q1 + q2);
+
+--
+-- test ability to generate a suitable plan for a star-schema query
+--
+
+-- explain (costs off)
+-- select * from
+-- tenk1, int8_tbl a, int8_tbl b
+-- where thousand = a.q1 and tenthous = b.q1 and a.q2 = 1 and b.q2 = 2;
+
+--
+-- test a corner case in which we shouldn't apply the star-schema optimization
+--
+
+-- explain (costs off)
+-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from
+-- tenk1 t1
+-- inner join int4_tbl i1
+-- left join (select v1.x2, v2.y1, 11 AS d1
+-- from (select 1,0 from onerow) v1(x1,x2)
+-- left join (select 3,1 from onerow) v2(y1,y2)
+-- on v1.x1 = v2.y2) subq1
+-- on (i1.f1 = subq1.x2)
+-- on (t1.unique2 = subq1.d1)
+-- left join tenk1 t2
+-- on (subq1.y1 = t2.unique1)
+-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2;
+
+-- [SPARK-20856] support statement using nested joins
+-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from
+-- tenk1 t1
+-- inner join int4_tbl i1
+-- left join (select v1.x2, v2.y1, 11 AS d1
+-- from (select 1,0 from onerow) v1(x1,x2)
+-- left join (select 3,1 from onerow) v2(y1,y2)
+-- on v1.x1 = v2.y2) subq1
+-- on (i1.f1 = subq1.x2)
+-- on (t1.unique2 = subq1.d1)
+-- left join tenk1 t2
+-- on (subq1.y1 = t2.unique1)
+-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2;
+
+-- variant that isn't quite a star-schema case
+
+-- [SPARK-16452] basic INFORMATION_SCHEMA support
+-- select ss1.d1 from
+-- tenk1 as t1
+-- inner join tenk1 as t2
+-- on t1.tenthous = t2.ten
+-- inner join
+-- int8_tbl as i8
+-- left join int4_tbl as i4
+-- inner join (select 64::information_schema.cardinal_number as d1
+-- from tenk1 t3,
+-- lateral (select abs(t3.unique1) + random()) ss0(x)
+-- where t3.fivethous < 0) as ss1
+-- on i4.f1 = ss1.d1
+-- on i8.q1 = i4.f1
+-- on t1.tenthous = ss1.d1
+-- where t1.unique1 < i4.f1;
+
+-- this variant is foldable by the remove-useless-RESULT-RTEs code
+
+-- explain (costs off)
+-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from
+-- tenk1 t1
+-- inner join int4_tbl i1
+-- left join (select v1.x2, v2.y1, 11 AS d1
+-- from (values(1,0)) v1(x1,x2)
+-- left join (values(3,1)) v2(y1,y2)
+-- on v1.x1 = v2.y2) subq1
+-- on (i1.f1 = subq1.x2)
+-- on (t1.unique2 = subq1.d1)
+-- left join tenk1 t2
+-- on (subq1.y1 = t2.unique1)
+-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2;
+
+-- [SPARK-20856] support statement using nested joins
+-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from
+-- tenk1 t1
+-- inner join int4_tbl i1
+-- left join (select v1.x2, v2.y1, 11 AS d1
+-- from (values(1,0)) v1(x1,x2)
+-- left join (values(3,1)) v2(y1,y2)
+-- on v1.x1 = v2.y2) subq1
+-- on (i1.f1 = subq1.x2)
+-- on (t1.unique2 = subq1.d1)
+-- left join tenk1 t2
+-- on (subq1.y1 = t2.unique1)
+-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2;
+
+--
+-- test extraction of restriction OR clauses from join OR clause
+-- (we used to only do this for indexable clauses)
+--
+
+-- explain (costs off)
+-- select * from tenk1 a join tenk1 b on
+-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.hundred = 4);
+-- explain (costs off)
+-- select * from tenk1 a join tenk1 b on
+-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.ten = 4);
+-- explain (costs off)
+-- select * from tenk1 a join tenk1 b on
+-- (a.unique1 = 1 and b.unique1 = 2) or
+-- ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4);
+
+--
+-- test placement of movable quals in a parameterized join tree
+--
+
+-- explain (costs off)
+-- select * from tenk1 t1 left join
+-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2)
+-- on t1.hundred = t2.hundred and t1.ten = t3.ten
+-- where t1.unique1 = 1;
+
+-- explain (costs off)
+-- select * from tenk1 t1 left join
+-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2)
+-- on t1.hundred = t2.hundred and t1.ten + t2.ten = t3.ten
+-- where t1.unique1 = 1;
+
+-- explain (costs off)
+-- select count(*) from
+-- tenk1 a join tenk1 b on a.unique1 = b.unique2
+-- left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand
+-- join int4_tbl on b.thousand = f1;
+
+select udf(count(*)) from
+ tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+ left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand))
+ join int4_tbl on udf(b.thousand) = f1;
+
+-- explain (costs off)
+-- select b.unique1 from
+-- tenk1 a join tenk1 b on a.unique1 = b.unique2
+-- left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand
+-- join int4_tbl i1 on b.thousand = f1
+-- right join int4_tbl i2 on i2.f1 = b.tenthous
+-- order by 1;
+
+select udf(b.unique1) from
+ tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+ left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand)
+ join int4_tbl i1 on udf(b.thousand) = udf(udf(f1))
+ right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous)
+ order by udf(1);
+
+-- explain (costs off)
+-- select * from
+-- (
+-- select unique1, q1, coalesce(unique1, -1) + q1 as fault
+-- from int8_tbl left join tenk1 on (q2 = unique2)
+-- ) ss
+-- where fault = 122
+-- order by fault;
+
+select * from
+(
+ select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault
+ from int8_tbl left join tenk1 on (udf(q2) = udf(unique2))
+) ss
+where udf(fault) = udf(122)
+order by udf(fault);
+
+-- explain (costs off)
+-- select * from
+-- (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys)
+-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x
+-- left join unnest(v1ys) as u1(u1y) on u1y = v2y;
+
+-- [SPARK-28382] Array Functions: unnest
+-- select * from
+-- (values (1, array(10,20)), (2, array(20,30))) as v1(v1x,v1ys)
+-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x
+-- left join unnest(v1ys) as u1(u1y) on u1y = v2y;
+
+--
+-- test handling of potential equivalence clauses above outer joins
+--
+
+-- explain (costs off)
+-- select q1, unique2, thousand, hundred
+-- from int8_tbl a left join tenk1 b on q1 = unique2
+-- where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123);
+
+select udf(q1), udf(unique2), udf(thousand), udf(hundred)
+ from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2)
+ where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123)));
+
+-- explain (costs off)
+-- select f1, unique2, case when unique2 is null then f1 else 0 end
+-- from int4_tbl a left join tenk1 b on f1 = unique2
+-- where (case when unique2 is null then f1 else 0 end) = 0;
+
+select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end
+ from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2))
+ where (case when udf(unique2) is null then udf(f1) else 0 end) = 0;
+
+--
+-- another case with equivalence clauses above outer joins (bug #8591)
+--
+
+-- explain (costs off)
+-- select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand)
+-- from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand)
+-- where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44;
+
+select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand))
+ from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand))
+ where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44);
+
+--
+-- check handling of join aliases when flattening multiple levels of subquery
+--
+
+-- explain (verbose, costs off)
+-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from
+-- (values (0),(1)) foo1(join_key)
+-- left join
+-- (select join_key, bug_field from
+-- (select ss1.join_key, ss1.bug_field from
+-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1
+-- ) foo2
+-- left join
+-- (select unique2 as join_key from tenk1 i2) ss2
+-- using (join_key)
+-- ) foo3
+-- using (join_key);
+
+
+-- [SPARK-28377] Fully support correlation names in the FROM clause
+-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from
+-- (values (0),(1)) foo1(join_key)
+-- left join
+-- (select join_key, bug_field from
+-- (select ss1.join_key, ss1.bug_field from
+-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1
+-- ) foo2
+-- left join
+-- (select unique2 as join_key from tenk1 i2) ss2
+-- using (join_key)
+-- ) foo3
+-- using (join_key);
+
+-- [SPARK-20856] Support statement using nested joins
+--
+-- test successful handling of nested outer joins with degenerate join quals
+--
+
+-- explain (verbose, costs off)
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, null::int as d2 from int8_tbl i8b2) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, int(null) as d2 from int8_tbl i8b2) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- explain (verbose, costs off)
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- explain (verbose, costs off)
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2
+-- where q1 = f1) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- select t1.* from
+-- text_tbl t1
+-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1
+-- left join int8_tbl i8
+-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2
+-- where q1 = f1) b2
+-- on (i8.q1 = b2.q1)
+-- on (b2.d2 = b1.q2)
+-- on (t1.f1 = b1.d1)
+-- left join int4_tbl i4
+-- on (i8.q2 = i4.f1);
+
+-- explain (verbose, costs off)
+-- select * from
+-- text_tbl t1
+-- inner join int8_tbl i8
+-- on i8.q2 = 456
+-- right join text_tbl t2
+-- on t1.f1 = 'doh!'
+-- left join int4_tbl i4
+-- on i8.q1 = i4.f1;
+
+select * from
+ text_tbl t1
+ inner join int8_tbl i8
+ on udf(i8.q2) = udf(udf(456))
+ right join text_tbl t2
+ on udf(t1.f1) = udf(udf('doh!'))
+ left join int4_tbl i4
+ on udf(udf(i8.q1)) = i4.f1;
+
+-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491)
+--
+-- test for appropriate join order in the presence of lateral references
+--
+
+-- explain (verbose, costs off)
+-- select * from
+-- text_tbl t1
+-- left join int8_tbl i8
+-- on i8.q2 = 123,
+-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss
+-- where t1.f1 = ss.f1;
+
+-- select * from
+-- text_tbl t1
+-- left join int8_tbl i8
+-- on i8.q2 = 123,
+-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss
+-- where t1.f1 = ss.f1;
+
+-- explain (verbose, costs off)
+-- select * from
+-- text_tbl t1
+-- left join int8_tbl i8
+-- on i8.q2 = 123,
+-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1,
+-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2
+-- where t1.f1 = ss2.f1;
+
+-- select * from
+-- text_tbl t1
+-- left join int8_tbl i8
+-- on i8.q2 = 123,
+-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1,
+-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2
+-- where t1.f1 = ss2.f1;
+
+-- explain (verbose, costs off)
+-- select 1 from
+-- text_tbl as tt1
+-- inner join text_tbl as tt2 on (tt1.f1 = 'foo')
+-- left join text_tbl as tt3 on (tt3.f1 = 'foo')
+-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1),
+-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1
+-- where tt1.f1 = ss1.c0;
+
+-- select 1 from
+-- text_tbl as tt1
+-- inner join text_tbl as tt2 on (tt1.f1 = 'foo')
+-- left join text_tbl as tt3 on (tt3.f1 = 'foo')
+-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1),
+-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1
+-- where tt1.f1 = ss1.c0;
+
+--
+-- check a case in which a PlaceHolderVar forces join order
+--
+
+-- explain (verbose, costs off)
+-- select ss2.* from
+-- int4_tbl i41
+-- left join int8_tbl i8
+-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3
+-- from int4_tbl i42, int4_tbl i43) ss1
+-- on i8.q1 = ss1.c2
+-- on i41.f1 = ss1.c1,
+-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2
+-- where ss1.c2 = 0;
+
+-- select ss2.* from
+-- int4_tbl i41
+-- left join int8_tbl i8
+-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3
+-- from int4_tbl i42, int4_tbl i43) ss1
+-- on i8.q1 = ss1.c2
+-- on i41.f1 = ss1.c1,
+-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2
+-- where ss1.c2 = 0;
+
+--
+-- test successful handling of full join underneath left join (bug #14105)
+--
+
+-- explain (costs off)
+-- select * from
+-- (select 1 as id) as xx
+-- left join
+-- (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id))
+-- on (xx.id = coalesce(yy.id));
+
+select * from
+ (select udf(udf(1)) as id) as xx
+ left join
+ (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id)))
+ on (xx.id = udf(udf(coalesce(yy.id))));
+
+--
+-- test ability to push constants through outer join clauses
+--
+
+-- explain (costs off)
+-- select * from int4_tbl a left join tenk1 b on f1 = unique2 where f1 = 0;
+
+-- explain (costs off)
+-- select * from tenk1 a full join tenk1 b using(unique2) where unique2 = 42;
+
+--
+-- test that quals attached to an outer join have correct semantics,
+-- specifically that they don't re-use expressions computed below the join;
+-- we force a mergejoin so that coalesce(b.q1, 1) appears as a join input
+--
+
+-- set enable_hashjoin to off;
+-- set enable_nestloop to off;
+
+-- explain (verbose, costs off)
+-- select a.q2, b.q1
+-- from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1)
+-- where coalesce(b.q1, 1) > 0;
+select udf(a.q2), udf(b.q1)
+ from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1)
+ where udf(udf(coalesce(b.q1, 1)) > 0);
+
+-- reset enable_hashjoin;
+-- reset enable_nestloop;
+
+-- Skip these test because it only test explain
+--
+-- test join removal
+--
+
+-- begin;
+
+-- CREATE TEMP TABLE a (id int PRIMARY KEY, b_id int);
+-- CREATE TEMP TABLE b (id int PRIMARY KEY, c_id int);
+-- CREATE TEMP TABLE c (id int PRIMARY KEY);
+-- CREATE TEMP TABLE d (a int, b int);
+-- INSERT INTO a VALUES (0, 0), (1, NULL);
+-- INSERT INTO b VALUES (0, 0), (1, NULL);
+-- INSERT INTO c VALUES (0), (1);
+-- INSERT INTO d VALUES (1,3), (2,2), (3,1);
+
+-- all three cases should be optimizable into a simple seqscan
+-- explain (costs off) SELECT a.* FROM a LEFT JOIN b ON a.b_id = b.id;
+-- explain (costs off) SELECT b.* FROM b LEFT JOIN c ON b.c_id = c.id;
+-- explain (costs off)
+-- SELECT a.* FROM a LEFT JOIN (b left join c on b.c_id = c.id)
+-- ON (a.b_id = b.id);
+
+-- check optimization of outer join within another special join
+-- explain (costs off)
+-- select id from a where id in (
+-- select b.id from b left join c on b.id = c.id
+-- );
+
+-- check that join removal works for a left join when joining a subquery
+-- that is guaranteed to be unique by its GROUP BY clause
+-- explain (costs off)
+-- select d.* from d left join (select * from b group by b.id, b.c_id) s
+-- on d.a = s.id and d.b = s.c_id;
+
+-- similarly, but keying off a DISTINCT clause
+-- explain (costs off)
+-- select d.* from d left join (select distinct * from b) s
+-- on d.a = s.id and d.b = s.c_id;
+
+-- join removal is not possible when the GROUP BY contains a column that is
+-- not in the join condition. (Note: as of 9.6, we notice that b.id is a
+-- primary key and so drop b.c_id from the GROUP BY of the resulting plan;
+-- but this happens too late for join removal in the outer plan level.)
+-- explain (costs off)
+-- select d.* from d left join (select * from b group by b.id, b.c_id) s
+-- on d.a = s.id;
+
+-- similarly, but keying off a DISTINCT clause
+-- explain (costs off)
+-- select d.* from d left join (select distinct * from b) s
+-- on d.a = s.id;
+
+-- check join removal works when uniqueness of the join condition is enforced
+-- by a UNION
+-- explain (costs off)
+-- select d.* from d left join (select id from a union select id from b) s
+-- on d.a = s.id;
+
+-- check join removal with a cross-type comparison operator
+-- explain (costs off)
+-- select i8.* from int8_tbl i8 left join (select f1 from int4_tbl group by f1) i4
+-- on i8.q1 = i4.f1;
+
+-- check join removal with lateral references
+-- explain (costs off)
+-- select 1 from (select a.id FROM a left join b on a.b_id = b.id) q,
+-- lateral generate_series(1, q.id) gs(i) where q.id = gs.i;
+
+-- rollback;
+
+create or replace temporary view parent as select * from
+ (values (1, 10), (2, 20), (3, 30))
+ as v(k, pd);
+create or replace temporary view child as select * from
+ (values (1, 100), (4, 400))
+ as v(k, cd);
+
+-- this case is optimizable
+select p.* from parent p left join child c on (udf(p.k) = udf(c.k));
+-- explain (costs off)
+-- select p.* from parent p left join child c on (p.k = c.k);
+
+-- this case is not
+select p.*, linked from parent p
+ left join (select c.*, udf(udf(true)) as linked from child c) as ss
+ on (udf(p.k) = udf(udf(ss.k)));
+-- explain (costs off)
+-- select p.*, linked from parent p
+-- left join (select c.*, true as linked from child c) as ss
+-- on (p.k = ss.k);
+
+-- check for a 9.0rc1 bug: join removal breaks pseudoconstant qual handling
+select p.* from
+ parent p left join child c on (udf(p.k) = c.k)
+ where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2));
+-- explain (costs off)
+-- select p.* from
+-- parent p left join child c on (p.k = c.k)
+-- where p.k = 1 and p.k = 2;
+
+select p.* from
+ (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k)
+ where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2));
+-- explain (costs off)
+-- select p.* from
+-- (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k
+-- where p.k = 1 and p.k = 2;
+
+-- bug 5255: this is not optimizable by join removal
+-- begin;
+
+create or replace temporary view a as select * from
+ (values (0), (1))
+ as v(id);
+create or replace temporary view b as select * from
+ (values (0, 0), (1, NULL))
+ as v(id, a_id);
+
+SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0);
+SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0);
+
+-- rollback;
+
+-- another join removal bug: this is not optimizable, either
+-- begin;
+
+create or replace temporary view innertab as select * from
+ (values (123L, 42L))
+ as v(id, dat1);
+
+SELECT * FROM
+ (SELECT udf(1) AS x) ss1
+ LEFT JOIN
+ (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y
+ FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2
+ ON true;
+
+-- rollback;
+
+-- another join removal bug: we must clean up correctly when removing a PHV
+-- begin;
+
+-- create temp table uniquetbl (f1 text unique);
+
+-- explain (costs off)
+-- select t1.* from
+-- uniquetbl as t1
+-- left join (select *, '***'::text as d1 from uniquetbl) t2
+-- on t1.f1 = t2.f1
+-- left join uniquetbl t3
+-- on t2.d1 = t3.f1;
+
+-- explain (costs off)
+-- select t0.*
+-- from
+-- text_tbl t0
+-- left join
+-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1,
+-- t1.stringu2
+-- from tenk1 t1
+-- join int4_tbl i4 ON i4.f1 = t1.unique2
+-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss
+-- on t0.f1 = ss.case1
+-- where ss.stringu2 !~* ss.case1;
+
+-- [SPARK-27987] Support POSIX Regular Expressions
+-- select t0.*
+-- from
+-- text_tbl t0
+-- left join
+-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1,
+-- t1.stringu2
+-- from tenk1 t1
+-- join int4_tbl i4 ON i4.f1 = t1.unique2
+-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss
+-- on t0.f1 = ss.case1
+-- where ss.stringu2 !~* ss.case1;
+
+-- rollback;
+
+-- bug #8444: we've historically allowed duplicate aliases within aliased JOINs
+
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1); -- error
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1); -- error
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)); -- ok
+
+--
+-- Test hints given on incorrect column references are useful
+--
+
+select udf(t1.uunique1) from
+ tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two); -- error, prefer "t1" suggestion
+select udf(udf(t2.uunique1)) from
+ tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two; -- error, prefer "t2" suggestion
+select udf(uunique1) from
+ tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two); -- error, suggest both at once
+
+-- Skip this test because it is a PostgreSQL specific case
+--
+-- Take care to reference the correct RTE
+--
+
+-- -- select atts.relid::regclass, s.* from pg_stats s join
+-- -- pg_attribute a on s.attname = a.attname and s.tablename =
+-- -- a.attrelid::regclass::text join (select unnest(indkey) attnum,
+-- -- indexrelid from pg_index i) atts on atts.attnum = a.attnum where
+-- schemaname != 'pg_catalog';
+
+-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491)
+--
+-- Test LATERAL
+--
+
+-- select unique2, x.*
+-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x;
+-- explain (costs off)
+-- select unique2, x.*
+-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x;
+-- select unique2, x.*
+-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss;
+-- explain (costs off)
+-- select unique2, x.*
+-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss;
+-- explain (costs off)
+-- select unique2, x.*
+-- from int4_tbl x cross join lateral (select unique2 from tenk1 where f1 = unique1) ss;
+-- select unique2, x.*
+-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true;
+-- explain (costs off)
+-- select unique2, x.*
+-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true;
+
+-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491)
+-- check scoping of lateral versus parent references
+-- the first of these should return int8_tbl.q2, the second int8_tbl.q1
+-- select *, (select r from (select q1 as q2) x, (select q2 as r) y) from int8_tbl;
+-- select *, (select r from (select q1 as q2) x, lateral (select q2 as r) y) from int8_tbl;
+
+-- lateral with function in FROM
+-- select count(*) from tenk1 a, lateral generate_series(1,two) g;
+-- explain (costs off)
+-- select count(*) from tenk1 a, lateral generate_series(1,two) g;
+-- explain (costs off)
+-- select count(*) from tenk1 a cross join lateral generate_series(1,two) g;
+-- don't need the explicit LATERAL keyword for functions
+-- explain (costs off)
+-- select count(*) from tenk1 a, generate_series(1,two) g;
+
+-- lateral with UNION ALL subselect
+-- explain (costs off)
+-- select * from generate_series(100,200) g,
+-- lateral (select * from int8_tbl a where g = q1 union all
+-- select * from int8_tbl b where g = q2) ss;
+-- select * from generate_series(100,200) g,
+-- lateral (select * from int8_tbl a where g = q1 union all
+-- select * from int8_tbl b where g = q2) ss;
+
+-- lateral with VALUES
+-- explain (costs off)
+-- select count(*) from tenk1 a,
+-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x;
+-- select count(*) from tenk1 a,
+-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x;
+
+-- lateral with VALUES, no flattening possible
+-- explain (costs off)
+-- select count(*) from tenk1 a,
+-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x;
+-- select count(*) from tenk1 a,
+-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x;
+
+-- lateral injecting a strange outer join condition
+-- explain (costs off)
+-- select * from int8_tbl a,
+-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z)
+-- on x.q2 = ss.z
+-- order by a.q1, a.q2, x.q1, x.q2, ss.z;
+-- select * from int8_tbl a,
+-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z)
+-- on x.q2 = ss.z
+-- order by a.q1, a.q2, x.q1, x.q2, ss.z;
+
+-- lateral reference to a join alias variable
+-- select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1,
+-- lateral (select x) ss2(y);
+-- select * from (select f1 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1,
+-- lateral (values(x)) ss2(y);
+-- select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j,
+-- lateral (select x) ss2(y);
+
+-- lateral references requiring pullup
+-- select * from (values(1)) x(lb),
+-- lateral generate_series(lb,4) x4;
+-- select * from (select f1/1000000000 from int4_tbl) x(lb),
+-- lateral generate_series(lb,4) x4;
+-- select * from (values(1)) x(lb),
+-- lateral (values(lb)) y(lbcopy);
+-- select * from (values(1)) x(lb),
+-- lateral (select lb from int4_tbl) y(lbcopy);
+-- select * from
+-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1,
+-- lateral (values(x.q1,y.q1,y.q2)) v(xq1,yq1,yq2);
+-- select * from
+-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1,
+-- lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2);
+-- select x.* from
+-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1,
+-- lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2);
+-- select v.* from
+-- (int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1)
+-- left join int4_tbl z on z.f1 = x.q2,
+-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy);
+-- select v.* from
+-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1)
+-- left join int4_tbl z on z.f1 = x.q2,
+-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy);
+-- select v.* from
+-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1)
+-- left join int4_tbl z on z.f1 = x.q2,
+-- lateral (select x.q1,y.q1 from onerow union all select x.q2,y.q2 from onerow) v(vx,vy);
+
+-- explain (verbose, costs off)
+-- select * from
+-- int8_tbl a left join
+-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1;
+-- select * from
+-- int8_tbl a left join
+-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1;
+-- explain (verbose, costs off)
+-- select * from
+-- int8_tbl a left join
+-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1;
+-- select * from
+-- int8_tbl a left join
+-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1;
+
+-- lateral can result in join conditions appearing below their
+-- real semantic level
+-- explain (verbose, costs off)
+-- select * from int4_tbl i left join
+-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true;
+-- select * from int4_tbl i left join
+-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true;
+-- explain (verbose, costs off)
+-- select * from int4_tbl i left join
+-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true;
+-- select * from int4_tbl i left join
+-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true;
+-- explain (verbose, costs off)
+-- select * from int4_tbl a,
+-- lateral (
+-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2)
+-- ) ss;
+-- select * from int4_tbl a,
+-- lateral (
+-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2)
+-- ) ss;
+
+-- lateral reference in a PlaceHolderVar evaluated at join level
+-- explain (verbose, costs off)
+-- select * from
+-- int8_tbl a left join lateral
+-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from
+-- int8_tbl b cross join int8_tbl c) ss
+-- on a.q2 = ss.bq1;
+-- select * from
+-- int8_tbl a left join lateral
+-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from
+-- int8_tbl b cross join int8_tbl c) ss
+-- on a.q2 = ss.bq1;
+
+-- case requiring nested PlaceHolderVars
+-- explain (verbose, costs off)
+-- select * from
+-- int8_tbl c left join (
+-- int8_tbl a left join (select q1, coalesce(q2,42) as x from int8_tbl b) ss1
+-- on a.q2 = ss1.q1
+-- cross join
+-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2
+-- ) on c.q2 = ss2.q1,
+-- lateral (select ss2.y offset 0) ss3;
+
+-- case that breaks the old ph_may_need optimization
+-- explain (verbose, costs off)
+-- select c.*,a.*,ss1.q1,ss2.q1,ss3.* from
+-- int8_tbl c left join (
+-- int8_tbl a left join
+-- (select q1, coalesce(q2,f1) as x from int8_tbl b, int4_tbl b2
+-- where q1 < f1) ss1
+-- on a.q2 = ss1.q1
+-- cross join
+-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2
+-- ) on c.q2 = ss2.q1,
+-- lateral (select * from int4_tbl i where ss2.y > f1) ss3;
+
+-- check processing of postponed quals (bug #9041)
+-- explain (verbose, costs off)
+-- select * from
+-- (select 1 as x offset 0) x cross join (select 2 as y offset 0) y
+-- left join lateral (
+-- select * from (select 3 as z offset 0) z where z.z = x.x
+-- ) zz on zz.z = y.y;
+
+-- check dummy rels with lateral references (bug #15694)
+-- explain (verbose, costs off)
+-- select * from int8_tbl i8 left join lateral
+-- (select *, i8.q2 from int4_tbl where false) ss on true;
+-- explain (verbose, costs off)
+-- select * from int8_tbl i8 left join lateral
+-- (select *, i8.q2 from int4_tbl i1, int4_tbl i2 where false) ss on true;
+
+-- check handling of nested appendrels inside LATERAL
+-- select * from
+-- ((select 2 as v) union all (select 3 as v)) as q1
+-- cross join lateral
+-- ((select * from
+-- ((select 4 as v) union all (select 5 as v)) as q3)
+-- union all
+-- (select q1.v)
+-- ) as q2;
+
+-- check we don't try to do a unique-ified semijoin with LATERAL
+-- explain (verbose, costs off)
+-- select * from
+-- (values (0,9998), (1,1000)) v(id,x),
+-- lateral (select f1 from int4_tbl
+-- where f1 = any (select unique1 from tenk1
+-- where unique2 = v.x offset 0)) ss;
+-- select * from
+-- (values (0,9998), (1,1000)) v(id,x),
+-- lateral (select f1 from int4_tbl
+-- where f1 = any (select unique1 from tenk1
+-- where unique2 = v.x offset 0)) ss;
+
+-- check proper extParam/allParam handling (this isn't exactly a LATERAL issue,
+-- but we can make the test case much more compact with LATERAL)
+-- explain (verbose, costs off)
+-- select * from (values (0), (1)) v(id),
+-- lateral (select * from int8_tbl t1,
+-- lateral (select * from
+-- (select * from int8_tbl t2
+-- where q1 = any (select q2 from int8_tbl t3
+-- where q2 = (select greatest(t1.q1,t2.q2))
+-- and (select v.id=0)) offset 0) ss2) ss
+-- where t1.q1 = ss.q2) ss0;
+
+-- select * from (values (0), (1)) v(id),
+-- lateral (select * from int8_tbl t1,
+-- lateral (select * from
+-- (select * from int8_tbl t2
+-- where q1 = any (select q2 from int8_tbl t3
+-- where q2 = (select greatest(t1.q1,t2.q2))
+-- and (select v.id=0)) offset 0) ss2) ss
+-- where t1.q1 = ss.q2) ss0;
+
+-- test some error cases where LATERAL should have been used but wasn't
+select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss;
+select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss;
+select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss;
+select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss;
+-- SQL:2008 says the left table is in scope but illegal to access here
+-- select f1,g from int4_tbl a right join lateral generate_series(0, a.f1) g on true;
+-- select f1,g from int4_tbl a full join lateral generate_series(0, a.f1) g on true;
+-- check we complain about ambiguous table references
+-- select * from
+-- int8_tbl x cross join (int4_tbl x cross join lateral (select x.f1) ss);
+-- LATERAL can be used to put an aggregate into the FROM clause of its query
+-- select 1 from tenk1 a, lateral (select max(a.unique1) from int4_tbl b) ss;
+
+-- check behavior of LATERAL in UPDATE/DELETE
+
+-- create temp table xx1 as select f1 as x1, -f1 as x2 from int4_tbl;
+
+-- error, can't do this:
+-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = x1) ss;
+-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = xx1.x1) ss;
+-- can't do it even with LATERAL:
+-- update xx1 set x2 = f1 from lateral (select * from int4_tbl where f1 = x1) ss;
+-- we might in future allow something like this, but for now it's an error:
+-- update xx1 set x2 = f1 from xx1, lateral (select * from int4_tbl where f1 = x1) ss;
+
+-- also errors:
+-- delete from xx1 using (select * from int4_tbl where f1 = x1) ss;
+-- delete from xx1 using (select * from int4_tbl where f1 = xx1.x1) ss;
+-- delete from xx1 using lateral (select * from int4_tbl where f1 = x1) ss;
+
+-- [SPARK-25411] Implement range partition in Spark
+--
+-- test LATERAL reference propagation down a multi-level inheritance hierarchy
+-- produced for a multi-level partitioned table hierarchy.
+--
+-- create table join_pt1 (a int, b int, c varchar) partition by range(a);
+-- create table join_pt1p1 partition of join_pt1 for values from (0) to (100) partition by range(b);
+-- create table join_pt1p2 partition of join_pt1 for values from (100) to (200);
+-- create table join_pt1p1p1 partition of join_pt1p1 for values from (0) to (100);
+-- insert into join_pt1 values (1, 1, 'x'), (101, 101, 'y');
+-- create table join_ut1 (a int, b int, c varchar);
+-- insert into join_ut1 values (101, 101, 'y'), (2, 2, 'z');
+-- explain (verbose, costs off)
+-- select t1.b, ss.phv from join_ut1 t1 left join lateral
+-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv
+-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss
+-- on t1.a = ss.t2a order by t1.a;
+-- select t1.b, ss.phv from join_ut1 t1 left join lateral
+-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv
+-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss
+-- on t1.a = ss.t2a order by t1.a;
+--
+-- drop table join_pt1;
+-- drop table join_ut1;
+--
+-- test that foreign key join estimation performs sanely for outer joins
+--
+
+-- begin;
+
+-- create table fkest (a int, b int, c int unique, primary key(a,b));
+-- create table fkest1 (a int, b int, primary key(a,b));
+
+-- insert into fkest select x/10, x%10, x from generate_series(1,1000) x;
+-- insert into fkest1 select x/10, x%10 from generate_series(1,1000) x;
+
+-- alter table fkest1
+-- add constraint fkest1_a_b_fkey foreign key (a,b) references fkest;
+
+-- analyze fkest;
+-- analyze fkest1;
+
+-- explain (costs off)
+-- select *
+-- from fkest f
+-- left join fkest1 f1 on f.a = f1.a and f.b = f1.b
+-- left join fkest1 f2 on f.a = f2.a and f.b = f2.b
+-- left join fkest1 f3 on f.a = f3.a and f.b = f3.b
+-- where f.c = 1;
+
+-- rollback;
+
+-- Skip these test because it only test explain
+--
+-- test planner's ability to mark joins as unique
+--
+
+-- create table j1 (id int primary key);
+-- create table j2 (id int primary key);
+-- create table j3 (id int);
+
+-- insert into j1 values(1),(2),(3);
+-- insert into j2 values(1),(2),(3);
+-- insert into j3 values(1),(1);
+
+-- analyze j1;
+-- analyze j2;
+-- analyze j3;
+
+-- ensure join is properly marked as unique
+-- explain (verbose, costs off)
+-- select * from j1 inner join j2 on j1.id = j2.id;
+
+-- ensure join is not unique when not an equi-join
+-- explain (verbose, costs off)
+-- select * from j1 inner join j2 on j1.id > j2.id;
+
+-- ensure non-unique rel is not chosen as inner
+-- explain (verbose, costs off)
+-- select * from j1 inner join j3 on j1.id = j3.id;
+
+-- ensure left join is marked as unique
+-- explain (verbose, costs off)
+-- select * from j1 left join j2 on j1.id = j2.id;
+
+-- ensure right join is marked as unique
+-- explain (verbose, costs off)
+-- select * from j1 right join j2 on j1.id = j2.id;
+
+-- ensure full join is marked as unique
+-- explain (verbose, costs off)
+-- select * from j1 full join j2 on j1.id = j2.id;
+
+-- a clauseless (cross) join can't be unique
+-- explain (verbose, costs off)
+-- select * from j1 cross join j2;
+
+-- ensure a natural join is marked as unique
+-- explain (verbose, costs off)
+-- select * from j1 natural join j2;
+
+-- ensure a distinct clause allows the inner to become unique
+-- explain (verbose, costs off)
+-- select * from j1
+-- inner join (select distinct id from j3) j3 on j1.id = j3.id;
+
+-- ensure group by clause allows the inner to become unique
+-- explain (verbose, costs off)
+-- select * from j1
+-- inner join (select id from j3 group by id) j3 on j1.id = j3.id;
+
+-- drop table if exists j1;
+-- drop table if exists j2;
+-- drop table if exists j3;
+
+-- test more complex permutations of unique joins
+
+CREATE TABLE j1 (id1 int, id2 int) USING parquet;
+CREATE TABLE j2 (id1 int, id2 int) USING parquet;
+-- create table j3 (id1 int, id2 int) using parquet;
+
+INSERT INTO j1 values(1,1),(1,2);
+INSERT INTO j2 values(1,1);
+-- insert into j3 values(1,1);
+
+-- analyze j1;
+-- analyze j2;
+-- analyze j3;
+
+-- ensure there's no unique join when not all columns which are part of the
+-- unique index are seen in the join clause
+-- explain (verbose, costs off)
+-- select * from j1
+-- inner join j2 on j1.id1 = j2.id1;
+
+-- ensure proper unique detection with multiple join quals
+-- explain (verbose, costs off)
+-- select * from j1
+-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2;
+
+-- ensure we don't detect the join to be unique when quals are not part of the
+-- join condition
+-- explain (verbose, costs off)
+-- select * from j1
+-- inner join j2 on j1.id1 = j2.id1 where j1.id2 = 1;
+
+-- as above, but for left joins.
+-- explain (verbose, costs off)
+-- select * from j1
+-- left join j2 on j1.id1 = j2.id1 where j1.id2 = 1;
+
+-- validate logic in merge joins which skips mark and restore.
+-- it should only do this if all quals which were used to detect the unique
+-- are present as join quals, and not plain quals.
+-- set enable_nestloop to 0;
+-- set enable_hashjoin to 0;
+-- set enable_sort to 0;
+
+-- create indexes that will be preferred over the PKs to perform the join
+-- create index j1_id1_idx on j1 (id1) where id1 % 1000 = 1;
+-- create index j2_id1_idx on j2 (id1) where id1 % 1000 = 1;
+
+-- need an additional row in j2, if we want j2_id1_idx to be preferred
+INSERT INTO j2 values(1,2);
+-- analyze j2;
+
+-- explain (costs off) select * from j1
+-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2
+-- where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1;
+
+select * from j1
+inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2)
+where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1;
+
+-- reset enable_nestloop;
+-- reset enable_hashjoin;
+-- reset enable_sort;
+
+drop table j1;
+drop table j2;
+-- drop table j3;
+
+-- Skip these tests because it only test explain
+-- check that semijoin inner is not seen as unique for a portion of the outerrel
+-- explain (verbose, costs off)
+-- select t1.unique1, t2.hundred
+-- from onek t1, tenk1 t2
+-- where exists (select 1 from tenk1 t3
+-- where t3.thousand = t1.unique1 and t3.tenthous = t2.hundred)
+-- and t1.unique1 < 1;
+
+-- ... unless it actually is unique
+-- create table j3 as select unique1, tenthous from onek;
+-- vacuum analyze j3;
+-- create unique index on j3(unique1, tenthous);
+
+-- explain (verbose, costs off)
+-- select t1.unique1, t2.hundred
+-- from onek t1, tenk1 t2
+-- where exists (select 1 from j3
+-- where j3.unique1 = t1.unique1 and j3.tenthous = t2.hundred)
+-- and t1.unique1 < 1;
+
+-- drop table j3;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql
index f52aa669e89db..c8e4346cedb89 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql
@@ -6,7 +6,7 @@
-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql
--
-- This test file was converted from inputs/pgSQL/select_having.sql
--- TODO: We should add UDFs in GROUP BY clause when [SPARK-28445] is resolved.
+-- TODO: We should add UDFs in GROUP BY clause when [SPARK-28386] and [SPARK-26741] is resolved.
-- load test data
CREATE TABLE test_having (a int, b int, c string, d string) USING parquet;
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out
new file mode 100644
index 0000000000000..6fcff129d7568
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-join.sql.out
@@ -0,0 +1,3408 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 185
+
+
+-- !query 0
+CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM
+ (VALUES (0), (123456), (-123456), (2147483647), (-2147483647))
+ AS v(f1)
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
+ (VALUES
+ (123, 456),
+ (123, 4567890123456789),
+ (4567890123456789, 123),
+ (4567890123456789, 4567890123456789),
+ (4567890123456789, -4567890123456789))
+ AS v(q1, q2)
+-- !query 1 schema
+struct<>
+-- !query 1 output
+
+
+
+-- !query 2
+CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM
+ (VALUES (0.0), (1004.30), (-34.84),
+ (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double)))
+ AS v(f1)
+-- !query 2 schema
+struct<>
+-- !query 2 output
+
+
+
+-- !query 3
+CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM
+ (VALUES ('doh!'), ('hi de ho neighbor'))
+ AS v(f1)
+-- !query 3 schema
+struct<>
+-- !query 3 output
+
+
+
+-- !query 4
+CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1
+-- !query 4 schema
+struct<>
+-- !query 4 output
+
+
+
+-- !query 5
+CREATE TABLE J1_TBL (
+ i integer,
+ j integer,
+ t string
+) USING parquet
+-- !query 5 schema
+struct<>
+-- !query 5 output
+
+
+
+-- !query 6
+CREATE TABLE J2_TBL (
+ i integer,
+ k integer
+) USING parquet
+-- !query 6 schema
+struct<>
+-- !query 6 output
+
+
+
+-- !query 7
+INSERT INTO J1_TBL VALUES (1, 4, 'one')
+-- !query 7 schema
+struct<>
+-- !query 7 output
+
+
+
+-- !query 8
+INSERT INTO J1_TBL VALUES (2, 3, 'two')
+-- !query 8 schema
+struct<>
+-- !query 8 output
+
+
+
+-- !query 9
+INSERT INTO J1_TBL VALUES (3, 2, 'three')
+-- !query 9 schema
+struct<>
+-- !query 9 output
+
+
+
+-- !query 10
+INSERT INTO J1_TBL VALUES (4, 1, 'four')
+-- !query 10 schema
+struct<>
+-- !query 10 output
+
+
+
+-- !query 11
+INSERT INTO J1_TBL VALUES (5, 0, 'five')
+-- !query 11 schema
+struct<>
+-- !query 11 output
+
+
+
+-- !query 12
+INSERT INTO J1_TBL VALUES (6, 6, 'six')
+-- !query 12 schema
+struct<>
+-- !query 12 output
+
+
+
+-- !query 13
+INSERT INTO J1_TBL VALUES (7, 7, 'seven')
+-- !query 13 schema
+struct<>
+-- !query 13 output
+
+
+
+-- !query 14
+INSERT INTO J1_TBL VALUES (8, 8, 'eight')
+-- !query 14 schema
+struct<>
+-- !query 14 output
+
+
+
+-- !query 15
+INSERT INTO J1_TBL VALUES (0, NULL, 'zero')
+-- !query 15 schema
+struct<>
+-- !query 15 output
+
+
+
+-- !query 16
+INSERT INTO J1_TBL VALUES (NULL, NULL, 'null')
+-- !query 16 schema
+struct<>
+-- !query 16 output
+
+
+
+-- !query 17
+INSERT INTO J1_TBL VALUES (NULL, 0, 'zero')
+-- !query 17 schema
+struct<>
+-- !query 17 output
+
+
+
+-- !query 18
+INSERT INTO J2_TBL VALUES (1, -1)
+-- !query 18 schema
+struct<>
+-- !query 18 output
+
+
+
+-- !query 19
+INSERT INTO J2_TBL VALUES (2, 2)
+-- !query 19 schema
+struct<>
+-- !query 19 output
+
+
+
+-- !query 20
+INSERT INTO J2_TBL VALUES (3, -3)
+-- !query 20 schema
+struct<>
+-- !query 20 output
+
+
+
+-- !query 21
+INSERT INTO J2_TBL VALUES (2, 4)
+-- !query 21 schema
+struct<>
+-- !query 21 output
+
+
+
+-- !query 22
+INSERT INTO J2_TBL VALUES (5, -5)
+-- !query 22 schema
+struct<>
+-- !query 22 output
+
+
+
+-- !query 23
+INSERT INTO J2_TBL VALUES (5, -5)
+-- !query 23 schema
+struct<>
+-- !query 23 output
+
+
+
+-- !query 24
+INSERT INTO J2_TBL VALUES (0, NULL)
+-- !query 24 schema
+struct<>
+-- !query 24 output
+
+
+
+-- !query 25
+INSERT INTO J2_TBL VALUES (NULL, NULL)
+-- !query 25 schema
+struct<>
+-- !query 25 output
+
+
+
+-- !query 26
+INSERT INTO J2_TBL VALUES (NULL, 0)
+-- !query 26 schema
+struct<>
+-- !query 26 output
+
+
+
+-- !query 27
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t)
+ FROM J1_TBL AS tx
+-- !query 27 schema
+struct
+-- !query 27 output
+ 0 NULL zero
+ 1 4 one
+ 2 3 two
+ 3 2 three
+ 4 1 four
+ 5 0 five
+ 6 6 six
+ 7 7 seven
+ 8 8 eight
+ NULL 0 zero
+ NULL NULL null
+
+
+-- !query 28
+SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t)
+ FROM J1_TBL tx
+-- !query 28 schema
+struct
+-- !query 28 output
+ 0 NULL zero
+ 1 4 one
+ 2 3 two
+ 3 2 three
+ 4 1 four
+ 5 0 five
+ 6 6 six
+ 7 7 seven
+ 8 8 eight
+ NULL 0 zero
+ NULL NULL null
+
+
+-- !query 29
+SELECT udf('') AS `xxx`, a, udf(udf(b)), c
+ FROM J1_TBL AS t1 (a, b, c)
+-- !query 29 schema
+struct
+-- !query 29 output
+ 0 NULL zero
+ 1 4 one
+ 2 3 two
+ 3 2 three
+ 4 1 four
+ 5 0 five
+ 6 6 six
+ 7 7 seven
+ 8 8 eight
+ NULL 0 zero
+ NULL NULL null
+
+
+-- !query 30
+SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c))
+ FROM J1_TBL t1 (a, b, c)
+-- !query 30 schema
+struct
+-- !query 30 output
+ 0 NULL zero
+ 1 4 one
+ 2 3 two
+ 3 2 three
+ 4 1 four
+ 5 0 five
+ 6 6 six
+ 7 7 seven
+ 8 8 eight
+ NULL 0 zero
+ NULL NULL null
+
+
+-- !query 31
+SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e
+ FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e)
+-- !query 31 schema
+struct
+-- !query 31 output
+ 0 NULL zero 0 NULL
+ 0 NULL zero 1 -1
+ 0 NULL zero 2 2
+ 0 NULL zero 2 4
+ 0 NULL zero 3 -3
+ 0 NULL zero 5 -5
+ 0 NULL zero 5 -5
+ 0 NULL zero NULL 0
+ 0 NULL zero NULL NULL
+ 1 4 one 0 NULL
+ 1 4 one 1 -1
+ 1 4 one 2 2
+ 1 4 one 2 4
+ 1 4 one 3 -3
+ 1 4 one 5 -5
+ 1 4 one 5 -5
+ 1 4 one NULL 0
+ 1 4 one NULL NULL
+ 2 3 two 0 NULL
+ 2 3 two 1 -1
+ 2 3 two 2 2
+ 2 3 two 2 4
+ 2 3 two 3 -3
+ 2 3 two 5 -5
+ 2 3 two 5 -5
+ 2 3 two NULL 0
+ 2 3 two NULL NULL
+ 3 2 three 0 NULL
+ 3 2 three 1 -1
+ 3 2 three 2 2
+ 3 2 three 2 4
+ 3 2 three 3 -3
+ 3 2 three 5 -5
+ 3 2 three 5 -5
+ 3 2 three NULL 0
+ 3 2 three NULL NULL
+ 4 1 four 0 NULL
+ 4 1 four 1 -1
+ 4 1 four 2 2
+ 4 1 four 2 4
+ 4 1 four 3 -3
+ 4 1 four 5 -5
+ 4 1 four 5 -5
+ 4 1 four NULL 0
+ 4 1 four NULL NULL
+ 5 0 five 0 NULL
+ 5 0 five 1 -1
+ 5 0 five 2 2
+ 5 0 five 2 4
+ 5 0 five 3 -3
+ 5 0 five 5 -5
+ 5 0 five 5 -5
+ 5 0 five NULL 0
+ 5 0 five NULL NULL
+ 6 6 six 0 NULL
+ 6 6 six 1 -1
+ 6 6 six 2 2
+ 6 6 six 2 4
+ 6 6 six 3 -3
+ 6 6 six 5 -5
+ 6 6 six 5 -5
+ 6 6 six NULL 0
+ 6 6 six NULL NULL
+ 7 7 seven 0 NULL
+ 7 7 seven 1 -1
+ 7 7 seven 2 2
+ 7 7 seven 2 4
+ 7 7 seven 3 -3
+ 7 7 seven 5 -5
+ 7 7 seven 5 -5
+ 7 7 seven NULL 0
+ 7 7 seven NULL NULL
+ 8 8 eight 0 NULL
+ 8 8 eight 1 -1
+ 8 8 eight 2 2
+ 8 8 eight 2 4
+ 8 8 eight 3 -3
+ 8 8 eight 5 -5
+ 8 8 eight 5 -5
+ 8 8 eight NULL 0
+ 8 8 eight NULL NULL
+ NULL 0 zero 0 NULL
+ NULL 0 zero 1 -1
+ NULL 0 zero 2 2
+ NULL 0 zero 2 4
+ NULL 0 zero 3 -3
+ NULL 0 zero 5 -5
+ NULL 0 zero 5 -5
+ NULL 0 zero NULL 0
+ NULL 0 zero NULL NULL
+ NULL NULL null 0 NULL
+ NULL NULL null 1 -1
+ NULL NULL null 2 2
+ NULL NULL null 2 4
+ NULL NULL null 3 -3
+ NULL NULL null 5 -5
+ NULL NULL null 5 -5
+ NULL NULL null NULL 0
+ NULL NULL null NULL NULL
+
+
+-- !query 32
+SELECT udf('') AS `xxx`, *
+ FROM J1_TBL CROSS JOIN J2_TBL
+-- !query 32 schema
+struct
+-- !query 32 output
+ 0 NULL zero 0 NULL
+ 0 NULL zero 1 -1
+ 0 NULL zero 2 2
+ 0 NULL zero 2 4
+ 0 NULL zero 3 -3
+ 0 NULL zero 5 -5
+ 0 NULL zero 5 -5
+ 0 NULL zero NULL 0
+ 0 NULL zero NULL NULL
+ 1 4 one 0 NULL
+ 1 4 one 1 -1
+ 1 4 one 2 2
+ 1 4 one 2 4
+ 1 4 one 3 -3
+ 1 4 one 5 -5
+ 1 4 one 5 -5
+ 1 4 one NULL 0
+ 1 4 one NULL NULL
+ 2 3 two 0 NULL
+ 2 3 two 1 -1
+ 2 3 two 2 2
+ 2 3 two 2 4
+ 2 3 two 3 -3
+ 2 3 two 5 -5
+ 2 3 two 5 -5
+ 2 3 two NULL 0
+ 2 3 two NULL NULL
+ 3 2 three 0 NULL
+ 3 2 three 1 -1
+ 3 2 three 2 2
+ 3 2 three 2 4
+ 3 2 three 3 -3
+ 3 2 three 5 -5
+ 3 2 three 5 -5
+ 3 2 three NULL 0
+ 3 2 three NULL NULL
+ 4 1 four 0 NULL
+ 4 1 four 1 -1
+ 4 1 four 2 2
+ 4 1 four 2 4
+ 4 1 four 3 -3
+ 4 1 four 5 -5
+ 4 1 four 5 -5
+ 4 1 four NULL 0
+ 4 1 four NULL NULL
+ 5 0 five 0 NULL
+ 5 0 five 1 -1
+ 5 0 five 2 2
+ 5 0 five 2 4
+ 5 0 five 3 -3
+ 5 0 five 5 -5
+ 5 0 five 5 -5
+ 5 0 five NULL 0
+ 5 0 five NULL NULL
+ 6 6 six 0 NULL
+ 6 6 six 1 -1
+ 6 6 six 2 2
+ 6 6 six 2 4
+ 6 6 six 3 -3
+ 6 6 six 5 -5
+ 6 6 six 5 -5
+ 6 6 six NULL 0
+ 6 6 six NULL NULL
+ 7 7 seven 0 NULL
+ 7 7 seven 1 -1
+ 7 7 seven 2 2
+ 7 7 seven 2 4
+ 7 7 seven 3 -3
+ 7 7 seven 5 -5
+ 7 7 seven 5 -5
+ 7 7 seven NULL 0
+ 7 7 seven NULL NULL
+ 8 8 eight 0 NULL
+ 8 8 eight 1 -1
+ 8 8 eight 2 2
+ 8 8 eight 2 4
+ 8 8 eight 3 -3
+ 8 8 eight 5 -5
+ 8 8 eight 5 -5
+ 8 8 eight NULL 0
+ 8 8 eight NULL NULL
+ NULL 0 zero 0 NULL
+ NULL 0 zero 1 -1
+ NULL 0 zero 2 2
+ NULL 0 zero 2 4
+ NULL 0 zero 3 -3
+ NULL 0 zero 5 -5
+ NULL 0 zero 5 -5
+ NULL 0 zero NULL 0
+ NULL 0 zero NULL NULL
+ NULL NULL null 0 NULL
+ NULL NULL null 1 -1
+ NULL NULL null 2 2
+ NULL NULL null 2 4
+ NULL NULL null 3 -3
+ NULL NULL null 5 -5
+ NULL NULL null 5 -5
+ NULL NULL null NULL 0
+ NULL NULL null NULL NULL
+
+
+-- !query 33
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t
+ FROM J1_TBL CROSS JOIN J2_TBL
+-- !query 33 schema
+struct<>
+-- !query 33 output
+org.apache.spark.sql.AnalysisException
+Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 29
+
+
+-- !query 34
+SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t)
+ FROM J1_TBL t1 CROSS JOIN J2_TBL t2
+-- !query 34 schema
+struct
+-- !query 34 output
+ 0 -1 zero
+ 0 -3 zero
+ 0 -5 zero
+ 0 -5 zero
+ 0 0 zero
+ 0 2 zero
+ 0 4 zero
+ 0 NULL zero
+ 0 NULL zero
+ 1 -1 one
+ 1 -3 one
+ 1 -5 one
+ 1 -5 one
+ 1 0 one
+ 1 2 one
+ 1 4 one
+ 1 NULL one
+ 1 NULL one
+ 2 -1 two
+ 2 -3 two
+ 2 -5 two
+ 2 -5 two
+ 2 0 two
+ 2 2 two
+ 2 4 two
+ 2 NULL two
+ 2 NULL two
+ 3 -1 three
+ 3 -3 three
+ 3 -5 three
+ 3 -5 three
+ 3 0 three
+ 3 2 three
+ 3 4 three
+ 3 NULL three
+ 3 NULL three
+ 4 -1 four
+ 4 -3 four
+ 4 -5 four
+ 4 -5 four
+ 4 0 four
+ 4 2 four
+ 4 4 four
+ 4 NULL four
+ 4 NULL four
+ 5 -1 five
+ 5 -3 five
+ 5 -5 five
+ 5 -5 five
+ 5 0 five
+ 5 2 five
+ 5 4 five
+ 5 NULL five
+ 5 NULL five
+ 6 -1 six
+ 6 -3 six
+ 6 -5 six
+ 6 -5 six
+ 6 0 six
+ 6 2 six
+ 6 4 six
+ 6 NULL six
+ 6 NULL six
+ 7 -1 seven
+ 7 -3 seven
+ 7 -5 seven
+ 7 -5 seven
+ 7 0 seven
+ 7 2 seven
+ 7 4 seven
+ 7 NULL seven
+ 7 NULL seven
+ 8 -1 eight
+ 8 -3 eight
+ 8 -5 eight
+ 8 -5 eight
+ 8 0 eight
+ 8 2 eight
+ 8 4 eight
+ 8 NULL eight
+ 8 NULL eight
+ NULL -1 null
+ NULL -1 zero
+ NULL -3 null
+ NULL -3 zero
+ NULL -5 null
+ NULL -5 null
+ NULL -5 zero
+ NULL -5 zero
+ NULL 0 null
+ NULL 0 zero
+ NULL 2 null
+ NULL 2 zero
+ NULL 4 null
+ NULL 4 zero
+ NULL NULL null
+ NULL NULL null
+ NULL NULL zero
+ NULL NULL zero
+
+
+-- !query 35
+SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk))
+ FROM (J1_TBL CROSS JOIN J2_TBL)
+ AS tx (ii, jj, tt, ii2, kk)
+-- !query 35 schema
+struct
+-- !query 35 output
+ 0 zero -1
+ 0 zero -3
+ 0 zero -5
+ 0 zero -5
+ 0 zero 0
+ 0 zero 2
+ 0 zero 4
+ 0 zero NULL
+ 0 zero NULL
+ 1 one -1
+ 1 one -3
+ 1 one -5
+ 1 one -5
+ 1 one 0
+ 1 one 2
+ 1 one 4
+ 1 one NULL
+ 1 one NULL
+ 2 two -1
+ 2 two -3
+ 2 two -5
+ 2 two -5
+ 2 two 0
+ 2 two 2
+ 2 two 4
+ 2 two NULL
+ 2 two NULL
+ 3 three -1
+ 3 three -3
+ 3 three -5
+ 3 three -5
+ 3 three 0
+ 3 three 2
+ 3 three 4
+ 3 three NULL
+ 3 three NULL
+ 4 four -1
+ 4 four -3
+ 4 four -5
+ 4 four -5
+ 4 four 0
+ 4 four 2
+ 4 four 4
+ 4 four NULL
+ 4 four NULL
+ 5 five -1
+ 5 five -3
+ 5 five -5
+ 5 five -5
+ 5 five 0
+ 5 five 2
+ 5 five 4
+ 5 five NULL
+ 5 five NULL
+ 6 six -1
+ 6 six -3
+ 6 six -5
+ 6 six -5
+ 6 six 0
+ 6 six 2
+ 6 six 4
+ 6 six NULL
+ 6 six NULL
+ 7 seven -1
+ 7 seven -3
+ 7 seven -5
+ 7 seven -5
+ 7 seven 0
+ 7 seven 2
+ 7 seven 4
+ 7 seven NULL
+ 7 seven NULL
+ 8 eight -1
+ 8 eight -3
+ 8 eight -5
+ 8 eight -5
+ 8 eight 0
+ 8 eight 2
+ 8 eight 4
+ 8 eight NULL
+ 8 eight NULL
+ NULL null -1
+ NULL null -3
+ NULL null -5
+ NULL null -5
+ NULL null 0
+ NULL null 2
+ NULL null 4
+ NULL null NULL
+ NULL null NULL
+ NULL zero -1
+ NULL zero -3
+ NULL zero -5
+ NULL zero -5
+ NULL zero 0
+ NULL zero 2
+ NULL zero 4
+ NULL zero NULL
+ NULL zero NULL
+
+
+-- !query 36
+SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k)
+ FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b
+-- !query 36 schema
+struct
+-- !query 36 output
+ 0 NULL zero 0 NULL 0 NULL
+ 0 NULL zero 0 NULL 1 -1
+ 0 NULL zero 0 NULL 2 2
+ 0 NULL zero 0 NULL 2 4
+ 0 NULL zero 0 NULL 3 -3
+ 0 NULL zero 0 NULL 5 -5
+ 0 NULL zero 0 NULL 5 -5
+ 0 NULL zero 0 NULL NULL 0
+ 0 NULL zero 0 NULL NULL NULL
+ 0 NULL zero 1 -1 0 NULL
+ 0 NULL zero 1 -1 1 -1
+ 0 NULL zero 1 -1 2 2
+ 0 NULL zero 1 -1 2 4
+ 0 NULL zero 1 -1 3 -3
+ 0 NULL zero 1 -1 5 -5
+ 0 NULL zero 1 -1 5 -5
+ 0 NULL zero 1 -1 NULL 0
+ 0 NULL zero 1 -1 NULL NULL
+ 0 NULL zero 2 2 0 NULL
+ 0 NULL zero 2 2 1 -1
+ 0 NULL zero 2 2 2 2
+ 0 NULL zero 2 2 2 4
+ 0 NULL zero 2 2 3 -3
+ 0 NULL zero 2 2 5 -5
+ 0 NULL zero 2 2 5 -5
+ 0 NULL zero 2 2 NULL 0
+ 0 NULL zero 2 2 NULL NULL
+ 0 NULL zero 2 4 0 NULL
+ 0 NULL zero 2 4 1 -1
+ 0 NULL zero 2 4 2 2
+ 0 NULL zero 2 4 2 4
+ 0 NULL zero 2 4 3 -3
+ 0 NULL zero 2 4 5 -5
+ 0 NULL zero 2 4 5 -5
+ 0 NULL zero 2 4 NULL 0
+ 0 NULL zero 2 4 NULL NULL
+ 0 NULL zero 3 -3 0 NULL
+ 0 NULL zero 3 -3 1 -1
+ 0 NULL zero 3 -3 2 2
+ 0 NULL zero 3 -3 2 4
+ 0 NULL zero 3 -3 3 -3
+ 0 NULL zero 3 -3 5 -5
+ 0 NULL zero 3 -3 5 -5
+ 0 NULL zero 3 -3 NULL 0
+ 0 NULL zero 3 -3 NULL NULL
+ 0 NULL zero 5 -5 0 NULL
+ 0 NULL zero 5 -5 0 NULL
+ 0 NULL zero 5 -5 1 -1
+ 0 NULL zero 5 -5 1 -1
+ 0 NULL zero 5 -5 2 2
+ 0 NULL zero 5 -5 2 2
+ 0 NULL zero 5 -5 2 4
+ 0 NULL zero 5 -5 2 4
+ 0 NULL zero 5 -5 3 -3
+ 0 NULL zero 5 -5 3 -3
+ 0 NULL zero 5 -5 5 -5
+ 0 NULL zero 5 -5 5 -5
+ 0 NULL zero 5 -5 5 -5
+ 0 NULL zero 5 -5 5 -5
+ 0 NULL zero 5 -5 NULL 0
+ 0 NULL zero 5 -5 NULL 0
+ 0 NULL zero 5 -5 NULL NULL
+ 0 NULL zero 5 -5 NULL NULL
+ 0 NULL zero NULL 0 0 NULL
+ 0 NULL zero NULL 0 1 -1
+ 0 NULL zero NULL 0 2 2
+ 0 NULL zero NULL 0 2 4
+ 0 NULL zero NULL 0 3 -3
+ 0 NULL zero NULL 0 5 -5
+ 0 NULL zero NULL 0 5 -5
+ 0 NULL zero NULL 0 NULL 0
+ 0 NULL zero NULL 0 NULL NULL
+ 0 NULL zero NULL NULL 0 NULL
+ 0 NULL zero NULL NULL 1 -1
+ 0 NULL zero NULL NULL 2 2
+ 0 NULL zero NULL NULL 2 4
+ 0 NULL zero NULL NULL 3 -3
+ 0 NULL zero NULL NULL 5 -5
+ 0 NULL zero NULL NULL 5 -5
+ 0 NULL zero NULL NULL NULL 0
+ 0 NULL zero NULL NULL NULL NULL
+ 1 4 one 0 NULL 0 NULL
+ 1 4 one 0 NULL 1 -1
+ 1 4 one 0 NULL 2 2
+ 1 4 one 0 NULL 2 4
+ 1 4 one 0 NULL 3 -3
+ 1 4 one 0 NULL 5 -5
+ 1 4 one 0 NULL 5 -5
+ 1 4 one 0 NULL NULL 0
+ 1 4 one 0 NULL NULL NULL
+ 1 4 one 1 -1 0 NULL
+ 1 4 one 1 -1 1 -1
+ 1 4 one 1 -1 2 2
+ 1 4 one 1 -1 2 4
+ 1 4 one 1 -1 3 -3
+ 1 4 one 1 -1 5 -5
+ 1 4 one 1 -1 5 -5
+ 1 4 one 1 -1 NULL 0
+ 1 4 one 1 -1 NULL NULL
+ 1 4 one 2 2 0 NULL
+ 1 4 one 2 2 1 -1
+ 1 4 one 2 2 2 2
+ 1 4 one 2 2 2 4
+ 1 4 one 2 2 3 -3
+ 1 4 one 2 2 5 -5
+ 1 4 one 2 2 5 -5
+ 1 4 one 2 2 NULL 0
+ 1 4 one 2 2 NULL NULL
+ 1 4 one 2 4 0 NULL
+ 1 4 one 2 4 1 -1
+ 1 4 one 2 4 2 2
+ 1 4 one 2 4 2 4
+ 1 4 one 2 4 3 -3
+ 1 4 one 2 4 5 -5
+ 1 4 one 2 4 5 -5
+ 1 4 one 2 4 NULL 0
+ 1 4 one 2 4 NULL NULL
+ 1 4 one 3 -3 0 NULL
+ 1 4 one 3 -3 1 -1
+ 1 4 one 3 -3 2 2
+ 1 4 one 3 -3 2 4
+ 1 4 one 3 -3 3 -3
+ 1 4 one 3 -3 5 -5
+ 1 4 one 3 -3 5 -5
+ 1 4 one 3 -3 NULL 0
+ 1 4 one 3 -3 NULL NULL
+ 1 4 one 5 -5 0 NULL
+ 1 4 one 5 -5 0 NULL
+ 1 4 one 5 -5 1 -1
+ 1 4 one 5 -5 1 -1
+ 1 4 one 5 -5 2 2
+ 1 4 one 5 -5 2 2
+ 1 4 one 5 -5 2 4
+ 1 4 one 5 -5 2 4
+ 1 4 one 5 -5 3 -3
+ 1 4 one 5 -5 3 -3
+ 1 4 one 5 -5 5 -5
+ 1 4 one 5 -5 5 -5
+ 1 4 one 5 -5 5 -5
+ 1 4 one 5 -5 5 -5
+ 1 4 one 5 -5 NULL 0
+ 1 4 one 5 -5 NULL 0
+ 1 4 one 5 -5 NULL NULL
+ 1 4 one 5 -5 NULL NULL
+ 1 4 one NULL 0 0 NULL
+ 1 4 one NULL 0 1 -1
+ 1 4 one NULL 0 2 2
+ 1 4 one NULL 0 2 4
+ 1 4 one NULL 0 3 -3
+ 1 4 one NULL 0 5 -5
+ 1 4 one NULL 0 5 -5
+ 1 4 one NULL 0 NULL 0
+ 1 4 one NULL 0 NULL NULL
+ 1 4 one NULL NULL 0 NULL
+ 1 4 one NULL NULL 1 -1
+ 1 4 one NULL NULL 2 2
+ 1 4 one NULL NULL 2 4
+ 1 4 one NULL NULL 3 -3
+ 1 4 one NULL NULL 5 -5
+ 1 4 one NULL NULL 5 -5
+ 1 4 one NULL NULL NULL 0
+ 1 4 one NULL NULL NULL NULL
+ 2 3 two 0 NULL 0 NULL
+ 2 3 two 0 NULL 1 -1
+ 2 3 two 0 NULL 2 2
+ 2 3 two 0 NULL 2 4
+ 2 3 two 0 NULL 3 -3
+ 2 3 two 0 NULL 5 -5
+ 2 3 two 0 NULL 5 -5
+ 2 3 two 0 NULL NULL 0
+ 2 3 two 0 NULL NULL NULL
+ 2 3 two 1 -1 0 NULL
+ 2 3 two 1 -1 1 -1
+ 2 3 two 1 -1 2 2
+ 2 3 two 1 -1 2 4
+ 2 3 two 1 -1 3 -3
+ 2 3 two 1 -1 5 -5
+ 2 3 two 1 -1 5 -5
+ 2 3 two 1 -1 NULL 0
+ 2 3 two 1 -1 NULL NULL
+ 2 3 two 2 2 0 NULL
+ 2 3 two 2 2 1 -1
+ 2 3 two 2 2 2 2
+ 2 3 two 2 2 2 4
+ 2 3 two 2 2 3 -3
+ 2 3 two 2 2 5 -5
+ 2 3 two 2 2 5 -5
+ 2 3 two 2 2 NULL 0
+ 2 3 two 2 2 NULL NULL
+ 2 3 two 2 4 0 NULL
+ 2 3 two 2 4 1 -1
+ 2 3 two 2 4 2 2
+ 2 3 two 2 4 2 4
+ 2 3 two 2 4 3 -3
+ 2 3 two 2 4 5 -5
+ 2 3 two 2 4 5 -5
+ 2 3 two 2 4 NULL 0
+ 2 3 two 2 4 NULL NULL
+ 2 3 two 3 -3 0 NULL
+ 2 3 two 3 -3 1 -1
+ 2 3 two 3 -3 2 2
+ 2 3 two 3 -3 2 4
+ 2 3 two 3 -3 3 -3
+ 2 3 two 3 -3 5 -5
+ 2 3 two 3 -3 5 -5
+ 2 3 two 3 -3 NULL 0
+ 2 3 two 3 -3 NULL NULL
+ 2 3 two 5 -5 0 NULL
+ 2 3 two 5 -5 0 NULL
+ 2 3 two 5 -5 1 -1
+ 2 3 two 5 -5 1 -1
+ 2 3 two 5 -5 2 2
+ 2 3 two 5 -5 2 2
+ 2 3 two 5 -5 2 4
+ 2 3 two 5 -5 2 4
+ 2 3 two 5 -5 3 -3
+ 2 3 two 5 -5 3 -3
+ 2 3 two 5 -5 5 -5
+ 2 3 two 5 -5 5 -5
+ 2 3 two 5 -5 5 -5
+ 2 3 two 5 -5 5 -5
+ 2 3 two 5 -5 NULL 0
+ 2 3 two 5 -5 NULL 0
+ 2 3 two 5 -5 NULL NULL
+ 2 3 two 5 -5 NULL NULL
+ 2 3 two NULL 0 0 NULL
+ 2 3 two NULL 0 1 -1
+ 2 3 two NULL 0 2 2
+ 2 3 two NULL 0 2 4
+ 2 3 two NULL 0 3 -3
+ 2 3 two NULL 0 5 -5
+ 2 3 two NULL 0 5 -5
+ 2 3 two NULL 0 NULL 0
+ 2 3 two NULL 0 NULL NULL
+ 2 3 two NULL NULL 0 NULL
+ 2 3 two NULL NULL 1 -1
+ 2 3 two NULL NULL 2 2
+ 2 3 two NULL NULL 2 4
+ 2 3 two NULL NULL 3 -3
+ 2 3 two NULL NULL 5 -5
+ 2 3 two NULL NULL 5 -5
+ 2 3 two NULL NULL NULL 0
+ 2 3 two NULL NULL NULL NULL
+ 3 2 three 0 NULL 0 NULL
+ 3 2 three 0 NULL 1 -1
+ 3 2 three 0 NULL 2 2
+ 3 2 three 0 NULL 2 4
+ 3 2 three 0 NULL 3 -3
+ 3 2 three 0 NULL 5 -5
+ 3 2 three 0 NULL 5 -5
+ 3 2 three 0 NULL NULL 0
+ 3 2 three 0 NULL NULL NULL
+ 3 2 three 1 -1 0 NULL
+ 3 2 three 1 -1 1 -1
+ 3 2 three 1 -1 2 2
+ 3 2 three 1 -1 2 4
+ 3 2 three 1 -1 3 -3
+ 3 2 three 1 -1 5 -5
+ 3 2 three 1 -1 5 -5
+ 3 2 three 1 -1 NULL 0
+ 3 2 three 1 -1 NULL NULL
+ 3 2 three 2 2 0 NULL
+ 3 2 three 2 2 1 -1
+ 3 2 three 2 2 2 2
+ 3 2 three 2 2 2 4
+ 3 2 three 2 2 3 -3
+ 3 2 three 2 2 5 -5
+ 3 2 three 2 2 5 -5
+ 3 2 three 2 2 NULL 0
+ 3 2 three 2 2 NULL NULL
+ 3 2 three 2 4 0 NULL
+ 3 2 three 2 4 1 -1
+ 3 2 three 2 4 2 2
+ 3 2 three 2 4 2 4
+ 3 2 three 2 4 3 -3
+ 3 2 three 2 4 5 -5
+ 3 2 three 2 4 5 -5
+ 3 2 three 2 4 NULL 0
+ 3 2 three 2 4 NULL NULL
+ 3 2 three 3 -3 0 NULL
+ 3 2 three 3 -3 1 -1
+ 3 2 three 3 -3 2 2
+ 3 2 three 3 -3 2 4
+ 3 2 three 3 -3 3 -3
+ 3 2 three 3 -3 5 -5
+ 3 2 three 3 -3 5 -5
+ 3 2 three 3 -3 NULL 0
+ 3 2 three 3 -3 NULL NULL
+ 3 2 three 5 -5 0 NULL
+ 3 2 three 5 -5 0 NULL
+ 3 2 three 5 -5 1 -1
+ 3 2 three 5 -5 1 -1
+ 3 2 three 5 -5 2 2
+ 3 2 three 5 -5 2 2
+ 3 2 three 5 -5 2 4
+ 3 2 three 5 -5 2 4
+ 3 2 three 5 -5 3 -3
+ 3 2 three 5 -5 3 -3
+ 3 2 three 5 -5 5 -5
+ 3 2 three 5 -5 5 -5
+ 3 2 three 5 -5 5 -5
+ 3 2 three 5 -5 5 -5
+ 3 2 three 5 -5 NULL 0
+ 3 2 three 5 -5 NULL 0
+ 3 2 three 5 -5 NULL NULL
+ 3 2 three 5 -5 NULL NULL
+ 3 2 three NULL 0 0 NULL
+ 3 2 three NULL 0 1 -1
+ 3 2 three NULL 0 2 2
+ 3 2 three NULL 0 2 4
+ 3 2 three NULL 0 3 -3
+ 3 2 three NULL 0 5 -5
+ 3 2 three NULL 0 5 -5
+ 3 2 three NULL 0 NULL 0
+ 3 2 three NULL 0 NULL NULL
+ 3 2 three NULL NULL 0 NULL
+ 3 2 three NULL NULL 1 -1
+ 3 2 three NULL NULL 2 2
+ 3 2 three NULL NULL 2 4
+ 3 2 three NULL NULL 3 -3
+ 3 2 three NULL NULL 5 -5
+ 3 2 three NULL NULL 5 -5
+ 3 2 three NULL NULL NULL 0
+ 3 2 three NULL NULL NULL NULL
+ 4 1 four 0 NULL 0 NULL
+ 4 1 four 0 NULL 1 -1
+ 4 1 four 0 NULL 2 2
+ 4 1 four 0 NULL 2 4
+ 4 1 four 0 NULL 3 -3
+ 4 1 four 0 NULL 5 -5
+ 4 1 four 0 NULL 5 -5
+ 4 1 four 0 NULL NULL 0
+ 4 1 four 0 NULL NULL NULL
+ 4 1 four 1 -1 0 NULL
+ 4 1 four 1 -1 1 -1
+ 4 1 four 1 -1 2 2
+ 4 1 four 1 -1 2 4
+ 4 1 four 1 -1 3 -3
+ 4 1 four 1 -1 5 -5
+ 4 1 four 1 -1 5 -5
+ 4 1 four 1 -1 NULL 0
+ 4 1 four 1 -1 NULL NULL
+ 4 1 four 2 2 0 NULL
+ 4 1 four 2 2 1 -1
+ 4 1 four 2 2 2 2
+ 4 1 four 2 2 2 4
+ 4 1 four 2 2 3 -3
+ 4 1 four 2 2 5 -5
+ 4 1 four 2 2 5 -5
+ 4 1 four 2 2 NULL 0
+ 4 1 four 2 2 NULL NULL
+ 4 1 four 2 4 0 NULL
+ 4 1 four 2 4 1 -1
+ 4 1 four 2 4 2 2
+ 4 1 four 2 4 2 4
+ 4 1 four 2 4 3 -3
+ 4 1 four 2 4 5 -5
+ 4 1 four 2 4 5 -5
+ 4 1 four 2 4 NULL 0
+ 4 1 four 2 4 NULL NULL
+ 4 1 four 3 -3 0 NULL
+ 4 1 four 3 -3 1 -1
+ 4 1 four 3 -3 2 2
+ 4 1 four 3 -3 2 4
+ 4 1 four 3 -3 3 -3
+ 4 1 four 3 -3 5 -5
+ 4 1 four 3 -3 5 -5
+ 4 1 four 3 -3 NULL 0
+ 4 1 four 3 -3 NULL NULL
+ 4 1 four 5 -5 0 NULL
+ 4 1 four 5 -5 0 NULL
+ 4 1 four 5 -5 1 -1
+ 4 1 four 5 -5 1 -1
+ 4 1 four 5 -5 2 2
+ 4 1 four 5 -5 2 2
+ 4 1 four 5 -5 2 4
+ 4 1 four 5 -5 2 4
+ 4 1 four 5 -5 3 -3
+ 4 1 four 5 -5 3 -3
+ 4 1 four 5 -5 5 -5
+ 4 1 four 5 -5 5 -5
+ 4 1 four 5 -5 5 -5
+ 4 1 four 5 -5 5 -5
+ 4 1 four 5 -5 NULL 0
+ 4 1 four 5 -5 NULL 0
+ 4 1 four 5 -5 NULL NULL
+ 4 1 four 5 -5 NULL NULL
+ 4 1 four NULL 0 0 NULL
+ 4 1 four NULL 0 1 -1
+ 4 1 four NULL 0 2 2
+ 4 1 four NULL 0 2 4
+ 4 1 four NULL 0 3 -3
+ 4 1 four NULL 0 5 -5
+ 4 1 four NULL 0 5 -5
+ 4 1 four NULL 0 NULL 0
+ 4 1 four NULL 0 NULL NULL
+ 4 1 four NULL NULL 0 NULL
+ 4 1 four NULL NULL 1 -1
+ 4 1 four NULL NULL 2 2
+ 4 1 four NULL NULL 2 4
+ 4 1 four NULL NULL 3 -3
+ 4 1 four NULL NULL 5 -5
+ 4 1 four NULL NULL 5 -5
+ 4 1 four NULL NULL NULL 0
+ 4 1 four NULL NULL NULL NULL
+ 5 0 five 0 NULL 0 NULL
+ 5 0 five 0 NULL 1 -1
+ 5 0 five 0 NULL 2 2
+ 5 0 five 0 NULL 2 4
+ 5 0 five 0 NULL 3 -3
+ 5 0 five 0 NULL 5 -5
+ 5 0 five 0 NULL 5 -5
+ 5 0 five 0 NULL NULL 0
+ 5 0 five 0 NULL NULL NULL
+ 5 0 five 1 -1 0 NULL
+ 5 0 five 1 -1 1 -1
+ 5 0 five 1 -1 2 2
+ 5 0 five 1 -1 2 4
+ 5 0 five 1 -1 3 -3
+ 5 0 five 1 -1 5 -5
+ 5 0 five 1 -1 5 -5
+ 5 0 five 1 -1 NULL 0
+ 5 0 five 1 -1 NULL NULL
+ 5 0 five 2 2 0 NULL
+ 5 0 five 2 2 1 -1
+ 5 0 five 2 2 2 2
+ 5 0 five 2 2 2 4
+ 5 0 five 2 2 3 -3
+ 5 0 five 2 2 5 -5
+ 5 0 five 2 2 5 -5
+ 5 0 five 2 2 NULL 0
+ 5 0 five 2 2 NULL NULL
+ 5 0 five 2 4 0 NULL
+ 5 0 five 2 4 1 -1
+ 5 0 five 2 4 2 2
+ 5 0 five 2 4 2 4
+ 5 0 five 2 4 3 -3
+ 5 0 five 2 4 5 -5
+ 5 0 five 2 4 5 -5
+ 5 0 five 2 4 NULL 0
+ 5 0 five 2 4 NULL NULL
+ 5 0 five 3 -3 0 NULL
+ 5 0 five 3 -3 1 -1
+ 5 0 five 3 -3 2 2
+ 5 0 five 3 -3 2 4
+ 5 0 five 3 -3 3 -3
+ 5 0 five 3 -3 5 -5
+ 5 0 five 3 -3 5 -5
+ 5 0 five 3 -3 NULL 0
+ 5 0 five 3 -3 NULL NULL
+ 5 0 five 5 -5 0 NULL
+ 5 0 five 5 -5 0 NULL
+ 5 0 five 5 -5 1 -1
+ 5 0 five 5 -5 1 -1
+ 5 0 five 5 -5 2 2
+ 5 0 five 5 -5 2 2
+ 5 0 five 5 -5 2 4
+ 5 0 five 5 -5 2 4
+ 5 0 five 5 -5 3 -3
+ 5 0 five 5 -5 3 -3
+ 5 0 five 5 -5 5 -5
+ 5 0 five 5 -5 5 -5
+ 5 0 five 5 -5 5 -5
+ 5 0 five 5 -5 5 -5
+ 5 0 five 5 -5 NULL 0
+ 5 0 five 5 -5 NULL 0
+ 5 0 five 5 -5 NULL NULL
+ 5 0 five 5 -5 NULL NULL
+ 5 0 five NULL 0 0 NULL
+ 5 0 five NULL 0 1 -1
+ 5 0 five NULL 0 2 2
+ 5 0 five NULL 0 2 4
+ 5 0 five NULL 0 3 -3
+ 5 0 five NULL 0 5 -5
+ 5 0 five NULL 0 5 -5
+ 5 0 five NULL 0 NULL 0
+ 5 0 five NULL 0 NULL NULL
+ 5 0 five NULL NULL 0 NULL
+ 5 0 five NULL NULL 1 -1
+ 5 0 five NULL NULL 2 2
+ 5 0 five NULL NULL 2 4
+ 5 0 five NULL NULL 3 -3
+ 5 0 five NULL NULL 5 -5
+ 5 0 five NULL NULL 5 -5
+ 5 0 five NULL NULL NULL 0
+ 5 0 five NULL NULL NULL NULL
+ 6 6 six 0 NULL 0 NULL
+ 6 6 six 0 NULL 1 -1
+ 6 6 six 0 NULL 2 2
+ 6 6 six 0 NULL 2 4
+ 6 6 six 0 NULL 3 -3
+ 6 6 six 0 NULL 5 -5
+ 6 6 six 0 NULL 5 -5
+ 6 6 six 0 NULL NULL 0
+ 6 6 six 0 NULL NULL NULL
+ 6 6 six 1 -1 0 NULL
+ 6 6 six 1 -1 1 -1
+ 6 6 six 1 -1 2 2
+ 6 6 six 1 -1 2 4
+ 6 6 six 1 -1 3 -3
+ 6 6 six 1 -1 5 -5
+ 6 6 six 1 -1 5 -5
+ 6 6 six 1 -1 NULL 0
+ 6 6 six 1 -1 NULL NULL
+ 6 6 six 2 2 0 NULL
+ 6 6 six 2 2 1 -1
+ 6 6 six 2 2 2 2
+ 6 6 six 2 2 2 4
+ 6 6 six 2 2 3 -3
+ 6 6 six 2 2 5 -5
+ 6 6 six 2 2 5 -5
+ 6 6 six 2 2 NULL 0
+ 6 6 six 2 2 NULL NULL
+ 6 6 six 2 4 0 NULL
+ 6 6 six 2 4 1 -1
+ 6 6 six 2 4 2 2
+ 6 6 six 2 4 2 4
+ 6 6 six 2 4 3 -3
+ 6 6 six 2 4 5 -5
+ 6 6 six 2 4 5 -5
+ 6 6 six 2 4 NULL 0
+ 6 6 six 2 4 NULL NULL
+ 6 6 six 3 -3 0 NULL
+ 6 6 six 3 -3 1 -1
+ 6 6 six 3 -3 2 2
+ 6 6 six 3 -3 2 4
+ 6 6 six 3 -3 3 -3
+ 6 6 six 3 -3 5 -5
+ 6 6 six 3 -3 5 -5
+ 6 6 six 3 -3 NULL 0
+ 6 6 six 3 -3 NULL NULL
+ 6 6 six 5 -5 0 NULL
+ 6 6 six 5 -5 0 NULL
+ 6 6 six 5 -5 1 -1
+ 6 6 six 5 -5 1 -1
+ 6 6 six 5 -5 2 2
+ 6 6 six 5 -5 2 2
+ 6 6 six 5 -5 2 4
+ 6 6 six 5 -5 2 4
+ 6 6 six 5 -5 3 -3
+ 6 6 six 5 -5 3 -3
+ 6 6 six 5 -5 5 -5
+ 6 6 six 5 -5 5 -5
+ 6 6 six 5 -5 5 -5
+ 6 6 six 5 -5 5 -5
+ 6 6 six 5 -5 NULL 0
+ 6 6 six 5 -5 NULL 0
+ 6 6 six 5 -5 NULL NULL
+ 6 6 six 5 -5 NULL NULL
+ 6 6 six NULL 0 0 NULL
+ 6 6 six NULL 0 1 -1
+ 6 6 six NULL 0 2 2
+ 6 6 six NULL 0 2 4
+ 6 6 six NULL 0 3 -3
+ 6 6 six NULL 0 5 -5
+ 6 6 six NULL 0 5 -5
+ 6 6 six NULL 0 NULL 0
+ 6 6 six NULL 0 NULL NULL
+ 6 6 six NULL NULL 0 NULL
+ 6 6 six NULL NULL 1 -1
+ 6 6 six NULL NULL 2 2
+ 6 6 six NULL NULL 2 4
+ 6 6 six NULL NULL 3 -3
+ 6 6 six NULL NULL 5 -5
+ 6 6 six NULL NULL 5 -5
+ 6 6 six NULL NULL NULL 0
+ 6 6 six NULL NULL NULL NULL
+ 7 7 seven 0 NULL 0 NULL
+ 7 7 seven 0 NULL 1 -1
+ 7 7 seven 0 NULL 2 2
+ 7 7 seven 0 NULL 2 4
+ 7 7 seven 0 NULL 3 -3
+ 7 7 seven 0 NULL 5 -5
+ 7 7 seven 0 NULL 5 -5
+ 7 7 seven 0 NULL NULL 0
+ 7 7 seven 0 NULL NULL NULL
+ 7 7 seven 1 -1 0 NULL
+ 7 7 seven 1 -1 1 -1
+ 7 7 seven 1 -1 2 2
+ 7 7 seven 1 -1 2 4
+ 7 7 seven 1 -1 3 -3
+ 7 7 seven 1 -1 5 -5
+ 7 7 seven 1 -1 5 -5
+ 7 7 seven 1 -1 NULL 0
+ 7 7 seven 1 -1 NULL NULL
+ 7 7 seven 2 2 0 NULL
+ 7 7 seven 2 2 1 -1
+ 7 7 seven 2 2 2 2
+ 7 7 seven 2 2 2 4
+ 7 7 seven 2 2 3 -3
+ 7 7 seven 2 2 5 -5
+ 7 7 seven 2 2 5 -5
+ 7 7 seven 2 2 NULL 0
+ 7 7 seven 2 2 NULL NULL
+ 7 7 seven 2 4 0 NULL
+ 7 7 seven 2 4 1 -1
+ 7 7 seven 2 4 2 2
+ 7 7 seven 2 4 2 4
+ 7 7 seven 2 4 3 -3
+ 7 7 seven 2 4 5 -5
+ 7 7 seven 2 4 5 -5
+ 7 7 seven 2 4 NULL 0
+ 7 7 seven 2 4 NULL NULL
+ 7 7 seven 3 -3 0 NULL
+ 7 7 seven 3 -3 1 -1
+ 7 7 seven 3 -3 2 2
+ 7 7 seven 3 -3 2 4
+ 7 7 seven 3 -3 3 -3
+ 7 7 seven 3 -3 5 -5
+ 7 7 seven 3 -3 5 -5
+ 7 7 seven 3 -3 NULL 0
+ 7 7 seven 3 -3 NULL NULL
+ 7 7 seven 5 -5 0 NULL
+ 7 7 seven 5 -5 0 NULL
+ 7 7 seven 5 -5 1 -1
+ 7 7 seven 5 -5 1 -1
+ 7 7 seven 5 -5 2 2
+ 7 7 seven 5 -5 2 2
+ 7 7 seven 5 -5 2 4
+ 7 7 seven 5 -5 2 4
+ 7 7 seven 5 -5 3 -3
+ 7 7 seven 5 -5 3 -3
+ 7 7 seven 5 -5 5 -5
+ 7 7 seven 5 -5 5 -5
+ 7 7 seven 5 -5 5 -5
+ 7 7 seven 5 -5 5 -5
+ 7 7 seven 5 -5 NULL 0
+ 7 7 seven 5 -5 NULL 0
+ 7 7 seven 5 -5 NULL NULL
+ 7 7 seven 5 -5 NULL NULL
+ 7 7 seven NULL 0 0 NULL
+ 7 7 seven NULL 0 1 -1
+ 7 7 seven NULL 0 2 2
+ 7 7 seven NULL 0 2 4
+ 7 7 seven NULL 0 3 -3
+ 7 7 seven NULL 0 5 -5
+ 7 7 seven NULL 0 5 -5
+ 7 7 seven NULL 0 NULL 0
+ 7 7 seven NULL 0 NULL NULL
+ 7 7 seven NULL NULL 0 NULL
+ 7 7 seven NULL NULL 1 -1
+ 7 7 seven NULL NULL 2 2
+ 7 7 seven NULL NULL 2 4
+ 7 7 seven NULL NULL 3 -3
+ 7 7 seven NULL NULL 5 -5
+ 7 7 seven NULL NULL 5 -5
+ 7 7 seven NULL NULL NULL 0
+ 7 7 seven NULL NULL NULL NULL
+ 8 8 eight 0 NULL 0 NULL
+ 8 8 eight 0 NULL 1 -1
+ 8 8 eight 0 NULL 2 2
+ 8 8 eight 0 NULL 2 4
+ 8 8 eight 0 NULL 3 -3
+ 8 8 eight 0 NULL 5 -5
+ 8 8 eight 0 NULL 5 -5
+ 8 8 eight 0 NULL NULL 0
+ 8 8 eight 0 NULL NULL NULL
+ 8 8 eight 1 -1 0 NULL
+ 8 8 eight 1 -1 1 -1
+ 8 8 eight 1 -1 2 2
+ 8 8 eight 1 -1 2 4
+ 8 8 eight 1 -1 3 -3
+ 8 8 eight 1 -1 5 -5
+ 8 8 eight 1 -1 5 -5
+ 8 8 eight 1 -1 NULL 0
+ 8 8 eight 1 -1 NULL NULL
+ 8 8 eight 2 2 0 NULL
+ 8 8 eight 2 2 1 -1
+ 8 8 eight 2 2 2 2
+ 8 8 eight 2 2 2 4
+ 8 8 eight 2 2 3 -3
+ 8 8 eight 2 2 5 -5
+ 8 8 eight 2 2 5 -5
+ 8 8 eight 2 2 NULL 0
+ 8 8 eight 2 2 NULL NULL
+ 8 8 eight 2 4 0 NULL
+ 8 8 eight 2 4 1 -1
+ 8 8 eight 2 4 2 2
+ 8 8 eight 2 4 2 4
+ 8 8 eight 2 4 3 -3
+ 8 8 eight 2 4 5 -5
+ 8 8 eight 2 4 5 -5
+ 8 8 eight 2 4 NULL 0
+ 8 8 eight 2 4 NULL NULL
+ 8 8 eight 3 -3 0 NULL
+ 8 8 eight 3 -3 1 -1
+ 8 8 eight 3 -3 2 2
+ 8 8 eight 3 -3 2 4
+ 8 8 eight 3 -3 3 -3
+ 8 8 eight 3 -3 5 -5
+ 8 8 eight 3 -3 5 -5
+ 8 8 eight 3 -3 NULL 0
+ 8 8 eight 3 -3 NULL NULL
+ 8 8 eight 5 -5 0 NULL
+ 8 8 eight 5 -5 0 NULL
+ 8 8 eight 5 -5 1 -1
+ 8 8 eight 5 -5 1 -1
+ 8 8 eight 5 -5 2 2
+ 8 8 eight 5 -5 2 2
+ 8 8 eight 5 -5 2 4
+ 8 8 eight 5 -5 2 4
+ 8 8 eight 5 -5 3 -3
+ 8 8 eight 5 -5 3 -3
+ 8 8 eight 5 -5 5 -5
+ 8 8 eight 5 -5 5 -5
+ 8 8 eight 5 -5 5 -5
+ 8 8 eight 5 -5 5 -5
+ 8 8 eight 5 -5 NULL 0
+ 8 8 eight 5 -5 NULL 0
+ 8 8 eight 5 -5 NULL NULL
+ 8 8 eight 5 -5 NULL NULL
+ 8 8 eight NULL 0 0 NULL
+ 8 8 eight NULL 0 1 -1
+ 8 8 eight NULL 0 2 2
+ 8 8 eight NULL 0 2 4
+ 8 8 eight NULL 0 3 -3
+ 8 8 eight NULL 0 5 -5
+ 8 8 eight NULL 0 5 -5
+ 8 8 eight NULL 0 NULL 0
+ 8 8 eight NULL 0 NULL NULL
+ 8 8 eight NULL NULL 0 NULL
+ 8 8 eight NULL NULL 1 -1
+ 8 8 eight NULL NULL 2 2
+ 8 8 eight NULL NULL 2 4
+ 8 8 eight NULL NULL 3 -3
+ 8 8 eight NULL NULL 5 -5
+ 8 8 eight NULL NULL 5 -5
+ 8 8 eight NULL NULL NULL 0
+ 8 8 eight NULL NULL NULL NULL
+ NULL 0 zero 0 NULL 0 NULL
+ NULL 0 zero 0 NULL 1 -1
+ NULL 0 zero 0 NULL 2 2
+ NULL 0 zero 0 NULL 2 4
+ NULL 0 zero 0 NULL 3 -3
+ NULL 0 zero 0 NULL 5 -5
+ NULL 0 zero 0 NULL 5 -5
+ NULL 0 zero 0 NULL NULL 0
+ NULL 0 zero 0 NULL NULL NULL
+ NULL 0 zero 1 -1 0 NULL
+ NULL 0 zero 1 -1 1 -1
+ NULL 0 zero 1 -1 2 2
+ NULL 0 zero 1 -1 2 4
+ NULL 0 zero 1 -1 3 -3
+ NULL 0 zero 1 -1 5 -5
+ NULL 0 zero 1 -1 5 -5
+ NULL 0 zero 1 -1 NULL 0
+ NULL 0 zero 1 -1 NULL NULL
+ NULL 0 zero 2 2 0 NULL
+ NULL 0 zero 2 2 1 -1
+ NULL 0 zero 2 2 2 2
+ NULL 0 zero 2 2 2 4
+ NULL 0 zero 2 2 3 -3
+ NULL 0 zero 2 2 5 -5
+ NULL 0 zero 2 2 5 -5
+ NULL 0 zero 2 2 NULL 0
+ NULL 0 zero 2 2 NULL NULL
+ NULL 0 zero 2 4 0 NULL
+ NULL 0 zero 2 4 1 -1
+ NULL 0 zero 2 4 2 2
+ NULL 0 zero 2 4 2 4
+ NULL 0 zero 2 4 3 -3
+ NULL 0 zero 2 4 5 -5
+ NULL 0 zero 2 4 5 -5
+ NULL 0 zero 2 4 NULL 0
+ NULL 0 zero 2 4 NULL NULL
+ NULL 0 zero 3 -3 0 NULL
+ NULL 0 zero 3 -3 1 -1
+ NULL 0 zero 3 -3 2 2
+ NULL 0 zero 3 -3 2 4
+ NULL 0 zero 3 -3 3 -3
+ NULL 0 zero 3 -3 5 -5
+ NULL 0 zero 3 -3 5 -5
+ NULL 0 zero 3 -3 NULL 0
+ NULL 0 zero 3 -3 NULL NULL
+ NULL 0 zero 5 -5 0 NULL
+ NULL 0 zero 5 -5 0 NULL
+ NULL 0 zero 5 -5 1 -1
+ NULL 0 zero 5 -5 1 -1
+ NULL 0 zero 5 -5 2 2
+ NULL 0 zero 5 -5 2 2
+ NULL 0 zero 5 -5 2 4
+ NULL 0 zero 5 -5 2 4
+ NULL 0 zero 5 -5 3 -3
+ NULL 0 zero 5 -5 3 -3
+ NULL 0 zero 5 -5 5 -5
+ NULL 0 zero 5 -5 5 -5
+ NULL 0 zero 5 -5 5 -5
+ NULL 0 zero 5 -5 5 -5
+ NULL 0 zero 5 -5 NULL 0
+ NULL 0 zero 5 -5 NULL 0
+ NULL 0 zero 5 -5 NULL NULL
+ NULL 0 zero 5 -5 NULL NULL
+ NULL 0 zero NULL 0 0 NULL
+ NULL 0 zero NULL 0 1 -1
+ NULL 0 zero NULL 0 2 2
+ NULL 0 zero NULL 0 2 4
+ NULL 0 zero NULL 0 3 -3
+ NULL 0 zero NULL 0 5 -5
+ NULL 0 zero NULL 0 5 -5
+ NULL 0 zero NULL 0 NULL 0
+ NULL 0 zero NULL 0 NULL NULL
+ NULL 0 zero NULL NULL 0 NULL
+ NULL 0 zero NULL NULL 1 -1
+ NULL 0 zero NULL NULL 2 2
+ NULL 0 zero NULL NULL 2 4
+ NULL 0 zero NULL NULL 3 -3
+ NULL 0 zero NULL NULL 5 -5
+ NULL 0 zero NULL NULL 5 -5
+ NULL 0 zero NULL NULL NULL 0
+ NULL 0 zero NULL NULL NULL NULL
+ NULL NULL null 0 NULL 0 NULL
+ NULL NULL null 0 NULL 1 -1
+ NULL NULL null 0 NULL 2 2
+ NULL NULL null 0 NULL 2 4
+ NULL NULL null 0 NULL 3 -3
+ NULL NULL null 0 NULL 5 -5
+ NULL NULL null 0 NULL 5 -5
+ NULL NULL null 0 NULL NULL 0
+ NULL NULL null 0 NULL NULL NULL
+ NULL NULL null 1 -1 0 NULL
+ NULL NULL null 1 -1 1 -1
+ NULL NULL null 1 -1 2 2
+ NULL NULL null 1 -1 2 4
+ NULL NULL null 1 -1 3 -3
+ NULL NULL null 1 -1 5 -5
+ NULL NULL null 1 -1 5 -5
+ NULL NULL null 1 -1 NULL 0
+ NULL NULL null 1 -1 NULL NULL
+ NULL NULL null 2 2 0 NULL
+ NULL NULL null 2 2 1 -1
+ NULL NULL null 2 2 2 2
+ NULL NULL null 2 2 2 4
+ NULL NULL null 2 2 3 -3
+ NULL NULL null 2 2 5 -5
+ NULL NULL null 2 2 5 -5
+ NULL NULL null 2 2 NULL 0
+ NULL NULL null 2 2 NULL NULL
+ NULL NULL null 2 4 0 NULL
+ NULL NULL null 2 4 1 -1
+ NULL NULL null 2 4 2 2
+ NULL NULL null 2 4 2 4
+ NULL NULL null 2 4 3 -3
+ NULL NULL null 2 4 5 -5
+ NULL NULL null 2 4 5 -5
+ NULL NULL null 2 4 NULL 0
+ NULL NULL null 2 4 NULL NULL
+ NULL NULL null 3 -3 0 NULL
+ NULL NULL null 3 -3 1 -1
+ NULL NULL null 3 -3 2 2
+ NULL NULL null 3 -3 2 4
+ NULL NULL null 3 -3 3 -3
+ NULL NULL null 3 -3 5 -5
+ NULL NULL null 3 -3 5 -5
+ NULL NULL null 3 -3 NULL 0
+ NULL NULL null 3 -3 NULL NULL
+ NULL NULL null 5 -5 0 NULL
+ NULL NULL null 5 -5 0 NULL
+ NULL NULL null 5 -5 1 -1
+ NULL NULL null 5 -5 1 -1
+ NULL NULL null 5 -5 2 2
+ NULL NULL null 5 -5 2 2
+ NULL NULL null 5 -5 2 4
+ NULL NULL null 5 -5 2 4
+ NULL NULL null 5 -5 3 -3
+ NULL NULL null 5 -5 3 -3
+ NULL NULL null 5 -5 5 -5
+ NULL NULL null 5 -5 5 -5
+ NULL NULL null 5 -5 5 -5
+ NULL NULL null 5 -5 5 -5
+ NULL NULL null 5 -5 NULL 0
+ NULL NULL null 5 -5 NULL 0
+ NULL NULL null 5 -5 NULL NULL
+ NULL NULL null 5 -5 NULL NULL
+ NULL NULL null NULL 0 0 NULL
+ NULL NULL null NULL 0 1 -1
+ NULL NULL null NULL 0 2 2
+ NULL NULL null NULL 0 2 4
+ NULL NULL null NULL 0 3 -3
+ NULL NULL null NULL 0 5 -5
+ NULL NULL null NULL 0 5 -5
+ NULL NULL null NULL 0 NULL 0
+ NULL NULL null NULL 0 NULL NULL
+ NULL NULL null NULL NULL 0 NULL
+ NULL NULL null NULL NULL 1 -1
+ NULL NULL null NULL NULL 2 2
+ NULL NULL null NULL NULL 2 4
+ NULL NULL null NULL NULL 3 -3
+ NULL NULL null NULL NULL 5 -5
+ NULL NULL null NULL NULL 5 -5
+ NULL NULL null NULL NULL NULL 0
+ NULL NULL null NULL NULL NULL NULL
+
+
+-- !query 37
+SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k)
+ FROM J1_TBL INNER JOIN J2_TBL USING (i)
+-- !query 37 schema
+struct
+-- !query 37 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+
+
+-- !query 38
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k
+ FROM J1_TBL JOIN J2_TBL USING (i)
+-- !query 38 schema
+struct
+-- !query 38 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+
+
+-- !query 39
+SELECT udf('') AS `xxx`, *
+ FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a)
+ ORDER BY udf(udf(a)), udf(d)
+-- !query 39 schema
+struct
+-- !query 39 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+
+
+-- !query 40
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL NATURAL JOIN J2_TBL
+-- !query 40 schema
+struct
+-- !query 40 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+
+
+-- !query 41
+SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d)
+ FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d)
+-- !query 41 schema
+struct
+-- !query 41 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+
+
+-- !query 42
+SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d
+ FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a)
+-- !query 42 schema
+struct
+-- !query 42 output
+ 0 NULL zero NULL
+ 2 3 two 2
+ 4 1 four 2
+
+
+-- !query 43
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+ FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i)
+-- !query 43 schema
+struct
+-- !query 43 output
+ 0 NULL zero 0 NULL
+ 1 4 one 1 -1
+ 2 3 two 2 2
+ 2 3 two 2 4
+ 3 2 three 3 -3
+ 5 0 five 5 -5
+ 5 0 five 5 -5
+
+
+-- !query 44
+SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k
+ FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k))
+-- !query 44 schema
+struct
+-- !query 44 output
+ 0 NULL zero NULL 0
+ 2 3 two 2 2
+ 4 1 four 2 4
+
+
+-- !query 45
+SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k)
+ FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k)))
+-- !query 45 schema
+struct
+-- !query 45 output
+ 0 NULL zero 2 2
+ 0 NULL zero 2 4
+ 0 NULL zero NULL 0
+ 1 4 one 2 2
+ 1 4 one 2 4
+ 2 3 two 2 2
+ 2 3 two 2 4
+ 3 2 three 2 4
+ 4 1 four 2 4
+
+
+-- !query 46
+SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(t)
+-- !query 46 schema
+struct
+-- !query 46 output
+ NULL NULL null NULL
+ NULL 0 zero NULL
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 4 1 four NULL
+ 5 0 five -5
+ 5 0 five -5
+ 6 6 six NULL
+ 7 7 seven NULL
+ 8 8 eight NULL
+
+
+-- !query 47
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i)
+ ORDER BY udf(i), udf(udf(k)), udf(t)
+-- !query 47 schema
+struct
+-- !query 47 output
+ NULL NULL null NULL
+ NULL 0 zero NULL
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 4 1 four NULL
+ 5 0 five -5
+ 5 0 five -5
+ 6 6 six NULL
+ 7 7 seven NULL
+ 8 8 eight NULL
+
+
+-- !query 48
+SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k)
+ FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i)
+-- !query 48 schema
+struct
+-- !query 48 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+ NULL NULL NULL 0
+ NULL NULL NULL NULL
+
+
+-- !query 49
+SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k)
+ FROM J1_TBL RIGHT JOIN J2_TBL USING (i)
+-- !query 49 schema
+struct
+-- !query 49 output
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 5 0 five -5
+ 5 0 five -5
+ NULL NULL NULL 0
+ NULL NULL NULL NULL
+
+
+-- !query 50
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k)
+ FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(t)
+-- !query 50 schema
+struct
+-- !query 50 output
+ NULL NULL NULL NULL
+ NULL NULL null NULL
+ NULL 0 zero NULL
+ NULL NULL NULL 0
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 4 1 four NULL
+ 5 0 five -5
+ 5 0 five -5
+ 6 6 six NULL
+ 7 7 seven NULL
+ 8 8 eight NULL
+
+
+-- !query 51
+SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k))
+ FROM J1_TBL FULL JOIN J2_TBL USING (i)
+ ORDER BY udf(udf(i)), udf(k), udf(udf(t))
+-- !query 51 schema
+struct
+-- !query 51 output
+ NULL NULL NULL NULL
+ NULL NULL null NULL
+ NULL 0 zero NULL
+ NULL NULL NULL 0
+ 0 NULL zero NULL
+ 1 4 one -1
+ 2 3 two 2
+ 2 3 two 4
+ 3 2 three -3
+ 4 1 four NULL
+ 5 0 five -5
+ 5 0 five -5
+ 6 6 six NULL
+ 7 7 seven NULL
+ 8 8 eight NULL
+
+
+-- !query 52
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k))
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1)
+-- !query 52 schema
+struct
+-- !query 52 output
+
+
+
+-- !query 53
+SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k)
+ FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1))
+-- !query 53 schema
+struct
+-- !query 53 output
+ 1 4 one -1
+
+
+-- !query 54
+CREATE TABLE t1 (name STRING, n INTEGER) USING parquet
+-- !query 54 schema
+struct<>
+-- !query 54 output
+
+
+
+-- !query 55
+CREATE TABLE t2 (name STRING, n INTEGER) USING parquet
+-- !query 55 schema
+struct<>
+-- !query 55 output
+
+
+
+-- !query 56
+CREATE TABLE t3 (name STRING, n INTEGER) USING parquet
+-- !query 56 schema
+struct<>
+-- !query 56 output
+
+
+
+-- !query 57
+INSERT INTO t1 VALUES ( 'bb', 11 )
+-- !query 57 schema
+struct<>
+-- !query 57 output
+
+
+
+-- !query 58
+INSERT INTO t2 VALUES ( 'bb', 12 )
+-- !query 58 schema
+struct<>
+-- !query 58 output
+
+
+
+-- !query 59
+INSERT INTO t2 VALUES ( 'cc', 22 )
+-- !query 59 schema
+struct<>
+-- !query 59 output
+
+
+
+-- !query 60
+INSERT INTO t2 VALUES ( 'ee', 42 )
+-- !query 60 schema
+struct<>
+-- !query 60 output
+
+
+
+-- !query 61
+INSERT INTO t3 VALUES ( 'bb', 13 )
+-- !query 61 schema
+struct<>
+-- !query 61 output
+
+
+
+-- !query 62
+INSERT INTO t3 VALUES ( 'cc', 23 )
+-- !query 62 schema
+struct<>
+-- !query 62 output
+
+
+
+-- !query 63
+INSERT INTO t3 VALUES ( 'dd', 33 )
+-- !query 63 schema
+struct<>
+-- !query 63 output
+
+
+
+-- !query 64
+SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name)
+-- !query 64 schema
+struct
+-- !query 64 output
+bb 11 12 13
+cc NULL 22 23
+dd NULL NULL 33
+ee NULL 42 NULL
+
+
+-- !query 65
+SELECT * FROM
+(SELECT udf(name) as name, t2.n FROM t2) as s2
+INNER JOIN
+(SELECT udf(udf(name)) as name, t3.n FROM t3) s3
+USING (name)
+-- !query 65 schema
+struct
+-- !query 65 output
+bb 12 13
+cc 22 23
+
+
+-- !query 66
+SELECT * FROM
+(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2
+LEFT JOIN
+(SELECT udf(name) as name, t3.n FROM t3) s3
+USING (name)
+-- !query 66 schema
+struct
+-- !query 66 output
+bb 12 13
+cc 22 23
+ee 42 NULL
+
+
+-- !query 67
+SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM
+(SELECT * FROM t2) as s2
+FULL JOIN
+(SELECT * FROM t3) s3
+USING (name)
+-- !query 67 schema
+struct
+-- !query 67 output
+bb 12 13
+cc 22 23
+dd NULL 33
+ee 42 NULL
+
+
+-- !query 68
+SELECT * FROM
+(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2
+NATURAL INNER JOIN
+(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3
+-- !query 68 schema
+struct
+-- !query 68 output
+bb 12 2 13 3
+cc 22 2 23 3
+
+
+-- !query 69
+SELECT * FROM
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL LEFT JOIN
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3
+-- !query 69 schema
+struct
+-- !query 69 output
+bb 12 2 13 3
+cc 22 2 23 3
+ee 42 2 NULL NULL
+
+
+-- !query 70
+SELECT * FROM
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL FULL JOIN
+(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3
+-- !query 70 schema
+struct
+-- !query 70 output
+bb 12 2 13 3
+cc 22 2 23 3
+dd NULL NULL 33 3
+ee 42 2 NULL NULL
+
+
+-- !query 71
+SELECT * FROM
+(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1
+NATURAL INNER JOIN
+(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2
+NATURAL INNER JOIN
+(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3
+-- !query 71 schema
+struct
+-- !query 71 output
+bb 11 1 12 2 13 3
+
+
+-- !query 72
+SELECT * FROM
+(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1
+NATURAL FULL JOIN
+(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2
+NATURAL FULL JOIN
+(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3
+-- !query 72 schema
+struct
+-- !query 72 output
+bb 11 1 12 2 13 3
+cc NULL NULL 22 2 23 3
+dd NULL NULL NULL NULL 33 3
+ee NULL NULL 42 2 NULL NULL
+
+
+-- !query 73
+SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM
+(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1
+NATURAL FULL JOIN
+ (SELECT * FROM
+ (SELECT name, udf(n) as s2_n FROM t2) as s2
+ NATURAL FULL JOIN
+ (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3
+ ) ss2
+-- !query 73 schema
+struct
+-- !query 73 output
+bb 11 12 13
+cc NULL 22 23
+dd NULL NULL 33
+ee NULL 42 NULL
+
+
+-- !query 74
+SELECT * FROM
+(SELECT name, n as s1_n FROM t1) as s1
+NATURAL FULL JOIN
+ (SELECT * FROM
+ (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2
+ NATURAL FULL JOIN
+ (SELECT name, udf(n) as s3_n FROM t3) as s3
+ ) ss2
+-- !query 74 schema
+struct
+-- !query 74 output
+bb 11 12 2 13
+cc NULL 22 2 23
+dd NULL NULL NULL 33
+ee NULL 42 2 NULL
+
+
+-- !query 75
+SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM
+ (SELECT name, udf(n) as s1_n FROM t1) as s1
+FULL JOIN
+ (SELECT name, 2 as s2_n FROM t2) as s2
+ON (udf(udf(s1_n)) = udf(s2_n))
+-- !query 75 schema
+struct
+-- !query 75 output
+NULL NULL bb 2
+NULL NULL cc 2
+NULL NULL ee 2
+bb 11 NULL NULL
+
+
+-- !query 76
+create or replace temporary view x as select * from
+ (values (1,11), (2,22), (3,null), (4,44), (5,null))
+ as v(x1, x2)
+-- !query 76 schema
+struct<>
+-- !query 76 output
+
+
+
+-- !query 77
+create or replace temporary view y as select * from
+ (values (1,111), (2,222), (3,333), (4,null))
+ as v(y1, y2)
+-- !query 77 schema
+struct<>
+-- !query 77 output
+
+
+
+-- !query 78
+select udf(udf(x1)), udf(x2) from x
+-- !query 78 schema
+struct
+-- !query 78 output
+1 11
+2 22
+3 NULL
+4 44
+5 NULL
+
+
+-- !query 79
+select udf(y1), udf(udf(y2)) from y
+-- !query 79 schema
+struct
+-- !query 79 output
+1 111
+2 222
+3 333
+4 NULL
+
+
+-- !query 80
+select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null)
+-- !query 80 schema
+struct
+-- !query 80 output
+1 11 1 111
+2 22 2 222
+3 NULL NULL NULL
+4 44 4 NULL
+5 NULL NULL NULL
+
+
+-- !query 81
+select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null)
+-- !query 81 schema
+struct
+-- !query 81 output
+1 11 1 111
+2 22 2 222
+3 NULL 3 333
+4 44 NULL NULL
+5 NULL NULL NULL
+
+
+-- !query 82
+select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1))
+-- !query 82 schema
+struct
+-- !query 82 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+3 NULL 3 333 3 NULL
+4 44 4 NULL 4 44
+5 NULL NULL NULL 5 NULL
+
+
+-- !query 83
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1 and udf(x2) is not null)
+-- !query 83 schema
+struct
+-- !query 83 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+3 NULL 3 333 NULL NULL
+4 44 4 NULL 4 44
+5 NULL NULL NULL NULL NULL
+
+
+-- !query 84
+select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null)
+-- !query 84 schema
+struct
+-- !query 84 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+3 NULL 3 333 3 NULL
+4 44 4 NULL NULL NULL
+5 NULL NULL NULL NULL NULL
+
+
+-- !query 85
+select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2)
+on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null)
+-- !query 85 schema
+struct
+-- !query 85 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+3 NULL 3 333 NULL NULL
+4 44 4 NULL 4 44
+5 NULL NULL NULL NULL NULL
+
+
+-- !query 86
+select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2)
+on (udf(x1) = udf(xx1)) where (udf(x2) is not null)
+-- !query 86 schema
+struct
+-- !query 86 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+4 44 4 NULL 4 44
+
+
+-- !query 87
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (udf(x1) = xx1) where (udf(y2) is not null)
+-- !query 87 schema
+struct
+-- !query 87 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+3 NULL 3 333 3 NULL
+
+
+-- !query 88
+select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2)
+on (x1 = udf(xx1)) where (xx2 is not null)
+-- !query 88 schema
+struct
+-- !query 88 output
+1 11 1 111 1 11
+2 22 2 222 2 22
+4 44 4 NULL 4 44
+
+
+-- !query 89
+select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in
+ (select udf(unique1) from tenk1 b join tenk1 c using (unique1)
+ where udf(udf(b.unique2)) = udf(42))
+-- !query 89 schema
+struct
+-- !query 89 output
+1
+
+
+-- !query 90
+select udf(count(*)) from tenk1 x where
+ udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+ udf(x.unique1) = 0 and
+ udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1)))
+-- !query 90 schema
+struct
+-- !query 90 output
+1
+
+
+-- !query 91
+select udf(udf(count(*))) from tenk1 x where
+ udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and
+ udf(x.unique1) = 0 and
+ udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1)))
+-- !query 91 schema
+struct
+-- !query 91 output
+1
+
+
+-- !query 92
+select * from int8_tbl i1 left join (int8_tbl i2 join
+ (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2))
+order by udf(udf(1)), 2
+-- !query 92 schema
+struct
+-- !query 92 output
+4567890123456789 -4567890123456789 NULL NULL NULL
+4567890123456789 123 NULL NULL NULL
+123 456 123 456 123
+123 4567890123456789 123 4567890123456789 123
+4567890123456789 4567890123456789 123 4567890123456789 123
+
+
+-- !query 93
+select udf(count(*))
+from
+ (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2
+ from tenk1 t1
+ left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1)
+ join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss,
+ tenk1 t4,
+ tenk1 t5
+where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1))
+-- !query 93 schema
+struct
+-- !query 93 output
+1000
+
+
+-- !query 94
+select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from
+ tenk1 t,
+ (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a,
+ (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b
+where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous))
+-- !query 94 schema
+struct
+-- !query 94 output
+
+
+
+-- !query 95
+select * from
+ j1_tbl full join
+ (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl
+ on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k)
+-- !query 95 schema
+struct
+-- !query 95 output
+0 NULL zero NULL NULL
+1 4 one NULL NULL
+2 3 two 2 2
+3 2 three NULL NULL
+4 1 four NULL NULL
+5 0 five NULL NULL
+6 6 six NULL NULL
+7 7 seven NULL NULL
+8 8 eight NULL NULL
+NULL 0 zero NULL NULL
+NULL NULL NULL 0 NULL
+NULL NULL NULL 1 -1
+NULL NULL NULL 2 4
+NULL NULL NULL 3 -3
+NULL NULL NULL 5 -5
+NULL NULL NULL 5 -5
+NULL NULL NULL NULL 0
+NULL NULL NULL NULL NULL
+NULL NULL null NULL NULL
+
+
+-- !query 96
+select udf(count(*)) from
+ (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x
+ left join
+ (select * from tenk1 y order by udf(y.unique2)) y
+ on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2
+-- !query 96 schema
+struct
+-- !query 96 output
+10000
+
+
+-- !query 97
+DROP TABLE t1
+-- !query 97 schema
+struct<>
+-- !query 97 output
+
+
+
+-- !query 98
+DROP TABLE t2
+-- !query 98 schema
+struct<>
+-- !query 98 output
+
+
+
+-- !query 99
+DROP TABLE t3
+-- !query 99 schema
+struct<>
+-- !query 99 output
+
+
+
+-- !query 100
+DROP TABLE J1_TBL
+-- !query 100 schema
+struct<>
+-- !query 100 output
+
+
+
+-- !query 101
+DROP TABLE J2_TBL
+-- !query 101 schema
+struct<>
+-- !query 101 output
+
+
+
+-- !query 102
+create or replace temporary view tt1 as select * from
+ (values (1, 11), (2, NULL))
+ as v(tt1_id, joincol)
+-- !query 102 schema
+struct<>
+-- !query 102 output
+
+
+
+-- !query 103
+create or replace temporary view tt2 as select * from
+ (values (21, 11), (22, 11))
+ as v(tt2_id, joincol)
+-- !query 103 schema
+struct<>
+-- !query 103 output
+
+
+
+-- !query 104
+select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol)
+-- !query 104 schema
+struct
+-- !query 104 output
+1 11 21 11
+1 11 22 11
+2 NULL NULL NULL
+
+
+-- !query 105
+select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol))
+-- !query 105 schema
+struct
+-- !query 105 output
+1 11 21 11
+1 11 22 11
+2 NULL NULL NULL
+
+
+-- !query 106
+select udf(count(*)) from tenk1 a, tenk1 b
+ where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10)
+-- !query 106 schema
+struct
+-- !query 106 output
+100000
+
+
+-- !query 107
+DROP TABLE IF EXISTS tt3
+-- !query 107 schema
+struct<>
+-- !query 107 output
+
+
+
+-- !query 108
+CREATE TABLE tt3(f1 int, f2 string) USING parquet
+-- !query 108 schema
+struct<>
+-- !query 108 output
+
+
+
+-- !query 109
+INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x
+-- !query 109 schema
+struct<>
+-- !query 109 output
+
+
+
+-- !query 110
+DROP TABLE IF EXISTS tt4
+-- !query 110 schema
+struct<>
+-- !query 110 output
+
+
+
+-- !query 111
+CREATE TABLE tt4(f1 int) USING parquet
+-- !query 111 schema
+struct<>
+-- !query 111 output
+
+
+
+-- !query 112
+INSERT INTO tt4 VALUES (0),(1),(9999)
+-- !query 112 schema
+struct<>
+-- !query 112 output
+
+
+
+-- !query 113
+SELECT udf(udf(a.f1)) as f1
+FROM tt4 a
+LEFT JOIN (
+ SELECT b.f1
+ FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1)
+ WHERE udf(c.f1) IS NULL
+) AS d ON udf(a.f1) = d.f1
+WHERE udf(udf(d.f1)) IS NULL
+-- !query 113 schema
+struct
+-- !query 113 output
+0
+1
+9999
+
+
+-- !query 114
+create or replace temporary view tt5 as select * from
+ (values (1, 10), (1, 11))
+ as v(f1, f2)
+-- !query 114 schema
+struct<>
+-- !query 114 output
+
+
+
+-- !query 115
+create or replace temporary view tt6 as select * from
+ (values (1, 9), (1, 2), (2, 9))
+ as v(f1, f2)
+-- !query 115 schema
+struct<>
+-- !query 115 output
+
+
+
+-- !query 116
+select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2))
+-- !query 116 schema
+struct
+-- !query 116 output
+1 10 1 9
+
+
+-- !query 117
+create or replace temporary view xx as select * from
+ (values (1), (2), (3))
+ as v(pkxx)
+-- !query 117 schema
+struct<>
+-- !query 117 output
+
+
+
+-- !query 118
+create or replace temporary view yy as select * from
+ (values (101, 1), (201, 2), (301, NULL))
+ as v(pkyy, pkxx)
+-- !query 118 schema
+struct<>
+-- !query 118 output
+
+
+
+-- !query 119
+select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy,
+ udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx
+from yy
+ left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy)
+ left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx))
+ left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx)
+-- !query 119 schema
+struct
+-- !query 119 output
+101 1 101 1 1
+201 2 NULL NULL 1
+301 NULL NULL NULL 1
+
+
+-- !query 120
+create or replace temporary view zt1 as select * from
+ (values (53))
+ as v(f1)
+-- !query 120 schema
+struct<>
+-- !query 120 output
+
+
+
+-- !query 121
+create or replace temporary view zt2 as select * from
+ (values (53))
+ as v(f2)
+-- !query 121 schema
+struct<>
+-- !query 121 output
+
+
+
+-- !query 122
+create or replace temporary view zt3(f3 int) using parquet
+-- !query 122 schema
+struct<>
+-- !query 122 output
+
+
+
+-- !query 123
+select * from
+ zt2 left join zt3 on (udf(f2) = udf(udf(f3)))
+ left join zt1 on (udf(udf(f3)) = udf(f1))
+where udf(f2) = 53
+-- !query 123 schema
+struct
+-- !query 123 output
+53 NULL NULL
+
+
+-- !query 124
+create temp view zv1 as select *,'dummy' AS junk from zt1
+-- !query 124 schema
+struct<>
+-- !query 124 output
+
+
+
+-- !query 125
+select * from
+ zt2 left join zt3 on (f2 = udf(f3))
+ left join zv1 on (udf(f3) = f1)
+where udf(udf(f2)) = 53
+-- !query 125 schema
+struct
+-- !query 125 output
+53 NULL NULL NULL
+
+
+-- !query 126
+select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred)
+from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous)
+where udf(a.unique1) = 42 and
+ ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3)))
+-- !query 126 schema
+struct
+-- !query 126 output
+
+
+
+-- !query 127
+create or replace temporary view a (i integer) using parquet
+-- !query 127 schema
+struct<>
+-- !query 127 output
+
+
+
+-- !query 128
+create or replace temporary view b (x integer, y integer) using parquet
+-- !query 128 schema
+struct<>
+-- !query 128 output
+
+
+
+-- !query 129
+select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i)
+-- !query 129 schema
+struct
+-- !query 129 output
+
+
+
+-- !query 130
+select udf(t1.q2), udf(count(t2.*))
+from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1)
+group by udf(t1.q2) order by 1
+-- !query 130 schema
+struct
+-- !query 130 output
+-4567890123456789 0
+123 2
+456 0
+4567890123456789 6
+
+
+-- !query 131
+select udf(udf(t1.q2)), udf(count(t2.*))
+from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1))
+group by udf(udf(t1.q2)) order by 1
+-- !query 131 schema
+struct
+-- !query 131 output
+-4567890123456789 0
+123 2
+456 0
+4567890123456789 6
+
+
+-- !query 132
+select udf(t1.q2) as q2, udf(udf(count(t2.*)))
+from int8_tbl t1 left join
+ (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2
+ on (udf(t1.q2) = udf(t2.q1))
+group by t1.q2 order by 1
+-- !query 132 schema
+struct
+-- !query 132 output
+-4567890123456789 0
+123 2
+456 0
+4567890123456789 6
+
+
+-- !query 133
+create or replace temporary view a as select * from
+ (values ('p'), ('q'))
+ as v(code)
+-- !query 133 schema
+struct<>
+-- !query 133 output
+
+
+
+-- !query 134
+create or replace temporary view b as select * from
+ (values ('p', 1), ('p', 2))
+ as v(a, num)
+-- !query 134 schema
+struct<>
+-- !query 134 output
+
+
+
+-- !query 135
+create or replace temporary view c as select * from
+ (values ('A', 'p'), ('B', 'q'), ('C', null))
+ as v(name, a)
+-- !query 135 schema
+struct<>
+-- !query 135 output
+
+
+
+-- !query 136
+select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const)
+from c left join
+ (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const
+ from a left join
+ (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp
+ on udf(a.code) = udf(udf(b_grp.a))
+ ) as ss
+ on (udf(udf(c.a)) = udf(ss.code))
+order by c.name
+-- !query 136 schema
+struct
+-- !query 136 output
+A p 2 -1
+B q 0 -1
+C NULL NULL NULL
+
+
+-- !query 137
+SELECT * FROM
+( SELECT 1 as key1 ) sub1
+LEFT JOIN
+( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM
+ ( SELECT 1 as key3 ) sub3
+ LEFT JOIN
+ ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM
+ ( SELECT 1 as key5 ) sub5
+ LEFT JOIN
+ ( SELECT 2 as key6, 42 as value1 ) sub6
+ ON sub5.key5 = udf(sub6.key6)
+ ) sub4
+ ON udf(sub4.key5) = sub3.key3
+) sub2
+ON udf(udf(sub1.key1)) = udf(udf(sub2.key3))
+-- !query 137 schema
+struct
+-- !query 137 output
+1 1 1 1
+
+
+-- !query 138
+SELECT * FROM
+( SELECT 1 as key1 ) sub1
+LEFT JOIN
+( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM
+ ( SELECT 1 as key3 ) sub3
+ LEFT JOIN
+ ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM
+ ( SELECT 1 as key5 ) sub5
+ LEFT JOIN
+ ( SELECT 2 as key6, 42 as value1 ) sub6
+ ON udf(udf(sub5.key5)) = sub6.key6
+ ) sub4
+ ON sub4.key5 = sub3.key3
+) sub2
+ON sub1.key1 = udf(udf(sub2.key3))
+-- !query 138 schema
+struct
+-- !query 138 output
+1 1 1 1
+
+
+-- !query 139
+SELECT udf(qq), udf(udf(unique1))
+ FROM
+ ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1
+ FULL OUTER JOIN
+ ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2
+ USING (qq)
+ INNER JOIN tenk1 c ON udf(qq) = udf(unique2)
+-- !query 139 schema
+struct
+-- !query 139 output
+123 4596
+123 4596
+456 7318
+
+
+-- !query 140
+create or replace temporary view nt1 as select * from
+ (values(1,true,true), (2,true,false), (3,false,false))
+ as v(id, a1, a2)
+-- !query 140 schema
+struct<>
+-- !query 140 output
+
+
+
+-- !query 141
+create or replace temporary view nt2 as select * from
+ (values(1,1,true,true), (2,2,true,false), (3,3,false,false))
+ as v(id, nt1_id, b1, b2)
+-- !query 141 schema
+struct<>
+-- !query 141 output
+
+
+
+-- !query 142
+create or replace temporary view nt3 as select * from
+ (values(1,1,true), (2,2,false), (3,3,true))
+ as v(id, nt2_id, c1)
+-- !query 142 schema
+struct<>
+-- !query 142 output
+
+
+
+-- !query 143
+select udf(nt3.id)
+from nt3 as nt3
+ left join
+ (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3
+ from nt2 as nt2
+ left join
+ (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1
+ on ss1.id = udf(udf(nt2.nt1_id))
+ ) as ss2
+ on udf(ss2.id) = nt3.nt2_id
+where udf(nt3.id) = 1 and udf(ss2.b3)
+-- !query 143 schema
+struct
+-- !query 143 output
+1
+
+
+-- !query 144
+select * from int4_tbl a full join int4_tbl b on true
+-- !query 144 schema
+struct
+-- !query 144 output
+-123456 -123456
+-123456 -2147483647
+-123456 0
+-123456 123456
+-123456 2147483647
+-2147483647 -123456
+-2147483647 -2147483647
+-2147483647 0
+-2147483647 123456
+-2147483647 2147483647
+0 -123456
+0 -2147483647
+0 0
+0 123456
+0 2147483647
+123456 -123456
+123456 -2147483647
+123456 0
+123456 123456
+123456 2147483647
+2147483647 -123456
+2147483647 -2147483647
+2147483647 0
+2147483647 123456
+2147483647 2147483647
+
+
+-- !query 145
+select * from int4_tbl a full join int4_tbl b on false
+-- !query 145 schema
+struct
+-- !query 145 output
+-123456 NULL
+-2147483647 NULL
+0 NULL
+123456 NULL
+2147483647 NULL
+NULL -123456
+NULL -2147483647
+NULL 0
+NULL 123456
+NULL 2147483647
+
+
+-- !query 146
+select udf(count(*)) from
+ tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+ left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand))
+ join int4_tbl on udf(b.thousand) = f1
+-- !query 146 schema
+struct
+-- !query 146 output
+10
+
+
+-- !query 147
+select udf(b.unique1) from
+ tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2)
+ left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand)
+ join int4_tbl i1 on udf(b.thousand) = udf(udf(f1))
+ right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous)
+ order by udf(1)
+-- !query 147 schema
+struct
+-- !query 147 output
+NULL
+NULL
+0
+NULL
+NULL
+
+
+-- !query 148
+select * from
+(
+ select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault
+ from int8_tbl left join tenk1 on (udf(q2) = udf(unique2))
+) ss
+where udf(fault) = udf(122)
+order by udf(fault)
+-- !query 148 schema
+struct
+-- !query 148 output
+NULL 123 122
+
+
+-- !query 149
+select udf(q1), udf(unique2), udf(thousand), udf(hundred)
+ from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2)
+ where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123)))
+-- !query 149 schema
+struct
+-- !query 149 output
+
+
+
+-- !query 150
+select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end
+ from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2))
+ where (case when udf(unique2) is null then udf(f1) else 0 end) = 0
+-- !query 150 schema
+struct
+-- !query 150 output
+0 0 0
+
+
+-- !query 151
+select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand))
+ from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand))
+ where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44)
+-- !query 151 schema
+struct
+-- !query 151 output
+
+
+
+-- !query 152
+select * from
+ text_tbl t1
+ inner join int8_tbl i8
+ on udf(i8.q2) = udf(udf(456))
+ right join text_tbl t2
+ on udf(t1.f1) = udf(udf('doh!'))
+ left join int4_tbl i4
+ on udf(udf(i8.q1)) = i4.f1
+-- !query 152 schema
+struct
+-- !query 152 output
+doh! 123 456 doh! NULL
+doh! 123 456 hi de ho neighbor NULL
+
+
+-- !query 153
+select * from
+ (select udf(udf(1)) as id) as xx
+ left join
+ (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id)))
+ on (xx.id = udf(udf(coalesce(yy.id))))
+-- !query 153 schema
+struct
+-- !query 153 output
+1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1
+
+
+-- !query 154
+select udf(a.q2), udf(b.q1)
+ from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1)
+ where udf(udf(coalesce(b.q1, 1)) > 0)
+-- !query 154 schema
+struct
+-- !query 154 output
+-4567890123456789 NULL
+123 123
+123 123
+456 NULL
+4567890123456789 4567890123456789
+4567890123456789 4567890123456789
+4567890123456789 4567890123456789
+4567890123456789 4567890123456789
+4567890123456789 4567890123456789
+4567890123456789 4567890123456789
+
+
+-- !query 155
+create or replace temporary view parent as select * from
+ (values (1, 10), (2, 20), (3, 30))
+ as v(k, pd)
+-- !query 155 schema
+struct<>
+-- !query 155 output
+
+
+
+-- !query 156
+create or replace temporary view child as select * from
+ (values (1, 100), (4, 400))
+ as v(k, cd)
+-- !query 156 schema
+struct<>
+-- !query 156 output
+
+
+
+-- !query 157
+select p.* from parent p left join child c on (udf(p.k) = udf(c.k))
+-- !query 157 schema
+struct
+-- !query 157 output
+1 10
+2 20
+3 30
+
+
+-- !query 158
+select p.*, linked from parent p
+ left join (select c.*, udf(udf(true)) as linked from child c) as ss
+ on (udf(p.k) = udf(udf(ss.k)))
+-- !query 158 schema
+struct
+-- !query 158 output
+1 10 true
+2 20 NULL
+3 30 NULL
+
+
+-- !query 159
+select p.* from
+ parent p left join child c on (udf(p.k) = c.k)
+ where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2))
+-- !query 159 schema
+struct
+-- !query 159 output
+
+
+
+-- !query 160
+select p.* from
+ (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k)
+ where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2))
+-- !query 160 schema
+struct
+-- !query 160 output
+
+
+
+-- !query 161
+create or replace temporary view a as select * from
+ (values (0), (1))
+ as v(id)
+-- !query 161 schema
+struct<>
+-- !query 161 output
+
+
+
+-- !query 162
+create or replace temporary view b as select * from
+ (values (0, 0), (1, NULL))
+ as v(id, a_id)
+-- !query 162 schema
+struct<>
+-- !query 162 output
+
+
+
+-- !query 163
+SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0)
+-- !query 163 schema
+struct
+-- !query 163 output
+1 NULL NULL
+
+
+-- !query 164
+SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0)
+-- !query 164 schema
+struct
+-- !query 164 output
+1 NULL
+
+
+-- !query 165
+create or replace temporary view innertab as select * from
+ (values (123L, 42L))
+ as v(id, dat1)
+-- !query 165 schema
+struct<>
+-- !query 165 output
+
+
+
+-- !query 166
+SELECT * FROM
+ (SELECT udf(1) AS x) ss1
+ LEFT JOIN
+ (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y
+ FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2
+ ON true
+-- !query 166 schema
+struct
+-- !query 166 output
+1 123 456 123
+1 123 4567890123456789 123
+1 4567890123456789 -4567890123456789 4567890123456789
+1 4567890123456789 123 42
+1 4567890123456789 4567890123456789 4567890123456789
+
+
+-- !query 167
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1)
+-- !query 167 schema
+struct<>
+-- !query 167 output
+org.apache.spark.sql.AnalysisException
+Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 72
+
+
+-- !query 168
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1)
+-- !query 168 schema
+struct<>
+-- !query 168 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72
+
+
+-- !query 169
+select * from
+ int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1))
+-- !query 169 schema
+struct
+-- !query 169 output
+
+
+
+-- !query 170
+select udf(t1.uunique1) from
+ tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two)
+-- !query 170 schema
+struct<>
+-- !query 170 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11
+
+
+-- !query 171
+select udf(udf(t2.uunique1)) from
+ tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two
+-- !query 171 schema
+struct<>
+-- !query 171 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 15
+
+
+-- !query 172
+select udf(uunique1) from
+ tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two)
+-- !query 172 schema
+struct<>
+-- !query 172 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11
+
+
+-- !query 173
+select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss
+-- !query 173 schema
+struct<>
+-- !query 173 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`f1`' given input columns: []; line 1 pos 55
+
+
+-- !query 174
+select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss
+-- !query 174 schema
+struct<>
+-- !query 174 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`a.f1`' given input columns: []; line 1 pos 42
+
+
+-- !query 175
+select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss
+-- !query 175 schema
+struct<>
+-- !query 175 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`f1`' given input columns: []; line 1 pos 61
+
+
+-- !query 176
+select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss
+-- !query 176 schema
+struct<>
+-- !query 176 output
+org.apache.spark.sql.AnalysisException
+cannot resolve '`a.f1`' given input columns: []; line 1 pos 60
+
+
+-- !query 177
+CREATE TABLE j1 (id1 int, id2 int) USING parquet
+-- !query 177 schema
+struct<>
+-- !query 177 output
+
+
+
+-- !query 178
+CREATE TABLE j2 (id1 int, id2 int) USING parquet
+-- !query 178 schema
+struct<>
+-- !query 178 output
+
+
+
+-- !query 179
+INSERT INTO j1 values(1,1),(1,2)
+-- !query 179 schema
+struct<>
+-- !query 179 output
+
+
+
+-- !query 180
+INSERT INTO j2 values(1,1)
+-- !query 180 schema
+struct<>
+-- !query 180 output
+
+
+
+-- !query 181
+INSERT INTO j2 values(1,2)
+-- !query 181 schema
+struct<>
+-- !query 181 output
+
+
+
+-- !query 182
+select * from j1
+inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2)
+where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1
+-- !query 182 schema
+struct
+-- !query 182 output
+1 1 1 1
+1 2 1 2
+
+
+-- !query 183
+drop table j1
+-- !query 183 schema
+struct<>
+-- !query 183 output
+
+
+
+-- !query 184
+drop table j2
+-- !query 184 schema
+struct<>
+-- !query 184 output
+
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala
index c7d86bc955d67..171e93c1bf002 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala
@@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, Literal}
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproxCountDistinctForIntervals
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
import org.apache.spark.sql.execution.QueryExecution
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSQLContext {
+class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
// ApproxCountDistinctForIntervals is used in equi-height histogram generation. An equi-height
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 52708f5fe4108..a4b142b7ab78e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -23,12 +23,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.PercentileDigest
import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* End-to-end tests for approximate percentile aggregate function.
*/
-class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext {
+class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
private val table = "percentile_test"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala
index c37d663941d8d..3fcb9892800b6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala
@@ -17,16 +17,14 @@
package org.apache.spark.sql
-import org.scalatest.BeforeAndAfterAll
-
import org.apache.spark.internal.config.Tests.IS_TESTING
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
-abstract class BenchmarkQueryTest extends QueryTest with SharedSQLContext with BeforeAndAfterAll {
+abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession {
// When Utils.isTesting is true, the RuleExecutor will issue an exception when hitting
// the max iteration of analyzer/optimizer batches.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index 267f255a11e86..6e1ee6da9200d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.columnar._
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.storage.{RDDBlockId, StorageLevel}
import org.apache.spark.storage.StorageLevel.{MEMORY_AND_DISK_2, MEMORY_ONLY}
@@ -40,7 +40,7 @@ import org.apache.spark.util.{AccumulatorContext, Utils}
private case class BigData(s: String)
-class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext {
+class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
index ee258f8ac3661..a52c6d503d147 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala
@@ -30,10 +30,10 @@ import org.apache.spark.sql.catalyst.expressions.NamedExpression
import org.apache.spark.sql.execution.ProjectExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
+class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
import testImplicits._
private lazy val booleanData = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala
index b74fe2f90df23..4d0eb04be751b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ComplexTypesSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql
import org.apache.spark.sql.catalyst.expressions.CreateNamedStruct
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ComplexTypesSuite extends QueryTest with SharedSQLContext {
+class ComplexTypesSuite extends QueryTest with SharedSparkSession {
override def beforeAll() {
super.beforeAll()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala
index 276496be3d62c..431e797e1686e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala
@@ -20,10 +20,10 @@ package org.apache.spark.sql
import org.apache.commons.math3.stat.inference.ChiSquareTest
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ConfigBehaviorSuite extends QueryTest with SharedSQLContext {
+class ConfigBehaviorSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala
index dea0d4c0c6d40..5a80bdcdc0f35 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala
@@ -17,13 +17,13 @@
package org.apache.spark.sql
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.sketch.CountMinSketch
/**
* End-to-end test suite for count_min_sketch.
*/
-class CountMinSketchAggQuerySuite extends QueryTest with SharedSQLContext {
+class CountMinSketchAggQuerySuite extends QueryTest with SharedSparkSession {
test("count-min sketch") {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
index 6b67fccf86b9f..52cf91cfade51 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala
@@ -25,10 +25,10 @@ import scala.collection.JavaConverters._
import org.apache.spark.SparkException
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class CsvFunctionsSuite extends QueryTest with SharedSQLContext {
+class CsvFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("from_csv with empty options") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index c56c93f708577..ec7b636c8f695 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -27,13 +27,13 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
import org.apache.spark.sql.expressions.Window
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.test.SQLTestData.DecimalData
import org.apache.spark.sql.types.DecimalType
case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double)
-class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
+class DataFrameAggregateSuite extends QueryTest with SharedSparkSession {
import testImplicits._
val absTol = 1e-8
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
index 1230b921aa279..e9179a39d3b6d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
@@ -19,12 +19,12 @@ package org.apache.spark.sql
import org.apache.spark.sql.catalyst.DefinedByConstructorParams
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* A test suite to test DataFrame/SQL functionalities with complex types (i.e. array, struct, map).
*/
-class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext {
+class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("UDF on struct") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 69d442a86c29e..7d044638db571 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -30,13 +30,13 @@ import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
* Test suite for functions in [[org.apache.spark.sql.functions]].
*/
-class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
+class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("array with column name") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala
index 7ef8b542c79a8..b33c26a0b75a2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameHintSuite extends AnalysisTest with SharedSQLContext {
+class DataFrameHintSuite extends AnalysisTest with SharedSparkSession {
import testImplicits._
lazy val df = spark.range(10)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
index 25e1d93ff0927..a454d91a8e62c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameImplicitsSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameImplicitsSuite extends QueryTest with SharedSQLContext {
+class DataFrameImplicitsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("RDD of tuples") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
index dc7928fde7791..3a217e6e28060 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala
@@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.plans.logical.Join
import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameJoinSuite extends QueryTest with SharedSQLContext {
+class DataFrameJoinSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("join - join using") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
index e6983b6be555a..aeee4577d3483 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql
import scala.collection.JavaConverters._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext {
+class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
def createDF(): DataFrame = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
index 8c2c11be9b6fd..bcd0c3f0d64a7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala
@@ -22,10 +22,10 @@ import java.util.Locale
import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class DataFramePivotSuite extends QueryTest with SharedSQLContext {
+class DataFramePivotSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("pivot courses") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
index 6c1a66cae2279..fc549e307c80f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala
@@ -27,10 +27,10 @@ import org.apache.spark.SparkException
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventually {
+class DataFrameRangeSuite extends QueryTest with SharedSparkSession with Eventually {
test("SPARK-7150 range api") {
// numSlice is greater than length
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
index 30452af1fad64..fbb7e903c3450 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala
@@ -22,11 +22,11 @@ import java.sql.{Date, Timestamp}
import org.apache.spark.sql.catalyst.plans.logical.Union
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
+import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
import org.apache.spark.sql.test.SQLTestData.NullStrings
import org.apache.spark.sql.types._
-class DataFrameSetOperationsSuite extends QueryTest with SharedSQLContext {
+class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("except") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala
index 2a74bfe4d3789..0572728d68226 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala
@@ -25,10 +25,10 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.execution.stat.StatFunctions
import org.apache.spark.sql.functions.{col, lit, struct}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
-class DataFrameStatSuite extends QueryTest with SharedSQLContext {
+class DataFrameStatSuite extends QueryTest with SharedSparkSession {
import testImplicits._
private def toLetter(i: Int): String = (i + 97).toChar.toString
@@ -449,7 +449,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext {
}
-class DataFrameStatPerfSuite extends QueryTest with SharedSQLContext with Logging {
+class DataFrameStatPerfSuite extends QueryTest with SharedSparkSession with Logging {
// Turn on this test if you want to test the performance of approximate quantiles.
ignore("computing quantiles should not take much longer than describe()") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 8782f95fe517f..c6daff1479fb9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -37,13 +37,13 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
-import org.apache.spark.sql.test.SQLTestData.{NullStrings, TestData2}
+import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
+import org.apache.spark.sql.test.SQLTestData.{DecimalData, NullStrings, TestData2}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
import org.apache.spark.util.random.XORShiftRandom
-class DataFrameSuite extends QueryTest with SharedSQLContext {
+class DataFrameSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("analysis error should be eagerly reported") {
@@ -156,6 +156,27 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
structDf.select(xxhash64($"a", $"record.*")))
}
+ test("SPARK-28224: Aggregate sum big decimal overflow") {
+ val largeDecimals = spark.sparkContext.parallelize(
+ DecimalData(BigDecimal("1"* 20 + ".123"), BigDecimal("1"* 20 + ".123")) ::
+ DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF()
+
+ Seq(true, false).foreach { nullOnOverflow =>
+ withSQLConf((SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key, nullOnOverflow.toString)) {
+ val structDf = largeDecimals.select("a").agg(sum("a"))
+ if (nullOnOverflow) {
+ checkAnswer(structDf, Row(null))
+ } else {
+ val e = intercept[SparkException] {
+ structDf.collect
+ }
+ assert(e.getCause.getClass.equals(classOf[ArithmeticException]))
+ assert(e.getCause.getMessage.contains("cannot be represented as Decimal"))
+ }
+ }
+ }
+ }
+
test("Star Expansion - explode should fail with a meaningful message if it takes a star") {
val df = Seq(("1,2"), ("4"), ("7,8,9")).toDF("csv")
val e = intercept[AnalysisException] {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala
index 2953425b1db49..8b0f46b9d1ddb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala
@@ -21,10 +21,10 @@ import org.scalatest.BeforeAndAfterEach
import org.apache.spark.sql.catalyst.plans.logical.Expand
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StringType
-class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
+class DataFrameTimeWindowingSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
index 0881212a64de8..a9f7d5bfab23e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
@@ -27,7 +27,7 @@ import org.apache.spark.sql.types._
* This is here for now so I can make sure Tungsten project is tested without refactoring existing
* end-to-end test infra. In the long run this should just go away.
*/
-class DataFrameTungstenSuite extends QueryTest with SharedSQLContext {
+class DataFrameTungstenSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("test simple types") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala
index 002c17f4cce4f..fbd399917e390 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala
@@ -21,12 +21,12 @@ import java.sql.Date
import org.apache.spark.sql.expressions.Window
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* Window frame testing for DataFrame API.
*/
-class DataFrameWindowFramesSuite extends QueryTest with SharedSQLContext {
+class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("lead/lag with empty data frame") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
index a07553008fd78..835630bff7099 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
@@ -25,13 +25,13 @@ import org.apache.spark.sql.execution.exchange.Exchange
import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
* Window function testing for DataFrame API.
*/
-class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext {
+class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala
index e581211e4e76f..817387b2845f5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.expressions.scalalang.typed
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType}
@@ -221,7 +221,7 @@ case class OptionBooleanIntAggregator(colName: String)
def OptionalBoolIntEncoder: Encoder[Option[(Boolean, Int)]] = ExpressionEncoder()
}
-class DatasetAggregatorSuite extends QueryTest with SharedSQLContext {
+class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
import testImplicits._
private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala
index d888585e120a4..33d9def0b44e5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala
@@ -22,11 +22,11 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec}
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.storage.StorageLevel
-class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits {
+class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimits {
import testImplicits._
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala
index ae051e43fbcb5..892122b94b977 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala
@@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.objects.ExternalMapToCatalyst
import org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
import org.apache.spark.sql.functions.expr
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DatasetOptimizationSuite extends QueryTest with SharedSQLContext {
+class DatasetOptimizationSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("SPARK-26619: Prune the unused serializers from SerializeFromObject") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala
index 4d7037f36b1fd..91a8f0a26b360 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala
@@ -22,7 +22,7 @@ import scala.collection.immutable.Queue
import scala.collection.mutable.{LinkedHashMap => LHMap}
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
case class IntClass(value: Int)
@@ -47,7 +47,7 @@ package object packageobject {
case class PackageClass(value: Int)
}
-class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext {
+class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("toDS") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
index 69728efcd1d9a..43de2663b1236 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
@@ -23,12 +23,12 @@ import com.esotericsoftware.kryo.io.{Input, Output}
import org.apache.spark.SparkConf
import org.apache.spark.internal.config.Kryo._
import org.apache.spark.serializer.KryoRegistrator
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* Test suite to test Kryo custom registrators.
*/
-class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext {
+class DatasetSerializerRegistratorSuite extends QueryTest with SharedSparkSession {
import testImplicits._
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 ff6143162ff2f..42e5ee58954e8 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
@@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.expressions.UserDefinedFunction
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2)
@@ -49,7 +49,7 @@ object TestForTypeAlias {
def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2))
}
-class DatasetSuite extends QueryTest with SharedSQLContext {
+class DatasetSuite extends QueryTest with SharedSparkSession {
import testImplicits._
private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b)
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 69f17f5e9c2dd..2fef05f97e57c 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
@@ -26,10 +26,10 @@ import java.util.concurrent.TimeUnit
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.unsafe.types.CalendarInterval
-class DateFunctionsSuite extends QueryTest with SharedSQLContext {
+class DateFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("function current_date") {
@@ -239,6 +239,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
df.select(date_add(col("ss"), 7)),
Seq(Row(Date.valueOf("2015-06-08")), Row(Date.valueOf("2015-06-09"))))
+ checkAnswer(
+ df.withColumn("x", lit(1)).select(date_add(col("d"), col("x"))),
+ Seq(Row(Date.valueOf("2015-06-02")), Row(Date.valueOf("2015-06-03"))))
+
checkAnswer(df.selectExpr("DATE_ADD(null, 1)"), Seq(Row(null), Row(null)))
checkAnswer(
df.selectExpr("""DATE_ADD(d, 1)"""),
@@ -270,6 +274,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
checkAnswer(
df.select(date_sub(lit(null), 1)).limit(1), Row(null))
+ checkAnswer(
+ df.withColumn("x", lit(1)).select(date_sub(col("d"), col("x"))),
+ Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+
checkAnswer(df.selectExpr("""DATE_SUB(d, null)"""), Seq(Row(null), Row(null)))
checkAnswer(
df.selectExpr("""DATE_SUB(d, 1)"""),
@@ -318,6 +326,9 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext {
checkAnswer(
df.selectExpr("add_months(d, -1)"),
Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-28"))))
+ checkAnswer(
+ df.withColumn("x", lit(1)).select(add_months(col("d"), col("x"))),
+ Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-28"))))
}
test("function months_between") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
index ec688282d5740..125cff0e6628a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
-class ExplainSuite extends QueryTest with SharedSQLContext {
+class ExplainSuite extends QueryTest with SharedSparkSession {
import testImplicits._
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala
index a41b465548622..9192370cfa620 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
case class FastOperator(output: Seq[Attribute]) extends SparkPlan {
@@ -46,7 +46,7 @@ object TestStrategy extends Strategy {
}
}
-class ExtraStrategiesSuite extends QueryTest with SharedSQLContext {
+class ExtraStrategiesSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("insert an extraStrategy") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
index 51e26d42812ce..4b24add42d8d8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
@@ -24,19 +24,21 @@ import java.util.Locale
import scala.collection.mutable
import org.apache.hadoop.fs.Path
-import org.scalatest.BeforeAndAfterAll
import org.apache.spark.SparkException
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with BeforeAndAfterAll {
+class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession {
import testImplicits._
override def beforeAll(): Unit = {
@@ -707,6 +709,27 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo
}
}
}
+
+ test("File table location should include both values of option `path` and `paths`") {
+ withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") {
+ withTempPaths(3) { paths =>
+ paths.zipWithIndex.foreach { case (path, index) =>
+ Seq(index).toDF("a").write.mode("overwrite").parquet(path.getCanonicalPath)
+ }
+ val df = spark
+ .read
+ .option("path", paths.head.getCanonicalPath)
+ .parquet(paths(1).getCanonicalPath, paths(2).getCanonicalPath)
+ df.queryExecution.optimizedPlan match {
+ case PhysicalOperation(_, _, DataSourceV2Relation(table: ParquetTable, _, _)) =>
+ assert(table.paths.toSet == paths.map(_.getCanonicalPath).toSet)
+ case _ =>
+ throw new AnalysisException("Can not match ParquetTable in the query.")
+ }
+ checkAnswer(df, Seq(0, 1, 2).map(Row(_)))
+ }
+ }
+ }
}
object TestingUDT {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
index 8280a3ce39845..951e246072d39 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala
@@ -22,10 +22,10 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Generator}
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StructType}
-class GeneratorFunctionSuite extends QueryTest with SharedSQLContext {
+class GeneratorFunctionSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("stack") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala
index 6bda1afdfdd42..6b154253e6e6c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala
@@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class JoinHintSuite extends PlanTest with SharedSQLContext {
+class JoinHintSuite extends PlanTest with SharedSparkSession {
import testImplicits._
lazy val df = spark.range(10)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
index 059dbf892c653..1e97347f6751a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala
@@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec}
import org.apache.spark.sql.execution.joins._
import org.apache.spark.sql.execution.python.BatchEvalPythonExec
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
-class JoinSuite extends QueryTest with SharedSQLContext {
+class JoinSuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index ae6e9037acd21..92a4acc130be5 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -25,10 +25,10 @@ import collection.JavaConverters._
import org.apache.spark.SparkException
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class JsonFunctionsSuite extends QueryTest with SharedSQLContext {
+class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("function get_json_object") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala
index d3db3b40e59a1..567bcdd1878a8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala
@@ -21,14 +21,14 @@ import java.nio.charset.StandardCharsets
import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions.{log => logarithm}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
private object MathFunctionsTestData {
case class DoubleData(a: java.lang.Double, b: java.lang.Double)
case class NullDoubles(a: java.lang.Double)
}
-class MathFunctionsSuite extends QueryTest with SharedSQLContext {
+class MathFunctionsSuite extends QueryTest with SharedSparkSession {
import MathFunctionsTestData._
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala
index 602951bc48a1e..a8993bca657c3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala
@@ -21,12 +21,12 @@ import java.io.File
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* Test suite to handle metadata cache related.
*/
-abstract class MetadataCacheSuite extends QueryTest with SharedSQLContext {
+abstract class MetadataCacheSuite extends QueryTest with SharedSparkSession {
/** Removes one data file in the given directory. */
protected def deleteOneFileInDirectory(dir: File): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
index a5b08f717767f..cad0821dbf5aa 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class MiscFunctionsSuite extends QueryTest with SharedSQLContext {
+class MiscFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("reflect and java_method") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
index d034e10e0d077..3039a4ccb677c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -416,7 +416,7 @@ object QueryTest {
}
}
-class QueryTestSuite extends QueryTest with test.SharedSQLContext {
+class QueryTestSuite extends QueryTest with test.SharedSparkSession {
test("SPARK-16940: checkAnswer should raise TestFailedException for wrong results") {
intercept[org.scalatest.exceptions.TestFailedException] {
checkAnswer(sql("SELECT 1"), Row(2) :: Nil)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala
index 1729c3c0c557a..bdbb741f24bc6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala
@@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.expressions.{CaseWhen, If, Literal}
import org.apache.spark.sql.execution.LocalTableScanExec
import org.apache.spark.sql.functions.{lit, when}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.BooleanType
-class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSQLContext {
+class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("SPARK-25860: Replace Literal(null, _) with FalseLiteral whenever possible") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
index c53fd5bd7a158..fd9655fdbef42 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala
@@ -19,11 +19,11 @@ package org.apache.spark.sql
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, SpecificInternalRow}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
-class RowSuite extends SparkFunSuite with SharedSQLContext {
+class RowSuite extends SparkFunSuite with SharedSparkSession {
import testImplicits._
test("create row") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 972950669198c..a8664ad66c66e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -33,11 +33,11 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
+import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext}
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._
-class SQLQuerySuite extends QueryTest with SharedSQLContext {
+class SQLQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
index 4bdf25051127c..a32bc8b2f8a35 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile}
import org.apache.spark.sql.execution.HiveResult.hiveResultString
import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
/**
@@ -102,7 +102,7 @@ import org.apache.spark.sql.types.StructType
* Therefore, UDF test cases should have single input and output files but executed by three
* different types of UDFs. See 'udf/udf-inner-join.sql' as an example.
*/
-class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
+class SQLQueryTestSuite extends QueryTest with SharedSparkSession {
import IntegratedUDFTestUtils._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
index c9bd05d0e4e36..7e305e0504729 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql
import java.sql.{Date, Timestamp}
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
case class ReflectData(
stringField: String,
@@ -74,7 +74,7 @@ case class ComplexReflectData(
mapFieldContainsNull: Map[Int, Option[Long]],
dataField: Data)
-class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext {
+class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession {
import testImplicits._
test("query case class RDD") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
index 1a1c956aed3d9..3ca21a3d02f08 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.serializer.JavaSerializer
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SerializationSuite extends SparkFunSuite with SharedSQLContext {
+class SerializationSuite extends SparkFunSuite with SharedSparkSession {
test("[SPARK-5235] SQLContext should be serializable") {
val spark = SparkSession.builder.getOrCreate()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala
index ada3baf3cd720..42307b1b9734e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala
@@ -19,10 +19,10 @@ package org.apache.spark.sql
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTable
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
import org.apache.spark.util.Utils
-class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSQLContext
+class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSparkSession
abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
index 4c78f85a3ac69..e9ceab6724659 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.test.SQLTestData.ArrayData
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -39,7 +39,7 @@ import org.apache.spark.util.Utils
/**
* End-to-end suite testing statistics collection and use on both entire table and columns.
*/
-class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSQLContext {
+class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession {
import testImplicits._
test("estimates the size of a limit 0 on outer join") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
index cc5af327cf599..88b3e5ec61f8a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class StringFunctionsSuite extends QueryTest with SharedSQLContext {
+class StringFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("string concat") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index b46abdb48e738..a1d7792941ed9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.FileScanRDD
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SubquerySuite extends QueryTest with SharedSQLContext {
+class SubquerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala
index c5fb17345222a..f2b608bbcb21b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala
@@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggreg
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.expressions.Window
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext {
+class TypedImperativeAggregateSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 08a98e29d2ad0..2a034bcdc3f00 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectComm
import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
import org.apache.spark.sql.functions.{lit, udf}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.test.SQLTestData._
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.QueryExecutionListener
@@ -36,7 +36,7 @@ import org.apache.spark.sql.util.QueryExecutionListener
private case class FunctionResult(f1: String, f2: String)
-class UDFSuite extends QueryTest with SharedSQLContext {
+class UDFSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("built-in fixed arity expressions") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
index 49f0000212554..2b2fedd3ca218 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal}
import org.apache.spark.sql.execution.datasources.parquet.ParquetTest
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseVector) {
@@ -101,7 +101,7 @@ private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType]
override def userClass: Class[IExampleSubType] = classOf[IExampleSubType]
}
-class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest
+class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with ParquetTest
with ExpressionEvalHelper {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala
index 1d33e7970be8e..a25cca7af50bd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala
@@ -17,12 +17,12 @@
package org.apache.spark.sql
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* End-to-end tests for xpath expressions.
*/
-class XPathFunctionsSuite extends QueryTest with SharedSQLContext {
+class XPathFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("xpath_boolean") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
index 7cfee49575570..3fb5a4aa4f3ed 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql.api.r
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SQLUtilsSuite extends SharedSQLContext {
+class SQLUtilsSuite extends SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala
index 4e39df928603f..43e29c2d50786 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala
@@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec
import org.apache.spark.sql.execution.joins.HashedRelation
import org.apache.spark.sql.functions.broadcast
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class BroadcastExchangeSuite extends SparkPlanTest with SharedSQLContext {
+class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits._
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 20e864ab24b71..1051410cc0997 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
@@ -23,12 +23,12 @@ import org.apache.spark.sql.{DataFrame, QueryTest}
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* Test suite base for testing the redaction of DataSourceScanExec/BatchScanExec.
*/
-abstract class DataSourceScanRedactionTest extends QueryTest with SharedSQLContext {
+abstract class DataSourceScanRedactionTest extends QueryTest with SharedSparkSession {
override protected def sparkConf: SparkConf = super.sparkConf
.set("spark.redaction.string.regex", "file:/[^\\]\\s]+")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
index bde2de5b39fd7..4828c4973731f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala
@@ -25,9 +25,9 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityB
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ExchangeSuite extends SparkPlanTest with SharedSQLContext {
+class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits._
test("shuffling UnsafeRows in exchange") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala
index 972b47e96fe06..7fbfa73623c85 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala
@@ -21,10 +21,10 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.catalog.Table
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
-class GlobalTempViewSuite extends QueryTest with SharedSQLContext {
+class GlobalTempViewSuite extends QueryTest with SharedSparkSession {
import testImplicits._
override protected def beforeAll(): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
index bbce4705871df..104cf4c58d617 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala
@@ -19,10 +19,9 @@ package org.apache.spark.sql.execution
import java.sql.{Date, Timestamp}
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
+import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
-class HiveResultSuite extends SparkFunSuite with SharedSQLContext {
+class HiveResultSuite extends SharedSparkSession {
import testImplicits._
test("date formatting in hive result") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala
index e20a82ba9bc48..1ed3e07d1dbe9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala
@@ -23,9 +23,9 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_METADATA_ONLY
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext {
+class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
override def beforeAll(): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
index e30fb13d10df1..2bafc87bc25f2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -29,10 +29,10 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchan
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class PlannerSuite extends SharedSQLContext {
+class PlannerSuite extends SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
index 39c87c9eeb47d..eca39f3f81726 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
case class QueryExecutionTestRecord(
c0: Int, c1: Int, c2: Int, c3: Int, c4: Int,
@@ -33,7 +33,7 @@ case class QueryExecutionTestRecord(
c20: Int, c21: Int, c22: Int, c23: Int, c24: Int,
c25: Int, c26: Int)
-class QueryExecutionSuite extends SharedSQLContext {
+class QueryExecutionSuite extends SharedSparkSession {
import testImplicits._
def checkDumpedPlans(path: String, expected: Int): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala
index e42177c156ee9..76006efda992f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql.execution
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext {
+class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession {
test("programmatic API") {
val df = spark.range(1000).selectExpr("count(*)")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala
index 35c33a7157d38..b5dbdd0b18b49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala
@@ -587,4 +587,22 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA
}
withSparkSession(test, 200, None)
}
+
+ test("Union two datasets with different pre-shuffle partition number") {
+ val test: SparkSession => Unit = { spark: SparkSession =>
+ val df1 = spark.range(3).join(spark.range(3), "id").toDF()
+ val df2 = spark.range(3).groupBy().sum()
+
+ val resultDf = df1.union(df2)
+
+ checkAnswer(resultDf, Seq((0), (1), (2), (3)).map(i => Row(i)))
+
+ val finalPlan = resultDf.queryExecution.executedPlan
+ .asInstanceOf[AdaptiveSparkPlanExec].executedPlan
+ // As the pre-shuffle partition number are different, we will skip reducing
+ // the shuffle partition numbers.
+ assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 0)
+ }
+ withSparkSession(test, 100, None)
+ }
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
index 8d4a9ae6a5760..347a14c9129f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
@@ -21,9 +21,9 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
-class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext
+class SimpleSQLViewSuite extends SQLViewSuite with SharedSparkSession
/**
* A suite for testing view related functionality.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
index 971fd842f046a..7aabf4d039f08 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
import org.apache.spark.TestUtils.assertSpilled
import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
case class WindowData(month: Int, area: String, product: Int)
@@ -28,7 +28,7 @@ case class WindowData(month: Int, area: String, product: Int)
/**
* Test suite for SQL window functions.
*/
-class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext {
+class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala
index b252100d890e3..de1acd3ff2734 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala
@@ -23,13 +23,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project}
import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.IntegerType
/**
* Tests for the sameResult function for [[SparkPlan]]s.
*/
-class SameResultSuite extends QueryTest with SharedSQLContext {
+class SameResultSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("FileSourceScanExec: different orders of data filters and partition filters") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala
index a7bbe34f4eedb..7654a9d982059 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala
@@ -22,14 +22,14 @@ import scala.util.Random
import org.apache.spark.AccumulatorSuite
import org.apache.spark.sql.{RandomDataGenerator, Row}
import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
* Test sorting. Many of the test cases generate random data and compares the sorted result with one
* sorted by a reference implementation ([[ReferenceSort]]).
*/
-class SortSuite extends SparkPlanTest with SharedSQLContext {
+class SortSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits.newProductEncoder
import testImplicits.localSeqToDatasetHolder
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala
index e9d0556ebb513..1a29c471a77af 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala
@@ -21,9 +21,9 @@ import org.apache.spark.SparkEnv
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SparkPlanSuite extends QueryTest with SharedSQLContext {
+class SparkPlanSuite extends QueryTest with SharedSparkSession {
test("SPARK-21619 execution of a canonicalized plan should fail") {
val plan = spark.range(10).queryExecution.executedPlan.canonicalized
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala
index 5828f9783da42..9107f8afa83d7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql.execution
import org.apache.spark.sql.Strategy
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SparkPlannerSuite extends SharedSQLContext {
+class SparkPlannerSuite extends SharedSparkSession {
import testImplicits._
test("Ensure to go down only the first branch, not any other possible branches") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
index 7e317a4d80265..376d330ebeb70 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
@@ -22,11 +22,11 @@ import scala.util.Random
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext {
+class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession {
private var rand: Random = _
private var seed: Long = 0
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala
index 5c15ecd42fa0c..392cce54ebede 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala
@@ -31,7 +31,7 @@ import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED
import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String
class UnsafeFixedWidthAggregationMapSuite
extends SparkFunSuite
with Matchers
- with SharedSQLContext {
+ with SharedSparkSession {
import UnsafeFixedWidthAggregationMap._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala
index c882a9dd2148c..8aa003a3dfeb0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala
@@ -27,14 +27,14 @@ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
import org.apache.spark.sql.{RandomDataGenerator, Row}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeProjection, UnsafeRow}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.map.BytesToBytesMap
/**
* Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data.
*/
-class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext {
+class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSparkSession {
private val keyTypes = Seq(IntegerType, FloatType, DoubleType, StringType)
private val valueTypes = Seq(IntegerType, FloatType, DoubleType, StringType)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
index 55dff16887cb8..0ea16a1a15d66 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala
@@ -26,10 +26,10 @@ import org.apache.spark.sql.execution.joins.SortMergeJoinExec
import org.apache.spark.sql.expressions.scalalang.typed
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StringType, StructType}
-class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
+class WholeStageCodegenSuite extends QueryTest with SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
index d8efca323d519..720e7edb72f8d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
@@ -23,9 +23,9 @@ import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec
import org.apache.spark.sql.execution.exchange.Exchange
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext {
+class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
index 67c3fa0d3bf59..2eb4ac52aca90 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
@@ -27,20 +27,19 @@ import org.apache.arrow.memory.RootAllocator
import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
import org.apache.arrow.vector.ipc.JsonFileReader
import org.apache.arrow.vector.util.{ByteArrayReadableSeekableByteChannel, Validator}
-import org.scalatest.BeforeAndAfterAll
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BinaryType, Decimal, IntegerType, StructField, StructType}
import org.apache.spark.sql.util.ArrowUtils
import org.apache.spark.util.Utils
-class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll {
+class ArrowConvertersSuite extends SharedSparkSession {
import testImplicits._
private var tempDataPath: String = _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala
new file mode 100644
index 0000000000000..dbbad43efa08c
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.benchmark
+
+import java.time.Instant
+
+/**
+ * Synthetic benchmark for the extract function.
+ * To run this benchmark:
+ * {{{
+ * 1. without sbt:
+ * bin/spark-submit --class --jars
+ * 2. build/sbt "sql/test:runMain "
+ * 3. generate result:
+ * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain "
+ * Results will be written to "benchmarks/ExtractBenchmark-results.txt".
+ * }}}
+ */
+object ExtractBenchmark extends SqlBasedBenchmark {
+ private def doBenchmark(cardinality: Long, exprs: String*): Unit = {
+ val sinceSecond = Instant.parse("2010-01-01T00:00:00Z").getEpochSecond
+ spark
+ .range(sinceSecond, sinceSecond + cardinality, 1, 1)
+ .selectExpr(exprs: _*)
+ .write
+ .format("noop")
+ .save()
+ }
+
+ private def run(cardinality: Long, name: String, exprs: String*): Unit = {
+ codegenBenchmark(name, cardinality) {
+ doBenchmark(cardinality, exprs: _*)
+ }
+ }
+
+ private def run(cardinality: Long, field: String): Unit = {
+ codegenBenchmark(s"$field of timestamp", cardinality) {
+ doBenchmark(cardinality, s"EXTRACT($field FROM (cast(id as timestamp)))")
+ }
+ }
+
+ override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+ val N = 10000000L
+ runBenchmark("Extract") {
+ run(N, "cast to timestamp", "cast(id as timestamp)")
+ run(N, "MILLENNIUM")
+ run(N, "CENTURY")
+ run(N, "DECADE")
+ run(N, "YEAR")
+ run(N, "ISOYEAR")
+ run(N, "QUARTER")
+ run(N, "MONTH")
+ run(N, "WEEK")
+ run(N, "DAY")
+ run(N, "DAYOFWEEK")
+ run(N, "DOW")
+ run(N, "ISODOW")
+ run(N, "DOY")
+ run(N, "HOUR")
+ run(N, "MINUTE")
+ run(N, "SECOND")
+ run(N, "MILLISECONDS")
+ run(N, "MICROSECONDS")
+ run(N, "EPOCH")
+ }
+ }
+}
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 711ecf1738ab1..2f6843cb5b6d5 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
@@ -27,13 +27,13 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
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 {
+class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
import testImplicits._
setupTestData()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala
index 7b2ccca403bb2..a22cb66474493 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala
@@ -17,19 +17,12 @@
package org.apache.spark.sql.execution.columnar
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.test.SQLTestData._
-class PartitionBatchPruningSuite
- extends SparkFunSuite
- with BeforeAndAfterEach
- with SharedSQLContext {
+class PartitionBatchPruningSuite extends SharedSparkSession {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala
index 83452cdd8927b..74ef81f7181da 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala
@@ -35,10 +35,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, Log
import org.apache.spark.sql.execution.SparkSqlParser
import org.apache.spark.sql.execution.datasources.CreateTable
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
-class DDLParserSuite extends AnalysisTest with SharedSQLContext {
+class DDLParserSuite extends AnalysisTest with SharedSparkSession {
private lazy val parser = new SparkSqlParser(new SQLConf)
private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index b777db750a1bb..fd1da2011f28e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -33,12 +33,12 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with BeforeAndAfterEach {
+class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession {
import testImplicits._
override def afterEach(): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
index ce209666024d0..1040bda157584 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
@@ -20,8 +20,12 @@ package org.apache.spark.sql.execution.command
import java.net.URI
import java.util.Locale
+import org.mockito.ArgumentMatchers.any
+import org.mockito.Mockito.{mock, when}
+import org.mockito.invocation.InvocationOnMock
+
import org.apache.spark.sql.{AnalysisException, SaveMode}
-import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog, TableCatalog, TestTableCatalog}
+import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog, TestTableCatalog}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType}
@@ -50,37 +54,49 @@ class PlanResolutionSuite extends AnalysisTest {
newCatalog
}
- private val lookupWithDefault: LookupCatalog = new LookupCatalog {
- override protected def defaultCatalogName: Option[String] = Some("testcat")
-
- override protected def lookupCatalog(name: String): CatalogPlugin = name match {
- case "testcat" =>
- testCat
- case "session" =>
- v2SessionCatalog
- case _ =>
- throw new CatalogNotFoundException(s"No such catalog: $name")
- }
+ private val catalogManagerWithDefault = {
+ val manager = mock(classOf[CatalogManager])
+ when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => {
+ invocation.getArgument[String](0) match {
+ case "testcat" =>
+ testCat
+ case "session" =>
+ v2SessionCatalog
+ case name =>
+ throw new CatalogNotFoundException(s"No such catalog: $name")
+ }
+ })
+ when(manager.defaultCatalog).thenReturn(Some(testCat))
+ when(manager.v2SessionCatalog).thenCallRealMethod()
+ manager
}
- private val lookupWithoutDefault: LookupCatalog = new LookupCatalog {
- override protected def defaultCatalogName: Option[String] = None
-
- override protected def lookupCatalog(name: String): CatalogPlugin = name match {
- case "testcat" =>
- testCat
- case "session" =>
- v2SessionCatalog
- case _ =>
- throw new CatalogNotFoundException(s"No such catalog: $name")
- }
+ private val catalogManagerWithoutDefault = {
+ val manager = mock(classOf[CatalogManager])
+ when(manager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => {
+ invocation.getArgument[String](0) match {
+ case "testcat" =>
+ testCat
+ case "session" =>
+ v2SessionCatalog
+ case name =>
+ throw new CatalogNotFoundException(s"No such catalog: $name")
+ }
+ })
+ when(manager.defaultCatalog).thenReturn(None)
+ when(manager.v2SessionCatalog).thenCallRealMethod()
+ manager
}
def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = {
val newConf = conf.copy()
newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat")
- DataSourceResolution(newConf, if (withDefault) lookupWithDefault else lookupWithoutDefault)
- .apply(parsePlan(query))
+ val catalogManager = if (withDefault) {
+ catalogManagerWithDefault
+ } else {
+ catalogManagerWithoutDefault
+ }
+ DataSourceResolution(newConf, catalogManager).apply(parsePlan(query))
}
private def parseResolveCompare(query: String, expected: LogicalPlan): Unit =
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala
index 2f5d5551c5df0..a44a94aaa4f94 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala
@@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.sources
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataSourceStrategySuite extends PlanTest with SharedSQLContext {
+class DataSourceStrategySuite extends PlanTest with SharedSparkSession {
test("translate simple expression") {
val attrInt = 'cint.int
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala
index e09ec0d7bbb49..ce511842e6356 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala
@@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
class FileFormatWriterSuite
extends QueryTest
- with SharedSQLContext
+ with SharedSparkSession
with CodegenInterpretedPlanTest{
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
index 2a5c5a2dd0ff8..4b086e830e456 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala
@@ -30,11 +30,11 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
import org.apache.spark.util.KnownSizeEstimation
-class FileIndexSuite extends SharedSQLContext {
+class FileIndexSuite extends SharedSparkSession {
private class TestInMemoryFileIndex(
spark: SparkSession,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
index eaff5a2352a0c..7845895f3ce1e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
@@ -35,11 +35,11 @@ import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StructType}
import org.apache.spark.util.Utils
-class FileSourceStrategySuite extends QueryTest with SharedSQLContext with PredicateHelper {
+class FileSourceStrategySuite extends QueryTest with SharedSparkSession with PredicateHelper {
import testImplicits._
protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala
index 508614a7e476c..3c83388461ce1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala
@@ -24,9 +24,9 @@ import java.nio.file.Files
import org.apache.hadoop.conf.Configuration
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class HadoopFileLinesReaderSuite extends SharedSQLContext {
+class HadoopFileLinesReaderSuite extends SharedSparkSession {
def getLines(
path: File,
text: String,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala
index d42809529ceed..fd70b6529ff51 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala
@@ -21,7 +21,7 @@ import java.io.File
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
/**
* The reader schema is said to be evolved (or projected) when it changed after the data is
@@ -67,7 +67,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
* -> ToDecimalTypeTest
*/
-trait ReadSchemaTest extends QueryTest with SQLTestUtils with SharedSQLContext {
+trait ReadSchemaTest extends QueryTest with SharedSparkSession {
val format: String
val options: Map[String, String] = Map.empty[String, String]
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala
index e8bf21a2a9dbe..6420081a9757b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala
@@ -25,11 +25,11 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class RowDataSourceStrategySuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext {
+class RowDataSourceStrategySuite extends SharedSparkSession with BeforeAndAfter {
import testImplicits._
val url = "jdbc:h2:mem:testdb0"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
index 8b06b175a2103..233978289f068 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources
import org.apache.spark.SparkConf
import org.apache.spark.sql.SaveMode
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class SaveIntoDataSourceCommandSuite extends SharedSQLContext {
+class SaveIntoDataSourceCommandSuite extends SharedSparkSession {
test("simpleString is redacted") {
val URL = "connection.url"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
index 09ca42851836b..bb3cec579016e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala
@@ -27,14 +27,14 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
abstract class SchemaPruningSuite
extends QueryTest
with FileBasedDataSourceTest
with SchemaPruningTest
- with SharedSQLContext {
+ with SharedSparkSession {
case class FullName(first: String, middle: String, last: String)
case class Company(name: String, address: String)
case class Employer(id: Int, company: Company)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
index a66b34fe367fa..70ec9bbf4819d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
@@ -34,11 +34,11 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.internal.SQLConf.SOURCES_BINARY_FILE_MAX_LENGTH
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class BinaryFileFormatSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+class BinaryFileFormatSuite extends QueryTest with SharedSparkSession {
import BinaryFileFormat._
private var testDir: String = _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index cf17025980ee3..a6c2e5bfab2eb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -39,10 +39,10 @@ import org.apache.spark.{SparkException, TestUtils}
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with TestCsvData {
+class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData {
import testImplicits._
private val carsFile = "test-data/cars.csv"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
index 316c5183fddf1..bafb6769af69c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
@@ -19,12 +19,12 @@ package org.apache.spark.sql.execution.datasources.json
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.catalyst.json.JSONOptions
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
/**
* Test cases for various [[JSONOptions]].
*/
-class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext {
+class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("allowComments off") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index 21b5cbb3ace29..2998e673bd45c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.ExternalRDD
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.sql.types.StructType.fromDDL
import org.apache.spark.util.Utils
@@ -45,7 +45,7 @@ class TestFileFilter extends PathFilter {
override def accept(path: Path): Boolean = path.getParent.getName != "p=2"
}
-class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData {
+class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData {
import testImplicits._
test("Type promotion") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala
index 59de28688ec1d..c5a03cb8ef6d3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql.execution.datasources.noop
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class NoopSuite extends SharedSQLContext {
+class NoopSuite extends SharedSparkSession {
import testImplicits._
test("materialisation of all rows") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala
index c16fcc67f8dd1..719bf91e1786b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala
@@ -22,11 +22,11 @@ import org.apache.orc.TypeDescription
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector}
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{StructField, StructType}
import org.apache.spark.unsafe.types.UTF8String.fromString
-class OrcColumnarBatchReaderSuite extends QueryTest with SQLTestUtils with SharedSQLContext {
+class OrcColumnarBatchReaderSuite extends QueryTest with SharedSparkSession {
private val dataSchema = StructType.fromDDL("col1 int, col2 int")
private val partitionSchema = StructType.fromDDL("p1 string, p2 string")
private val partitionValues = InternalRow(fromString("partValue1"), fromString("partValue2"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala
index 143e3f0997201..42ea161cb30c7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala
@@ -24,7 +24,7 @@ import org.apache.hadoop.fs.{Path, PathFilter}
import org.apache.spark.SparkConf
import org.apache.spark.sql._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
// The data where the partitioning key exists only in the directory structure.
case class OrcParData(intField: Int, stringField: String)
@@ -168,7 +168,7 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest {
}
}
-class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext {
+class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSparkSession {
test("read partitioned table - partition key included in orc file") {
withTempDir { base =>
for {
@@ -252,7 +252,7 @@ class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQ
}
}
-class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext {
+class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSparkSession {
override protected def sparkConf: SparkConf =
super
.sparkConf
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
index 18ec3e3ebed0c..b5b9e398f5ae7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -621,7 +621,7 @@ abstract class OrcQueryTest extends OrcTest {
}
}
-class OrcQuerySuite extends OrcQueryTest with SharedSQLContext {
+class OrcQuerySuite extends OrcQueryTest with SharedSparkSession {
import testImplicits._
test("LZO compression options for writing to an ORC file") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
index c9f5d9cb23e6b..0d904a09c07e8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala
@@ -35,7 +35,7 @@ import org.apache.spark.{SPARK_VERSION_SHORT, SparkException}
import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY}
import org.apache.spark.sql.execution.datasources.SchemaMergeUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{LongType, StructField, StructType}
import org.apache.spark.util.Utils
@@ -480,7 +480,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll {
}
}
-class OrcSourceSuite extends OrcSuite with SharedSQLContext {
+class OrcSourceSuite extends OrcSuite with SharedSparkSession {
protected override def beforeAll(): Unit = {
super.beforeAll()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
index 1b99fbedca047..d7727d93ddf98 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala
@@ -31,9 +31,9 @@ import org.apache.parquet.hadoop.ParquetWriter
import org.apache.spark.sql.Row
import org.apache.spark.sql.execution.datasources.parquet.test.avro._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext {
+class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession {
private def withWriter[T <: IndexedRecord]
(path: String, schema: Schema)
(f: ParquetWriter[T] => Unit): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala
index 09de715e87a11..900f4fdca395d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala
@@ -25,9 +25,9 @@ import org.apache.hadoop.fs.Path
import org.apache.parquet.hadoop.ParquetOutputFormat
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSQLContext {
+class ParquetCompressionCodecPrecedenceSuite extends ParquetTest with SharedSparkSession {
test("Test `spark.sql.parquet.compression.codec` config") {
Seq("NONE", "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO", "LZ4", "BROTLI", "ZSTD").foreach { c =>
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> c) {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
index db73bfa149aa0..6d681afd23b18 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala
@@ -20,11 +20,11 @@ import scala.collection.JavaConverters._
import org.apache.parquet.hadoop.ParquetOutputFormat
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
// TODO: this needs a lot more testing but it's currently not easy to test with the parquet
// writer abstractions. Revisit.
-class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContext {
+class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSparkSession {
import testImplicits._
val ROW = ((1).toByte, 2, 3L, "abc")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
index 94abf115cef35..e65f4d12bf7f2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala
@@ -22,9 +22,9 @@ import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkException
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSQLContext {
+class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSparkSession {
test("read parquet footers in parallel") {
def testReadFooters(ignoreCorruptFiles: Boolean): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index 577d1bc8d6a4f..365cb137eceb2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.{AccumulatorContext, AccumulatorV2}
@@ -61,7 +61,7 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2}
* dependent on this configuration, don't forget you better explicitly set this configuration
* within the test.
*/
-abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext {
+abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSparkSession {
protected def createParquetFilters(
schema: MessageType,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index 6f2218ba82dc8..026ba5deffdfd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -47,7 +47,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -74,7 +74,7 @@ private[parquet] class TestGroupWriteSupport(schema: MessageType) extends WriteS
/**
* A test suite that tests basic Parquet I/O.
*/
-class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
+class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession {
import testImplicits._
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
index edbc2493ac26d..1ded34f24e436 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala
@@ -28,9 +28,9 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSQLContext {
+class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSparkSession {
test("parquet files with different physical schemas but share the same logical schema") {
import ParquetCompatibilityTest._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
index 04ace0a236e6d..34f9c2e630e49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala
@@ -40,7 +40,7 @@ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition}
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable}
import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -51,7 +51,7 @@ case class ParquetData(intField: Int, stringField: String)
case class ParquetDataWithKey(intField: Int, pi: Int, stringField: String, ps: String)
abstract class ParquetPartitionDiscoverySuite
- extends QueryTest with ParquetTest with SharedSQLContext {
+ extends QueryTest with ParquetTest with SharedSparkSession {
import PartitioningUtils._
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala
index fa88019298a69..f24647bf41dd4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala
@@ -18,9 +18,9 @@
package org.apache.spark.sql.execution.datasources.parquet
import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext {
+class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession {
test("unannotated array of primitive type") {
checkAnswer(readResourceParquetFile("test-data/old-repeated-int.parquet"), Row(Seq(1, 2, 3)))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
index a6429bfc52924..3aa594ba44c31 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
@@ -33,14 +33,14 @@ import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStru
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
/**
* A test suite that tests various Parquet queries.
*/
-abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext {
+abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSparkSession {
import testImplicits._
test("simple select queries") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index f8e4822af11f1..1274995fd6779 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -28,10 +28,10 @@ import org.apache.spark.sql.catalyst.ScalaReflection
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
-abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
+abstract class ParquetSchemaTest extends ParquetTest with SharedSparkSession {
/**
* Checks whether the reflected Parquet message type for product type `T` conforms `messageType`.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala
index 4157a5b46dc42..c59c049612389 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala
@@ -18,9 +18,9 @@
package org.apache.spark.sql.execution.datasources.parquet
import org.apache.spark.sql.Row
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext {
+class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSparkSession {
import ParquetCompatibilityTest._
private val parquetFilePath = Thread.currentThread().getContextClassLoader.getResource(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
index a86d5ee37f3db..62a779528cec1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala
@@ -27,11 +27,11 @@ import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.TestUtils
import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.util.Utils
-class TextSuite extends QueryTest with SharedSQLContext {
+class TextSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("reading text file") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala
index a302d67b5cbf7..5e3b3441aa74f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala
@@ -21,10 +21,10 @@ import java.io.File
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{StringType, StructType}
-class WholeTextFileSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+class WholeTextFileSuite extends QueryTest with SharedSparkSession {
// Hadoop's FileSystem caching does not use the Configuration as part of its cache key, which
// can cause Filesystem.get(Configuration) to return a cached instance created with a different
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala
index ac1d5672af68c..ad0dfadacca15 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.execution.datasources.text.TextFileFormat
import org.apache.spark.sql.sources.v2.reader.ScanBuilder
import org.apache.spark.sql.sources.v2.writer.WriteBuilder
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
@@ -51,7 +51,7 @@ class DummyFileTable(
override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[TextFileFormat]
}
-class FileTableSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
+class FileTableSuite extends QueryTest with SharedSparkSession {
test("Data type validation should check data schema only") {
withTempPath { dir =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
index 4f14ecc28680d..64460d0338054 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
@@ -30,12 +30,12 @@ import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, TableCatalog, Tabl
import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
class V2SessionCatalogSuite
- extends SparkFunSuite with SharedSQLContext with BeforeAndAfter with BeforeAndAfterAll {
+ extends SparkFunSuite with SharedSparkSession with BeforeAndAfter {
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String]
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala
index 8a0450fce76a1..e2d801104da7e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala
@@ -21,7 +21,6 @@ import java.util
import scala.collection.JavaConverters._
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext}
import org.apache.spark.sql.catalyst.plans.logical.Union
import org.apache.spark.sql.execution.datasources.DataSource
@@ -32,7 +31,7 @@ import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-class V2StreamingScanSupportCheckSuite extends SparkFunSuite with SharedSparkSession {
+class V2StreamingScanSupportCheckSuite extends SharedSparkSession {
import TableCapability._
private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala
index e423420c2914a..aaf1fe4807fab 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala
@@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.debug
import java.io.ByteArrayOutputStream
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.functions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.test.SQLTestData.TestData
-class DebuggingSuite extends SparkFunSuite with SharedSQLContext {
+class DebuggingSuite extends SharedSparkSession {
test("DataFrame.debug()") {
testData.debug()
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala
index 771a9730247af..5490246baceea 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala
@@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint}
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest}
import org.apache.spark.sql.execution.exchange.EnsureRequirements
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructType}
-class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext {
+class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession {
private lazy val left = spark.createDataFrame(
sparkContext.parallelize(Seq(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
index 02e474ce83372..1a3d3e819213e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
@@ -28,13 +28,13 @@ import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager}
import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.map.BytesToBytesMap
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.collection.CompactBuffer
-class HashedRelationSuite extends SparkFunSuite with SharedSQLContext {
+class HashedRelationSuite extends SharedSparkSession {
val mm = new TaskMemoryManager(
new UnifiedMemoryManager(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala
index f99a278bb2427..08898f80034e6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala
@@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint}
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.exchange.EnsureRequirements
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StringType, StructType}
-class InnerJoinSuite extends SparkPlanTest with SharedSQLContext {
+class InnerJoinSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits.newProductEncoder
import testImplicits.localSeqToDatasetHolder
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
index 1f04fcf6ca451..a5ade0d8d7508 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala
@@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint}
import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest}
import org.apache.spark.sql.execution.exchange.EnsureRequirements
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType}
-class OuterJoinSuite extends SparkPlanTest with SharedSQLContext {
+class OuterJoinSuite extends SparkPlanTest with SharedSparkSession {
private lazy val left = spark.createDataFrame(
sparkContext.parallelize(Seq(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
index dc4a2998a908f..0a254ab210bad 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala
@@ -22,7 +22,6 @@ import java.io.File
import scala.reflect.{classTag, ClassTag}
import scala.util.Random
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial}
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
@@ -30,10 +29,10 @@ import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeSt
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.{AccumulatorContext, JsonProtocol}
-class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with SharedSQLContext {
+class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils {
import testImplicits._
/**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
index ac5752b41cf28..d26989b00a651 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
@@ -24,10 +24,10 @@ import org.apache.spark.api.python.{PythonEvalType, PythonFunction}
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In}
import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BooleanType, DoubleType}
-class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext {
+class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits.newProductEncoder
import testImplicits.localSeqToDatasetHolder
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala
index 311a8ef3257da..4f1c28d36e384 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala
@@ -22,9 +22,9 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSQLContext {
+class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession {
import testImplicits._
val batchedPythonUDF = new MyDummyPythonUDF
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
index 1a971b0deb7f5..8cf1b7fc71079 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.python
import org.apache.spark.sql.{IntegratedUDFTestUtils, QueryTest}
import org.apache.spark.sql.functions.count
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class PythonUDFSuite extends QueryTest with SharedSQLContext {
+class PythonUDFSuite extends QueryTest with SharedSparkSession {
import testImplicits._
import IntegratedUDFTestUtils._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
index cbac1c13cdd33..c57b40c977e49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala
@@ -90,7 +90,7 @@ abstract class CheckpointFileManagerTests extends SparkFunSuite with SQLHelper {
}
}
-class CheckpointFileManagerSuite extends SparkFunSuite with SharedSparkSession {
+class CheckpointFileManagerSuite extends SharedSparkSession {
test("CheckpointFileManager.create() should pick up user-specified class from conf") {
withSQLConf(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
index 71dc3776bcaf6..ead17d50b4e1e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
@@ -20,11 +20,10 @@ package org.apache.spark.sql.execution.streaming
import java.io._
import java.nio.charset.StandardCharsets._
-import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class CompactibleFileStreamLogSuite extends SparkFunSuite with SharedSQLContext {
+class CompactibleFileStreamLogSuite extends SharedSparkSession {
import CompactibleFileStreamLog._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
index dd3a414659c23..f95daafdfe19b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala
@@ -22,9 +22,9 @@ import java.nio.charset.StandardCharsets.UTF_8
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext {
+class FileStreamSinkLogSuite extends SparkFunSuite with SharedSparkSession {
import CompactibleFileStreamLog._
import FileStreamSinkLog._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
index 04fe1e787909d..c09756cd1b248 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
@@ -25,11 +25,10 @@ import scala.language.implicitConversions
import org.scalatest.concurrent.Waiters._
import org.scalatest.time.SpanSugar._
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.UninterruptibleThread
-class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext {
+class HDFSMetadataLogSuite extends SharedSparkSession {
private implicit def toOption[A](a: A): Option[A] = Option(a)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
index e6cdc063c4e9f..e5dfa33164903 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
@@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.streaming
import java.io.File
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.util.stringToFile
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext {
+class OffsetSeqLogSuite extends SharedSparkSession {
/** test string offset type */
case class StringOffset(override val json: String) extends Offset
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala
index e60c339bc9cc1..0c17320acade9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala
@@ -28,9 +28,9 @@ import org.apache.spark.sql.ForeachWriter
import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.functions.{count, window}
import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAndAfter {
+class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeAndAfter {
import testImplicits._
@@ -154,6 +154,8 @@ class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAnd
val errorEvent = allEvents(0)(2).asInstanceOf[ForeachWriterSuite.Close]
assert(errorEvent.error.get.isInstanceOf[RuntimeException])
assert(errorEvent.error.get.getMessage === "ForeachSinkSuite error")
+ // 'close' shouldn't be called with abort message if close with error has been called
+ assert(allEvents(0).size == 3)
}
}
@@ -258,6 +260,34 @@ class ForeachWriterSuite extends StreamTest with SharedSQLContext with BeforeAnd
query.stop()
}
}
+
+ testQuietly("foreach with error not caused by ForeachWriter") {
+ withTempDir { checkpointDir =>
+ val input = MemoryStream[Int]
+ val query = input.toDS().repartition(1).map(_ / 0).writeStream
+ .option("checkpointLocation", checkpointDir.getCanonicalPath)
+ .foreach(new TestForeachWriter)
+ .start()
+ input.addData(1, 2, 3, 4)
+
+ val e = intercept[StreamingQueryException] {
+ query.processAllAvailable()
+ }
+
+ assert(e.getCause.isInstanceOf[SparkException])
+ assert(e.getCause.getCause.getCause.getMessage === "/ by zero")
+ assert(query.isActive === false)
+
+ val allEvents = ForeachWriterSuite.allEvents()
+ assert(allEvents.size === 1)
+ assert(allEvents(0)(0) === ForeachWriterSuite.Open(partition = 0, version = 0))
+ // `close` should be called with the error
+ val errorEvent = allEvents(0)(1).asInstanceOf[ForeachWriterSuite.Close]
+ assert(errorEvent.error.get.isInstanceOf[SparkException])
+ assert(errorEvent.error.get.getMessage ===
+ "Foreach writer has been aborted due to a task failure")
+ }
+ }
}
/** A global object to collect events in the executor */
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
index 5ff250dd83867..e1284ea03267e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala
@@ -27,8 +27,6 @@ import java.util.concurrent.TimeUnit._
import scala.collection.JavaConverters._
-import org.scalatest.BeforeAndAfterEach
-
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.DataSource
@@ -38,11 +36,11 @@ import org.apache.spark.sql.execution.streaming.continuous._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream}
import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach {
+class TextSocketStreamSuite extends StreamTest with SharedSparkSession {
override def afterEach() {
sqlContext.streams.active.foreach(_.stop())
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
index 95a6af3720d85..9e42056c19a0c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala
@@ -29,11 +29,11 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.scheduler.{JobFailed, SparkListenerJobEnd, SparkListenerJobStart}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.status.ElementTrackingStore
import org.apache.spark.util.kvstore.InMemoryStore
-class AllExecutionsPageSuite extends SharedSQLContext with BeforeAndAfter {
+class AllExecutionsPageSuite extends SharedSparkSession with BeforeAndAfter {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
index bb3f6d8236fd7..90966d2efec23 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
@@ -39,13 +39,13 @@ 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.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.status.ElementTrackingStore
import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator}
import org.apache.spark.util.kvstore.InMemoryStore
-class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils
+class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils
with BeforeAndAfter {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
index 6acac1a9aa317..d6a1fde2147b1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
@@ -19,16 +19,13 @@ package org.apache.spark.sql.internal
import java.io.File
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalog.{Column, Database, Function, Table}
import org.apache.spark.sql.catalyst.{FunctionIdentifier, ScalaReflection, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo}
import org.apache.spark.sql.catalyst.plans.logical.Range
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
@@ -36,10 +33,7 @@ import org.apache.spark.storage.StorageLevel
/**
* Tests for the user-facing [[org.apache.spark.sql.catalog.Catalog]].
*/
-class CatalogSuite
- extends SparkFunSuite
- with BeforeAndAfterEach
- with SharedSQLContext {
+class CatalogSuite extends SharedSparkSession {
import testImplicits._
private def sessionCatalog: SessionCatalog = spark.sessionState.catalog
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
index 829dea48a22be..1dfbca64f5778 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
@@ -21,10 +21,10 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.sql._
import org.apache.spark.sql.internal.StaticSQLConf._
-import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
+import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext}
import org.apache.spark.util.Utils
-class SQLConfSuite extends QueryTest with SharedSQLContext {
+class SQLConfSuite extends QueryTest with SharedSparkSession {
import testImplicits._
private val testKey = "test.key.0"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 158085c35351b..72a564506bd23 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -36,12 +36,12 @@ import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartiti
import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
class JDBCSuite extends QueryTest
- with BeforeAndAfter with PrivateMethodTester with SharedSQLContext {
+ with BeforeAndAfter with PrivateMethodTester with SharedSparkSession {
import testImplicits._
val url = "jdbc:h2:mem:testdb0"
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index b751ec2de4825..b28c6531d42b2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -29,11 +29,11 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter {
+class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter {
val url = "jdbc:h2:mem:testdb2"
var conn: java.sql.Connection = null
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
index c3edec39979ae..7043b6d396977 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala
@@ -34,11 +34,11 @@ import org.apache.spark.sql.execution.joins.SortMergeJoinExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
import org.apache.spark.util.Utils
import org.apache.spark.util.collection.BitSet
-class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedSQLContext {
+class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedSparkSession {
protected override def beforeAll(): Unit = {
super.beforeAll()
assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
index 75f68dea96bf0..9713de988e379 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala
@@ -28,9 +28,9 @@ import org.apache.spark.sql.execution.datasources.BucketingUtils
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
-import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
+import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
-class BucketedWriteWithoutHiveSupportSuite extends BucketedWriteSuite with SharedSQLContext {
+class BucketedWriteWithoutHiveSupportSuite extends BucketedWriteSuite with SharedSparkSession {
protected override def beforeAll(): Unit = {
super.beforeAll()
assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
index 5f9856656ac3b..08f0865c1e128 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala
@@ -19,21 +19,16 @@ package org.apache.spark.sql.sources
import java.io.File
-import org.scalatest.BeforeAndAfterEach
-
import org.apache.spark.SparkException
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.internal.SQLConf.BUCKETING_MAX_BUCKETS
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
-class CreateTableAsSelectSuite
- extends DataSourceTest
- with SharedSQLContext
- with BeforeAndAfterEach {
+class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession {
import testImplicits._
protected override lazy val sql = spark.sql _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
index f22d843bfabde..5d4ddeac66375 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala
@@ -18,12 +18,12 @@
package org.apache.spark.sql.sources
import org.apache.spark.sql.{AnalysisException, SQLContext}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
// please note that the META-INF/services had to be modified for the test directory for this to work
-class DDLSourceLoadSuite extends DataSourceTest with SharedSQLContext {
+class DDLSourceLoadSuite extends DataSourceTest with SharedSparkSession {
test("data sources with the same name - internal data sources") {
val e = intercept[AnalysisException] {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
index 2b39bda97bd8d..657ef5ca13bd9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala
@@ -21,10 +21,9 @@ import java.util.Locale
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.PredicateHelper
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -133,7 +132,7 @@ object ColumnsRequired {
var set: Set[String] = Set.empty
}
-class FilteredScanSuite extends DataSourceTest with SharedSQLContext with PredicateHelper {
+class FilteredScanSuite extends DataSourceTest with SharedSparkSession {
protected override lazy val sql = spark.sql _
override def beforeAll(): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
index 4f1ae069d4b89..237f197bdec50 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -50,7 +50,7 @@ case class SimpleInsert(userSpecifiedSchema: StructType)(@transient val sparkSes
}
}
-class InsertSuite extends DataSourceTest with SharedSQLContext {
+class InsertSuite extends DataSourceTest with SharedSparkSession {
import testImplicits._
protected override lazy val sql = spark.sql _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
index 27c983f270bf6..ab1d1f80e7397 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String)
@@ -43,7 +43,7 @@ private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String
}
}
-class PartitionedWriteSuite extends QueryTest with SharedSQLContext {
+class PartitionedWriteSuite extends QueryTest with SharedSparkSession {
import testImplicits._
test("write many partitions") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
index 85da3f0e38468..87dce376a09dd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogUtils
import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, Metadata, MetadataBuilder, StructType}
class TestOptionsSource extends SchemaRelationProvider with CreatableRelationProvider {
@@ -65,7 +65,7 @@ class TestOptionsRelation(val options: Map[String, String])(@transient val sessi
}
}
-class PathOptionSuite extends DataSourceTest with SharedSQLContext {
+class PathOptionSuite extends DataSourceTest with SharedSparkSession {
test("path option always exist") {
withTable("src") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala
index 309591dd90f0f..d99c605b2e478 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.sources
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
class PrunedScanSource extends RelationProvider {
@@ -53,7 +53,7 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: Spa
}
}
-class PrunedScanSuite extends DataSourceTest with SharedSQLContext {
+class PrunedScanSuite extends DataSourceTest with SharedSparkSession {
protected override lazy val sql = spark.sql _
override def beforeAll(): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
index 0aa67bf1b0d48..36fb418b09cb6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala
@@ -17,13 +17,12 @@
package org.apache.spark.sql.sources
-import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.datasources.DataSource
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class ResolvedDataSourceSuite extends SparkFunSuite with SharedSQLContext {
+class ResolvedDataSourceSuite extends SharedSparkSession {
private def getProvidingClass(name: String): Class[_] =
DataSource(
sparkSession = spark,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
index 7680f61b8b6c7..ce3ec7f97a537 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala
@@ -25,11 +25,11 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
-class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter {
+class SaveLoadSuite extends DataSourceTest with SharedSparkSession with BeforeAndAfter {
import testImplicits._
protected override lazy val sql = spark.sql _
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
index 13a126ff963d5..d4e117953942e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
@@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
class DefaultSource extends SimpleScanSource
@@ -108,7 +108,7 @@ case class AllDataTypesScan(
}
}
-class TableScanSuite extends DataSourceTest with SharedSQLContext {
+class TableScanSuite extends DataSourceTest with SharedSparkSession {
protected override lazy val sql = spark.sql _
private lazy val tableWithSchemaExpected = (1 to 10).map { i =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala
new file mode 100644
index 0000000000000..a104b8835c613
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.sources.v2
+
+import java.util
+import java.util.concurrent.ConcurrentHashMap
+
+import scala.collection.JavaConverters._
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode}
+import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier}
+import org.apache.spark.sql.catalog.v2.expressions.Transform
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class DataSourceV2DataFrameSessionCatalogSuite
+ extends QueryTest
+ with SharedSparkSession
+ with BeforeAndAfter {
+ import testImplicits._
+
+ private def catalog(name: String): CatalogPlugin = {
+ spark.sessionState.catalogManager.catalog(name)
+ }
+
+ private val v2Format = classOf[InMemoryTableProvider].getName
+
+ before {
+ spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[TestV2SessionCatalog].getName)
+ }
+
+ override def afterEach(): Unit = {
+ super.afterEach()
+ catalog("session").asInstanceOf[TestV2SessionCatalog].clearTables()
+ spark.conf.set(SQLConf.V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName)
+ }
+
+ private def verifyTable(tableName: String, expected: DataFrame): Unit = {
+ checkAnswer(spark.table(tableName), expected)
+ checkAnswer(sql(s"SELECT * FROM $tableName"), expected)
+ checkAnswer(sql(s"SELECT * FROM default.$tableName"), expected)
+ checkAnswer(sql(s"TABLE $tableName"), expected)
+ }
+
+ test("saveAsTable: v2 table - table doesn't exist and default mode (ErrorIfExists)") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ df.write.format(v2Format).saveAsTable(t1)
+ verifyTable(t1, df)
+ }
+
+ test("saveAsTable: v2 table - table doesn't exist and append mode") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ df.write.format(v2Format).mode("append").saveAsTable(t1)
+ verifyTable(t1, df)
+ }
+
+ test("saveAsTable: Append mode should not fail if the table not exists " +
+ "but a same-name temp view exist") {
+ withTable("same_name") {
+ withTempView("same_name") {
+ spark.range(10).createTempView("same_name")
+ spark.range(20).write.format(v2Format).mode(SaveMode.Append).saveAsTable("same_name")
+ assert(
+ spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default"))))
+ }
+ }
+ }
+
+ test("saveAsTable: Append mode should not fail if the table already exists " +
+ "and a same-name temp view exist") {
+ withTable("same_name") {
+ withTempView("same_name") {
+ val format = spark.sessionState.conf.defaultDataSourceName
+ sql(s"CREATE TABLE same_name(id LONG) USING $format")
+ spark.range(10).createTempView("same_name")
+ spark.range(20).write.format(v2Format).mode(SaveMode.Append).saveAsTable("same_name")
+ checkAnswer(spark.table("same_name"), spark.range(10).toDF())
+ checkAnswer(spark.table("default.same_name"), spark.range(20).toDF())
+ }
+ }
+ }
+
+ test("saveAsTable: v2 table - table exists") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ spark.sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format")
+ intercept[TableAlreadyExistsException] {
+ df.select("id", "data").write.format(v2Format).saveAsTable(t1)
+ }
+ df.write.format(v2Format).mode("append").saveAsTable(t1)
+ verifyTable(t1, df)
+
+ // Check that appends are by name
+ df.select('data, 'id).write.format(v2Format).mode("append").saveAsTable(t1)
+ verifyTable(t1, df.union(df))
+ }
+
+ test("saveAsTable: v2 table - table overwrite and table doesn't exist") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ df.write.format(v2Format).mode("overwrite").saveAsTable(t1)
+ verifyTable(t1, df)
+ }
+
+ test("saveAsTable: v2 table - table overwrite and table exists") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'")
+ df.write.format(v2Format).mode("overwrite").saveAsTable(t1)
+ verifyTable(t1, df)
+ }
+
+ test("saveAsTable: Overwrite mode should not drop the temp view if the table not exists " +
+ "but a same-name temp view exist") {
+ withTable("same_name") {
+ withTempView("same_name") {
+ spark.range(10).createTempView("same_name")
+ spark.range(20).write.format(v2Format).mode(SaveMode.Overwrite).saveAsTable("same_name")
+ assert(spark.sessionState.catalog.getTempView("same_name").isDefined)
+ assert(
+ spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default"))))
+ }
+ }
+ }
+
+ test("saveAsTable with mode Overwrite should not fail if the table already exists " +
+ "and a same-name temp view exist") {
+ withTable("same_name") {
+ withTempView("same_name") {
+ sql(s"CREATE TABLE same_name(id LONG) USING $v2Format")
+ spark.range(10).createTempView("same_name")
+ spark.range(20).write.format(v2Format).mode(SaveMode.Overwrite).saveAsTable("same_name")
+ checkAnswer(spark.table("same_name"), spark.range(10).toDF())
+ checkAnswer(spark.table("default.same_name"), spark.range(20).toDF())
+ }
+ }
+ }
+
+ test("saveAsTable: v2 table - ignore mode and table doesn't exist") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ df.write.format(v2Format).mode("ignore").saveAsTable(t1)
+ verifyTable(t1, df)
+ }
+
+ test("saveAsTable: v2 table - ignore mode and table exists") {
+ val t1 = "tbl"
+ val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data")
+ spark.sql(s"CREATE TABLE $t1 USING $v2Format AS SELECT 'c', 'd'")
+ df.write.format(v2Format).mode("ignore").saveAsTable(t1)
+ verifyTable(t1, Seq(("c", "d")).toDF("id", "data"))
+ }
+}
+
+class InMemoryTableProvider extends TableProvider {
+ override def getTable(options: CaseInsensitiveStringMap): Table = {
+ throw new UnsupportedOperationException("D'oh!")
+ }
+}
+
+/** A SessionCatalog that always loads an in memory Table, so we can test write code paths. */
+class TestV2SessionCatalog extends V2SessionCatalog {
+
+ protected val tables: util.Map[Identifier, InMemoryTable] =
+ new ConcurrentHashMap[Identifier, InMemoryTable]()
+
+ private def fullIdentifier(ident: Identifier): Identifier = {
+ if (ident.namespace().isEmpty) {
+ Identifier.of(Array("default"), ident.name())
+ } else {
+ ident
+ }
+ }
+
+ override def loadTable(ident: Identifier): Table = {
+ val fullIdent = fullIdentifier(ident)
+ if (tables.containsKey(fullIdent)) {
+ tables.get(fullIdent)
+ } else {
+ // Table was created through the built-in catalog
+ val t = super.loadTable(fullIdent)
+ val table = new InMemoryTable(t.name(), t.schema(), t.partitioning(), t.properties())
+ tables.put(fullIdent, table)
+ table
+ }
+ }
+
+ override def createTable(
+ ident: Identifier,
+ schema: StructType,
+ partitions: Array[Transform],
+ properties: util.Map[String, String]): Table = {
+ val created = super.createTable(ident, schema, partitions, properties)
+ val t = new InMemoryTable(created.name(), schema, partitions, properties)
+ val fullIdent = fullIdentifier(ident)
+ tables.put(fullIdent, t)
+ t
+ }
+
+ def clearTables(): Unit = {
+ assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session catalog code path?")
+ tables.keySet().asScala.foreach(super.dropTable)
+ tables.clear()
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala
index 8909c41ddaa8f..af9e56a3b9816 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala
@@ -21,9 +21,9 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataSourceV2DataFrameSuite extends QueryTest with SharedSQLContext with BeforeAndAfter {
+class DataSourceV2DataFrameSuite extends QueryTest with SharedSparkSession with BeforeAndAfter {
import testImplicits._
before {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
index 9b1a23a1f2bbf..08082d88c0738 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala
@@ -23,23 +23,28 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.SparkException
import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
-import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog}
+import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog}
import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException}
import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog
import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG}
-import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType}
+import org.apache.spark.sql.sources.v2.internal.UnresolvedTable
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
-class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter {
+class DataSourceV2SQLSuite extends QueryTest with SharedSparkSession with BeforeAndAfter {
import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._
private val orc2 = classOf[OrcDataSourceV2].getName
private val v2Source = classOf[FakeV2Provider].getName
+ private def catalog(name: String): CatalogPlugin = {
+ spark.sessionState.catalogManager.catalog(name)
+ }
+
before {
spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName)
spark.conf.set(
@@ -54,15 +59,14 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
after {
- spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables()
- spark.catalog("testcat_atomic").asInstanceOf[TestInMemoryTableCatalog].clearTables()
- spark.catalog("session").asInstanceOf[TestInMemoryTableCatalog].clearTables()
+ spark.sessionState.catalogManager.reset()
+ spark.sessionState.conf.clear()
}
test("CreateTable: use v2 plan because catalog is set") {
spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -127,7 +131,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("CreateTable: use v2 plan and session catalog when provider is v2") {
spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $orc2")
- val testCatalog = spark.catalog("session").asTableCatalog
+ val testCatalog = catalog("session").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "session.table_name")
@@ -142,7 +146,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("CreateTable: fail if table exists") {
spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -173,7 +177,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
spark.sql(
"CREATE TABLE IF NOT EXISTS testcat.table_name (id bigint, data string) USING foo")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -196,12 +200,10 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
test("CreateTable: use default catalog for v2 sources when default catalog is set") {
- val sparkSession = spark.newSession()
- sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName)
- sparkSession.conf.set("spark.sql.default.catalog", "testcat")
- sparkSession.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo")
+ spark.conf.set("spark.sql.default.catalog", "testcat")
+ spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo")
- val testCatalog = sparkSession.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -210,13 +212,13 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
assert(table.schema == new StructType().add("id", LongType).add("data", StringType))
// check that the table is empty
- val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows)
+ val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows)
checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty)
}
test("CreateTableAsSelect: use v2 plan because catalog is set") {
- val basicCatalog = spark.catalog("testcat").asTableCatalog
- val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog
+ val basicCatalog = catalog("testcat").asTableCatalog
+ val atomicCatalog = catalog("testcat_atomic").asTableCatalog
val basicIdentifier = "testcat.table_name"
val atomicIdentifier = "testcat_atomic.table_name"
@@ -239,8 +241,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
test("ReplaceTableAsSelect: basic v2 implementation.") {
- val basicCatalog = spark.catalog("testcat").asTableCatalog
- val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog
+ val basicCatalog = catalog("testcat").asTableCatalog
+ val atomicCatalog = catalog("testcat_atomic").asTableCatalog
val basicIdentifier = "testcat.table_name"
val atomicIdentifier = "testcat_atomic.table_name"
@@ -268,7 +270,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("ReplaceTableAsSelect: Non-atomic catalog drops the table if the write fails.") {
spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty)
@@ -285,7 +287,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("ReplaceTableAsSelect: Non-atomic catalog drops the table permanently if the" +
" subsequent table creation fails.") {
spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty)
@@ -302,7 +304,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("ReplaceTableAsSelect: Atomic catalog does not drop the table when replace fails.") {
spark.sql("CREATE TABLE testcat_atomic.table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat_atomic").asTableCatalog
+ val testCatalog = catalog("testcat_atomic").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
intercept[Exception] {
@@ -328,7 +330,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("ReplaceTable: Erases the table contents and changes the metadata.") {
spark.sql(s"CREATE TABLE testcat.table_name USING $orc2 AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty)
@@ -344,12 +346,12 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
test("ReplaceTableAsSelect: CREATE OR REPLACE new table has same behavior as CTAS.") {
- Seq("testcat", "testcat_atomic").foreach { catalog =>
- spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source")
+ Seq("testcat", "testcat_atomic").foreach { catalogName =>
+ spark.sql(s"CREATE TABLE $catalogName.created USING $orc2 AS SELECT id, data FROM source")
spark.sql(
- s"CREATE OR REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source")
+ s"CREATE OR REPLACE TABLE $catalogName.replaced USING $orc2 AS SELECT id, data FROM source")
- val testCatalog = spark.catalog(catalog).asTableCatalog
+ val testCatalog = catalog(catalogName).asTableCatalog
val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created"))
val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced"))
@@ -382,7 +384,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") {
spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("session").asTableCatalog
+ val testCatalog = catalog("session").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "session.table_name")
@@ -399,7 +401,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
test("CreateTableAsSelect: fail if table exists") {
spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -437,7 +439,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
spark.sql(
"CREATE TABLE IF NOT EXISTS testcat.table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -459,18 +461,16 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
test("CreateTableAsSelect: use default catalog for v2 sources when default catalog is set") {
- val sparkSession = spark.newSession()
- sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName)
- sparkSession.conf.set("spark.sql.default.catalog", "testcat")
+ spark.conf.set("spark.sql.default.catalog", "testcat")
- val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+ val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
df.createOrReplaceTempView("source")
// setting the default catalog breaks the reference to source because the default catalog is
// used and AsTableIdentifier no longer matches
- sparkSession.sql(s"CREATE TABLE table_name USING foo AS SELECT id, data FROM source")
+ spark.sql(s"CREATE TABLE table_name USING foo AS SELECT id, data FROM source")
- val testCatalog = sparkSession.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array(), "table_name"))
assert(table.name == "testcat.table_name")
@@ -480,30 +480,33 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
.add("id", LongType, nullable = false)
.add("data", StringType))
- val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows)
- checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), sparkSession.table("source"))
+ val rdd = sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows)
+ checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source"))
}
test("CreateTableAsSelect: v2 session catalog can load v1 source table") {
- val sparkSession = spark.newSession()
- sparkSession.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName)
+ spark.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName)
- val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+ val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
df.createOrReplaceTempView("source")
- sparkSession.sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source")
+ sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source")
- // use the catalog name to force loading with the v2 catalog
- checkAnswer(sparkSession.sql(s"TABLE session.table_name"), sparkSession.table("source"))
+ checkAnswer(sql(s"TABLE default.table_name"), spark.table("source"))
+ // The fact that the following line doesn't throw an exception means, the session catalog
+ // can load the table.
+ val t = catalog("session").asTableCatalog
+ .loadTable(Identifier.of(Array.empty, "table_name"))
+ assert(t.isInstanceOf[UnresolvedTable], "V1 table wasn't returned as an unresolved table")
}
test("DropTable: basic") {
val tableName = "testcat.ns1.ns2.tbl"
val ident = Identifier.of(Array("ns1", "ns2"), "tbl")
sql(s"CREATE TABLE $tableName USING foo AS SELECT id, data FROM source")
- assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === true)
+ assert(catalog("testcat").asTableCatalog.tableExists(ident) === true)
sql(s"DROP TABLE $tableName")
- assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === false)
+ assert(catalog("testcat").asTableCatalog.tableExists(ident) === false)
}
test("DropTable: if exists") {
@@ -594,7 +597,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
assert(exc.getMessage.contains("Unsupported table change"))
assert(exc.getMessage.contains("Cannot drop all fields")) // from the implementation
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -608,7 +611,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN data string")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -622,7 +625,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -638,7 +641,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts timestamp")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -655,7 +658,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point struct) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN point.z double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -674,7 +677,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN points.key.z double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -693,7 +696,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map>) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN points.value.z double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -712,7 +715,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN points.element.z double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -731,7 +734,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN points array>")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -749,7 +752,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t ADD COLUMN points.element.z double COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -782,7 +785,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -796,7 +799,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point struct) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -820,7 +823,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
assert(exc.getMessage.contains("point"))
assert(exc.getMessage.contains("update a struct by adding, deleting, or updating its fields"))
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -843,7 +846,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
assert(exc.getMessage.contains("update the element by updating points.element"))
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -859,7 +862,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.element TYPE long")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -880,7 +883,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
assert(exc.getMessage.contains("update a map by updating m.key or m.value"))
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -896,7 +899,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, m map) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN m.value TYPE long")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -912,7 +915,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.key.x TYPE double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -930,7 +933,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.value.x TYPE double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -948,7 +951,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.element.x TYPE double")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1008,7 +1011,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN id COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1022,7 +1025,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1036,7 +1039,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point struct) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN point.y COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1054,7 +1057,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.key.y COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1072,7 +1075,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.value.y COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1090,7 +1093,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t ALTER COLUMN points.element.y COMMENT 'doc'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1136,7 +1139,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t RENAME COLUMN id TO user_id")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1150,7 +1153,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point struct) USING foo")
sql(s"ALTER TABLE $t RENAME COLUMN point.y TO t")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1168,7 +1171,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo")
sql(s"ALTER TABLE $t RENAME COLUMN point.key.y TO t")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1186,7 +1189,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map>) USING foo")
sql(s"ALTER TABLE $t RENAME COLUMN points.value.y TO t")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1204,7 +1207,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t RENAME COLUMN points.element.y TO t")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1250,7 +1253,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, data string) USING foo")
sql(s"ALTER TABLE $t DROP COLUMN data")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1264,7 +1267,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point struct) USING foo")
sql(s"ALTER TABLE $t DROP COLUMN point.t")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1282,7 +1285,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo")
sql(s"ALTER TABLE $t DROP COLUMN point.key.y")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1299,7 +1302,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points map>) USING foo")
sql(s"ALTER TABLE $t DROP COLUMN points.value.y")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1316,7 +1319,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int, points array>) USING foo")
sql(s"ALTER TABLE $t DROP COLUMN points.element.y")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1361,7 +1364,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t SET LOCATION 's3://bucket/path'")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1375,7 +1378,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
sql(s"CREATE TABLE $t (id int) USING foo")
sql(s"ALTER TABLE $t SET TBLPROPERTIES ('test'='34')")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1388,7 +1391,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
withTable(t) {
sql(s"CREATE TABLE $t (id int) USING foo TBLPROPERTIES('test' = '34')")
- val testCatalog = spark.catalog("testcat").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name"))
assert(table.name == "testcat.ns1.table_name")
@@ -1701,8 +1704,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn
}
test("tableCreation: partition column case insensitive resolution") {
- val testCatalog = spark.catalog("testcat").asTableCatalog
- val sessionCatalog = spark.catalog("session").asTableCatalog
+ val testCatalog = catalog("testcat").asTableCatalog
+ val sessionCatalog = catalog("session").asTableCatalog
def checkPartitioning(cat: TableCatalog, partition: String): Unit = {
val table = cat.loadTable(Identifier.of(Array.empty, "tbl"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
index 379c9c4303cd6..8f7dbe8d13c39 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala
@@ -36,12 +36,12 @@ import org.apache.spark.sql.sources.{Filter, GreaterThan}
import org.apache.spark.sql.sources.v2.TableCapability._
import org.apache.spark.sql.sources.v2.reader._
import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Distribution, Partitioning}
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.vectorized.ColumnarBatch
-class DataSourceV2Suite extends QueryTest with SharedSQLContext {
+class DataSourceV2Suite extends QueryTest with SharedSparkSession {
import testImplicits._
private def getBatch(query: DataFrame): AdvancedBatch = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala
index 493aee6c1a9de..050292a4566bf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.reader.ScanBuilder
import org.apache.spark.sql.sources.v2.writer.WriteBuilder
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener}
@@ -80,7 +80,7 @@ class DummyWriteOnlyFileTable extends Table with SupportsWrite {
Set(TableCapability.BATCH_WRITE, TableCapability.ACCEPT_ANY_SCHEMA).asJava
}
-class FileDataSourceV2FallBackSuite extends QueryTest with SharedSQLContext {
+class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession {
private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].getName
private val dummyParquetWriterV2 = classOf[DummyWriteOnlyFileDataSourceV2].getName
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 f3f03715ee83a..f59f819c9c108 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
@@ -35,12 +35,12 @@ import org.apache.spark.sql.execution.streaming.sources.MemorySink
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._
import org.apache.spark.sql.streaming.util.StreamManualClock
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
abstract class FileStreamSourceTest
- extends StreamTest with SharedSQLContext with PrivateMethodTester {
+ extends StreamTest with SharedSparkSession with PrivateMethodTester {
import testImplicits._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
index 3a4414f6e6ecf..7914a713f0baa 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
@@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.streaming.sources.MemorySink
import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream}
import org.apache.spark.sql.streaming.StreamingQueryListener._
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.{Clock, SystemClock, Utils}
/**
@@ -71,7 +71,7 @@ import org.apache.spark.util.{Clock, SystemClock, Utils}
* avoid hanging forever in the case of failures. However, individual suites can change this
* by overriding `streamingTimeout`.
*/
-trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with BeforeAndAfterAll {
+trait StreamTest extends QueryTest with SharedSparkSession with TimeLimits with BeforeAndAfterAll {
// Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x
implicit val defaultSignaler: Signaler = ThreadSignaler
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
index 15a000b45a7b6..23efcf48844e0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
@@ -142,7 +142,7 @@ class MessageCapturingCommitProtocol(jobId: String, path: String)
}
-class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with BeforeAndAfter {
+class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with BeforeAndAfter {
import testImplicits._
private val userSchema = new StructType().add("s", StringType)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
index 14ac479e89754..2861b80190abe 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFlatSpecSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset
* The purpose of this suite is to make sure that generic FlatSpec-based scala
* tests work with a shared spark session
*/
-class GenericFlatSpecSuite extends FlatSpec with SharedSparkSession {
+class GenericFlatSpecSuite extends FlatSpec with SharedSparkSessionBase {
import testImplicits._
private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
index e8971e36d112d..efdaac3ae1f0c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericFunSpecSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset
* The purpose of this suite is to make sure that generic FunSpec-based scala
* tests work with a shared spark session
*/
-class GenericFunSpecSuite extends FunSpec with SharedSparkSession {
+class GenericFunSpecSuite extends FunSpec with SharedSparkSessionBase {
import testImplicits._
private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
index 44655a5345ca4..3a3540c1dbdbf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/GenericWordSpecSuite.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.Dataset
* The purpose of this suite is to make sure that generic WordSpec-based scala
* tests work with a shared spark session
*/
-class GenericWordSpecSuite extends WordSpec with SharedSparkSession {
+class GenericWordSpecSuite extends WordSpec with SharedSparkSessionBase {
import testImplicits._
private def ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
deleted file mode 100644
index 0dd24d2d56b82..0000000000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.test
-
-trait SharedSQLContext extends SQLTestUtils with SharedSparkSession {
-
- /**
- * Suites extending [[SharedSQLContext]] are sharing resources (eg. SparkSession) in their tests.
- * That trait initializes the spark session in its [[beforeAll()]] implementation before the
- * automatic thread snapshot is performed, so the audit code could fail to report threads leaked
- * by that shared session.
- *
- * The behavior is overridden here to take the snapshot before the spark session is initialized.
- */
- override protected val enableAutoThreadAudit = false
-
- protected override def beforeAll(): Unit = {
- doThreadPreAudit()
- super.beforeAll()
- }
-
- protected override def afterAll(): Unit = {
- try {
- super.afterAll()
- } finally {
- doThreadPostAudit()
- }
- }
-}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala
index ff6211b95042f..ee29b4b8fb32b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala
@@ -28,10 +28,36 @@ import org.apache.spark.sql.{SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+trait SharedSparkSession extends SQLTestUtils with SharedSparkSessionBase {
+
+ /**
+ * Suites extending [[SharedSparkSession]] are sharing resources (eg. SparkSession) in their
+ * tests. That trait initializes the spark session in its [[beforeAll()]] implementation before
+ * the automatic thread snapshot is performed, so the audit code could fail to report threads
+ * leaked by that shared session.
+ *
+ * The behavior is overridden here to take the snapshot before the spark session is initialized.
+ */
+ override protected val enableAutoThreadAudit = false
+
+ protected override def beforeAll(): Unit = {
+ doThreadPreAudit()
+ super.beforeAll()
+ }
+
+ protected override def afterAll(): Unit = {
+ try {
+ super.afterAll()
+ } finally {
+ doThreadPostAudit()
+ }
+ }
+}
+
/**
* Helper trait for SQL test suites where all tests share a single [[TestSparkSession]].
*/
-trait SharedSparkSession
+trait SharedSparkSessionBase
extends SQLTestUtilsBase
with BeforeAndAfterEach
with Eventually { self: Suite =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
index a6f7f2250b586..a8e1a44f3d5d2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala
@@ -26,9 +26,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoTable,
import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand}
import org.apache.spark.sql.execution.datasources.json.JsonFileFormat
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
-class DataFrameCallbackSuite extends QueryTest with SharedSQLContext {
+class DataFrameCallbackSuite extends QueryTest with SharedSparkSession {
import testImplicits._
import functions._
diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
index 535c32396b593..b1a907f9cba27 100644
--- a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
+++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
@@ -42,7 +42,7 @@ import org.apache.spark.sql.types._
* - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package.
* - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package.
*/
-class OrcFilterSuite extends OrcTest with SharedSQLContext {
+class OrcFilterSuite extends OrcTest with SharedSparkSession {
protected def checkFilterPredicate(
df: DataFrame,
diff --git a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
index 0f19c9e40b933..65b0537a0a8c1 100644
--- a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
+++ b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala
@@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable
-import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._
/**
@@ -42,7 +42,7 @@ import org.apache.spark.sql.types._
* - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package.
* - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package.
*/
-class OrcFilterSuite extends OrcTest with SharedSQLContext {
+class OrcFilterSuite extends OrcTest with SharedSparkSession {
protected def checkFilterPredicate(
df: DataFrame,
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
index d4df35c8ec69c..03874d005a6e6 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
@@ -363,11 +363,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
(None, message)
// our bucketing is un-compatible with hive(different hash function)
- case _ if table.bucketSpec.nonEmpty =>
+ case Some(serde) if table.bucketSpec.nonEmpty =>
val message =
s"Persisting bucketed data source table $qualifiedTableName into " +
- "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. "
- (None, message)
+ "Hive metastore in Spark SQL specific format, which is NOT compatible with " +
+ "Hive bucketed table. But Hive can read this table as a non-bucketed table."
+ (Some(newHiveCompatibleMetastoreTable(serde)), message)
case Some(serde) =>
val message =
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
index cd609002410a0..d68395812a74c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
@@ -75,7 +75,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session
new FindDataSourceTable(session) +:
new ResolveSQLOnFile(session) +:
new FallBackFileSourceV2(session) +:
- DataSourceResolution(conf, this) +:
+ DataSourceResolution(conf, this.catalogManager) +:
customResolutionRules
override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =
@@ -94,8 +94,6 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session
V2WriteSupportCheck +:
V2StreamingScanSupportCheck +:
customCheckRules
-
- override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name)
}
/**
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index deb0a1085714e..007694543df15 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
import org.apache.spark.sql.hive.test.TestHiveSingleton
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils}
import org.apache.spark.sql.types._
@@ -284,4 +284,40 @@ class DataSourceWithHiveMetastoreCatalogSuite
}
}
+
+ test("SPARK-27592 set the bucketed data source table SerDe correctly") {
+ val provider = "parquet"
+ withTable("t") {
+ spark.sql(
+ s"""
+ |CREATE TABLE t
+ |USING $provider
+ |CLUSTERED BY (c1)
+ |SORTED BY (c1)
+ |INTO 2 BUCKETS
+ |AS SELECT 1 AS c1, 2 AS c2
+ """.stripMargin)
+
+ val metadata = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default")))
+
+ val hiveSerDe = HiveSerDe.sourceToSerDe(provider).get
+ assert(metadata.storage.serde === hiveSerDe.serde)
+ assert(metadata.storage.inputFormat === hiveSerDe.inputFormat)
+ assert(metadata.storage.outputFormat === hiveSerDe.outputFormat)
+
+ // It's a bucketed table at Spark side
+ assert(sql("DESC FORMATTED t").collect().containsSlice(
+ Seq(Row("Num Buckets", "2", ""), Row("Bucket Columns", "[`c1`]", ""))
+ ))
+ checkAnswer(table("t"), Row(1, 2))
+
+ // It's not a bucketed table at Hive side
+ val hiveSide = sparkSession.metadataHive.runSqlHive("DESC FORMATTED t")
+ assert(hiveSide.contains("Num Buckets: \t-1 \t "))
+ assert(hiveSide.contains("Bucket Columns: \t[] \t "))
+ assert(hiveSide.contains("\tspark.sql.sources.schema.numBuckets\t2 "))
+ assert(hiveSide.contains("\tspark.sql.sources.schema.bucketCol.0\tc1 "))
+ assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === Seq("1\t2"))
+ }
+ }
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index a907fcae526c0..cd8e2eaa2b4dc 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -548,6 +548,14 @@ class HiveDDLSuite
assert(e.message == "Found duplicate column(s) in the table definition of `default`.`tbl`: `a`")
}
+ test("create partitioned table without specifying data type for the partition columns") {
+ val e = intercept[AnalysisException] {
+ sql("CREATE TABLE tbl(a int) PARTITIONED BY (b) STORED AS parquet")
+ }
+ assert(e.message.contains("Must specify a data type for each partition column while creating " +
+ "Hive partitioned table."))
+ }
+
test("add/drop partition with location - managed table") {
val tab = "tab_with_partitions"
withTempDir { tmpDir =>