diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 5da7d7a430d79..34d04c9856d2c 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -63,7 +63,12 @@ if not "x%JAVA_HOME%"=="x" ( rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. +:gen set LAUNCHER_OUTPUT=%temp%\spark-class-launcher-output-%RANDOM%.txt +rem SPARK-28302: %RANDOM% would return the same number if we call it instantly after last call, +rem so we should make it sure to generate unique file to avoid process collision of writing into +rem the same file concurrently. +if exist %LAUNCHER_OUTPUT% goto :gen "%RUNNER%" -Xmx128m -cp "%LAUNCH_CLASSPATH%" org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% for /f "tokens=*" %%i in (%LAUNCHER_OUTPUT%) do ( set SPARK_CMD=%%i diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 3438f5ffb36c4..908ff1983e6be 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -54,8 +54,8 @@ private static String unitRegex(String unit) { private static Pattern yearMonthPattern = Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); - private static Pattern dayTimePattern = - Pattern.compile("^(?:['|\"])?([+|-])?((\\d+) )?(\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); + private static Pattern dayTimePattern = Pattern.compile( + "^(?:['|\"])?([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); @@ -160,6 +160,20 @@ public static CalendarInterval fromYearMonthString(String s) throws IllegalArgum * adapted from HiveIntervalDayTime.valueOf */ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException { + return fromDayTimeString(s, "day", "second"); + } + + /** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf. + * Below interval conversion patterns are supported: + * - DAY TO (HOUR|MINUTE|SECOND) + * - HOUR TO (MINUTE|SECOND) + * - MINUTE TO SECOND + */ + public static CalendarInterval fromDayTimeString(String s, String from, String to) + throws IllegalArgumentException { CalendarInterval result = null; if (s == null) { throw new IllegalArgumentException("Interval day-time string was null"); @@ -174,12 +188,40 @@ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumen int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3), 0, Integer.MAX_VALUE); - long hours = toLongWithRange("hour", m.group(4), 0, 23); - long minutes = toLongWithRange("minute", m.group(5), 0, 59); - long seconds = toLongWithRange("second", m.group(6), 0, 59); + long hours = 0; + long minutes; + long seconds = 0; + if (m.group(5) != null || from.equals("minute")) { // 'HH:mm:ss' or 'mm:ss minute' + hours = toLongWithRange("hour", m.group(5), 0, 23); + minutes = toLongWithRange("minute", m.group(6), 0, 59); + seconds = toLongWithRange("second", m.group(7), 0, 59); + } else if (m.group(8) != null){ // 'mm:ss.nn' + minutes = toLongWithRange("minute", m.group(6), 0, 59); + seconds = toLongWithRange("second", m.group(7), 0, 59); + } else { // 'HH:mm' + hours = toLongWithRange("hour", m.group(6), 0, 23); + minutes = toLongWithRange("second", m.group(7), 0, 59); + } // Hive allow nanosecond precision interval - String nanoStr = m.group(8) == null ? null : (m.group(8) + "000000000").substring(0, 9); + String nanoStr = m.group(9) == null ? null : (m.group(9) + "000000000").substring(0, 9); long nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L); + switch (to) { + case "hour": + minutes = 0; + seconds = 0; + nanos = 0; + break; + case "minute": + seconds = 0; + nanos = 0; + break; + case "second": + // No-op + break; + default: + throw new IllegalArgumentException( + String.format("Cannot support (interval '%s' %s to %s) expression", s, from, to)); + } result = new CalendarInterval(0, sign * ( days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + seconds * MICROS_PER_SECOND + nanos / 1000L)); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index c125ba5f083ec..c307d74e0ba07 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -185,6 +185,14 @@ public void fromDayTimeStringTest() { } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("not match day-time format")); } + + try { + input = "5 1:12:20"; + fromDayTimeString(input, "hour", "microsecond"); + fail("Expected to throw an exception for the invalid convention type"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Cannot support (interval")); + } } @Test diff --git a/core/pom.xml b/core/pom.xml index 8a872dea1de4b..4446dbdb5ed06 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -378,7 +378,7 @@ net.razorvine pyrolite - 4.23 + 4.30 net.razorvine diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bceb26cfd4f8a..5114cf70e3f26 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -94,6 +94,7 @@ private[spark] class ExecutorAllocationManager( client: ExecutorAllocationClient, listenerBus: LiveListenerBus, conf: SparkConf, + cleaner: Option[ContextCleaner] = None, clock: Clock = new SystemClock()) extends Logging { @@ -148,7 +149,7 @@ private[spark] class ExecutorAllocationManager( // Listener for Spark events that impact the allocation policy val listener = new ExecutorAllocationListener - val executorMonitor = new ExecutorMonitor(conf, client, clock) + val executorMonitor = new ExecutorMonitor(conf, client, listenerBus, clock) // Executor that handles the scheduling task. private val executor = @@ -194,11 +195,13 @@ private[spark] class ExecutorAllocationManager( throw new SparkException( s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!") } - // Require external shuffle service for dynamic allocation - // Otherwise, we may lose shuffle files when killing executors - if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !testing) { - throw new SparkException("Dynamic allocation of executors requires the external " + - "shuffle service. You may enable this through spark.shuffle.service.enabled.") + if (!conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING)) { + logWarning("Dynamic allocation without a shuffle service is an experimental feature.") + } else if (!testing) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } } if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) { @@ -214,6 +217,7 @@ private[spark] class ExecutorAllocationManager( def start(): Unit = { listenerBus.addToManagementQueue(listener) listenerBus.addToManagementQueue(executorMonitor) + cleaner.foreach(_.attachListener(executorMonitor)) val scheduleTask = new Runnable() { override def run(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 1632e0c69eef5..446a8e5eba73e 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -68,7 +68,7 @@ private[spark] case class SSLOptions( */ def createJettySslContextFactory(): Option[SslContextFactory] = { if (enabled) { - val sslContextFactory = new SslContextFactory() + val sslContextFactory = new SslContextFactory.Server() keyStore.foreach(file => sslContextFactory.setKeyStorePath(file.getAbsolutePath)) keyStorePassword.foreach(sslContextFactory.setKeyStorePassword) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a0d7aa743223c..75182b0c9008b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -553,14 +553,22 @@ class SparkContext(config: SparkConf) extends Logging { None } - // Optionally scale number of executors dynamically based on workload. Exposed for testing. + _cleaner = + if (_conf.get(CLEANER_REFERENCE_TRACKING)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf) _executorAllocationManager = if (dynamicAllocationEnabled) { schedulerBackend match { case b: ExecutorAllocationClient => Some(new ExecutorAllocationManager( - schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf)) + schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf, + cleaner = cleaner)) case _ => None } @@ -569,14 +577,6 @@ class SparkContext(config: SparkConf) extends Logging { } _executorAllocationManager.foreach(_.start()) - _cleaner = - if (_conf.get(CLEANER_REFERENCE_TRACKING)) { - Some(new ContextCleaner(this)) - } else { - None - } - _cleaner.foreach(_.start()) - setupAndStartListenerBus() postEnvironmentUpdate() postApplicationStart() @@ -1791,7 +1791,7 @@ class SparkContext(config: SparkConf) extends Logging { * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addJar(path: String) { - def addJarFile(file: File): String = { + def addLocalJarFile(file: File): String = { try { if (!file.exists()) { throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not found") @@ -1808,12 +1808,36 @@ class SparkContext(config: SparkConf) extends Logging { } } + def checkRemoteJarFile(path: String): String = { + val hadoopPath = new Path(path) + val scheme = new URI(path).getScheme + if (!Array("http", "https", "ftp").contains(scheme)) { + try { + val fs = hadoopPath.getFileSystem(hadoopConfiguration) + if (!fs.exists(hadoopPath)) { + throw new FileNotFoundException(s"Jar ${path} not found") + } + if (fs.isDirectory(hadoopPath)) { + throw new IllegalArgumentException( + s"Directory ${path} is not allowed for addJar") + } + path + } catch { + case NonFatal(e) => + logError(s"Failed to add $path to Spark environment", e) + null + } + } else { + path + } + } + if (path == null) { logWarning("null specified as parameter to addJar") } else { val key = if (path.contains("\\")) { // For local paths with backslashes on Windows, URI throws an exception - addJarFile(new File(path)) + addLocalJarFile(new File(path)) } else { val uri = new URI(path) // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies @@ -1822,12 +1846,12 @@ class SparkContext(config: SparkConf) extends Logging { // A JAR file which exists only on the driver node case null => // SPARK-22585 path without schema is not url encoded - addJarFile(new File(uri.getRawPath)) + addLocalJarFile(new File(uri.getRawPath)) // A JAR file which exists only on the driver node - case "file" => addJarFile(new File(uri.getPath)) + case "file" => addLocalJarFile(new File(uri.getPath)) // A JAR file which exists locally on every worker node case "local" => "file:" + uri.getPath - case _ => path + case _ => checkRemoteJarFile(path) } } if (key != null) { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index c5ef1908ac436..330c2f6e6117e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -35,6 +35,7 @@ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} +import org.apache.spark.resource.ResourceInformation /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -114,6 +115,8 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { def appName: String = sc.appName + def resources: JMap[String, ResourceInformation] = sc.resources.asJava + def jars: util.List[String] = sc.jars.asJava def startTime: java.lang.Long = sc.startTime diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 414d208a1d039..dc6c59673d142 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -281,6 +281,16 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( dataOut.writeInt(context.partitionId()) dataOut.writeInt(context.attemptNumber()) dataOut.writeLong(context.taskAttemptId()) + val resources = context.resources() + dataOut.writeInt(resources.size) + resources.foreach { case (k, v) => + PythonRDD.writeUTF(k, dataOut) + PythonRDD.writeUTF(v.name, dataOut) + dataOut.writeInt(v.addresses.size) + v.addresses.foreach { case addr => + PythonRDD.writeUTF(addr, dataOut) + } + } val localProps = context.getLocalProperties.asScala dataOut.writeInt(localProps.size) localProps.foreach { case (k, v) => diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index eee6e4b28ac47..62d60475985b3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -81,4 +81,8 @@ private[spark] object PythonUtils { def isEncryptionEnabled(sc: JavaSparkContext): Boolean = { sc.conf.get(org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED) } + + def getBroadcastThreshold(sc: JavaSparkContext): Long = { + sc.conf.get(org.apache.spark.internal.config.BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) + } } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 9462dfd950bab..01e64b6972ae2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -186,9 +186,6 @@ private[spark] object SerDeUtil extends Logging { val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. - unpickle.close() if (batched) { obj match { case array: Array[Any] => array.toSeq diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index f2ee5994a8f74..5f9b18ce01279 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -805,6 +805,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private[history] def cleanLogs(): Unit = Utils.tryLog { val maxTime = clock.getTimeMillis() - conf.get(MAX_LOG_AGE_S) * 1000 + val maxNum = conf.get(MAX_LOG_NUM) val expired = listing.view(classOf[ApplicationInfoWrapper]) .index("oldestAttempt") @@ -817,23 +818,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val (remaining, toDelete) = app.attempts.partition { attempt => attempt.info.lastUpdated.getTime() >= maxTime } - - if (remaining.nonEmpty) { - val newApp = new ApplicationInfoWrapper(app.info, remaining) - listing.write(newApp) - } - - toDelete.foreach { attempt => - logInfo(s"Deleting expired event log for ${attempt.logPath}") - val logPath = new Path(logDir, attempt.logPath) - listing.delete(classOf[LogInfo], logPath.toString()) - cleanAppData(app.id, attempt.info.attemptId, logPath.toString()) - deleteLog(fs, logPath) - } - - if (remaining.isEmpty) { - listing.delete(app.getClass(), app.id) - } + deleteAttemptLogs(app, remaining, toDelete) } // Delete log files that don't have a valid application and exceed the configured max age. @@ -851,10 +836,59 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) listing.delete(classOf[LogInfo], log.logPath) } } + + // If the number of files is bigger than MAX_LOG_NUM, + // clean up all completed attempts per application one by one. + val num = listing.view(classOf[LogInfo]).index("lastProcessed").asScala.size + var count = num - maxNum + if (count > 0) { + logInfo(s"Try to delete $count old event logs to keep $maxNum logs in total.") + val oldAttempts = listing.view(classOf[ApplicationInfoWrapper]) + .index("oldestAttempt") + .asScala + oldAttempts.foreach { app => + if (count > 0) { + // Applications may have multiple attempts, some of which may not be completed yet. + val (toDelete, remaining) = app.attempts.partition(_.info.completed) + count -= deleteAttemptLogs(app, remaining, toDelete) + } + } + if (count > 0) { + logWarning(s"Fail to clean up according to MAX_LOG_NUM policy ($maxNum).") + } + } + // Clean the blacklist from the expired entries. clearBlacklist(CLEAN_INTERVAL_S) } + private def deleteAttemptLogs( + app: ApplicationInfoWrapper, + remaining: List[AttemptInfoWrapper], + toDelete: List[AttemptInfoWrapper]): Int = { + if (remaining.nonEmpty) { + val newApp = new ApplicationInfoWrapper(app.info, remaining) + listing.write(newApp) + } + + var countDeleted = 0 + toDelete.foreach { attempt => + logInfo(s"Deleting expired event log for ${attempt.logPath}") + val logPath = new Path(logDir, attempt.logPath) + listing.delete(classOf[LogInfo], logPath.toString()) + cleanAppData(app.id, attempt.info.attemptId, logPath.toString()) + if (deleteLog(fs, logPath)) { + countDeleted += 1 + } + } + + if (remaining.isEmpty) { + listing.delete(app.getClass(), app.id) + } + + countDeleted + } + /** * Delete driver logs from the configured spark dfs dir that exceed the configured max age */ @@ -1066,12 +1100,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) throw new NoSuchElementException(s"Cannot find attempt $attemptId of $appId.")) } - private def deleteLog(fs: FileSystem, log: Path): Unit = { + private def deleteLog(fs: FileSystem, log: Path): Boolean = { + var deleted = false if (isBlacklisted(log)) { logDebug(s"Skipping deleting $log as we don't have permissions on it.") } else { try { - fs.delete(log, true) + deleted = fs.delete(log, true) } catch { case _: AccessControlException => logInfo(s"No permission to delete $log, ignoring.") @@ -1079,6 +1114,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logError(s"IOException in cleaning $log", ioe) } } + deleted } private def isCompleted(name: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index a70754c6e2c48..f912ed64c80bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -24,6 +24,7 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ import org.apache.spark.internal.config +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -135,6 +136,7 @@ private[rest] class StandaloneSubmitRequestServlet( val sparkProperties = request.sparkProperties val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) val driverCores = sparkProperties.get(config.DRIVER_CORES.key) + val driverDefaultJavaOptions = sparkProperties.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) @@ -160,9 +162,11 @@ private[rest] class StandaloneSubmitRequestServlet( .set("spark.master", updatedMasters) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val defaultJavaOpts = driverDefaultJavaOptions.map(Utils.splitCommandString) + .getOrElse(Seq.empty) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaOpts = sparkJavaOpts ++ extraJavaOpts + val javaOpts = sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.worker.DriverWrapper", Seq("{{WORKER_URL}}", "{{USER_JAR}}", mainClass) ++ appArgs, // args to the DriverWrapper diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index a33c2874d1a50..759d857d56e0e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -263,12 +263,14 @@ private[spark] class HadoopDelegationTokenManager( val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) logInfo("Successfully logged into KDC.") ugi - } else { + } else if (!SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser())) { logInfo(s"Attempting to load user's ticket cache.") val ccache = sparkConf.getenv("KRB5CCNAME") val user = Option(sparkConf.getenv("KRB5PRINCIPAL")).getOrElse( UserGroupInformation.getCurrentUser().getUserName()) UserGroupInformation.getUGIFromTicketCache(ccache, user) + } else { + UserGroupInformation.getCurrentUser() } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index f27aca03773a9..68e1994f0f94f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -127,8 +127,9 @@ private[spark] class TypedConfigBuilder[T]( /** Creates a [[ConfigEntry]] that does not have a default value. */ def createOptional: OptionalConfigEntry[T] = { - val entry = new OptionalConfigEntry[T](parent.key, parent._alternatives, converter, - stringConverter, parent._doc, parent._public) + val entry = new OptionalConfigEntry[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, converter, stringConverter, parent._doc, + parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -141,8 +142,9 @@ private[spark] class TypedConfigBuilder[T]( createWithDefaultString(default.asInstanceOf[String]) } else { val transformedDefault = converter(stringConverter(default)) - val entry = new ConfigEntryWithDefault[T](parent.key, parent._alternatives, - transformedDefault, converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefault[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, transformedDefault, converter, + stringConverter, parent._doc, parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -150,8 +152,9 @@ private[spark] class TypedConfigBuilder[T]( /** Creates a [[ConfigEntry]] with a function to determine the default value */ def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = { - val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._alternatives, defaultFunc, - converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, defaultFunc, converter, stringConverter, + parent._doc, parent._public) parent._onCreate.foreach(_ (entry)) entry } @@ -161,8 +164,9 @@ private[spark] class TypedConfigBuilder[T]( * [[String]] and must be a valid value for the entry. */ def createWithDefaultString(default: String): ConfigEntry[T] = { - val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._alternatives, default, - converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, default, converter, stringConverter, + parent._doc, parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -178,6 +182,8 @@ private[spark] case class ConfigBuilder(key: String) { import ConfigHelpers._ + private[config] var _prependedKey: Option[String] = None + private[config] var _prependSeparator: String = "" private[config] var _public = true private[config] var _doc = "" private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None @@ -202,24 +208,34 @@ private[spark] case class ConfigBuilder(key: String) { this } + def withPrepended(key: String, separator: String = " "): ConfigBuilder = { + _prependedKey = Option(key) + _prependSeparator = separator + this + } + def withAlternative(key: String): ConfigBuilder = { _alternatives = _alternatives :+ key this } def intConf: TypedConfigBuilder[Int] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) } def longConf: TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) } def doubleConf: TypedConfigBuilder[Double] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) } def booleanConf: TypedConfigBuilder[Boolean] = { + checkPrependConfig new TypedConfigBuilder(this, toBoolean(_, key)) } @@ -228,20 +244,30 @@ private[spark] case class ConfigBuilder(key: String) { } def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) } def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) } def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { - val entry = new FallbackConfigEntry(key, _alternatives, _doc, _public, fallback) + val entry = new FallbackConfigEntry(key, _prependedKey, _prependSeparator, _alternatives, _doc, + _public, fallback) _onCreate.foreach(_(entry)) entry } def regexConf: TypedConfigBuilder[Regex] = { + checkPrependConfig new TypedConfigBuilder(this, regexFromString(_, this.key), _.toString) } + + private def checkPrependConfig = { + if (_prependedKey.isDefined) { + throw new IllegalArgumentException(s"$key type must be string if prepend used") + } + } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index ede3ace4f9aac..c5df4c8820098 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -28,6 +28,8 @@ package org.apache.spark.internal.config * value declared as a string. * * @param key the key for the configuration + * @param prependedKey the key for the configuration which will be prepended + * @param prependSeparator the separator which is used for prepending * @param valueConverter how to convert a string to the value. It should throw an exception if the * string does not have the required format. * @param stringConverter how to convert a value to a string that the user can use it as a valid @@ -41,6 +43,8 @@ package org.apache.spark.internal.config */ private[spark] abstract class ConfigEntry[T] ( val key: String, + val prependedKey: Option[String], + val prependSeparator: String, val alternatives: List[String], val valueConverter: String => T, val stringConverter: T => String, @@ -54,7 +58,15 @@ private[spark] abstract class ConfigEntry[T] ( def defaultValueString: String protected def readString(reader: ConfigReader): Option[String] = { - alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + val values = Seq( + prependedKey.flatMap(reader.get(_)), + alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + ).flatten + if (values.nonEmpty) { + Some(values.mkString(prependSeparator)) + } else { + None + } } def readFrom(reader: ConfigReader): T @@ -68,13 +80,24 @@ private[spark] abstract class ConfigEntry[T] ( private class ConfigEntryWithDefault[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], _defaultValue: T, valueConverter: String => T, stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(_defaultValue) @@ -86,14 +109,25 @@ private class ConfigEntryWithDefault[T] ( } private class ConfigEntryWithDefaultFunction[T] ( - key: String, - alternatives: List[String], - _defaultFunction: () => T, - valueConverter: String => T, - stringConverter: T => String, - doc: String, - isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + key: String, + prependedKey: Option[String], + prependSeparator: String, + alternatives: List[String], + _defaultFunction: () => T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(_defaultFunction()) @@ -106,13 +140,24 @@ private class ConfigEntryWithDefaultFunction[T] ( private class ConfigEntryWithDefaultString[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], _defaultValue: String, valueConverter: String => T, stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) @@ -130,14 +175,23 @@ private class ConfigEntryWithDefaultString[T] ( */ private[spark] class OptionalConfigEntry[T]( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], val rawValueConverter: String => T, val rawStringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry[Option[T]](key, alternatives, + extends ConfigEntry[Option[T]]( + key, + prependedKey, + prependSeparator, + alternatives, s => Some(rawValueConverter(s)), - v => v.map(rawStringConverter).orNull, doc, isPublic) { + v => v.map(rawStringConverter).orNull, + doc, + isPublic + ) { override def defaultValueString: String = ConfigEntry.UNDEFINED @@ -151,12 +205,22 @@ private[spark] class OptionalConfigEntry[T]( */ private[spark] class FallbackConfigEntry[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], doc: String, isPublic: Boolean, val fallback: ConfigEntry[T]) - extends ConfigEntry[T](key, alternatives, - fallback.valueConverter, fallback.stringConverter, doc, isPublic) { + extends ConfigEntry[T]( + key, + prependedKey, + prependSeparator, + alternatives, + fallback.valueConverter, + fallback.stringConverter, + doc, + isPublic + ) { override def defaultValueString: String = s"" diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 1d73f01cb84d0..ca9af316dffd0 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -49,6 +49,11 @@ private[spark] object History { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("7d") + val MAX_LOG_NUM = ConfigBuilder("spark.history.fs.cleaner.maxNum") + .doc("The maximum number of log files in the event log directory.") + .intConf + .createWithDefault(Int.MaxValue) + val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path") .doc("Local directory where to cache application history information. By default this is " + "not set, meaning all history information will be kept in memory.") diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 7c332fdb85721..f2b88fe00cdf9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -48,7 +48,10 @@ package object config { ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional private[spark] val DRIVER_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.createOptional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS) + .withPrepended(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) + .stringConf + .createOptional private[spark] val DRIVER_LIBRARY_PATH = ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional @@ -174,7 +177,10 @@ package object config { ConfigBuilder("spark.executor.heartbeat.maxFailures").internal().intConf.createWithDefault(60) private[spark] val EXECUTOR_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.createOptional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS) + .withPrepended(SparkLauncher.EXECUTOR_DEFAULT_JAVA_OPTIONS) + .stringConf + .createOptional private[spark] val EXECUTOR_LIBRARY_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional @@ -363,6 +369,17 @@ package object config { .checkValue(_ >= 0L, "Timeout must be >= 0.") .createWithDefault(60) + private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING = + ConfigBuilder("spark.dynamicAllocation.shuffleTracking.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val DYN_ALLOCATION_SHUFFLE_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.shuffleTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(_ >= 0L, "Timeout must be >= 0.") + .createWithDefault(Long.MaxValue) + private[spark] val DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT = ConfigBuilder("spark.dynamicAllocation.schedulerBacklogTimeout") .timeConf(TimeUnit.SECONDS).createWithDefault(1) @@ -1240,6 +1257,14 @@ package object config { "mechanisms to guarantee data won't be corrupted during broadcast") .booleanConf.createWithDefault(true) + private[spark] val BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD = + ConfigBuilder("spark.broadcast.UDFCompressionThreshold") + .doc("The threshold at which user-defined functions (UDFs) and Python RDD commands " + + "are compressed by broadcast in bytes unless otherwise specified") + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v >= 0, "The threshold should be non-negative.") + .createWithDefault(1L * 1024 * 1024) + private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") .doc("Whether to compress serialized RDD partitions " + "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 33a68f24bd53a..e3216151462bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -37,7 +37,8 @@ class StageInfo( val parentIds: Seq[Int], val details: String, val taskMetrics: TaskMetrics = null, - private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { + private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + private[spark] val shuffleDepId: Option[Int] = None) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -90,6 +91,10 @@ private[spark] object StageInfo { ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + val shuffleDepId = stage match { + case sms: ShuffleMapStage => Option(sms.shuffleDep).map(_.shuffleId) + case _ => None + } new StageInfo( stage.id, attemptId, @@ -99,6 +104,7 @@ private[spark] object StageInfo { stage.parents.map(_.id), stage.details, taskMetrics, - taskLocalityPreferences) + taskLocalityPreferences, + shuffleDepId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index 9aac4d2281ecf..f5beb403555e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -36,14 +36,19 @@ import org.apache.spark.util.Clock private[spark] class ExecutorMonitor( conf: SparkConf, client: ExecutorAllocationClient, - clock: Clock) extends SparkListener with Logging { + listenerBus: LiveListenerBus, + clock: Clock) extends SparkListener with CleanerListener with Logging { private val idleTimeoutMs = TimeUnit.SECONDS.toMillis( conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT)) private val storageTimeoutMs = TimeUnit.SECONDS.toMillis( conf.get(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT)) + private val shuffleTimeoutMs = conf.get(DYN_ALLOCATION_SHUFFLE_TIMEOUT) + private val fetchFromShuffleSvcEnabled = conf.get(SHUFFLE_SERVICE_ENABLED) && conf.get(SHUFFLE_SERVICE_FETCH_RDD_ENABLED) + private val shuffleTrackingEnabled = !conf.get(SHUFFLE_SERVICE_ENABLED) && + conf.get(DYN_ALLOCATION_SHUFFLE_TRACKING) private val executors = new ConcurrentHashMap[String, Tracker]() @@ -64,6 +69,26 @@ private[spark] class ExecutorMonitor( private val nextTimeout = new AtomicLong(Long.MaxValue) private var timedOutExecs = Seq.empty[String] + // Active job tracking. + // + // The following state is used when an external shuffle service is not in use, and allows Spark + // to scale down based on whether the shuffle data stored in executors is in use. + // + // The algorithm works as following: when jobs start, some state is kept that tracks which stages + // are part of that job, and which shuffle ID is attached to those stages. As tasks finish, the + // executor tracking code is updated to include the list of shuffles for which it's storing + // shuffle data. + // + // If executors hold shuffle data that is related to an active job, then the executor is + // considered to be in "shuffle busy" state; meaning that the executor is not allowed to be + // removed. If the executor has shuffle data but it doesn't relate to any active job, then it + // may be removed when idle, following the shuffle-specific timeout configuration. + // + // The following fields are not thread-safe and should be only used from the event thread. + private val shuffleToActiveJobs = new mutable.HashMap[Int, mutable.ArrayBuffer[Int]]() + private val stageToShuffleID = new mutable.HashMap[Int, Int]() + private val jobToStageIDs = new mutable.HashMap[Int, Seq[Int]]() + def reset(): Unit = { executors.clear() nextTimeout.set(Long.MaxValue) @@ -85,7 +110,7 @@ private[spark] class ExecutorMonitor( var newNextTimeout = Long.MaxValue timedOutExecs = executors.asScala - .filter { case (_, exec) => !exec.pendingRemoval } + .filter { case (_, exec) => !exec.pendingRemoval && !exec.hasActiveShuffle } .filter { case (_, exec) => val deadline = exec.timeoutAt if (deadline > now) { @@ -124,6 +149,109 @@ private[spark] class ExecutorMonitor( def pendingRemovalCount: Int = executors.asScala.count { case (_, exec) => exec.pendingRemoval } + override def onJobStart(event: SparkListenerJobStart): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + val shuffleStages = event.stageInfos.flatMap { s => + s.shuffleDepId.toSeq.map { shuffleId => + s.stageId -> shuffleId + } + } + + var updateExecutors = false + shuffleStages.foreach { case (stageId, shuffle) => + val jobIDs = shuffleToActiveJobs.get(shuffle) match { + case Some(jobs) => + // If a shuffle is being re-used, we need to re-scan the executors and update their + // tracker with the information that the shuffle data they're storing is in use. + logDebug(s"Reusing shuffle $shuffle in job ${event.jobId}.") + updateExecutors = true + jobs + + case _ => + logDebug(s"Registered new shuffle $shuffle (from stage $stageId).") + val jobs = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs(shuffle) = jobs + jobs + } + jobIDs += event.jobId + } + + if (updateExecutors) { + val activeShuffleIds = shuffleStages.map(_._2).toSeq + var needTimeoutUpdate = false + val activatedExecs = new mutable.ArrayBuffer[String]() + executors.asScala.foreach { case (id, exec) => + if (!exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffleIds) + if (exec.hasActiveShuffle) { + needTimeoutUpdate = true + activatedExecs += id + } + } + } + + logDebug(s"Activated executors ${activatedExecs.mkString(",")} due to shuffle data " + + s"needed by new job ${event.jobId}.") + + if (needTimeoutUpdate) { + nextTimeout.set(Long.MinValue) + } + } + + stageToShuffleID ++= shuffleStages + jobToStageIDs(event.jobId) = shuffleStages.map(_._1).toSeq + } + + override def onJobEnd(event: SparkListenerJobEnd): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + var updateExecutors = false + val activeShuffles = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs.foreach { case (shuffleId, jobs) => + jobs -= event.jobId + if (jobs.nonEmpty) { + activeShuffles += shuffleId + } else { + // If a shuffle went idle we need to update all executors to make sure they're correctly + // tracking active shuffles. + updateExecutors = true + } + } + + if (updateExecutors) { + if (log.isDebugEnabled()) { + if (activeShuffles.nonEmpty) { + logDebug( + s"Job ${event.jobId} ended, shuffles ${activeShuffles.mkString(",")} still active.") + } else { + logDebug(s"Job ${event.jobId} ended, no active shuffles remain.") + } + } + + val deactivatedExecs = new mutable.ArrayBuffer[String]() + executors.asScala.foreach { case (id, exec) => + if (exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffles) + if (!exec.hasActiveShuffle) { + deactivatedExecs += id + } + } + } + + logDebug(s"Executors ${deactivatedExecs.mkString(",")} do not have active shuffle data " + + s"after job ${event.jobId} finished.") + } + + jobToStageIDs.remove(event.jobId).foreach { stages => + stages.foreach { id => stageToShuffleID -= id } + } + } + override def onTaskStart(event: SparkListenerTaskStart): Unit = { val executorId = event.taskInfo.executorId // Guard against a late arriving task start event (SPARK-26927). @@ -137,6 +265,21 @@ private[spark] class ExecutorMonitor( val executorId = event.taskInfo.executorId val exec = executors.get(executorId) if (exec != null) { + // If the task succeeded and the stage generates shuffle data, record that this executor + // holds data for the shuffle. This code will track all executors that generate shuffle + // for the stage, even if speculative tasks generate duplicate shuffle data and end up + // being ignored by the map output tracker. + // + // This means that an executor may be marked as having shuffle data, and thus prevented + // from being removed, even though the data may not be used. + if (shuffleTrackingEnabled && event.reason == Success) { + stageToShuffleID.get(event.stageId).foreach { shuffleId => + exec.addShuffle(shuffleId) + } + } + + // Update the number of running tasks after checking for shuffle data, so that the shuffle + // information is up-to-date in case the executor is going idle. exec.updateRunningTasks(-1) } } @@ -171,7 +314,6 @@ private[spark] class ExecutorMonitor( // available. So don't count blocks that can be served by the external service. if (storageLevel.isValid && (!fetchFromShuffleSvcEnabled || !storageLevel.useDisk)) { val hadCachedBlocks = exec.cachedBlocks.nonEmpty - val blocks = exec.cachedBlocks.getOrElseUpdate(blockId.rddId, new mutable.BitSet(blockId.splitIndex)) blocks += blockId.splitIndex @@ -201,6 +343,25 @@ private[spark] class ExecutorMonitor( } } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case ShuffleCleanedEvent(id) => cleanupShuffle(id) + case _ => + } + + override def rddCleaned(rddId: Int): Unit = { } + + override def shuffleCleaned(shuffleId: Int): Unit = { + // Because this is called in a completely separate thread, we post a custom event to the + // listener bus so that the internal state is safely updated. + listenerBus.post(ShuffleCleanedEvent(shuffleId)) + } + + override def broadcastCleaned(broadcastId: Long): Unit = { } + + override def accumCleaned(accId: Long): Unit = { } + + override def checkpointCleaned(rddId: Long): Unit = { } + // Visible for testing. private[dynalloc] def isExecutorIdle(id: String): Boolean = { Option(executors.get(id)).map(_.isIdle).getOrElse(throw new NoSuchElementException(id)) @@ -209,7 +370,7 @@ private[spark] class ExecutorMonitor( // Visible for testing private[dynalloc] def timedOutExecutors(when: Long): Seq[String] = { executors.asScala.flatMap { case (id, tracker) => - if (tracker.timeoutAt <= when) Some(id) else None + if (tracker.isIdle && tracker.timeoutAt <= when) Some(id) else None }.toSeq } @@ -236,6 +397,14 @@ private[spark] class ExecutorMonitor( } } + private def cleanupShuffle(id: Int): Unit = { + logDebug(s"Cleaning up state related to shuffle $id.") + shuffleToActiveJobs -= id + executors.asScala.foreach { case (_, exec) => + exec.removeShuffle(id) + } + } + private class Tracker { @volatile var timeoutAt: Long = Long.MaxValue @@ -244,6 +413,7 @@ private[spark] class ExecutorMonitor( @volatile var timedOut: Boolean = false var pendingRemoval: Boolean = false + var hasActiveShuffle: Boolean = false private var idleStart: Long = -1 private var runningTasks: Int = 0 @@ -252,8 +422,11 @@ private[spark] class ExecutorMonitor( // This should only be used in the event thread. val cachedBlocks = new mutable.HashMap[Int, mutable.BitSet]() - // For testing. - def isIdle: Boolean = idleStart >= 0 + // The set of shuffles for which shuffle data is held by the executor. + // This should only be used in the event thread. + private val shuffleIds = if (shuffleTrackingEnabled) new mutable.HashSet[Int]() else null + + def isIdle: Boolean = idleStart >= 0 && !hasActiveShuffle def updateRunningTasks(delta: Int): Unit = { runningTasks = math.max(0, runningTasks + delta) @@ -264,7 +437,18 @@ private[spark] class ExecutorMonitor( def updateTimeout(): Unit = { val oldDeadline = timeoutAt val newDeadline = if (idleStart >= 0) { - idleStart + (if (cachedBlocks.nonEmpty) storageTimeoutMs else idleTimeoutMs) + val timeout = if (cachedBlocks.nonEmpty || (shuffleIds != null && shuffleIds.nonEmpty)) { + val _cacheTimeout = if (cachedBlocks.nonEmpty) storageTimeoutMs else Long.MaxValue + val _shuffleTimeout = if (shuffleIds != null && shuffleIds.nonEmpty) { + shuffleTimeoutMs + } else { + Long.MaxValue + } + math.min(_cacheTimeout, _shuffleTimeout) + } else { + idleTimeoutMs + } + idleStart + timeout } else { Long.MaxValue } @@ -279,5 +463,32 @@ private[spark] class ExecutorMonitor( updateNextTimeout(newDeadline) } } + + def addShuffle(id: Int): Unit = { + if (shuffleIds.add(id)) { + hasActiveShuffle = true + } + } + + def removeShuffle(id: Int): Unit = { + if (shuffleIds.remove(id) && shuffleIds.isEmpty) { + hasActiveShuffle = false + if (isIdle) { + updateTimeout() + } + } + } + + def updateActiveShuffles(ids: Iterable[Int]): Unit = { + val hadActiveShuffle = hasActiveShuffle + hasActiveShuffle = ids.exists(shuffleIds.contains) + if (hadActiveShuffle && isIdle) { + updateTimeout() + } + } + } + + private case class ShuffleCleanedEvent(id: Int) extends SparkListenerEvent { + override protected[spark] def logEvent: Boolean = false } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 838fc82d2ee37..54f0f8e226791 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -641,18 +641,22 @@ private[ui] class TaskPagedTable( {accumulatorsInfo(task)} }} {if (hasInput(stage)) { - metricInfo(task) { m => - val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) - val records = m.inputMetrics.recordsRead - {bytesRead} / {records} - } + { + metricInfo(task) { m => + val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) + val records = m.inputMetrics.recordsRead + Unparsed(s"$bytesRead / $records") + } + } }} {if (hasOutput(stage)) { - metricInfo(task) { m => - val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) - val records = m.outputMetrics.recordsWritten - {bytesWritten} / {records} - } + { + metricInfo(task) { m => + val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) + val records = m.outputMetrics.recordsWritten + Unparsed(s"$bytesWritten / $records") + } + } }} {if (hasShuffleRead(stage)) { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3ba33e358ef03..191b516661e49 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -1008,7 +1008,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private def createManager( conf: SparkConf, clock: Clock = new SystemClock()): ExecutorAllocationManager = { - val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock) + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) managers += manager manager.start() manager diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 6be1fedc123d8..202b85dcf5695 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -389,6 +389,19 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst """.stripMargin.trim) } + test("SPARK-28355: Use Spark conf for threshold at which UDFs are compressed by broadcast") { + val conf = new SparkConf() + + // Check the default value + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024 * 1024) + + // Set the conf + conf.set(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD, 1L * 1024) + + // Verify that it has been set properly + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024) + } + val defaultIllegalValue = "SomeIllegalValue" val illegalValueTests : Map[String, (SparkConf, String) => Any] = Map( "getTimeAsSeconds" -> (_.getTimeAsSeconds(_)), diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 628ac60fa7672..fed3ae35ee0e7 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -170,6 +170,17 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("add FS jar files not exists") { + try { + val jarPath = "hdfs:///no/path/to/TestUDTF.jar" + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(jarPath) + assert(sc.listJars().forall(!_.contains("TestUDTF.jar"))) + } finally { + sc.stop() + } + } + test("SPARK-17650: malformed url's throw exceptions before bricking Executors") { try { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 571c6e3e579ba..aaf068e81db0a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1185,6 +1185,56 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) } + test("log cleaner with the maximum number of log files") { + val clock = new ManualClock(0) + (5 to 0 by -1).foreach { num => + val log1_1 = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(log1_1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), + SparkListenerApplicationEnd(2L) + ) + log1_1.setLastModified(2L) + + val log2_1 = newLogFile("app2", Some("attempt1"), inProgress = false) + writeFile(log2_1, true, None, + SparkListenerApplicationStart("app2", Some("app2"), 3L, "test", Some("attempt1")), + SparkListenerApplicationEnd(4L) + ) + log2_1.setLastModified(4L) + + val log3_1 = newLogFile("app3", Some("attempt1"), inProgress = false) + writeFile(log3_1, true, None, + SparkListenerApplicationStart("app3", Some("app3"), 5L, "test", Some("attempt1")), + SparkListenerApplicationEnd(6L) + ) + log3_1.setLastModified(6L) + + val log1_2_incomplete = newLogFile("app1", Some("attempt2"), inProgress = false) + writeFile(log1_2_incomplete, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 7L, "test", Some("attempt2")) + ) + log1_2_incomplete.setLastModified(8L) + + val log3_2 = newLogFile("app3", Some("attempt2"), inProgress = false) + writeFile(log3_2, true, None, + SparkListenerApplicationStart("app3", Some("app3"), 9L, "test", Some("attempt2")), + SparkListenerApplicationEnd(10L) + ) + log3_2.setLastModified(10L) + + val provider = new FsHistoryProvider(createTestConf().set(MAX_LOG_NUM.key, s"$num"), clock) + updateAndCheck(provider) { list => + assert(log1_1.exists() == (num > 4)) + assert(log1_2_incomplete.exists()) // Always exists for all configurations + + assert(log2_1.exists() == (num > 3)) + + assert(log3_1.exists() == (num > 2)) + assert(log3_2.exists() == (num > 2)) + } + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala index 02514dc7daef4..c3bfa7ddee5bf 100644 --- a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala @@ -70,8 +70,8 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: fallback") { val conf = new SparkConf() - val parentConf = ConfigBuilder(testKey("parent")).intConf.createWithDefault(1) - val confWithFallback = ConfigBuilder(testKey("fallback")).fallbackConf(parentConf) + val parentConf = ConfigBuilder(testKey("parent1")).intConf.createWithDefault(1) + val confWithFallback = ConfigBuilder(testKey("fallback1")).fallbackConf(parentConf) assert(conf.get(confWithFallback) === 1) conf.set(confWithFallback, 2) assert(conf.get(parentConf) === 1) @@ -289,6 +289,92 @@ class ConfigEntrySuite extends SparkFunSuite { assert(conf.get(iConf) === 3) } + test("conf entry: prepend with default separator") { + val conf = new SparkConf() + val prependedKey = testKey("prepended1") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend1")) + .withPrepended(prependedKey) + .stringConf + .createOptional + + conf.set(derivedConf, "1") + assert(conf.get(derivedConf) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(derivedConf) === Some("2 1")) + } + + test("conf entry: prepend with custom separator") { + val conf = new SparkConf() + val prependedKey = testKey("prepended2") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend2")) + .withPrepended(prependedKey, ",") + .stringConf + .createOptional + + conf.set(derivedConf, "1") + assert(conf.get(derivedConf) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(derivedConf) === Some("2,1")) + } + + test("conf entry: prepend with fallback") { + val conf = new SparkConf() + val prependedKey = testKey("prepended3") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend3")) + .withPrepended(prependedKey) + .stringConf + .createOptional + val confWithFallback = ConfigBuilder(testKey("fallback2")).fallbackConf(derivedConf) + + assert(conf.get(confWithFallback) === None) + + conf.set(derivedConf, "1") + assert(conf.get(confWithFallback) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(confWithFallback) === Some("2 1")) + + conf.set(confWithFallback, Some("3")) + assert(conf.get(confWithFallback) === Some("3")) + } + + test("conf entry: prepend should work only with string type") { + var i = 0 + def testPrependFail(createConf: (String, String) => Unit): Unit = { + intercept[IllegalArgumentException] { + createConf(testKey(s"prependedFail$i"), testKey(s"prependFail$i")) + }.getMessage.contains("type must be string if prepend used") + i += 1 + } + + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).intConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).longConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).doubleConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).booleanConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).timeConf(TimeUnit.MILLISECONDS) + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).bytesConf(ByteUnit.BYTE) + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).regexConf + ) + } + test("onCreate") { var onCreateCalled = false ConfigBuilder(testKey("oc1")).onCreate(_ => onCreateCalled = true).intConf.createWithDefault(1) diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 8d1577e835d27..e11ee97469b00 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{mock, when} +import org.mockito.Mockito.{doAnswer, mock, when} import org.apache.spark._ import org.apache.spark.internal.config._ @@ -34,10 +34,13 @@ class ExecutorMonitorSuite extends SparkFunSuite { private val idleTimeoutMs = TimeUnit.SECONDS.toMillis(60L) private val storageTimeoutMs = TimeUnit.SECONDS.toMillis(120L) + private val shuffleTimeoutMs = TimeUnit.SECONDS.toMillis(240L) private val conf = new SparkConf() .set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "60s") .set(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT.key, "120s") + .set(DYN_ALLOCATION_SHUFFLE_TIMEOUT.key, "240s") + .set(SHUFFLE_SERVICE_ENABLED, true) private var monitor: ExecutorMonitor = _ private var client: ExecutorAllocationClient = _ @@ -55,7 +58,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { when(client.isExecutorActive(any())).thenAnswer { invocation => knownExecs.contains(invocation.getArguments()(0).asInstanceOf[String]) } - monitor = new ExecutorMonitor(conf, client, clock) + monitor = new ExecutorMonitor(conf, client, null, clock) } test("basic executor timeout") { @@ -205,7 +208,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) conf.set(SHUFFLE_SERVICE_ENABLED, true).set(SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) - monitor = new ExecutorMonitor(conf, client, clock) + monitor = new ExecutorMonitor(conf, client, null, clock) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.MEMORY_ONLY)) @@ -259,8 +262,119 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors().toSet === Set("2")) } + test("shuffle block tracking") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + // 3 jobs: 2 and 3 share a shuffle, 1 has a separate shuffle. + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + + val stage5 = stageInfo(5, shuffleId = 1) + val stage6 = stageInfo(6) + + // Start jobs 1 and 2. Finish a task on each, but don't finish the jobs. This should prevent the + // executor from going idle since there are active shuffles. + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + // First a failed task, to make sure it does not count. + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", TaskResultLost, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + // Finish the jobs, now the executor should be idle, but with the shuffle timeout, since the + // shuffles are not active. + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(!monitor.isExecutorIdle("1")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Start job 3. Since it shares a shuffle with job 2, the executor should not be considered + // idle anymore, even if no tasks are run. + monitor.onJobStart(SparkListenerJobStart(3, clock.getTimeMillis(), Seq(stage5, stage6))) + assert(!monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(shuffleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(3, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Clean up the shuffles, executor now should now time out at the idle deadline. + monitor.shuffleCleaned(0) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + monitor.shuffleCleaned(1) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + } + + test("shuffle tracking with multiple executors and concurrent jobs") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + + // Two separate jobs with separate shuffles. The first job will only run tasks on + // executor 1, the second on executor 2. Ensures that jobs finishing don't affect + // executors that are active in other jobs. + + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("2")) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("2", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("2", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(!monitor.isExecutorIdle("2")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("2")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.shuffleCleaned(0) + monitor.shuffleCleaned(1) + assert(monitor.timedOutExecutors(idleDeadline).toSet === Set("1", "2")) + } + private def idleDeadline: Long = clock.getTimeMillis() + idleTimeoutMs + 1 private def storageDeadline: Long = clock.getTimeMillis() + storageTimeoutMs + 1 + private def shuffleDeadline: Long = clock.getTimeMillis() + shuffleTimeoutMs + 1 + + private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { + new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + } private def taskInfo( execId: String, @@ -286,4 +400,16 @@ class ExecutorMonitorSuite extends SparkFunSuite { RDDBlockId(rddId, splitIndex), level, 1L, 0L)) } + /** + * Mock the listener bus *only* for the functionality needed by the shuffle tracking code. + * Any other event sent through the mock bus will fail. + */ + private def mockListenerBus(): LiveListenerBus = { + val bus = mock(classOf[LiveListenerBus]) + doAnswer { invocation => + monitor.onOtherEvent(invocation.getArguments()(0).asInstanceOf[SparkListenerEvent]) + }.when(bus).post(any()) + bus + } + } diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 242163931f7ac..f5f93ece660b8 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -128,7 +128,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { val files = testRolling(appender, testOutputStream, textToAppend, 0, isCompressed = true) files.foreach { file => logInfo(file.toString + ": " + file.length + " bytes") - assert(file.length < rolloverSize) + assert(file.length <= rolloverSize) } } diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 2f660ccfd92fe..79158bb6edfe3 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -170,7 +170,7 @@ parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar -pyrolite-4.23.jar +pyrolite-4.30.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index e1e114fa08cad..5e03a5951db04 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -189,7 +189,7 @@ parquet-hadoop-1.10.1.jar parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar -pyrolite-4.23.jar +pyrolite-4.30.jar re2j-1.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 593e34983aa09..e51e9560b59d4 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -353,7 +353,7 @@ def choose_jira_assignee(issue, asf_jira): except: # assume it's a user id, and try to assign (might fail, we just prompt again) assignee = asf_jira.user(raw_assignee) - asf_jira.assign_issue(issue.key, assignee.key) + asf_jira.assign_issue(issue.key, assignee.name) return assignee except KeyboardInterrupt: raise diff --git a/docs/building-spark.md b/docs/building-spark.md index 391ad223ccaa4..5453a218205e2 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -83,11 +83,14 @@ Example: To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 1.2.1 bindings. +By default, Spark will use Hive 1.2.1 with the `hadoop-2.7` profile, and Hive 2.3.5 with the `hadoop-3.2` profile. # With Hive 1.2.1 support ./build/mvn -Pyarn -Phive -Phive-thriftserver -DskipTests clean package + # With Hive 2.3.5 support + ./build/mvn -Pyarn -Phive -Phive-thriftserver -Phadoop-3.2 -DskipTests clean package + ## Packaging without Hadoop Dependencies for YARN The assembly directory produced by `mvn package` will, by default, include all of Spark's diff --git a/docs/configuration.md b/docs/configuration.md index 211dfbb3f459e..108862416f8da 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -410,11 +410,31 @@ Apart from these, the following properties are also available, and may be useful your default properties file. + + spark.driver.defaultJavaOptions + (none) + + A string of default JVM options to prepend to spark.driver.extraJavaOptions. + This is intended to be set by administrators. + + For instance, GC settings or other logging. + Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap + size settings can be set with spark.driver.memory in the cluster mode and through + the --driver-memory command line option in the client mode. + +
Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-java-options command line option or in + your default properties file. + + spark.driver.extraJavaOptions (none) - A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. + A string of extra JVM options to pass to the driver. This is intended to be set by users. + + For instance, GC settings or other logging. Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap size settings can be set with spark.driver.memory in the cluster mode and through the --driver-memory command line option in the client mode. @@ -423,6 +443,8 @@ Apart from these, the following properties are also available, and may be useful directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-java-options command line option or in your default properties file. + + spark.driver.defaultJavaOptions will be prepended to this configuration. @@ -457,11 +479,31 @@ Apart from these, the following properties are also available, and may be useful this option. + + spark.executor.defaultJavaOptions + (none) + + A string of default JVM options to prepend to spark.executor.extraJavaOptions. + This is intended to be set by administrators. + + For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or maximum heap size (-Xmx) settings with this + option. Spark properties should be set using a SparkConf object or the spark-defaults.conf file + used with the spark-submit script. Maximum heap size settings can be set with spark.executor.memory. + + The following symbols, if present will be interpolated: {{APP_ID}} will be replaced by + application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable + verbose gc logging to a file named for the executor ID of the app in /tmp, pass a 'value' of: + -verbose:gc -Xloggc:/tmp/{{APP_ID}}-{{EXECUTOR_ID}}.gc + + spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + A string of extra JVM options to pass to executors. This is intended to be set by users. + + For instance, GC settings or other logging. Note that it is illegal to set Spark properties or maximum heap size (-Xmx) settings with this option. Spark properties should be set using a SparkConf object or the spark-defaults.conf file used with the spark-submit script. Maximum heap size settings can be set with spark.executor.memory. @@ -470,6 +512,8 @@ Apart from these, the following properties are also available, and may be useful application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable verbose gc logging to a file named for the executor ID of the app in /tmp, pass a 'value' of: -verbose:gc -Xloggc:/tmp/{{APP_ID}}-{{EXECUTOR_ID}}.gc + + spark.executor.defaultJavaOptions will be prepended to this configuration. @@ -2070,6 +2114,26 @@ Apart from these, the following properties are also available, and may be useful description. + + spark.dynamicAllocation.shuffleTracking.enabled + false + + Experimental. Enables shuffle file tracking for executors, which allows dynamic allocation + without the need for an external shuffle service. This option will try to keep alive executors + that are storing shuffle data for active jobs. + + + + spark.dynamicAllocation.shuffleTimeout + infinity + + When shuffle tracking is enabled, controls the timeout for executors that are holding shuffle + data. The default value means that Spark will rely on the shuffles being garbage collected to be + able to release executors. If for some reason garbage collection is not cleaning up shuffles + quickly enough, this option can be used to control when to time out executors even when they are + storing shuffle data. + + ### Thread Configurations diff --git a/docs/monitoring.md b/docs/monitoring.md index 4017677861a75..0f7210c3b8bb2 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -190,7 +190,11 @@ Security options for the Spark History Server are covered more detail in the 1d How often the filesystem job history cleaner checks for files to delete. - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge + Files are deleted if at least one of two conditions holds. + First, they're deleted if they're older than spark.history.fs.cleaner.maxAge. + They are also deleted if the number of files is more than + spark.history.fs.cleaner.maxNum, Spark tries to clean up the completed attempts + from the applications based on the order of their oldest attempt time. @@ -200,6 +204,16 @@ Security options for the Spark History Server are covered more detail in the Job history files older than this will be deleted when the filesystem history cleaner runs. + + spark.history.fs.cleaner.maxNum + Int.MaxValue + + The maximum number of files in the event log directory. + Spark tries to clean up the completed attempt logs to maintain the log directory under this limit. + This should be smaller than the underlying file system limit like + `dfs.namenode.fs-limits.max-directory-items` in HDFS. + + spark.history.fs.endEventReparseChunkSize 1m diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index dc93e9cea5bce..9d9b253a5c8ea 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -142,20 +142,20 @@ To use a custom metrics.properties for the application master and executors, upd - spark.yarn.am.resource.{resource-type} + spark.yarn.am.resource.{resource-type}.amount (none) Amount of resource to use for the YARN Application Master in client mode. - In cluster mode, use spark.yarn.driver.resource.<resource-type> instead. + In cluster mode, use spark.yarn.driver.resource.<resource-type>.amount instead. Please note that this feature can be used only with YARN 3.0+ For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu.amount - spark.yarn.driver.resource.{resource-type} + spark.yarn.driver.resource.{resource-type}.amount (none) Amount of resource to use for the YARN Application Master in cluster mode. @@ -163,11 +163,11 @@ To use a custom metrics.properties for the application master and executors, upd For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu.amount - spark.yarn.executor.resource.{resource-type} + spark.yarn.executor.resource.{resource-type}.amount (none) Amount of resource to use per executor process. @@ -175,7 +175,7 @@ To use a custom metrics.properties for the application master and executors, upd For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu.amount diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 816b4eaa0ca63..13058cba7564f 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -117,6 +117,7 @@ Below is a list of all the keywords in Spark SQL. FIELDSnon-reservednon-reservednon-reserved FILEFORMATnon-reservednon-reservednon-reserved FIRSTnon-reservednon-reservednon-reserved + FIRST_VALUEreservednon-reservedreserved FOLLOWINGnon-reservednon-reservednon-reserved FORreservednon-reservedreserved FOREIGNreservednon-reservedreserved @@ -151,6 +152,7 @@ Below is a list of all the keywords in Spark SQL. JOINreservedstrict-non-reservedreserved KEYSnon-reservednon-reservednon-reserved LASTnon-reservednon-reservednon-reserved + LAST_VALUEreservednon-reservedreserved LATERALnon-reservednon-reservedreserved LAZYnon-reservednon-reservednon-reserved LEADINGreservednon-reservedreserved @@ -219,6 +221,7 @@ Below is a list of all the keywords in Spark SQL. REPAIRnon-reservednon-reservednon-reserved REPLACEnon-reservednon-reservednon-reserved RESETnon-reservednon-reservednon-reserved + RESPECTnon-reservednon-reservednon-reserved RESTRICTnon-reservednon-reservednon-reserved REVOKEnon-reservednon-reservedreserved RIGHTreservedstrict-non-reservedreserved diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index d39bd933427fc..f13d298674b2d 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -149,6 +149,10 @@ license: | - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. + + - Since Spark 3.0, the `add_months` function adjusts the resulting date to a last day of month only if it is invalid. For example, `select add_months(DATE'2019-01-31', 1)` results `2019-02-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when it is invalid, or the original date is a last day of months. For example, adding a month to `2019-02-28` resultes in `2019-03-31`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index d5224da2cf3f7..fe3c60040d0a0 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -818,6 +818,11 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} +#### Kafka Specific Configurations + +Kafka's own configurations can be set with `kafka.` prefix, e.g, `--conf spark.kafka.clusters.${cluster}.kafka.retries=1`. +For possible Kafka parameters, see [Kafka adminclient config docs](http://kafka.apache.org/documentation.html#adminclientconfigs). + #### Caveats - Obtaining delegation token for proxy user is not yet supported ([KAFKA-6945](https://issues.apache.org/jira/browse/KAFKA-6945)). diff --git a/docs/tuning.md b/docs/tuning.md index 222f8720ce35f..1faf7cfe0d68e 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -248,7 +248,7 @@ Our experience suggests that the effect of GC tuning depends on your application There are [many more tuning options](https://docs.oracle.com/javase/8/docs/technotes/guides/vm/gctuning/index.html) described online, but at a high level, managing how frequently full GC takes place can help in reducing the overhead. -GC tuning flags for executors can be specified by setting `spark.executor.extraJavaOptions` in +GC tuning flags for executors can be specified by setting `spark.executor.defaultJavaOptions` or `spark.executor.extraJavaOptions` in a job's configuration. # Other Considerations 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 40bf3b1530fbb..924bf374c7370 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 @@ -50,7 +50,7 @@ abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUti override protected def beforeAll(): Unit = { super.beforeAll() - spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) + spark.conf.set(SQLConf.FILES_MAX_PARTITION_BYTES.key, 1024) } def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 82ce16c2b7e5a..efd7ca74c796b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -120,24 +120,24 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types.length == 12) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Integer")) - assert(types(2).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Long")) assert(types(5).equals("class java.lang.Double")) - assert(types(6).equals("class java.lang.Double")) - assert(types(7).equals("class java.lang.Double")) + assert(types(6).equals("class java.lang.Float")) + assert(types(7).equals("class java.lang.Float")) assert(types(8).equals("class java.math.BigDecimal")) assert(types(9).equals("class java.math.BigDecimal")) assert(types(10).equals("class java.math.BigDecimal")) assert(types(11).equals("class java.math.BigDecimal")) assert(row.getBoolean(0) == false) assert(row.getInt(1) == 255) - assert(row.getInt(2) == 32767) + assert(row.getShort(2) == 32767) assert(row.getInt(3) == 2147483647) assert(row.getLong(4) == 9223372036854775807L) assert(row.getDouble(5) == 1.2345678901234512E14) // float = float(53) has 15-digits precision - assert(row.getDouble(6) == 1.23456788103168E14) // float(24) has 7-digits precision - assert(row.getDouble(7) == 1.23456788103168E14) // real = float(24) + assert(row.getFloat(6) == 1.23456788103168E14) // float(24) has 7-digits precision + assert(row.getFloat(7) == 1.23456788103168E14) // real = float(24) assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00"))) assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000"))) assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800"))) 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 64b9837cc5fac..8cdc4a1806b2b 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 @@ -376,8 +376,8 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo val e = intercept[org.apache.spark.SparkException] { spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", new Properties()).collect() } - assert(e.getMessage.contains( - "requirement failed: Decimal precision 39 exceeds max precision 38")) + assert(e.getCause().isInstanceOf[ArithmeticException]) + assert(e.getMessage.contains("Decimal precision 39 exceeds max precision 38")) // custom schema can read data val props = new Properties() diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 462f88ff14a8d..89da9a1de6f74 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -206,4 +206,17 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { """.stripMargin.replaceAll("\n", " ")) assert(sql("select c1, c3 from queryOption").collect.toSet == expectedResult) } + + test("write byte as smallint") { + sqlContext.createDataFrame(Seq((1.toByte, 2.toShort))) + .write.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val df = sqlContext.read.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val schema = df.schema + assert(schema.head.dataType == ShortType) + assert(schema(1).dataType == ShortType) + val rows = df.collect() + assert(rows.length === 1) + assert(rows(0).getShort(0) === 1) + assert(rows(0).getShort(1) === 2) + } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 9b3e78c84c34a..76c25980fc33f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -21,7 +21,7 @@ import org.apache.kafka.clients.producer.ProducerRecord import org.apache.spark.sql.Dataset import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.ContinuousTrigger import org.apache.spark.sql.streaming.Trigger // Run tests in KafkaSourceSuiteBase in continuous execution mode. 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 e089e36eba5f3..ba8340ea59c14 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 @@ -135,7 +135,7 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe test("failOnDataLoss=false should not return duplicated records: microbatch v1") { withSQLConf( - "spark.sql.streaming.disabledV2MicroBatchReaders" -> + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key -> classOf[KafkaSourceProvider].getCanonicalName) { verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = true) { (df, table) => val query = df.writeStream.format("memory").queryName(table).start() 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 3d14ebe267c49..bb9b3696fe8f6 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 @@ -1066,7 +1066,7 @@ class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { override def beforeAll(): Unit = { super.beforeAll() spark.conf.set( - "spark.sql.streaming.disabledV2MicroBatchReaders", + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key, classOf[KafkaSourceProvider].getCanonicalName) } diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala index 84d58d8c419ad..e1f3c800a51f8 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT private[spark] case class KafkaTokenClusterConf( identifier: String, @@ -35,7 +36,8 @@ private[spark] case class KafkaTokenClusterConf( keyStoreLocation: Option[String], keyStorePassword: Option[String], keyPassword: Option[String], - tokenMechanism: String) { + tokenMechanism: String, + specifiedKafkaParams: Map[String, String]) { override def toString: String = s"KafkaTokenClusterConf{" + s"identifier=$identifier, " + s"authBootstrapServers=$authBootstrapServers, " + @@ -43,11 +45,12 @@ private[spark] case class KafkaTokenClusterConf( s"securityProtocol=$securityProtocol, " + s"kerberosServiceName=$kerberosServiceName, " + s"trustStoreLocation=$trustStoreLocation, " + - s"trustStorePassword=${trustStorePassword.map(_ => "xxx")}, " + + s"trustStorePassword=${trustStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + s"keyStoreLocation=$keyStoreLocation, " + - s"keyStorePassword=${keyStorePassword.map(_ => "xxx")}, " + - s"keyPassword=${keyPassword.map(_ => "xxx")}, " + - s"tokenMechanism=$tokenMechanism}" + s"keyStorePassword=${keyStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"keyPassword=${keyPassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"tokenMechanism=$tokenMechanism, " + + s"specifiedKafkaParams=${KafkaRedactionUtil.redactParams(specifiedKafkaParams.toSeq)}}" } private [kafka010] object KafkaTokenSparkConf extends Logging { @@ -59,6 +62,8 @@ private [kafka010] object KafkaTokenSparkConf extends Logging { def getClusterConfig(sparkConf: SparkConf, identifier: String): KafkaTokenClusterConf = { val configPrefix = s"$CLUSTERS_CONFIG_PREFIX$identifier." val sparkClusterConf = sparkConf.getAllWithPrefix(configPrefix).toMap + val configKafkaPrefix = s"${configPrefix}kafka." + val sparkClusterKafkaConf = sparkConf.getAllWithPrefix(configKafkaPrefix).toMap val result = KafkaTokenClusterConf( identifier, sparkClusterConf @@ -76,7 +81,8 @@ private [kafka010] object KafkaTokenSparkConf extends Logging { sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG), sparkClusterConf.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG), sparkClusterConf.getOrElse("sasl.token.mechanism", - KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM), + sparkClusterKafkaConf ) logDebug(s"getClusterConfig($identifier): $result") result diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala index da21d2e2413da..950df867e1e8a 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -134,6 +134,16 @@ private[spark] object KafkaTokenUtil extends Logging { } } + logDebug("AdminClient params before specified params: " + + s"${KafkaRedactionUtil.redactParams(adminClientProperties.asScala.toSeq)}") + + clusterConf.specifiedKafkaParams.foreach { param => + adminClientProperties.setProperty(param._1, param._2) + } + + logDebug("AdminClient params after specified params: " + + s"${KafkaRedactionUtil.redactParams(adminClientProperties.asScala.toSeq)}") + adminClientProperties } diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala index 74f1cdcf73462..eebbf96afa470 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala @@ -107,7 +107,8 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { protected def createClusterConf( identifier: String, - securityProtocol: String): KafkaTokenClusterConf = { + securityProtocol: String, + specifiedKafkaParams: Map[String, String] = Map.empty): KafkaTokenClusterConf = { KafkaTokenClusterConf( identifier, bootStrapServers, @@ -119,6 +120,7 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { Some(keyStoreLocation), Some(keyStorePassword), Some(keyPassword), - KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM, + specifiedKafkaParams) } } diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala index 60bb8a2bc6c31..61184a6fac33d 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala @@ -96,6 +96,16 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { assert(clusterConfig.tokenMechanism === tokenMechanism) } + test("getClusterConfig should return specified kafka params") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", authBootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.kafka.customKey", "customValue") + + val clusterConfig = KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1) + assert(clusterConfig.identifier === identifier1) + assert(clusterConfig.authBootstrapServers === authBootStrapServers) + assert(clusterConfig.specifiedKafkaParams === Map("customKey" -> "customValue")) + } + test("getAllClusterConfigs should return empty list when nothing configured") { assert(KafkaTokenSparkConf.getAllClusterConfigs(sparkConf).isEmpty) } diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala index bcca920eed4e4..5496195b41490 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala @@ -155,6 +155,15 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { assert(saslJaasConfig.contains("useTicketCache=true")) } + test("createAdminClientProperties with specified params should include it") { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name, + Map("customKey" -> "customValue")) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get("customKey") === "customValue") + } + test("isGlobalJaasConfigurationProvided without global config should return false") { assert(!KafkaTokenUtil.isGlobalJaasConfigurationProvided) } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 422f53d6c8774..4d3e476e7cc58 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kafka010 import java.io.File import java.lang.{ Long => JLong } import java.util.{ Arrays, HashMap => JHashMap, Map => JMap, UUID } +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicLong @@ -430,7 +431,7 @@ class DirectKafkaStreamSuite ) val collectedData = new ConcurrentLinkedQueue[String]() - val committed = new JHashMap[TopicPartition, OffsetAndMetadata]() + val committed = new ConcurrentHashMap[TopicPartition, OffsetAndMetadata]() // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { @@ -458,6 +459,7 @@ class DirectKafkaStreamSuite logError("commit failed", e) } else { committed.putAll(m) + logDebug(s"commit succeeded: $m") } }) } @@ -467,8 +469,10 @@ class DirectKafkaStreamSuite for (i <- (1 to 10).grouped(4)) { sendDataAndWaitForReceive(i) } + eventually(timeout(10.seconds), interval(50.milliseconds)) { + assert(!committed.isEmpty) + } ssc.stop() - assert(! committed.isEmpty) val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer.poll(0) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index f86d40015bd22..84940d96b563f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -48,6 +48,8 @@ public class SparkLauncher extends AbstractLauncher { public static final String DRIVER_MEMORY = "spark.driver.memory"; /** Configuration key for the driver class path. */ public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath"; + /** Configuration key for the default driver VM options. */ + public static final String DRIVER_DEFAULT_JAVA_OPTIONS = "spark.driver.defaultJavaOptions"; /** Configuration key for the driver VM options. */ public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions"; /** Configuration key for the driver native library path. */ @@ -57,6 +59,8 @@ public class SparkLauncher extends AbstractLauncher { public static final String EXECUTOR_MEMORY = "spark.executor.memory"; /** Configuration key for the executor class path. */ public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath"; + /** Configuration key for the default executor VM options. */ + public static final String EXECUTOR_DEFAULT_JAVA_OPTIONS = "spark.executor.defaultJavaOptions"; /** Configuration key for the executor VM options. */ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index e3ee843f62449..3479e0c3422bd 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -267,13 +267,10 @@ private List buildSparkSubmitCommand(Map env) // We don't want the client to specify Xmx. These have to be set by their corresponding // memory flag --driver-memory or configuration entry spark.driver.memory + String driverDefaultJavaOptions = config.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS); + checkJavaOptions(driverDefaultJavaOptions); String driverExtraJavaOptions = config.get(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS); - if (!isEmpty(driverExtraJavaOptions) && driverExtraJavaOptions.contains("Xmx")) { - String msg = String.format("Not allowed to specify max heap(Xmx) memory settings through " + - "java options (was %s). Use the corresponding --driver-memory or " + - "spark.driver.memory configuration instead.", driverExtraJavaOptions); - throw new IllegalArgumentException(msg); - } + checkJavaOptions(driverExtraJavaOptions); if (isClientMode) { // Figuring out where the memory value come from is a little tricky due to precedence. @@ -289,6 +286,7 @@ private List buildSparkSubmitCommand(Map env) String memory = firstNonEmpty(tsMemory, config.get(SparkLauncher.DRIVER_MEMORY), System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); cmd.add("-Xmx" + memory); + addOptionString(cmd, driverDefaultJavaOptions); addOptionString(cmd, driverExtraJavaOptions); mergeEnvPathList(env, getLibPathEnvName(), config.get(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH)); @@ -299,6 +297,15 @@ private List buildSparkSubmitCommand(Map env) return cmd; } + private void checkJavaOptions(String javaOptions) { + if (!isEmpty(javaOptions) && javaOptions.contains("Xmx")) { + String msg = String.format("Not allowed to specify max heap(Xmx) memory settings through " + + "java options (was %s). Use the corresponding --driver-memory or " + + "spark.driver.memory configuration instead.", javaOptions); + throw new IllegalArgumentException(msg); + } + } + private List buildPySparkShellCommand(Map env) throws IOException { // For backwards compatibility, if a script is specified in // the pyspark command line, then run it using spark-submit. diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index e694e9066f12e..32a91b1789412 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -251,6 +251,8 @@ public void testMissingAppResource() { } private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception { + final String DRIVER_DEFAULT_PARAM = "-Ddriver-default"; + final String DRIVER_EXTRA_PARAM = "-Ddriver-extra"; String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = @@ -270,7 +272,8 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th launcher.setPropertiesFile(dummyPropsFile.getAbsolutePath()); launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g"); launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver"); - launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver"); + launcher.conf.put(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS, DRIVER_DEFAULT_PARAM); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, DRIVER_EXTRA_PARAM); launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native"); } else { launcher.childEnv.put("SPARK_CONF_DIR", System.getProperty("spark.test.home") @@ -284,6 +287,9 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th if (isDriver) { assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx1g")); + assertTrue("Driver default options should be configured.", + cmd.contains(DRIVER_DEFAULT_PARAM)); + assertTrue("Driver extra options should be configured.", cmd.contains(DRIVER_EXTRA_PARAM)); } else { boolean found = false; for (String arg : cmd) { @@ -293,6 +299,10 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th } } assertFalse("Memory arguments should not be set.", found); + assertFalse("Driver default options should not be configured.", + cmd.contains(DRIVER_DEFAULT_PARAM)); + assertFalse("Driver extra options should not be configured.", + cmd.contains(DRIVER_EXTRA_PARAM)); } String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); diff --git a/launcher/src/test/resources/spark-defaults.conf b/launcher/src/test/resources/spark-defaults.conf index 3a51208c7c244..22c253693dcff 100644 --- a/launcher/src/test/resources/spark-defaults.conf +++ b/launcher/src/test/resources/spark-defaults.conf @@ -17,5 +17,6 @@ spark.driver.memory=1g spark.driver.extraClassPath=/driver -spark.driver.extraJavaOptions=-Ddriver +spark.driver.defaultJavaOptions=-Ddriver-default +spark.driver.extraJavaOptions=-Ddriver-extra spark.driver.extraLibraryPath=/native \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index d8f3dfa874439..58815434cbdaf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -204,8 +204,8 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, if ($(predictionCol).nonEmpty) { transformImpl(dataset) } else { - this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: Predictor.transform() does nothing" + + " because no output columns were set.") dataset.toDF } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index e35e6ce7fdadb..568cdd11a12a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -204,8 +204,8 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } if (numColsOutput == 0) { - logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + logWarning(s"$uid: ClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index e1fceb1fc96a4..675315e3bb070 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -169,9 +169,9 @@ final class OneVsRestModel private[ml] ( // Check schema transformSchema(dataset.schema, logging = true) - if (getPredictionCol == "" && getRawPredictionCol == "") { - logWarning(s"$uid: OneVsRestModel.transform() was called as NOOP" + - " since no output columns were set.") + if (getPredictionCol.isEmpty && getRawPredictionCol.isEmpty) { + logWarning(s"$uid: OneVsRestModel.transform() does nothing" + + " because no output columns were set.") return dataset.toDF } @@ -218,7 +218,7 @@ final class OneVsRestModel private[ml] ( var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] - if (getRawPredictionCol != "") { + if (getRawPredictionCol.nonEmpty) { val numClass = models.length // output the RawPrediction as vector @@ -228,18 +228,18 @@ final class OneVsRestModel private[ml] ( Vectors.dense(predArray) } - predictionColNames = predictionColNames :+ getRawPredictionCol - predictionColumns = predictionColumns :+ rawPredictionUDF(col(accColName)) + predictionColNames :+= getRawPredictionCol + predictionColumns :+= rawPredictionUDF(col(accColName)) } - if (getPredictionCol != "") { + if (getPredictionCol.nonEmpty) { // output the index of the classifier with highest confidence as prediction val labelUDF = udf { (predictions: Map[Int, Double]) => predictions.maxBy(_._2)._1.toDouble } - predictionColNames = predictionColNames :+ getPredictionCol - predictionColumns = predictionColumns :+ labelUDF(col(accColName)) + predictionColNames :+= getPredictionCol + predictionColumns :+= labelUDF(col(accColName)) .as(getPredictionCol, labelMetadata) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 730fcab333e11..5046caa568d5c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -147,8 +147,8 @@ abstract class ProbabilisticClassificationModel[ } if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index fb4698ab55643..9a51d2f188460 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -33,7 +33,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -110,11 +110,29 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predUDF = udf((vector: Vector) => predict(vector)) - val probUDF = udf((vector: Vector) => predictProbability(vector)) - dataset - .withColumn($(predictionCol), predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) - .withColumn($(probabilityCol), probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predUDF = udf((vector: Vector) => predict(vector)) + predictionColNames :+= $(predictionCol) + predictionColumns :+= predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if ($(probabilityCol).nonEmpty) { + val probUDF = udf((vector: Vector) => predictProbability(vector)) + predictionColNames :+= $(probabilityCol) + predictionColumns :+= probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index aa81037014451..91201e7bd03f9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -461,17 +461,10 @@ abstract class LDAModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - if ($(topicDistributionCol).nonEmpty) { - val func = getTopicDistributionMethod - val transformer = udf(func) - - dataset.withColumn($(topicDistributionCol), - transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) - } else { - logWarning("LDAModel.transform was called without any output columns. Set an output column" + - " such as topicDistributionCol to produce results.") - dataset.toDF() - } + val func = getTopicDistributionMethod + val transformer = udf(func) + dataset.withColumn($(topicDistributionCol), + transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) } /** @@ -490,7 +483,7 @@ abstract class LDAModel private[ml] ( Vectors.zeros(k) } else { val (ids: List[Int], cts: Array[Double]) = vector match { - case v: DenseVector => ((0 until v.size).toList, v.values) + case v: DenseVector => (List.range(0, v.size), v.values) case v: SparseVector => (v.indices.toList, v.values) case other => throw new UnsupportedOperationException( diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 17f2c17c9552e..81cf2e1a4ff79 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -169,7 +169,7 @@ class StandardScalerModel private[ml] ( case d: DenseVector => d.values.clone() case v: Vector => v.toArray } - val newValues = scaler.transfromWithMean(values) + val newValues = scaler.transformWithMean(values) Vectors.dense(newValues) } else if ($(withStd)) { vector: Vector => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 067dfa43433e4..1565782dd6314 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructType} import org.apache.spark.storage.StorageLevel @@ -355,13 +355,28 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predictUDF = udf { features: Vector => predict(features) } - val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predictUDF = udf { features: Vector => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) + } + if (hasQuantilesCol) { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) - .withColumn($(quantilesCol), predictQuantilesUDF(col($(featuresCol)))) + val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + predictionColNames :+= $(quantilesCol) + predictionColumns :+= predictQuantilesUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) } else { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + this.logWarning(s"$uid: AFTSurvivalRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index f4f4e56a35787..6348289de516e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -34,7 +34,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType @@ -216,16 +216,28 @@ class DecisionTreeRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector) => predict(features) } - val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } - var output = dataset.toDF() + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + val predictUDF = udf { (features: Vector) => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) } + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { - output = output.withColumn($(varianceCol), predictVarianceUDF(col($(featuresCol)))) + val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } + predictionColNames :+= $(varianceCol) + predictionColumns :+= predictVarianceUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: DecisionTreeRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } - output } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 885b13bf8dac3..b1a8f95c12618 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1041,18 +1041,31 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } - val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) - var output = dataset + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol)), offset) } + if (hasLinkPredictionCol) { - output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) + val predictLinkUDF = + udf { (features: Vector, offset: Double) => predictLink(features, offset) } + predictionColNames :+= $(linkPredictionCol) + predictionColumns :+= predictLinkUDF(col($(featuresCol)), offset) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } - output.toDF() } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 322ef93473da0..4617073f9decd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -53,6 +53,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTree import org.apache.spark.mllib.util.{LinearDataGenerator, MLUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.LongType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -1142,12 +1143,21 @@ private[python] class PythonMLLibAPI extends Serializable { new RowMatrix(rows.rdd, numRows, numCols) } + def createRowMatrix(df: DataFrame, numRows: Long, numCols: Int): RowMatrix = { + require(df.schema.length == 1 && df.schema.head.dataType.getClass == classOf[VectorUDT], + "DataFrame must have a single vector type column") + new RowMatrix(df.rdd.map { case Row(vector: Vector) => vector }, numRows, numCols) + } + /** * Wrapper around IndexedRowMatrix constructor. */ def createIndexedRowMatrix(rows: DataFrame, numRows: Long, numCols: Int): IndexedRowMatrix = { // We use DataFrames for serialization of IndexedRows from Python, // so map each Row in the DataFrame back to an IndexedRow. + require(rows.schema.length == 2 && rows.schema.head.dataType == LongType && + rows.schema(1).dataType.getClass == classOf[VectorUDT], + "DataFrame must consist of a long type index column and a vector type column") val indexedRows = rows.rdd.map { case Row(index: Long, vector: Vector) => IndexedRow(index, vector) } @@ -1347,10 +1357,6 @@ private[spark] abstract class SerDeBase { val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. Pyrolite - // doesn't clear it up, so we manually clear it. - unpickle.close() if (batched) { obj match { case list: JArrayList[_] => list.asScala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 578b779cd52d2..19e53e7eac844 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -141,7 +141,7 @@ class StandardScalerModel @Since("1.3.0") ( case d: DenseVector => d.values.clone() case v: Vector => v.toArray } - val newValues = transfromWithMean(values) + val newValues = transformWithMean(values) Vectors.dense(newValues) } else if (withStd) { vector match { @@ -161,7 +161,7 @@ class StandardScalerModel @Since("1.3.0") ( } } - private[spark] def transfromWithMean(values: Array[Double]): Array[Double] = { + private[spark] def transformWithMean(values: Array[Double]): Array[Double] = { // By default, Scala generates Java methods for member variables. So every time when // the member variables are accessed, `invokespecial` will be called which is expensive. // This can be avoid by having a local reference of `shift`. diff --git a/pom.xml b/pom.xml index 81e6e7956ab74..dae5eb0070155 100644 --- a/pom.xml +++ b/pom.xml @@ -832,7 +832,7 @@ org.mockito mockito-core - 2.23.4 + 2.28.2 test diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index cb3b803096432..5978f88d6a468 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -372,7 +372,11 @@ object MimaExcludes { // [SPARK-26616][MLlib] Expose document frequency in IDFModel ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf") + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), + + // [SPARK-28199][SS] Remove deprecated ProcessingTime + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime$") ) // Exclude rules for 2.4.x diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index ee153af18c88c..70c0b27a6aa33 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -54,6 +54,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast +from pyspark.resourceinformation import ResourceInformation from pyspark.serializers import MarshalSerializer, PickleSerializer from pyspark.status import * from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo @@ -118,5 +119,5 @@ def wrapper(self, *args, **kwargs): "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", - "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", + "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation", ] diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 69020e6585ffb..8d2848841c109 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -37,6 +37,7 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer, ChunkedStream from pyspark.storagelevel import StorageLevel +from pyspark.resourceinformation import ResourceInformation from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker @@ -1107,6 +1108,17 @@ def getConf(self): conf.setAll(self._conf.getAll()) return conf + @property + def resources(self): + resources = {} + jresources = self._jsc.resources() + for x in jresources: + name = jresources[x].name() + jaddresses = jresources[x].addresses() + addrs = [addr for addr in jaddresses] + resources[name] = ResourceInformation(name, addrs) + return resources + def _test(): import atexit diff --git a/python/pyspark/mllib/linalg/distributed.py b/python/pyspark/mllib/linalg/distributed.py index b7f09782be9dd..56701758c89c9 100644 --- a/python/pyspark/mllib/linalg/distributed.py +++ b/python/pyspark/mllib/linalg/distributed.py @@ -30,6 +30,7 @@ from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import _convert_to_vector, DenseMatrix, Matrix, QRDecomposition from pyspark.mllib.stat import MultivariateStatisticalSummary +from pyspark.sql import DataFrame from pyspark.storagelevel import StorageLevel @@ -57,7 +58,8 @@ class RowMatrix(DistributedMatrix): Represents a row-oriented distributed Matrix with no meaningful row indices. - :param rows: An RDD of vectors. + :param rows: An RDD or DataFrame of vectors. If a DataFrame is provided, it must have a single + vector typed column. :param numRows: Number of rows in the matrix. A non-positive value means unknown, at which point the number of rows will be determined by the number of @@ -73,7 +75,7 @@ def __init__(self, rows, numRows=0, numCols=0): Create a wrapper over a Java RowMatrix. - Publicly, we require that `rows` be an RDD. However, for + Publicly, we require that `rows` be an RDD or DataFrame. However, for internal usage, `rows` can also be a Java RowMatrix object, in which case we can wrap it directly. This assists in clean matrix conversions. @@ -94,6 +96,8 @@ def __init__(self, rows, numRows=0, numCols=0): if isinstance(rows, RDD): rows = rows.map(_convert_to_vector) java_matrix = callMLlibFunc("createRowMatrix", rows, long(numRows), int(numCols)) + elif isinstance(rows, DataFrame): + java_matrix = callMLlibFunc("createRowMatrix", rows, long(numRows), int(numCols)) elif (isinstance(rows, JavaObject) and rows.getClass().getSimpleName() == "RowMatrix"): java_matrix = rows @@ -461,7 +465,8 @@ class IndexedRowMatrix(DistributedMatrix): """ Represents a row-oriented distributed Matrix with indexed rows. - :param rows: An RDD of IndexedRows or (long, vector) tuples. + :param rows: An RDD of IndexedRows or (long, vector) tuples or a DataFrame consisting of a + long typed column of indices and a vector typed column. :param numRows: Number of rows in the matrix. A non-positive value means unknown, at which point the number of rows will be determined by the max row @@ -477,7 +482,7 @@ def __init__(self, rows, numRows=0, numCols=0): Create a wrapper over a Java IndexedRowMatrix. - Publicly, we require that `rows` be an RDD. However, for + Publicly, we require that `rows` be an RDD or DataFrame. However, for internal usage, `rows` can also be a Java IndexedRowMatrix object, in which case we can wrap it directly. This assists in clean matrix conversions. @@ -506,6 +511,8 @@ def __init__(self, rows, numRows=0, numCols=0): # IndexedRows on the Scala side. java_matrix = callMLlibFunc("createIndexedRowMatrix", rows.toDF(), long(numRows), int(numCols)) + elif isinstance(rows, DataFrame): + java_matrix = callMLlibFunc("createIndexedRowMatrix", rows, long(numRows), int(numCols)) elif (isinstance(rows, JavaObject) and rows.getClass().getSimpleName() == "IndexedRowMatrix"): java_matrix = rows diff --git a/python/pyspark/mllib/tests/test_linalg.py b/python/pyspark/mllib/tests/test_linalg.py index b3ab25f2ef5e0..312730e8aff8b 100644 --- a/python/pyspark/mllib/tests/test_linalg.py +++ b/python/pyspark/mllib/tests/test_linalg.py @@ -25,10 +25,15 @@ from pyspark.serializers import PickleSerializer from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector, \ DenseMatrix, SparseMatrix, Vectors, Matrices, MatrixUDT +from pyspark.mllib.linalg.distributed import RowMatrix, IndexedRowMatrix from pyspark.mllib.regression import LabeledPoint +from pyspark.sql import Row from pyspark.testing.mllibutils import MLlibTestCase from pyspark.testing.utils import have_scipy +if sys.version >= '3': + long = int + class VectorTests(MLlibTestCase): @@ -431,6 +436,24 @@ def test_infer_schema(self): else: raise TypeError("expecting a vector but got %r of type %r" % (v, type(v))) + def test_row_matrix_from_dataframe(self): + from pyspark.sql.utils import IllegalArgumentException + df = self.spark.createDataFrame([Row(Vectors.dense(1))]) + row_matrix = RowMatrix(df) + self.assertEqual(row_matrix.numRows(), 1) + self.assertEqual(row_matrix.numCols(), 1) + with self.assertRaises(IllegalArgumentException): + RowMatrix(df.selectExpr("'monkey'")) + + def test_indexed_row_matrix_from_dataframe(self): + from pyspark.sql.utils import IllegalArgumentException + df = self.spark.createDataFrame([Row(long(0), Vectors.dense(1))]) + matrix = IndexedRowMatrix(df) + self.assertEqual(matrix.numRows(), 1) + self.assertEqual(matrix.numCols(), 1) + with self.assertRaises(IllegalArgumentException): + IndexedRowMatrix(df.drop("_1")) + class MatrixUDTTests(MLlibTestCase): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8bcc67ab1c3e6..96fdf5f33b39d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2490,7 +2490,7 @@ def _prepare_for_python_RDD(sc, command): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) - if len(pickled_command) > (1 << 20): # 1M + if len(pickled_command) > sc._jvm.PythonUtils.getBroadcastThreshold(sc._jsc): # Default 1M # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) diff --git a/python/pyspark/resourceinformation.py b/python/pyspark/resourceinformation.py new file mode 100644 index 0000000000000..aaed21374b6ee --- /dev/null +++ b/python/pyspark/resourceinformation.py @@ -0,0 +1,43 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +class ResourceInformation(object): + + """ + .. note:: Evolving + + Class to hold information about a type of Resource. A resource could be a GPU, FPGA, etc. + The array of addresses are resource specific and its up to the user to interpret the address. + + One example is GPUs, where the addresses would be the indices of the GPUs + + @param name the name of the resource + @param addresses an array of strings describing the addresses of the resource + """ + + def __init__(self, name, addresses): + self._name = name + self._addresses = addresses + + @property + def name(self): + return self._name + + @property + def addresses(self): + return self._addresses diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f8be8ee5d4c36..398471234d2b7 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -22,8 +22,10 @@ basestring = unicode = str long = int from functools import reduce + from html import escape as html_escape else: from itertools import imap as map + from cgi import escape as html_escape import warnings @@ -375,7 +377,6 @@ def _repr_html_(self): by 'spark.sql.repl.eagerEval.enabled', this only called by REPL you are using support eager evaluation with HTML. """ - import cgi if not self._support_repr_html: self._support_repr_html = True if self.sql_ctx._conf.isReplEagerEvalEnabled(): @@ -390,11 +391,11 @@ def _repr_html_(self): html = "\n" # generate table head - html += "\n" % "\n" % "\n" % "
%s
".join(map(lambda x: cgi.escape(x), head)) + html += "
%s
".join(map(lambda x: html_escape(x), head)) # generate table rows for row in row_data: html += "
%s
".join( - map(lambda x: cgi.escape(x), row)) + map(lambda x: html_escape(x), row)) html += "
\n" if has_more_data: html += "only showing top %d %s\n" % ( diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6bb7da6b2edb2..e531000f3295c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -36,6 +36,7 @@ from pyspark.sql.types import StringType, DataType # Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409 from pyspark.sql.udf import UserDefinedFunction, _create_udf +from pyspark.sql.utils import to_str # Note to developers: all of PySpark functions here take string as column names whenever possible. # Namely, if columns are referred as arguments, they can be always both Column or string, @@ -114,6 +115,10 @@ def _(): _.__doc__ = 'Window function: ' + doc return _ + +def _options_to_str(options): + return {key: to_str(value) for (key, value) in options.items()} + _lit_doc = """ Creates a :class:`Column` of literal value. @@ -2343,7 +2348,7 @@ def from_json(col, schema, options={}): schema = schema.json() elif isinstance(schema, Column): schema = _to_java_column(schema) - jc = sc._jvm.functions.from_json(_to_java_column(col), schema, options) + jc = sc._jvm.functions.from_json(_to_java_column(col), schema, _options_to_str(options)) return Column(jc) @@ -2384,7 +2389,7 @@ def to_json(col, options={}): """ sc = SparkContext._active_spark_context - jc = sc._jvm.functions.to_json(_to_java_column(col), options) + jc = sc._jvm.functions.to_json(_to_java_column(col), _options_to_str(options)) return Column(jc) @@ -2415,7 +2420,7 @@ def schema_of_json(json, options={}): raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_json(col, options) + jc = sc._jvm.functions.schema_of_json(col, _options_to_str(options)) return Column(jc) @@ -2442,7 +2447,7 @@ def schema_of_csv(csv, options={}): raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_csv(col, options) + jc = sc._jvm.functions.schema_of_csv(col, _options_to_str(options)) return Column(jc) @@ -2464,7 +2469,7 @@ def to_csv(col, options={}): """ sc = SparkContext._active_spark_context - jc = sc._jvm.functions.to_csv(_to_java_column(col), options) + jc = sc._jvm.functions.to_csv(_to_java_column(col), _options_to_str(options)) return Column(jc) @@ -2693,7 +2698,10 @@ def array_repeat(col, count): [Row(r=[u'ab', u'ab', u'ab'])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.array_repeat(_to_java_column(col), count)) + return Column(sc._jvm.functions.array_repeat( + _to_java_column(col), + _to_java_column(count) if isinstance(count, Column) else count + )) @since(2.4) @@ -2775,6 +2783,11 @@ def from_csv(col, schema, options={}): >>> value = data[0][0] >>> df.select(from_csv(df.value, schema_of_csv(value)).alias("csv")).collect() [Row(csv=Row(_c0=1, _c1=2, _c2=3))] + >>> data = [(" abc",)] + >>> df = spark.createDataFrame(data, ("value",)) + >>> options = {'ignoreLeadingWhiteSpace': True} + >>> df.select(from_csv(df.value, "s string", options).alias("csv")).collect() + [Row(csv=Row(s=u'abc'))] """ sc = SparkContext._active_spark_context @@ -2785,7 +2798,7 @@ def from_csv(col, schema, options={}): else: raise TypeError("schema argument should be a column or string") - jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, options) + jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, _options_to_str(options)) return Column(jc) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index aa5bf635d1874..f9bc2ff72a505 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -27,23 +27,11 @@ from pyspark.sql.column import _to_seq from pyspark.sql.types import * from pyspark.sql import utils +from pyspark.sql.utils import to_str __all__ = ["DataFrameReader", "DataFrameWriter"] -def to_str(value): - """ - A wrapper over str(), but converts bool values to lower case strings. - If None is given, just returns None, instead of converting it to string "None". - """ - if isinstance(value, bool): - return str(value).lower() - elif value is None: - return value - else: - return str(value) - - class OptionUtils(object): def _set_opts(self, schema=None, **options): @@ -757,7 +745,7 @@ def save(self, path=None, format=None, mode=None, partitionBy=None, **options): self._jwrite.save(path) @since(1.4) - def insertInto(self, tableName, overwrite=False): + def insertInto(self, tableName, overwrite=None): """Inserts the content of the :class:`DataFrame` to the specified table. It requires that the schema of the class:`DataFrame` is the same as the @@ -765,7 +753,9 @@ def insertInto(self, tableName, overwrite=False): Optionally overwriting any existing data. """ - self._jwrite.mode("overwrite" if overwrite else "append").insertInto(tableName) + if overwrite is not None: + self.mode("overwrite" if overwrite else "append") + self._jwrite.insertInto(tableName) @since(1.4) def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options): diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index e5c3b48d548e2..5550a093bf808 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -750,6 +750,7 @@ def test_query_execution_listener_on_collect(self): self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should not be called before 'collect'") self.spark.sql("SELECT * FROM range(1)").collect() + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty(10000) self.assertTrue( self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should be called after 'collect'") @@ -764,6 +765,7 @@ def test_query_execution_listener_on_collect_with_arrow(self): "The callback from the query execution listener should not be " "called before 'toPandas'") self.spark.sql("SELECT * FROM range(1)").toPandas() + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty(10000) self.assertTrue( self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should be called after 'toPandas'") diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 7dfc757970091..64f2fd6a3919f 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -294,6 +294,16 @@ def test_input_file_name_reset_for_rdd(self): for result in results: self.assertEqual(result[0], '') + def test_array_repeat(self): + from pyspark.sql.functions import array_repeat, lit + + df = self.spark.range(1) + + self.assertEquals( + df.select(array_repeat("id", 3)).toDF("val").collect(), + df.select(array_repeat("id", lit(3))).toDF("val").collect(), + ) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index a708072489601..2530cc2ebf224 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -141,6 +141,27 @@ def count_bucketed_cols(names, table="pyspark_bucket"): .mode("overwrite").saveAsTable("pyspark_bucket")) self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + def test_insert_into(self): + df = self.spark.createDataFrame([("a", 1), ("b", 2)], ["C1", "C2"]) + with self.table("test_table"): + df.write.saveAsTable("test_table") + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table") + self.assertEqual(4, self.spark.sql("select * from test_table").count()) + + df.write.mode("overwrite").insertInto("test_table") + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table", True) + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table", False) + self.assertEqual(4, self.spark.sql("select * from test_table").count()) + + df.write.mode("overwrite").insertInto("test_table", False) + self.assertEqual(6, self.spark.sql("select * from test_table").count()) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/tests/test_serde.py b/python/pyspark/sql/tests/test_serde.py index f9bed7604b13f..ea2a686cddaa2 100644 --- a/python/pyspark/sql/tests/test_serde.py +++ b/python/pyspark/sql/tests/test_serde.py @@ -128,10 +128,6 @@ def test_BinaryType_serialization(self): def test_int_array_serialization(self): # Note that this test seems dependent on parallelism. - # This issue is because internal object map in Pyrolite is not cleared after op code - # STOP. If we use protocol 4 to pickle Python objects, op code MEMOIZE will store - # objects in the map. We need to clear up it to make sure next unpickling works on - # clear map. data = self.spark.sparkContext.parallelize([[1, 2, 3, 4]] * 100, numSlices=12) df = self.spark.createDataFrame(data, "array") self.assertEqual(len(list(filter(lambda r: None in r.value, df.collect()))), 0) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 0dafa18743fa8..803d471c8c6fd 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -197,6 +197,8 @@ def test_udf_in_join_condition(self): left = self.spark.createDataFrame([Row(a=1)]) right = self.spark.createDataFrame([Row(b=1)]) f = udf(lambda a, b: a == b, BooleanType()) + # The udf uses attributes from both sides of join, so it is pulled out as Filter + + # Cross join. df = left.join(right, f("a", "b")) with self.assertRaisesRegexp(AnalysisException, 'Detected implicit cartesian product'): df.collect() @@ -243,6 +245,14 @@ def runWithJoinType(join_type, type_string): runWithJoinType("leftanti", "LeftAnti") runWithJoinType("leftsemi", "LeftSemi") + def test_udf_as_join_condition(self): + left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) + right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) + f = udf(lambda a: a, IntegerType()) + + df = left.join(right, [f("a") == f("b"), left.a1 == right.b1]) + self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) + def test_udf_without_arguments(self): self.spark.catalog.registerFunction("foo", lambda: "bar") [row] = self.spark.sql("SELECT foo()").collect() diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index ca5e85bb3a9bb..c30cc1482750a 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -207,3 +207,16 @@ def call(self, jdf, batch_id): class Java: implements = ['org.apache.spark.sql.execution.streaming.sources.PythonForeachBatchFunction'] + + +def to_str(value): + """ + A wrapper over str(), but converts bool values to lower case strings. + If None is given, just returns None, instead of converting it to string "None". + """ + if isinstance(value, bool): + return str(value).lower() + elif value is None: + return value + else: + return str(value) diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index 6d28491e4aff3..790de0b9bf41a 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -38,6 +38,7 @@ class TaskContext(object): _stageId = None _taskAttemptId = None _localProperties = None + _resources = None def __new__(cls): """Even if users construct TaskContext instead of using get, give them the singleton.""" @@ -95,6 +96,13 @@ def getLocalProperty(self, key): """ return self._localProperties.get(key, None) + def resources(self): + """ + Resources allocated to the task. The key is the resource name and the value is information + about the resource. + """ + return self._resources + BARRIER_FUNCTION = 1 diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index 4048ac5b03375..bcd5d06c1b67f 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -16,13 +16,14 @@ # import os import shutil +import stat import tempfile import threading import time import unittest from collections import namedtuple -from pyspark import SparkFiles, SparkContext +from pyspark import SparkConf, SparkFiles, SparkContext from pyspark.testing.utils import ReusedPySparkTestCase, PySparkTestCase, QuietTest, SPARK_HOME @@ -256,6 +257,38 @@ def test_forbid_insecure_gateway(self): SparkContext(gateway=mock_insecure_gateway) self.assertIn("insecure Py4j gateway", str(context.exception)) + def test_resources(self): + """Test the resources are empty by default.""" + with SparkContext() as sc: + resources = sc.resources + self.assertEqual(len(resources), 0) + + +class ContextTestsWithResources(unittest.TestCase): + + def setUp(self): + class_name = self.__class__.__name__ + self.tempFile = tempfile.NamedTemporaryFile(delete=False) + self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') + self.tempFile.close() + os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | + stat.S_IROTH | stat.S_IXOTH) + conf = SparkConf().set("spark.driver.resource.gpu.amount", "1") + conf = conf.set("spark.driver.resource.gpu.discoveryScript", self.tempFile.name) + self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) + + def test_resources(self): + """Test the resources are available.""" + resources = self.sc.resources + self.assertEqual(len(resources), 1) + self.assertTrue('gpu' in resources) + self.assertEqual(resources['gpu'].name, 'gpu') + self.assertEqual(resources['gpu'].addresses, ['0']) + + def tearDown(self): + os.unlink(self.tempFile.name) + self.sc.stop() + if __name__ == "__main__": from pyspark.tests.test_context import * diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index d7d1d80ea115b..66357b61c79ee 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -16,7 +16,9 @@ # import os import random +import stat import sys +import tempfile import time import unittest @@ -43,6 +45,15 @@ def test_stage_id(self): self.assertEqual(stage1 + 2, stage3) self.assertEqual(stage2 + 1, stage3) + def test_resources(self): + """Test the resources are empty by default.""" + rdd = self.sc.parallelize(range(10)) + resources1 = rdd.map(lambda x: TaskContext.get().resources()).take(1)[0] + # Test using the constructor directly rather than the get() + resources2 = rdd.map(lambda x: TaskContext().resources()).take(1)[0] + self.assertEqual(len(resources1), 0) + self.assertEqual(len(resources2), 0) + def test_partition_id(self): """Test the partition id.""" rdd1 = self.sc.parallelize(range(10), 1) @@ -174,6 +185,33 @@ def tearDown(self): self.sc.stop() +class TaskContextTestsWithResources(unittest.TestCase): + + def setUp(self): + class_name = self.__class__.__name__ + self.tempFile = tempfile.NamedTemporaryFile(delete=False) + self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') + self.tempFile.close() + os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | + stat.S_IROTH | stat.S_IXOTH) + conf = SparkConf().set("spark.task.resource.gpu.amount", "1") + conf = conf.set("spark.executor.resource.gpu.amount", "1") + conf = conf.set("spark.executor.resource.gpu.discoveryScript", self.tempFile.name) + self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) + + def test_resources(self): + """Test the resources are available.""" + rdd = self.sc.parallelize(range(10)) + resources = rdd.map(lambda x: TaskContext.get().resources()).take(1)[0] + self.assertEqual(len(resources), 1) + self.assertTrue('gpu' in resources) + self.assertEqual(resources['gpu'].name, 'gpu') + self.assertEqual(resources['gpu'].addresses, ['0']) + + def tearDown(self): + os.unlink(self.tempFile.name) + self.sc.stop() + if __name__ == "__main__": import unittest from pyspark.tests.test_taskcontext import * diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index b34abd07e8c8c..7f38c27360ed9 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -35,6 +35,7 @@ from pyspark.java_gateway import local_connect_and_auth from pyspark.taskcontext import BarrierTaskContext, TaskContext from pyspark.files import SparkFiles +from pyspark.resourceinformation import ResourceInformation from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ @@ -435,6 +436,16 @@ def main(infile, outfile): taskContext._partitionId = read_int(infile) taskContext._attemptNumber = read_int(infile) taskContext._taskAttemptId = read_long(infile) + taskContext._resources = {} + for r in range(read_int(infile)): + key = utf8_deserializer.loads(infile) + name = utf8_deserializer.loads(infile) + addresses = [] + taskContext._resources = {} + for a in range(read_int(infile)): + addresses.append(utf8_deserializer.loads(infile)) + taskContext._resources[key] = ResourceInformation(name, addresses) + taskContext._localProperties = dict() for i in range(read_int(infile)): k = utf8_deserializer.loads(infile) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 3ff68348be7b1..2fd13a5903243 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -28,6 +28,7 @@ import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest._ import org.apache.spark.internal.config +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils @@ -97,6 +98,7 @@ private[mesos] class MesosSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties + val driverDefaultJavaOptions = sparkProperties.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) @@ -110,9 +112,11 @@ private[mesos] class MesosSubmitRequestServlet( val conf = new SparkConf(false).setAll(sparkProperties) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val defaultJavaOpts = driverDefaultJavaOptions.map(Utils.splitCommandString) + .getOrElse(Seq.empty) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaOpts = sparkJavaOpts ++ extraJavaOpts + val javaOpts = sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts val command = new Command( mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) 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 5b361d17c01a9..651e706021fcb 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 @@ -51,7 +51,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.api.python.PythonUtils import org.apache.spark.deploy.{SparkApplication, SparkHadoopUtil} import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -241,12 +241,12 @@ private[spark] class Client( newApp: YarnClientApplication, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val yarnAMResources = - if (isClusterMode) { - sparkConf.getAllWithPrefix(config.YARN_DRIVER_RESOURCE_TYPES_PREFIX).toMap - } else { - sparkConf.getAllWithPrefix(config.YARN_AM_RESOURCE_TYPES_PREFIX).toMap - } + val componentName = if (isClusterMode) { + config.YARN_DRIVER_RESOURCE_TYPES_PREFIX + } else { + config.YARN_AM_RESOURCE_TYPES_PREFIX + } + val yarnAMResources = getYarnResourcesAndAmounts(sparkConf, componentName) val amResources = yarnAMResources ++ getYarnResourcesFromSparkResources(SPARK_DRIVER_PREFIX, sparkConf) logDebug(s"AM resources: $amResources") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index cb0c68d1d346d..522c16b3a1082 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -26,11 +26,11 @@ import scala.util.Try import org.apache.hadoop.yarn.api.records.Resource import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.{AMOUNT, FPGA, GPU} import org.apache.spark.util.{CausedBy, Utils} /** @@ -40,6 +40,45 @@ import org.apache.spark.util.{CausedBy, Utils} private object ResourceRequestHelper extends Logging { private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" + val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" + val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" + + private[yarn] def getYarnResourcesAndAmounts( + sparkConf: SparkConf, + componentName: String): Map[String, String] = { + sparkConf.getAllWithPrefix(s"$componentName").map { case (key, value) => + val splitIndex = key.lastIndexOf('.') + if (splitIndex == -1) { + val errorMessage = s"Missing suffix for ${componentName}${key}, you must specify" + + s" a suffix - $AMOUNT is currently the only supported suffix." + throw new IllegalArgumentException(errorMessage.toString()) + } + val resourceName = key.substring(0, splitIndex) + val resourceSuffix = key.substring(splitIndex + 1) + if (!AMOUNT.equals(resourceSuffix)) { + val errorMessage = s"Unsupported suffix: $resourceSuffix in: ${componentName}${key}, " + + s"only .$AMOUNT is supported." + throw new IllegalArgumentException(errorMessage.toString()) + } + (resourceName, value) + }.toMap + } + + /** + * Convert Spark resources into YARN resources. + * The only resources we know how to map from spark configs to yarn configs are + * gpus and fpgas, everything else the user has to specify them in both the + * spark.yarn.*.resource and the spark.*.resource configs. + */ + private[yarn] def getYarnResourcesFromSparkResources( + confPrefix: String, + sparkConf: SparkConf + ): Map[String, String] = { + Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { + case (rName, yarnName) => + (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) + }.filter { case (_, count) => count.toLong > 0 } + } /** * Validates sparkConf and throws a SparkException if any of standard resources (memory or cores) @@ -81,8 +120,9 @@ private object ResourceRequestHelper extends Logging { val errorMessage = new mutable.StringBuilder() resourceDefinitions.foreach { case (sparkName, resourceRequest) => - if (sparkConf.contains(resourceRequest)) { - errorMessage.append(s"Error: Do not use $resourceRequest, " + + val resourceRequestAmount = s"${resourceRequest}.${AMOUNT}" + if (sparkConf.contains(resourceRequestAmount)) { + errorMessage.append(s"Error: Do not use $resourceRequestAmount, " + s"please use $sparkName instead!\n") } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 6e634b921fcd1..8ec7bd66b2507 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{SecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging @@ -142,8 +143,8 @@ private[yarn] class YarnAllocator( protected val executorCores = sparkConf.get(EXECUTOR_CORES) private val executorResourceRequests = - sparkConf.getAllWithPrefix(config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX).toMap ++ - getYarnResourcesFromSparkResources(SPARK_EXECUTOR_PREFIX, sparkConf) + getYarnResourcesAndAmounts(sparkConf, config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) ++ + getYarnResourcesFromSparkResources(SPARK_EXECUTOR_PREFIX, sparkConf) // Resource capability requested for each executor private[yarn] val resource: Resource = { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 6b87eec795f9f..11035520ae185 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -41,24 +41,6 @@ object YarnSparkHadoopUtil { val MEMORY_OVERHEAD_MIN = 384L val ANY_HOST = "*" - val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" - val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" - - /** - * Convert Spark resources into YARN resources. - * The only resources we know how to map from spark configs to yarn configs are - * gpus and fpgas, everything else the user has to specify them in both the - * spark.yarn.*.resource and the spark.*.resource configs. - */ - private[yarn] def getYarnResourcesFromSparkResources( - confPrefix: String, - sparkConf: SparkConf - ): Map[String, String] = { - Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { - case (rName, yarnName) => - (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) - }.filter { case (_, count) => count.toLong > 0 } - } // All RM requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index d5f1992a09f53..847fc3773de59 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -38,10 +38,11 @@ import org.mockito.Mockito.{spy, verify} import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TestUtils} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.AMOUNT import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { @@ -372,7 +373,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, deployMode) resources.foreach { case (name, v) => - conf.set(prefix + name, v.toString) + conf.set(s"${prefix}${name}.${AMOUNT}", v.toString) } val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) @@ -397,7 +398,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") resources.keys.foreach { yarnName => - conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}", "2") + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") @@ -407,9 +408,9 @@ class ClientSuite extends SparkFunSuite with Matchers { ResourceRequestHelper.validateResources(conf) }.getMessage() - assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/fpga," + + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/fpga.amount," + " please use spark.driver.resource.fpga.amount")) - assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/gpu," + + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/gpu.amount," + " please use spark.driver.resource.gpu.amount")) } @@ -420,7 +421,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") resources.keys.foreach { yarnName => - conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}", "2") + conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => conf.set(ResourceID(SPARK_EXECUTOR_PREFIX, rName).amountConf, "3") @@ -430,9 +431,9 @@ class ClientSuite extends SparkFunSuite with Matchers { ResourceRequestHelper.validateResources(conf) }.getMessage() - assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/fpga," + + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/fpga.amount," + " please use spark.executor.resource.fpga.amount")) - assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/gpu," + + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/gpu.amount," + " please use spark.executor.resource.gpu.amount")) } @@ -450,7 +451,7 @@ class ClientSuite extends SparkFunSuite with Matchers { conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") } // also just set yarn one that we don't convert - conf.set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + yarnMadeupResource, "5") + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}", "5") val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala index 9e3cc6ec01dfd..f5ec531e26e0c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -22,9 +22,11 @@ import org.apache.hadoop.yarn.util.Records import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config.{DRIVER_CORES, DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_MEMORY} +import org.apache.spark.resource.ResourceUtils.AMOUNT class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { @@ -32,16 +34,18 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { private val CUSTOM_RES_2 = "custom-resource-type-2" private val MEMORY = "memory" private val CORES = "cores" - private val NEW_CONFIG_EXECUTOR_MEMORY = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_EXECUTOR_CORES = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_AM_MEMORY = YARN_AM_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_AM_CORES = YARN_AM_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_DRIVER_MEMORY = YARN_DRIVER_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_DRIVER_CORES = YARN_DRIVER_RESOURCE_TYPES_PREFIX + CORES + private val NEW_CONFIG_EXECUTOR_MEMORY = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_EXECUTOR_CORES = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_AM_MEMORY = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_AM_CORES = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_MEMORY = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_CORES = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" test("empty SparkConf should be valid") { val sparkConf = new SparkConf() - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) } test("just normal resources are defined") { @@ -50,7 +54,44 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { sparkConf.set(DRIVER_CORES.key, "4") sparkConf.set(EXECUTOR_MEMORY.key, "4G") sparkConf.set(EXECUTOR_CORES.key, "2") - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) + } + + test("get yarn resources from configs") { + val sparkConf = new SparkConf() + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "2G", + YARN_FPGA_RESOURCE_CONFIG -> "3G", "custom" -> "4") + resources.foreach { case (name, value) => + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_AM_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + } + var parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_DRIVER_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_AM_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + } + + test("get invalid yarn resources from configs") { + val sparkConf = new SparkConf() + + val missingAmountConfig = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}missingAmount" + // missing .amount + sparkConf.set(missingAmountConfig, "2g") + var thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Missing suffix for") + + sparkConf.remove(missingAmountConfig) + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}customResource.invalid", "2g") + + thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Unsupported suffix") } Seq( @@ -60,14 +101,14 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { ResourceInformation(CUSTOM_RES_2, 10, "G")) ).foreach { case (name, resources) => test(s"valid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val resourceDefs = resources.map { r => r.name } val requests = resources.map { r => (r.name, r.value.toString + r.unit) }.toMap ResourceRequestTestHelper.initializeResourceTypes(resourceDefs) val resource = createResource() - ResourceRequestHelper.setResourceRequests(requests, resource) + setResourceRequests(requests, resource) resources.foreach { r => val requested = ResourceRequestTestHelper.getResourceInformationByName(resource, r.name) @@ -82,12 +123,12 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { ("invalid unit", CUSTOM_RES_1, "123ppp") ).foreach { case (name, key, value) => test(s"invalid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) ResourceRequestTestHelper.initializeResourceTypes(Seq(key)) val resource = createResource() val thrown = intercept[IllegalArgumentException] { - ResourceRequestHelper.setResourceRequests(Map(key -> value), resource) + setResourceRequests(Map(key -> value), resource) } thrown.getMessage should include (key) } @@ -95,20 +136,20 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { Seq( NEW_CONFIG_EXECUTOR_MEMORY -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb" -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb" -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}memory-mb.$AMOUNT" -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}mb.$AMOUNT" -> "30G", NEW_CONFIG_EXECUTOR_CORES -> "5", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores" -> "5", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}vcores.$AMOUNT" -> "5", NEW_CONFIG_AM_MEMORY -> "1G", NEW_CONFIG_DRIVER_MEMORY -> "1G", NEW_CONFIG_AM_CORES -> "3", NEW_CONFIG_DRIVER_CORES -> "1G" ).foreach { case (key, value) => test(s"disallowed resource request: $key") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val conf = new SparkConf(false).set(key, value) val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(conf) + validateResources(conf) } thrown.getMessage should include (key) } @@ -126,7 +167,7 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { sparkConf.set(NEW_CONFIG_DRIVER_MEMORY, "2G") val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) } thrown.getMessage should ( include(NEW_CONFIG_EXECUTOR_MEMORY) and diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index ca89af26230f9..4ac27ede64831 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -31,9 +31,11 @@ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils.{AMOUNT, GPU} import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo @@ -160,12 +162,12 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("custom resource requested from yarn") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) val handler = createAllocator(1, mockAmClient, - Map(YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "gpu" -> "2G")) + Map(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${GPU}.${AMOUNT}" -> "2G")) handler.updateResourceRequests() val container = createContainer("host1", resource = handler.resource) @@ -174,7 +176,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter // get amount of memory and vcores from resource, so effectively skipping their validation val expectedResources = Resource.newInstance(handler.resource.getMemory(), handler.resource.getVirtualCores) - ResourceRequestHelper.setResourceRequests(Map("gpu" -> "2G"), expectedResources) + setResourceRequests(Map("gpu" -> "2G"), expectedResources) val captor = ArgumentCaptor.forClass(classOf[ContainerRequest]) verify(mockAmClient).addContainerRequest(captor.capture()) @@ -183,15 +185,16 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("custom spark resource mapped to yarn resource configs") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val yarnMadeupResource = "yarn.io/madeup" val yarnResources = Seq(YARN_GPU_RESOURCE_CONFIG, YARN_FPGA_RESOURCE_CONFIG, yarnMadeupResource) ResourceRequestTestHelper.initializeResourceTypes(yarnResources) val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) + val madeupConfigName = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}" val sparkResources = Map(EXECUTOR_GPU_ID.amountConf -> "3", EXECUTOR_FPGA_ID.amountConf -> "2", - s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}" -> "5") + madeupConfigName -> "5") val handler = createAllocator(1, mockAmClient, sparkResources) handler.updateResourceRequests() diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a1c11504a9036..d991e7cf7e898 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -680,8 +680,8 @@ primaryExpression | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct - | FIRST '(' expression (IGNORE NULLS)? ')' #first - | LAST '(' expression (IGNORE NULLS)? ')' #last + | (FIRST | FIRST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #first + | (LAST | LAST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #last | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position | constant #constantDefault | ASTERISK #star @@ -1023,6 +1023,7 @@ ansiNonReserved | REPAIR | REPLACE | RESET + | RESPECT | RESTRICT | REVOKE | RLIKE @@ -1184,6 +1185,7 @@ nonReserved | FIELDS | FILEFORMAT | FIRST + | FIRST_VALUE | FOLLOWING | FOR | FOREIGN @@ -1214,6 +1216,7 @@ nonReserved | ITEMS | KEYS | LAST + | LAST_VALUE | LATERAL | LAZY | LEADING @@ -1278,6 +1281,7 @@ nonReserved | REPAIR | REPLACE | RESET + | RESPECT | RESTRICT | REVOKE | RLIKE @@ -1435,6 +1439,7 @@ FETCH: 'FETCH'; FIELDS: 'FIELDS'; FILEFORMAT: 'FILEFORMAT'; FIRST: 'FIRST'; +FIRST_VALUE: 'FIRST_VALUE'; FOLLOWING: 'FOLLOWING'; FOR: 'FOR'; FOREIGN: 'FOREIGN'; @@ -1469,6 +1474,7 @@ ITEMS: 'ITEMS'; JOIN: 'JOIN'; KEYS: 'KEYS'; LAST: 'LAST'; +LAST_VALUE: 'LAST_VALUE'; LATERAL: 'LATERAL'; LAZY: 'LAZY'; LEADING: 'LEADING'; @@ -1536,6 +1542,7 @@ RENAME: 'RENAME'; REPAIR: 'REPAIR'; REPLACE: 'REPLACE'; RESET: 'RESET'; +RESPECT: 'RESPECT'; RESTRICT: 'RESTRICT'; REVOKE: 'REVOKE'; RIGHT: 'RIGHT'; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java index 9b87e676d9b2d..7eef6aea88120 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java @@ -227,6 +227,10 @@ public String property() { } } + interface ColumnChange extends TableChange { + String[] fieldNames(); + } + /** * A TableChange to add a field. *

@@ -234,7 +238,7 @@ public String property() { * If the new field is nested and its parent does not exist or is not a struct, the change must * result in an {@link IllegalArgumentException}. */ - final class AddColumn implements TableChange { + final class AddColumn implements ColumnChange { private final String[] fieldNames; private final DataType dataType; private final boolean isNullable; @@ -247,6 +251,7 @@ private AddColumn(String[] fieldNames, DataType dataType, boolean isNullable, St this.comment = comment; } + @Override public String[] fieldNames() { return fieldNames; } @@ -272,7 +277,7 @@ public String comment() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class RenameColumn implements TableChange { + final class RenameColumn implements ColumnChange { private final String[] fieldNames; private final String newName; @@ -281,6 +286,7 @@ private RenameColumn(String[] fieldNames, String newName) { this.newName = newName; } + @Override public String[] fieldNames() { return fieldNames; } @@ -297,7 +303,7 @@ public String newName() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class UpdateColumnType implements TableChange { + final class UpdateColumnType implements ColumnChange { private final String[] fieldNames; private final DataType newDataType; private final boolean isNullable; @@ -308,6 +314,7 @@ private UpdateColumnType(String[] fieldNames, DataType newDataType, boolean isNu this.isNullable = isNullable; } + @Override public String[] fieldNames() { return fieldNames; } @@ -328,7 +335,7 @@ public boolean isNullable() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class UpdateColumnComment implements TableChange { + final class UpdateColumnComment implements ColumnChange { private final String[] fieldNames; private final String newComment; @@ -337,6 +344,7 @@ private UpdateColumnComment(String[] fieldNames, String newComment) { this.newComment = newComment; } + @Override public String[] fieldNames() { return fieldNames; } @@ -351,13 +359,14 @@ public String newComment() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class DeleteColumn implements TableChange { + final class DeleteColumn implements ColumnChange { private final String[] fieldNames; private DeleteColumn(String[] fieldNames) { this.fieldNames = fieldNames; } + @Override public String[] fieldNames() { return fieldNames; } 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 6de1ef5660e55..7cc80c41a9013 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, 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.sources.v2.Table -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} object CatalogV2Util { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ @@ -132,16 +132,45 @@ object CatalogV2Util { val pos = struct.getFieldIndex(fieldNames.head) .getOrElse(throw new IllegalArgumentException(s"Cannot find field: ${fieldNames.head}")) val field = struct.fields(pos) - val replacement: Option[StructField] = if (fieldNames.tail.isEmpty) { - update(field) - } else { - field.dataType match { - case nestedStruct: StructType => - val updatedType: StructType = replace(nestedStruct, fieldNames.tail, update) - Some(StructField(field.name, updatedType, field.nullable, field.metadata)) - case _ => - throw new IllegalArgumentException(s"Not a struct: ${fieldNames.head}") - } + val replacement: Option[StructField] = (fieldNames.tail, field.dataType) match { + case (Seq(), _) => + update(field) + + case (names, struct: StructType) => + val updatedType: StructType = replace(struct, names, update) + Some(StructField(field.name, updatedType, field.nullable, field.metadata)) + + case (Seq("key"), map @ MapType(keyType, _, _)) => + val updated = update(StructField("key", keyType, nullable = false)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map key")) + Some(field.copy(dataType = map.copy(keyType = updated.dataType))) + + case (Seq("key", names @ _*), map @ MapType(keyStruct: StructType, _, _)) => + Some(field.copy(dataType = map.copy(keyType = replace(keyStruct, names, update)))) + + case (Seq("value"), map @ MapType(_, mapValueType, isNullable)) => + val updated = update(StructField("value", mapValueType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map value")) + Some(field.copy(dataType = map.copy( + valueType = updated.dataType, + valueContainsNull = updated.nullable))) + + case (Seq("value", names @ _*), map @ MapType(_, valueStruct: StructType, _)) => + Some(field.copy(dataType = map.copy(valueType = replace(valueStruct, names, update)))) + + case (Seq("element"), array @ ArrayType(elementType, isNullable)) => + val updated = update(StructField("element", elementType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete array element")) + Some(field.copy(dataType = array.copy( + elementType = updated.dataType, + containsNull = updated.nullable))) + + case (Seq("element", names @ _*), array @ ArrayType(elementStruct: StructType, _)) => + Some(field.copy(dataType = array.copy(elementType = replace(elementStruct, names, update)))) + + case (names, dataType) => + throw new IllegalArgumentException( + s"Cannot find field: ${names.head} in ${dataType.simpleString}") } val newFields = struct.fields.zipWithIndex.flatMap { 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 5464a7496d23d..5f7ee30cdab79 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,36 +17,91 @@ 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 /** * A trait to encapsulate catalog lookup function and helpful extractors. */ @Experimental -trait LookupCatalog { +trait LookupCatalog extends Logging { + + import LookupCatalog._ + protected def defaultCatalogName: Option[String] = None protected def lookupCatalog(name: String): CatalogPlugin - type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier) + /** + * Returns the default catalog. When set, this catalog is used for all identifiers that do not + * set a specific catalog. When this is None, the session catalog is responsible for the + * identifier. + * + * 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 + } + } /** - * Extract catalog plugin and identifier from a multi-part identifier. + * This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the + * session catalog is responsible for an identifier, but the source requires the v2 catalog API. + * This happens when the source implementation extends the v2 TableProvider API and is not listed + * in the fallback configuration, spark.sql.sources.write.useV1SourceList */ - object CatalogObjectIdentifier { - def unapply(parts: Seq[String]): Some[CatalogObjectIdentifier] = parts match { - case Seq(name) => - Some((None, Identifier.of(Array.empty, name))) + def sessionCatalog: Option[CatalogPlugin] = { + try { + Some(lookupCatalog(SESSION_CATALOG_NAME)) + } catch { + case NonFatal(e) => + logError("Cannot load v2 session catalog", e) + None + } + } + + /** + * Extract catalog plugin and remaining identifier names. + * + * This does not substitute the default catalog if no catalog is set in the identifier. + */ + private object CatalogAndIdentifier { + def unapply(parts: Seq[String]): Some[(Option[CatalogPlugin], Seq[String])] = parts match { + case Seq(_) => + Some((None, parts)) case Seq(catalogName, tail @ _*) => try { - Some((Some(lookupCatalog(catalogName)), Identifier.of(tail.init.toArray, tail.last))) + Some((Some(lookupCatalog(catalogName)), tail)) } catch { case _: CatalogNotFoundException => - Some((None, Identifier.of(parts.init.toArray, parts.last))) + Some((None, parts)) } } } + type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier) + + /** + * Extract catalog and identifier from a multi-part identifier with the default catalog if needed. + */ + object CatalogObjectIdentifier { + def unapply(parts: Seq[String]): Some[CatalogObjectIdentifier] = parts match { + case CatalogAndIdentifier(maybeCatalog, nameParts) => + Some(( + maybeCatalog.orElse(defaultCatalog), + Identifier.of(nameParts.init.toArray, nameParts.last) + )) + } + } + /** * Extract legacy table identifier from a multi-part identifier. * @@ -54,12 +109,12 @@ trait LookupCatalog { */ object AsTableIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogObjectIdentifier(None, ident) => - ident.namespace match { - case Array() => - Some(TableIdentifier(ident.name)) - case Array(database) => - Some(TableIdentifier(ident.name, Some(database))) + case CatalogAndIdentifier(None, names) if defaultCatalog.isEmpty => + names match { + case Seq(name) => + Some(TableIdentifier(name)) + case Seq(database, name) => + Some(TableIdentifier(name, Some(database))) case _ => None } @@ -67,4 +122,22 @@ trait LookupCatalog { None } } + + /** + * For temp views, extract a table identifier from a multi-part identifier if it has no catalog. + */ + object AsTemporaryViewIdentifier { + def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { + case CatalogAndIdentifier(None, Seq(table)) => + Some(TableIdentifier(table)) + case CatalogAndIdentifier(None, Seq(database, table)) => + Some(TableIdentifier(table, Some(database))) + case _ => + None + } + } +} + +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 5d37e909f80aa..e55cdfedd3234 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} +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -104,6 +105,8 @@ 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") @@ -163,6 +166,7 @@ class Analyzer( new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: + ResolveAlterTable :: ResolveTables :: ResolveRelations :: ResolveReferences :: @@ -209,38 +213,6 @@ class Analyzer( CleanupAliases) ) - /** - * Analyze cte definitions and substitute child plan with analyzed cte definitions. - */ - object CTESubstitution extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case With(child, relations) => - // substitute CTE expressions right-to-left to resolve references to previous CTEs: - // with a as (select * from t), b as (select * from a) select * from b - relations.foldRight(child) { - case ((cteName, ctePlan), currentPlan) => - substituteCTE(currentPlan, cteName, ctePlan) - } - case other => other - } - - private def substituteCTE( - plan: LogicalPlan, - cteName: String, - ctePlan: LogicalPlan): LogicalPlan = { - plan resolveOperatorsUp { - case UnresolvedRelation(Seq(table)) if resolver(cteName, table) => - ctePlan - case other => - // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. - other transformExpressions { - case e: SubqueryExpression => - e.withNewPlan(substituteCTE(e.plan, cteName, ctePlan)) - } - } - } - } - /** * Substitute child plan with WindowSpecDefinitions. */ @@ -667,6 +639,10 @@ class Analyzer( import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util._ def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) + 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) } @@ -704,6 +680,10 @@ class Analyzer( // Note this is compatible with the views defined by older versions of Spark(before 2.2), which // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { + case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) + if catalog.isTemporaryTable(ident) => + resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) + case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => val defaultDatabase = AnalysisContext.get.defaultDatabase val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase) @@ -777,6 +757,86 @@ class Analyzer( } } + /** + * Resolve ALTER TABLE statements that use a DSv2 catalog. + * + * This rule converts unresolved ALTER TABLE statements to v2 when a v2 catalog is responsible + * for the table identifier. A v2 catalog is responsible for an identifier when the identifier + * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and + * the table identifier does not include a catalog. + */ + object ResolveAlterTable extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case alter @ AlterTableAddColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map { col => + TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableAlterColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), colName, dataType, comment) => + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName.toArray, newDataType, true) + } + + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName.toArray, newComment) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + typeChange.toSeq ++ commentChange.toSeq) + + case alter @ AlterTableRenameColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), col, newName) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.renameColumn(col.toArray, newName))) + + case alter @ AlterTableDropColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableSetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), props) => + val changes = props.map { + case (key, value) => + TableChange.setProperty(key, value) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes.toSeq) + + case alter @ AlterTableUnsetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), keys, _) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + keys.map(key => TableChange.removeProperty(key))) + + case alter @ AlterTableSetLocationStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), newLoc) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.setProperty("location", newLoc))) + } + } + /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from * a logical plan node's children. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala new file mode 100644 index 0000000000000..60e6bf8db06d7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -0,0 +1,136 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, With} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_ENABLED + +/** + * Analyze WITH nodes and substitute child plan with CTE definitions. + */ +object CTESubstitution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)) { + legacyTraverseAndSubstituteCTE(plan) + } else { + traverseAndSubstituteCTE(plan, false) + } + } + + private def legacyTraverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child, relations) => + // substitute CTE expressions right-to-left to resolve references to previous CTEs: + // with a as (select * from t), b as (select * from a) select * from b + relations.foldRight(child) { + case ((cteName, ctePlan), currentPlan) => substituteCTE(currentPlan, cteName, ctePlan) + } + } + } + + /** + * Traverse the plan and expression nodes as a tree and replace matching references to CTE + * definitions. + * - If the rule encounters a WITH node then it substitutes the child of the node with CTE + * definitions of the node right-to-left order as a definition can reference to a previous + * one. + * For example the following query is valid: + * WITH + * t AS (SELECT 1), + * t2 AS (SELECT * FROM t) + * SELECT * FROM t2 + * - If a CTE definition contains an inner WITH node then substitution of inner should take + * precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH + * t AS (SELECT 1), + * t2 AS ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * SELECT * FROM t2 + * - If a CTE definition contains a subquery that contains an inner WITH node then substitution + * of inner should take precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1 AS c) + * SELECT max(c) FROM ( + * WITH t AS (SELECT 2 AS c) + * SELECT * FROM t + * ) + * - If a CTE definition contains a subquery expression that contains an inner WITH node then + * substitution of inner should take precedence because it can shadow an outer CTE + * definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1) + * SELECT ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * @param plan the plan to be traversed + * @param inTraverse whether the current traverse is called from another traverse, only in this + * case name collision can occur + * @return the plan where CTE substitution is applied + */ + private def traverseAndSubstituteCTE(plan: LogicalPlan, inTraverse: Boolean): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child: LogicalPlan, relations) => + // child might contain an inner CTE that has priority so traverse and substitute inner CTEs + // in child first + val traversedChild: LogicalPlan = child transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + + // Substitute CTE definitions from last to first as a CTE definition can reference a + // previous one + relations.foldRight(traversedChild) { + case ((cteName, ctePlan), currentPlan) => + // A CTE definition might contain an inner CTE that has priority, so traverse and + // substitute CTE defined in ctePlan. + // A CTE definition might not be used at all or might be used multiple times. To avoid + // computation if it is not used and to avoid multiple recomputation if it is used + // multiple times we use a lazy construct with call-by-name parameter passing. + lazy val substitutedCTEPlan = traverseAndSubstituteCTE(ctePlan, true) + substituteCTE(currentPlan, cteName, substitutedCTEPlan) + } + + // CTE name collision can occur only when inTraverse is true, it helps to avoid eager CTE + // substitution in a subquery expression. + case other if inTraverse => + other.transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + } + } + + private def substituteCTE( + plan: LogicalPlan, + cteName: String, + ctePlan: => LogicalPlan): LogicalPlan = + plan resolveOperatorsUp { + case UnresolvedRelation(Seq(table)) if plan.conf.resolver(cteName, table) => ctePlan + + case other => + // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. + other transformExpressions { + case e: SubqueryExpression => e.withNewPlan(substituteCTE(e.plan, cteName, ctePlan)) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 02031e758d83d..ae19d02e4475e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -353,6 +354,59 @@ trait CheckAnalysis extends PredicateHelper { case _ => } + case alter: AlterTable if alter.childrenResolved => + val table = alter.table + def findField(operation: String, fieldName: Array[String]): StructField = { + // include collections because structs nested in maps and arrays may be altered + val field = table.schema.findNestedField(fieldName, includeCollections = true) + if (field.isEmpty) { + throw new AnalysisException( + s"Cannot $operation missing field in ${table.name} schema: ${fieldName.quoted}") + } + field.get + } + + alter.changes.foreach { + case add: AddColumn => + val parent = add.fieldNames.init + if (parent.nonEmpty) { + findField("add to", parent) + } + case update: UpdateColumnType => + val field = findField("update", update.fieldNames) + val fieldName = update.fieldNames.quoted + update.newDataType match { + case _: StructType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a struct by adding, deleting, or updating its fields") + case _: MapType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a map by updating $fieldName.key or $fieldName.value") + case _: ArrayType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update the element by updating $fieldName.element") + case _: AtomicType => + // update is okay + } + if (!Cast.canUpCast(field.dataType, update.newDataType)) { + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName: " + + s"${field.dataType.simpleString} cannot be cast to " + + s"${update.newDataType.simpleString}") + } + case rename: RenameColumn => + findField("rename", rename.fieldNames) + case update: UpdateColumnComment => + findField("update", update.fieldNames) + case delete: DeleteColumn => + findField("delete", delete.fieldNames) + case _ => + // no validation needed for set and remove property + } + case _ => // Fallbacks to the following checks } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 9fe95671cda08..c72400a8b72c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -222,9 +222,12 @@ object FunctionRegistry { // math functions expression[Acos]("acos"), + expression[Acosh]("acosh"), expression[Asin]("asin"), + expression[Asinh]("asinh"), expression[Atan]("atan"), expression[Atan2]("atan2"), + expression[Atanh]("atanh"), expression[Bin]("bin"), expression[BRound]("bround"), expression[Cbrt]("cbrt"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 1fdec89e258a7..3125f8cb732db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -59,7 +59,7 @@ object TypeCoercion { CaseWhenCoercion :: IfCoercion :: StackCoercion :: - Division :: + Division(conf) :: ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: @@ -666,7 +666,7 @@ object TypeCoercion { * Hive only performs integral division with the DIV operator. The arguments to / are always * converted to fractional types. */ - object Division extends TypeCoercionRule { + case class Division(conf: SQLConf) extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, @@ -677,7 +677,12 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + (left.dataType, right.dataType) match { + case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => + IntegralDivide(left, right) + case _ => + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + } } private def isNumericOrNull(ex: Expression): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala index 2210e180bc75f..3eae34da7e502 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala @@ -37,7 +37,7 @@ object UpdateAttributeNullability extends Rule[LogicalPlan] { case p if !p.resolved => p // Skip leaf node, as it has no child and no need to update nullability. case p: LeafNode => p - case p: LogicalPlan => + case p: LogicalPlan if p.childrenResolved => val nullabilities = p.children.flatMap(c => c.output).groupBy(_.exprId).map { // If there are multiple Attributes having the same ExprId, we need to resolve // the conflict of nullable field. We do not really expect this to happen. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b700c336e6ae8..9e0e0d528a968 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -40,12 +40,15 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str * * @param multipartIdentifier table name */ -case class UnresolvedRelation(multipartIdentifier: Seq[String]) extends LeafNode { +case class UnresolvedRelation( + multipartIdentifier: Seq[String]) extends LeafNode with NamedRelation { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ /** Returns a `.` separated name for this relation. */ def tableName: String = multipartIdentifier.quoted + override def name: String = tableName + override def output: Seq[Attribute] = Nil override lazy val resolved = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 8477e63135e30..5314821ea3a59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -16,7 +16,8 @@ */ package org.apache.spark.sql.catalyst.expressions -import java.util.{Comparator, TimeZone} +import java.time.ZoneId +import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag @@ -2459,10 +2460,10 @@ case class Sequence( new IntegralSequenceImpl(iType)(ct, iType.integral) case TimestampType => - new TemporalSequenceImpl[Long](LongType, 1, identity, timeZone) + new TemporalSequenceImpl[Long](LongType, 1, identity, zoneId) case DateType => - new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, timeZone) + new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, zoneId) } override def eval(input: InternalRow): Any = { @@ -2603,7 +2604,7 @@ object Sequence { } private class TemporalSequenceImpl[T: ClassTag] - (dt: IntegralType, scale: Long, fromLong: Long => T, timeZone: TimeZone) + (dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId) (implicit num: Integral[T]) extends SequenceImpl { override val defaultStep: DefaultStep = new DefaultStep( @@ -2641,8 +2642,8 @@ object Sequence { while (t < exclusiveItem ^ stepSign < 0) { arr(i) = fromLong(t / scale) - t = timestampAddInterval(t, stepMonths, stepMicros, timeZone) i += 1 + t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, zoneId) } // truncate array to the correct length @@ -2668,7 +2669,7 @@ object Sequence { val exclusiveItem = ctx.freshName("exclusiveItem") val t = ctx.freshName("t") val i = ctx.freshName("i") - val genTimeZone = ctx.addReferenceObj("timeZone", timeZone, classOf[TimeZone].getName) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val sequenceLengthCode = s""" @@ -2676,12 +2677,6 @@ object Sequence { |${genSequenceLengthCode(ctx, startMicros, stopMicros, intervalInMicros, arrLength)} """.stripMargin - val timestampAddIntervalCode = - s""" - |$t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( - | $t, $stepMonths, $stepMicros, $genTimeZone); - """.stripMargin - s""" |final int $stepMonths = $step.months; |final long $stepMicros = $step.microseconds; @@ -2705,8 +2700,9 @@ object Sequence { | | while ($t < $exclusiveItem ^ $stepSign < 0) { | $arr[$i] = ($elemType) ($t / ${scale}L); - | $timestampAddIntervalCode | $i += 1; + | $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( + | $startMicros, $i * $stepMonths, $i * $stepMicros, $zid); | } | | if ($arr.length > $i) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala index 2917b0b8c9c53..5bfae7b77e096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala @@ -21,15 +21,21 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, FalseLiteral} import org.apache.spark.sql.types.DataType -case class KnownNotNull(child: Expression) extends UnaryExpression { - override def nullable: Boolean = false +trait TaggingExpression extends UnaryExpression { + override def nullable: Boolean = child.nullable override def dataType: DataType = child.dataType + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.genCode(ctx) + + override def eval(input: InternalRow): Any = child.eval(input) +} + +case class KnownNotNull(child: Expression) extends TaggingExpression { + override def nullable: Boolean = false + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { child.genCode(ctx).copy(isNull = FalseLiteral) } - - override def eval(input: InternalRow): Any = { - child.eval(input) - } } + +case class KnownFloatingPointNormalized(child: Expression) extends TaggingExpression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index ccf6b36effa08..53329fd10a9c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -996,14 +996,14 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], itvl.months, itvl.microseconds, timeZone) + start.asInstanceOf[Long], itvl.months, itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $zid)""" }) } } @@ -1111,14 +1111,14 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, timeZone) + start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $zid)""" }) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index bdeb9ed29e0ac..e873f8ed1a21c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -287,6 +287,29 @@ case class Cos(child: Expression) extends UnaryMathExpression(math.cos, "COS") """) case class Cosh(child: Expression) extends UnaryMathExpression(math.cosh, "COSH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic cosine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(1); + 0.0 + > SELECT _FUNC_(0); + NaN + """, + since = "3.0.0") +case class Acosh(child: Expression) + extends UnaryMathExpression((x: Double) => math.log(x + math.sqrt(x * x - 1.0)), "ACOSH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c - 1.0))") + } +} + /** * Convert a num from one base to another * @@ -557,6 +580,31 @@ case class Sin(child: Expression) extends UnaryMathExpression(math.sin, "SIN") """) case class Sinh(child: Expression) extends UnaryMathExpression(math.sinh, "SINH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic sine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """, + since = "3.0.0") +case class Asinh(child: Expression) + extends UnaryMathExpression((x: Double) => x match { + case Double.NegativeInfinity => Double.NegativeInfinity + case _ => math.log(x + math.sqrt(x * x + 1.0)) }, "ASINH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => + s"$c == Double.NEGATIVE_INFINITY ? Double.NEGATIVE_INFINITY : " + + s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c + 1.0))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Returns the square root of `expr`.", examples = """ @@ -617,6 +665,29 @@ case class Cot(child: Expression) """) case class Tanh(child: Expression) extends UnaryMathExpression(math.tanh, "TANH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic tangent of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + > SELECT _FUNC_(2); + NaN + """, + since = "3.0.0") +case class Atanh(child: Expression) + extends UnaryMathExpression((x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x)), "ATANH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"0.5 * java.lang.Math.log((1.0 + $c)/(1.0 - $c))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Converts radians to degrees.", arguments = """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 06985ac85b70e..02d5a1f27aa7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -116,6 +116,10 @@ trait PredicateHelper { // non-correlated subquery will be replaced as literal e.children.isEmpty case a: AttributeReference => true + // PythonUDF will be executed by dedicated physical operator later. + // For PythonUDFs that can't be evaluated in join condition, `PullOutPythonUDFInJoinCondition` + // will pull them out later. + case _: PythonUDF => true case e: Unevaluable => false case e => e.children.forall(canEvaluateWithinJoin) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index a5921ebe7751a..b036092cf1fcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, LambdaFunction, NamedLambdaVariable, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, KnownFloatingPointNormalized, LambdaFunction, NamedLambdaVariable, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, Window} @@ -61,7 +61,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case _: Subquery => plan case _ => plan transform { - case w: Window if w.partitionSpec.exists(p => needNormalize(p.dataType)) => + case w: Window if w.partitionSpec.exists(p => needNormalize(p)) => // Although the `windowExpressions` may refer to `partitionSpec` expressions, we don't need // to normalize the `windowExpressions`, as they are executed per input row and should take // the input row as it is. @@ -73,7 +73,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, condition, _, _, _) // The analyzer guarantees left and right joins keys are of the same data type. Here we // only need to check join keys of one side. - if leftKeys.exists(k => needNormalize(k.dataType)) => + if leftKeys.exists(k => needNormalize(k)) => val newLeftJoinKeys = leftKeys.map(normalize) val newRightJoinKeys = rightKeys.map(normalize) val newConditions = newLeftJoinKeys.zip(newRightJoinKeys).map { @@ -87,6 +87,14 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { } } + /** + * Short circuit if the underlying expression is already normalized + */ + private def needNormalize(expr: Expression): Boolean = expr match { + case KnownFloatingPointNormalized(_) => false + case _ => needNormalize(expr.dataType) + } + private def needNormalize(dt: DataType): Boolean = dt match { case FloatType | DoubleType => true case StructType(fields) => fields.exists(f => needNormalize(f.dataType)) @@ -98,7 +106,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { } private[sql] def normalize(expr: Expression): Expression = expr match { - case _ if !needNormalize(expr.dataType) => expr + case _ if !needNormalize(expr) => expr case a: Alias => a.withNewChildren(Seq(normalize(a.child))) @@ -116,7 +124,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { CreateMap(children.map(normalize)) case _ if expr.dataType == FloatType || expr.dataType == DoubleType => - NormalizeNaNAndZero(expr) + KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) case _ if expr.dataType.isInstanceOf[StructType] => val fields = expr.dataType.asInstanceOf[StructType].fields.indices.map { i => @@ -128,7 +136,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { val ArrayType(et, containsNull) = expr.dataType val lv = NamedLambdaVariable("arg", et, containsNull) val function = normalize(lv) - ArrayTransform(expr, LambdaFunction(function, Seq(lv))) + KnownFloatingPointNormalized(ArrayTransform(expr, LambdaFunction(function, Seq(lv)))) case _ => throw new IllegalStateException(s"fail to normalize $expr") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6c5ad55e88bea..d9f8b9a7203ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1851,7 +1851,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[CalendarInterval]] for a unit value pair. Two unit configuration types are * supported: * - Single unit. - * - From-To unit (only 'YEAR TO MONTH' and 'DAY TO SECOND' and 'HOUR to SECOND' are supported). + * - From-To unit ('YEAR TO MONTH', 'DAY TO HOUR', 'DAY TO MINUTE', 'DAY TO SECOND', + * 'HOUR TO MINUTE', 'HOUR TO SECOND' and 'MINUTE TO SECOND' are supported). */ override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { import ctx._ @@ -1866,10 +1867,18 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CalendarInterval.fromSingleUnitString(u, s) case ("year", Some("month")) => CalendarInterval.fromYearMonthString(s) + case ("day", Some("hour")) => + CalendarInterval.fromDayTimeString(s, "day", "hour") + case ("day", Some("minute")) => + CalendarInterval.fromDayTimeString(s, "day", "minute") case ("day", Some("second")) => - CalendarInterval.fromDayTimeString(s) + CalendarInterval.fromDayTimeString(s, "day", "second") + case ("hour", Some("minute")) => + CalendarInterval.fromDayTimeString(s, "hour", "minute") case ("hour", Some("second")) => - CalendarInterval.fromDayTimeString(s) + CalendarInterval.fromDayTimeString(s, "hour", "second") + case ("minute", Some("second")) => + CalendarInterval.fromDayTimeString(s, "minute", "second") case (from, Some(t)) => throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index a816922f49aee..51d2a73ea97b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -118,19 +118,23 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { // Replace null with default value for joining key, then those rows with null in it could // be joined together case EqualNullSafe(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => - Some((Coalesce(Seq(l, Literal.default(l.dataType))), - Coalesce(Seq(r, Literal.default(r.dataType))))) + Seq((Coalesce(Seq(l, Literal.default(l.dataType))), + Coalesce(Seq(r, Literal.default(r.dataType)))), + (IsNull(l), IsNull(r)) + ) case EqualNullSafe(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => - Some((Coalesce(Seq(r, Literal.default(r.dataType))), - Coalesce(Seq(l, Literal.default(l.dataType))))) + Seq((Coalesce(Seq(r, Literal.default(r.dataType))), + Coalesce(Seq(l, Literal.default(l.dataType)))), + (IsNull(r), IsNull(l)) + ) case other => None } val otherPredicates = predicates.filterNot { case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => false - case EqualTo(l, r) => + case Equality(l, r) => canEvaluate(l, left) && canEvaluate(r, right) || canEvaluate(l, right) && canEvaluate(r, left) - case other => false + case _ => false } if (joinKeys.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 6fd446017b149..2cb04c9ec70c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} @@ -432,7 +433,7 @@ case class CreateTableAsSelect( override def children: Seq[LogicalPlan] = Seq(query) - override lazy val resolved: Boolean = { + override lazy val resolved: Boolean = childrenResolved && { // the table schema is created from the query schema, so the only resolution needed is to check // that the columns referenced by the table's partitioning exist in the query schema val references = partitioning.flatMap(_.references).toSet @@ -507,6 +508,40 @@ case class DropTable( ident: Identifier, ifExists: Boolean) extends Command +/** + * Alter a table. + */ +case class AlterTable( + catalog: TableCatalog, + ident: Identifier, + table: NamedRelation, + changes: Seq[TableChange]) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = childrenResolved && { + changes.forall { + case add: AddColumn => + add.fieldNames match { + case Array(_) => + // a top-level field can always be added + true + case _ => + // the parent field must exist + table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined + } + + case colChange: ColumnChange => + // the column that will be changed must exist + table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined + + case _ => + // property changes require no resolution checks + true + } + } +} + /** * Insert some data into a table. Note that this plan is unresolved and has to be replaced by the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 50fa6fb87e74f..10a7f9bd550e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.time._ -import java.time.Year.isLeap -import java.time.temporal.IsoFields +import java.time.temporal.{ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -45,17 +44,19 @@ object DateTimeUtils { // it's 2440587.5, rounding up to compatible with Hive final val JULIAN_DAY_OF_EPOCH = 2440588 - final val NANOS_PER_MICROS = MICROSECONDS.toNanos(1) - final val NANOS_PER_MILLIS = MILLISECONDS.toNanos(1) - final val NANOS_PER_SECOND = SECONDS.toNanos(1) - final val MICROS_PER_MILLIS = MILLISECONDS.toMicros(1) - final val MICROS_PER_SECOND = SECONDS.toMicros(1) - final val MICROS_PER_DAY = DAYS.toMicros(1) - final val MILLIS_PER_SECOND = SECONDS.toMillis(1) - final val MILLIS_PER_MINUTE = MINUTES.toMillis(1) - final val MILLIS_PER_HOUR = HOURS.toMillis(1) - final val MILLIS_PER_DAY = DAYS.toMillis(1) - final val SECONDS_PER_DAY = DAYS.toSeconds(1) + // Pre-calculated values can provide an opportunity of additional optimizations + // to the compiler like constants propagation and folding. + final val NANOS_PER_MICROS: Long = 1000 + final val MICROS_PER_MILLIS: Long = 1000 + final val MILLIS_PER_SECOND: Long = 1000 + final val SECONDS_PER_DAY: Long = 24 * 60 * 60 + final val MICROS_PER_SECOND: Long = MILLIS_PER_SECOND * MICROS_PER_MILLIS + final val NANOS_PER_MILLIS: Long = NANOS_PER_MICROS * MICROS_PER_MILLIS + final val NANOS_PER_SECOND: Long = NANOS_PER_MICROS * MICROS_PER_SECOND + final val MICROS_PER_DAY: Long = SECONDS_PER_DAY * MICROS_PER_SECOND + final val MILLIS_PER_MINUTE: Long = 60 * MILLIS_PER_SECOND + final val MILLIS_PER_HOUR: Long = 60 * MILLIS_PER_MINUTE + final val MILLIS_PER_DAY: Long = SECONDS_PER_DAY * MILLIS_PER_SECOND // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 @@ -406,6 +407,10 @@ object DateTimeUtils { // year should have exact four digits return None } + if (i < 2 && j < bytes.length) { + // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed. + return None + } segments(i) = currentSegmentValue try { val localDate = LocalDate.of(segments(0), segments(1), segments(2)) @@ -499,60 +504,12 @@ object DateTimeUtils { LocalDate.ofEpochDay(date).getDayOfMonth } - /** - * The number of days for each month (not leap year) - */ - private val monthDays = Array(31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31) - - /** - * Returns the date value for the first day of the given month. - * The month is expressed in months since year zero (17999 BC), starting from 0. - */ - private def firstDayOfMonth(absoluteMonth: Int): SQLDate = { - val absoluteYear = absoluteMonth / 12 - var monthInYear = absoluteMonth - absoluteYear * 12 - var date = getDateFromYear(absoluteYear) - if (monthInYear >= 2 && isLeap(absoluteYear + YearZero)) { - date += 1 - } - while (monthInYear > 0) { - date += monthDays(monthInYear - 1) - monthInYear -= 1 - } - date - } - - /** - * Returns the date value for January 1 of the given year. - * The year is expressed in years since year zero (17999 BC), starting from 0. - */ - private def getDateFromYear(absoluteYear: Int): SQLDate = { - val absoluteDays = (absoluteYear * 365 + absoluteYear / 400 - absoluteYear / 100 - + absoluteYear / 4) - absoluteDays - toYearZero - } - /** * Add date and year-month interval. * Returns a date value, expressed in days since 1.1.1970. */ def dateAddMonths(days: SQLDate, months: Int): SQLDate = { - val (year, monthInYear, dayOfMonth, daysToMonthEnd) = splitDate(days) - val absoluteMonth = (year - YearZero) * 12 + monthInYear - 1 + months - val nonNegativeMonth = if (absoluteMonth >= 0) absoluteMonth else 0 - val currentMonthInYear = nonNegativeMonth % 12 - val currentYear = nonNegativeMonth / 12 - - val leapDay = if (currentMonthInYear == 1 && isLeap(currentYear + YearZero)) 1 else 0 - val lastDayOfMonth = monthDays(currentMonthInYear) + leapDay - - val currentDayInMonth = if (daysToMonthEnd == 0 || dayOfMonth >= lastDayOfMonth) { - // last day of the month - lastDayOfMonth - } else { - dayOfMonth - } - firstDayOfMonth(nonNegativeMonth) + currentDayInMonth - 1 + LocalDate.ofEpochDay(days).plusMonths(months).toEpochDay.toInt } /** @@ -563,12 +520,12 @@ object DateTimeUtils { start: SQLTimestamp, months: Int, microseconds: Long, - timeZone: TimeZone): SQLTimestamp = { - val days = millisToDays(MICROSECONDS.toMillis(start), timeZone) - val newDays = dateAddMonths(days, months) - start + - MILLISECONDS.toMicros(daysToMillis(newDays, timeZone) - daysToMillis(days, timeZone)) + - microseconds + zoneId: ZoneId): SQLTimestamp = { + val resultTimestamp = microsToInstant(start) + .atZone(zoneId) + .plusMonths(months) + .plus(microseconds, ChronoUnit.MICROS) + instantToMicros(resultTimestamp.toInstant) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index af67632706dfd..57f5128fd4fbe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -234,6 +234,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED = + buildConf("spark.sql.inMemoryTableScanStatistics.enable") + .internal() + .doc("When true, enable in-memory table scan accumulators.") + .booleanConf + .createWithDefault(false) + val CACHE_VECTORIZED_READER_ENABLED = buildConf("spark.sql.inMemoryColumnarStorage.enableVectorizedReader") .doc("Enables vectorized reader for columnar caching.") @@ -300,12 +307,6 @@ object SQLConf { .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024 * 1024) - val RUNTIME_REOPTIMIZATION_ENABLED = - buildConf("spark.sql.runtime.reoptimization.enabled") - .doc("When true, enable runtime query re-optimization.") - .booleanConf - .createWithDefault(false) - val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") .doc("When true, enable adaptive query execution.") .booleanConf @@ -1030,6 +1031,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ENABLE_VECTORIZED_HASH_MAP = + buildConf("spark.sql.codegen.aggregate.map.vectorized.enable") + .internal() + .doc("Enable vectorized aggregate hash map. This is for testing/benchmarking only.") + .booleanConf + .createWithDefault(false) + val MAX_NESTED_VIEW_DEPTH = buildConf("spark.sql.view.maxNestedViewDepth") .internal() @@ -1516,6 +1524,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") + .doc("When true, will perform integral division with the / operator " + + "if both sides are integral types.") + .booleanConf + .createWithDefault(false) + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() @@ -1833,11 +1847,22 @@ object SQLConf { .stringConf .createOptional + val V2_SESSION_CATALOG = buildConf("spark.sql.catalog.session") + .doc("Name of the default v2 catalog, used when a catalog is not identified in queries") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog") + val LEGACY_LOOSE_UPCAST = buildConf("spark.sql.legacy.looseUpcast") .doc("When true, the upcast will be loose and allows string to atomic types.") .booleanConf .createWithDefault(false) + val LEGACY_CTE_PRECEDENCE_ENABLED = buildConf("spark.sql.legacy.ctePrecedence.enabled") + .internal() + .doc("When true, outer CTE definitions takes precedence over inner definitions.") + .booleanConf + .createWithDefault(false) + val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic") .doc("When true, the ArrayExists will follow the three-valued boolean logic.") @@ -1948,10 +1973,7 @@ class SQLConf extends Serializable with Logging { def targetPostShuffleInputSize: Long = getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) - def runtimeReoptimizationEnabled: Boolean = getConf(RUNTIME_REOPTIMIZATION_ENABLED) - - def adaptiveExecutionEnabled: Boolean = - getConf(ADAPTIVE_EXECUTION_ENABLED) && !getConf(RUNTIME_REOPTIMIZATION_ENABLED) + def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) def reducePostShufflePartitionsEnabled: Boolean = getConf(REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED) @@ -2113,6 +2135,8 @@ class SQLConf extends Serializable with Logging { def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING) + def inMemoryTableScanStatisticsEnabled: Boolean = getConf(IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED) + def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED) def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD) @@ -2152,6 +2176,8 @@ class SQLConf extends Serializable with Logging { def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP) + def enableVectorizedHashMap: Boolean = getConf(ENABLE_VECTORIZED_HASH_MAP) + def useObjectHashAggregation: Boolean = getConf(USE_OBJECT_HASH_AGG) def objectAggSortBasedFallbackThreshold: Int = getConf(OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD) @@ -2274,6 +2300,8 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) + def preferIntegralDivision: Boolean = getConf(PREFER_INTEGRAL_DIVISION) + def allowCreatingManagedTableUsingNonemptyLocation: Boolean = getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index edf8d2c1b31a3..236f73ba3832c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -310,20 +310,46 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Returns a field in this struct and its child structs. * - * This does not support finding fields nested in maps or arrays. + * If includeCollections is true, this will return fields that are nested in maps and arrays. */ - private[sql] def findNestedField(fieldNames: Seq[String]): Option[StructField] = { + private[sql] def findNestedField( + fieldNames: Seq[String], + includeCollections: Boolean = false): Option[StructField] = { fieldNames.headOption.flatMap(nameToField.get) match { case Some(field) => - if (fieldNames.tail.isEmpty) { - Some(field) - } else { - field.dataType match { - case struct: StructType => - struct.findNestedField(fieldNames.tail) - case _ => - None - } + (fieldNames.tail, field.dataType, includeCollections) match { + case (Seq(), _, _) => + Some(field) + + case (names, struct: StructType, _) => + struct.findNestedField(names, includeCollections) + + case (_, _, false) => + None // types nested in maps and arrays are not used + + case (Seq("key"), MapType(keyType, _, _), true) => + // return the key type as a struct field to include nullability + Some(StructField("key", keyType, nullable = false)) + + case (Seq("key", names @ _*), MapType(struct: StructType, _, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("value"), MapType(_, valueType, isNullable), true) => + // return the value type as a struct field to include nullability + Some(StructField("value", valueType, nullable = isNullable)) + + case (Seq("value", names @ _*), MapType(_, struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("element"), ArrayType(elementType, isNullable), true) => + // return the element type as a struct field to include nullability + Some(StructField("element", elementType, nullable = isNullable)) + + case (Seq("element", names @ _*), ArrayType(struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case _ => + None } case _ => None diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 2c3ba1b0daf41..949bb30d15503 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1126,14 +1126,14 @@ class TypeCoercionSuite extends AnalysisTest { Concat(Seq(Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.concatBinaryAsString" -> "true") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "true") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.concatBinaryAsString" -> "false") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "false") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Literal("123".getBytes), Literal("456".getBytes)))) @@ -1180,14 +1180,14 @@ class TypeCoercionSuite extends AnalysisTest { Elt(Seq(Literal(2), Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.eltOutputAsString" -> "true") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "true") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.eltOutputAsString" -> "false") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "false") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes)))) @@ -1456,7 +1456,7 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + "in aggregation function like sum") { - val rules = Seq(FunctionArgumentConversion, Division) + val rules = Seq(FunctionArgumentConversion, Division(conf)) // Casts Integer to Double ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will @@ -1475,12 +1475,35 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) + val rules = Seq(FunctionArgumentConversion, Division(conf), ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } + test("SPARK-28395 Division operator support integral division") { + val rules = Seq(FunctionArgumentConversion, Division(conf)) + Seq(true, false).foreach { preferIntegralDivision => + withSQLConf(SQLConf.PREFER_INTEGRAL_DIVISION.key -> s"$preferIntegralDivision") { + val result1 = if (preferIntegralDivision) { + IntegralDivide(1L, 1L) + } else { + Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) + } + ruleTest(rules, Divide(1L, 1L), result1) + val result2 = if (preferIntegralDivision) { + IntegralDivide(1, Cast(1, ShortType)) + } else { + Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) + } + ruleTest(rules, Divide(1, Cast(1, ShortType)), result2) + + ruleTest(rules, Divide(1L, 1D), Divide(Cast(1L, DoubleType), Cast(1D, DoubleType))) + ruleTest(rules, Divide(Decimal(1.1), 1L), Divide(Decimal(1.1), 1L)) + } + } + } + test("binary comparison with string promotion") { val rule = TypeCoercion.PromoteStrings(conf) ruleTest(rule, @@ -1498,7 +1521,7 @@ class TypeCoercionSuite extends AnalysisTest { DoubleType))) Seq(true, false).foreach { convertToTS => withSQLConf( - "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) { + SQLConf.COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP.key -> convertToTS.toString) { val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) 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 783751ff79865..52543d16d4815 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 @@ -85,4 +85,111 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { } } } + + test("temporary table identifier") { + Seq( + ("tbl", TableIdentifier("tbl")), + ("db.tbl", TableIdentifier("tbl", Some("db"))), + ("`db.tbl`", TableIdentifier("db.tbl")), + ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { + case (sqlIdent: String, expectedTableIdent: TableIdentifier) => + // when there is no catalog and the namespace has one part, the rule should match + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(ident) => + ident shouldEqual expectedTableIdent + } + } + + Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") + .foreach { sqlIdent => + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(_) => + fail("AsTemporaryViewIdentifier should not match when " + + "the catalog is set or the namespace has multiple parts") + case _ => + // expected + } + } + } +} + +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") + + override def lookupCatalog(name: String): CatalogPlugin = + catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + + test("catalog object identifier") { + Seq( + ("tbl", catalogs.get("prod"), Seq.empty, "tbl"), + ("db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), + ("prod.func", catalogs.get("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", catalogs.get("prod"), Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs.get("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs.get("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", catalogs.get("prod"), Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", catalogs.get("prod"), Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", catalogs.get("prod"), + Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { + case (sql, expectedCatalog, namespace, name) => + inside(parseMultipartIdentifier(sql)) { + case CatalogObjectIdentifier(catalog, ident) => + catalog shouldEqual expectedCatalog + ident shouldEqual Identifier.of(namespace.toArray, name) + } + } + } + + test("table identifier") { + Seq( + "tbl", + "db.tbl", + "`db.tbl`", + "parquet.`file:/tmp/db.tbl`", + "`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + "prod.func", + "prod.db.tbl", + "ns1.ns2.tbl").foreach { sql => + parseMultipartIdentifier(sql) match { + case AsTableIdentifier(_) => + fail(s"$sql should not be resolved as TableIdentifier") + case _ => + } + } + } + + test("temporary table identifier") { + Seq( + ("tbl", TableIdentifier("tbl")), + ("db.tbl", TableIdentifier("tbl", Some("db"))), + ("`db.tbl`", TableIdentifier("db.tbl")), + ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { + case (sqlIdent: String, expectedTableIdent: TableIdentifier) => + // when there is no catalog and the namespace has one part, the rule should match + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(ident) => + ident shouldEqual expectedTableIdent + } + } + + Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") + .foreach { sqlIdent => + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(_) => + fail("AsTemporaryViewIdentifier should not match when " + + "the catalog is set or the namespace has multiple parts") + case _ => + // expected + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 04bb61a7486e3..4e8322d3c55d7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -462,13 +462,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(AddMonths(Literal.create(null, DateType), Literal(1)), null) checkEvaluation(AddMonths(Literal.create(null, DateType), Literal.create(null, IntegerType)), null) + // Valid range of DateType is [0001-01-01, 9999-12-31] + val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(Date.valueOf("2015-01-30")), Literal(Int.MinValue)), -7293498) + AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261) checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), positiveIntLit), 1014213) - checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), negativeIntLit), -980528) - checkConsistencyBetweenInterpretedAndCodegen(AddMonths, DateType, IntegerType) + AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) + // Test evaluation results between Interpreted mode and Codegen mode + forAll ( + LiteralGenerator.randomGen(DateType), + LiteralGenerator.monthIntervalLiterGen + ) { (l1: Literal, l2: Literal) => + cmpInterpretWithCodegen(EmptyRow, AddMonths(l1, l2)) + } } test("months_between") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 1c91adab71375..a2c0ce35df23c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -398,7 +398,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa } } - private def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { + def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { val interpret = try { evaluateWithoutCodegen(expr, inputRow) } catch { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index be5fdb5b42ea3..b111797c3588e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import java.time.{Duration, Instant, LocalDate} +import java.util.concurrent.TimeUnit import org.scalacheck.{Arbitrary, Gen} @@ -100,23 +102,44 @@ object LiteralGenerator { lazy val booleanLiteralGen: Gen[Literal] = for { b <- Arbitrary.arbBool.arbitrary } yield Literal.create(b, BooleanType) - lazy val dateLiteralGen: Gen[Literal] = - for { d <- Arbitrary.arbInt.arbitrary } yield Literal.create(new Date(d), DateType) + lazy val dateLiteralGen: Gen[Literal] = { + // Valid range for DateType is [0001-01-01, 9999-12-31] + val minDay = LocalDate.of(1, 1, 1).toEpochDay + val maxDay = LocalDate.of(9999, 12, 31).toEpochDay + for { day <- Gen.choose(minDay, maxDay) } + yield Literal.create(new Date(day * DateTimeUtils.MILLIS_PER_DAY), DateType) + } lazy val timestampLiteralGen: Gen[Literal] = { // Catalyst's Timestamp type stores number of microseconds since epoch in // a variable of Long type. To prevent arithmetic overflow of Long on // conversion from milliseconds to microseconds, the range of random milliseconds // since epoch is restricted here. - val maxMillis = Long.MaxValue / DateTimeUtils.MICROS_PER_MILLIS - val minMillis = Long.MinValue / DateTimeUtils.MICROS_PER_MILLIS + // Valid range for TimestampType is [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] + val minMillis = Instant.parse("0001-01-01T00:00:00.000000Z").toEpochMilli + val maxMillis = Instant.parse("9999-12-31T23:59:59.999999Z").toEpochMilli for { millis <- Gen.choose(minMillis, maxMillis) } yield Literal.create(new Timestamp(millis), TimestampType) } - lazy val calendarIntervalLiterGen: Gen[Literal] = - for { m <- Arbitrary.arbInt.arbitrary; s <- Arbitrary.arbLong.arbitrary} - yield Literal.create(new CalendarInterval(m, s), CalendarIntervalType) + // Valid range for DateType and TimestampType is [0001-01-01, 9999-12-31] + private val maxIntervalInMonths: Int = 10000 * 12 + + lazy val monthIntervalLiterGen: Gen[Literal] = { + for { months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) } + yield Literal.create(months, IntegerType) + } + + lazy val calendarIntervalLiterGen: Gen[Literal] = { + val maxDurationInSec = Duration.between( + Instant.parse("0001-01-01T00:00:00.000000Z"), + Instant.parse("9999-12-31T23:59:59.999999Z")).getSeconds + val maxMicros = TimeUnit.SECONDS.toMicros(maxDurationInSec) + for { + months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) + micros <- Gen.choose(-1 * maxMicros, maxMicros) + } yield Literal.create(new CalendarInterval(months, micros), CalendarIntervalType) + } // Sometimes, it would be quite expensive when unlimited value is used, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 48105571b2798..4c048f79741bc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -199,6 +199,18 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Sinh, DoubleType) } + test("asinh") { + testUnary(Asinh, (x: Double) => math.log(x + math.sqrt(x * x + 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Asinh, DoubleType) + + checkEvaluation(Asinh(Double.NegativeInfinity), Double.NegativeInfinity) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Asinh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Asinh(doubleNullLit)), null, EmptyRow) + } + test("cos") { testUnary(Cos, math.cos) checkConsistencyBetweenInterpretedAndCodegen(Cos, DoubleType) @@ -215,6 +227,16 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) } + test("acosh") { + testUnary(Acosh, (x: Double) => math.log(x + math.sqrt(x * x - 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Acosh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Acosh(doubleNullLit)), null, EmptyRow) + } + test("tan") { testUnary(Tan, math.tan) checkConsistencyBetweenInterpretedAndCodegen(Tan, DoubleType) @@ -244,6 +266,16 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Tanh, DoubleType) } + test("atanh") { + testUnary(Atanh, (x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x))) + checkConsistencyBetweenInterpretedAndCodegen(Atanh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Atanh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Atanh(doubleNullLit)), null, EmptyRow) + } + test("toDegrees") { testUnary(ToDegrees, math.toDegrees) checkConsistencyBetweenInterpretedAndCodegen(ToDegrees, DoubleType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 2db4667fd0561..3ec8d18bc871d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -24,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{BooleanType, IntegerType} import org.apache.spark.unsafe.types.CalendarInterval class FilterPushdownSuite extends PlanTest { @@ -41,9 +42,14 @@ class FilterPushdownSuite extends PlanTest { CollapseProject) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int - val testRelation1 = LocalRelation('d.int) + val testRelation = LocalRelation(attrA, attrB, attrC) + + val testRelation1 = LocalRelation(attrD) // This test already passes. test("eliminate subqueries") { @@ -1202,4 +1208,26 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + val pythonUDFJoinCond = { + val pythonUDF = PythonUDF("pythonUDF", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrD + } + + val query = testRelation.join( + testRelation1, + joinType = Cross).where(pythonUDFJoinCond) + + val expected = testRelation.join( + testRelation1, + joinType = Cross, + condition = Some(pythonUDFJoinCond)).analyze + + comparePlans(Optimize.execute(query.analyze), expected) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala new file mode 100644 index 0000000000000..f5af416602c9d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{And, IsNull, KnownFloatingPointNormalized} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class NormalizeFloatingPointNumbersSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("NormalizeFloatingPointNumbers", Once, NormalizeFloatingNumbers) :: Nil + } + + val testRelation1 = LocalRelation('a.double) + val a = testRelation1.output(0) + val testRelation2 = LocalRelation('a.double) + val b = testRelation2.output(0) + + test("normalize floating points in window function expressions") { + val query = testRelation1.window(Seq(sum(a).as("sum")), Seq(a), Seq(a.asc)) + + val optimized = Optimize.execute(query) + val correctAnswer = testRelation1.window(Seq(sum(a).as("sum")), + Seq(KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), Seq(a.asc)) + + comparePlans(optimized, correctAnswer) + } + + test("normalize floating points in window function expressions - idempotence") { + val query = testRelation1.window(Seq(sum(a).as("sum")), Seq(a), Seq(a.asc)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val correctAnswer = testRelation1.window(Seq(sum(a).as("sum")), + Seq(KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), Seq(a.asc)) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("normalize floating points in join keys") { + val query = testRelation1.join(testRelation2, condition = Some(a === b)) + + val optimized = Optimize.execute(query) + val joinCond = Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)) + === KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) + val correctAnswer = testRelation1.join(testRelation2, condition = joinCond) + + comparePlans(optimized, correctAnswer) + } + + test("normalize floating points in join keys - idempotence") { + val query = testRelation1.join(testRelation2, condition = Some(a === b)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val joinCond = Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)) + === KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) + val correctAnswer = testRelation1.join(testRelation2, condition = joinCond) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("normalize floating points in join keys (equal null safe) - idempotence") { + val query = testRelation1.join(testRelation2, condition = Some(a <=> b)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val joinCond = IsNull(a) === IsNull(b) && + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(a, 0.0))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(b, 0.0))) + val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) + + comparePlans(doubleOptimized, correctAnswer) + } +} + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index e16262ddb9cd3..6248e5724f063 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -737,6 +737,15 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) } + test("Support respect nulls keywords for first_value and last_value") { + assertEqual("first_value(a ignore nulls)", First('a, Literal(true)).toAggregateExpression()) + assertEqual("first_value(a respect nulls)", First('a, Literal(false)).toAggregateExpression()) + assertEqual("first_value(a)", First('a, Literal(false)).toAggregateExpression()) + assertEqual("last_value(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) + assertEqual("last_value(a respect nulls)", Last('a, Literal(false)).toAggregateExpression()) + assertEqual("last_value(a)", Last('a, Literal(false)).toAggregateExpression()) + } + test("timestamp literals") { DateTimeTestUtils.outstandingTimezones.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index ba01380558530..fc2ce12092190 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -381,6 +381,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "fields", "fileformat", "first", + "first_value", "following", "for", "foreign", @@ -415,6 +416,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "join", "keys", "last", + "last_value", "lateral", "lazy", "leading", @@ -483,6 +485,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "repair", "replace", "reset", + "respect", "restrict", "revoke", "right", @@ -579,6 +582,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "except", "false", "fetch", + "first_value", "for", "foreign", "from", @@ -593,6 +597,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "into", "join", "is", + "last_value", "leading", "left", "minute", 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 2de9c41a30b98..8ff691fb17f27 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 @@ -31,7 +31,6 @@ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite { val TimeZonePST = TimeZone.getTimeZone("PST") - private def defaultTz = DateTimeUtils.defaultTimeZone() private def defaultZoneId = ZoneId.systemDefault() test("nanoseconds truncation") { @@ -138,6 +137,9 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(stringToDate(UTF8String.fromString("015-03-18")).isEmpty) assert(stringToDate(UTF8String.fromString("015")).isEmpty) assert(stringToDate(UTF8String.fromString("02015")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999 08 01")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999-08 01")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) } test("string to timestamp") { @@ -242,6 +244,9 @@ class DateTimeUtilsSuite extends SparkFunSuite { checkStringToTimestamp("2015-03-18T12:03.17-20:0", None) checkStringToTimestamp("2015-03-18T12:03.17-0:70", None) checkStringToTimestamp("2015-03-18T12:03.17-1:0:0", None) + checkStringToTimestamp("1999 08 01", None) + checkStringToTimestamp("1999-08 01", None) + checkStringToTimestamp("1999 08", None) // Truncating the fractional seconds timeZone = TimeZone.getTimeZone("GMT+00:00") @@ -353,20 +358,20 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("date add months") { val input = days(1997, 2, 28, 10, 30) - assert(dateAddMonths(input, 36) === days(2000, 2, 29)) - assert(dateAddMonths(input, -13) === days(1996, 1, 31)) + assert(dateAddMonths(input, 36) === days(2000, 2, 28)) + assert(dateAddMonths(input, -13) === days(1996, 1, 28)) } test("timestamp add months") { val ts1 = date(1997, 2, 28, 10, 30, 0) - val ts2 = date(2000, 2, 29, 10, 30, 0, 123000) - assert(timestampAddInterval(ts1, 36, 123000, defaultTz) === ts2) + val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) + assert(timestampAddInterval(ts1, 36, 123000, defaultZoneId) === ts2) val ts3 = date(1997, 2, 27, 16, 0, 0, 0, TimeZonePST) val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, TimeZonePST) - val ts5 = date(2000, 2, 29, 0, 0, 0, 123000, TimeZoneGMT) - assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4) - assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5) + val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT) + assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5) } test("monthsBetween") { diff --git a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java index fd6f7be2abc5a..1bd7b825328db 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -20,9 +20,10 @@ import java.util.concurrent.TimeUnit; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; import scala.concurrent.duration.Duration; -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger; +import org.apache.spark.sql.execution.streaming.ContinuousTrigger; import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; /** @@ -40,7 +41,7 @@ public class Trigger { * @since 2.2.0 */ public static Trigger ProcessingTime(long intervalMs) { - return ProcessingTime.create(intervalMs, TimeUnit.MILLISECONDS); + return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS); } /** @@ -56,7 +57,7 @@ public static Trigger ProcessingTime(long intervalMs) { * @since 2.2.0 */ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { - return ProcessingTime.create(interval, timeUnit); + return ProcessingTimeTrigger.create(interval, timeUnit); } /** @@ -71,7 +72,7 @@ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { * @since 2.2.0 */ public static Trigger ProcessingTime(Duration interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** @@ -84,7 +85,7 @@ public static Trigger ProcessingTime(Duration interval) { * @since 2.2.0 */ public static Trigger ProcessingTime(String interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index a3cbea9021f22..0da52d432d25d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -520,6 +520,71 @@ class KeyValueGroupedDataset[K, V] private[sql]( col4: TypedColumn[V, U4]): Dataset[(K, U1, U2, U3, U4)] = aggUntyped(col1, col2, col3, col4).asInstanceOf[Dataset[(K, U1, U2, U3, U4)]] + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5]): Dataset[(K, U1, U2, U3, U4, U5)] = + aggUntyped(col1, col2, col3, col4, col5).asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6]): Dataset[(K, U1, U2, U3, U4, U5, U6)] = + aggUntyped(col1, col2, col3, col4, col5, col6) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7, U8]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7], + col8: TypedColumn[V, U8]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7, col8) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)]] + /** * Returns a [[Dataset]] that contains a tuple with each key and the number of items present * for that key. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index f0ef6e19b0aa0..bb05c76cfee6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -142,16 +142,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") val version = if (i == 0) "2.3.0" else "1.3.0" - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Register a deterministic Java UDF$i instance as user-defined function (UDF). | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | val func = f$anyCast.call($anyParams) + | val func = $funcCall | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, e.map(_ => false), udfName = Some(name)) + | ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -717,9 +717,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { - val func = f.asInstanceOf[UDF0[Any]].call() + val func = () => f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 315eba6635aac..4385843d90112 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -53,8 +53,8 @@ class ColumnarRule { * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. * - * The implementation is based off of similar implementations in [[ColumnarBatchScan]], - * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]], and + * The implementation is based off of similar implementations in + * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. */ case class ColumnarToRowExec(child: SparkPlan) @@ -96,9 +96,6 @@ case class ColumnarToRowExec(child: SparkPlan) /** * Generate [[ColumnVector]] expressions for our parent to consume as rows. * This is called once per [[ColumnVector]] in the batch. - * - * This code came unchanged from [[ColumnarBatchScan]] and will hopefully replace it - * at some point. */ private def genCodeColumnVector( ctx: CodegenContext, @@ -130,9 +127,6 @@ case class ColumnarToRowExec(child: SparkPlan) * Produce code to process the input iterator as [[ColumnarBatch]]es. * This produces an [[org.apache.spark.sql.catalyst.expressions.UnsafeRow]] for each row in * each batch. - * - * This code came almost completely unchanged from [[ColumnarBatchScan]] and will - * hopefully replace it at some point. */ override protected def doProduce(ctx: CodegenContext): String = { // PhysicalRDD always just has one input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala deleted file mode 100644 index b2e9f760d27ca..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ /dev/null @@ -1,167 +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.execution - -import org.apache.spark.sql.catalyst.expressions.{BoundReference, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} - - -/** - * Helper trait for abstracting scan functionality using [[ColumnarBatch]]es. - */ -private[sql] trait ColumnarBatchScan extends CodegenSupport { - - protected def supportsBatch: Boolean = true - - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) - - /** - * Generate [[ColumnVector]] expressions for our parent to consume as rows. - * This is called once per [[ColumnarBatch]]. - */ - private def genCodeColumnVector( - ctx: CodegenContext, - columnVar: String, - ordinal: String, - dataType: DataType, - nullable: Boolean): ExprCode = { - val javaType = CodeGenerator.javaType(dataType) - val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { - JavaCode.isNullVariable(ctx.freshName("isNull")) - } else { - FalseLiteral - } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = code"${ctx.registerComment(str)}" + (if (nullable) { - code""" - boolean $isNullVar = $columnVar.isNullAt($ordinal); - $javaType $valueVar = $isNullVar ? ${CodeGenerator.defaultValue(dataType)} : ($value); - """ - } else { - code"$javaType $valueVar = $value;" - }) - ExprCode(code, isNullVar, JavaCode.variable(valueVar, dataType)) - } - - /** - * Produce code to process the input iterator as [[ColumnarBatch]]es. - * This produces an [[UnsafeRow]] for each row in each batch. - */ - // TODO: return ColumnarBatch.Rows instead - override protected def doProduce(ctx: CodegenContext): String = { - // PhysicalRDD always just has one input - val input = ctx.addMutableState("scala.collection.Iterator", "input", - v => s"$v = inputs[0];") - if (supportsBatch) { - produceBatches(ctx, input) - } else { - produceRows(ctx, input) - } - } - - private def produceBatches(ctx: CodegenContext, input: String): String = { - // metrics - val numOutputRows = metricTerm(ctx, "numOutputRows") - val scanTimeMetric = metricTerm(ctx, "scanTime") - val scanTimeTotalNs = - ctx.addMutableState(CodeGenerator.JAVA_LONG, "scanTime") // init as scanTime = 0 - - val columnarBatchClz = classOf[ColumnarBatch].getName - val batch = ctx.addMutableState(columnarBatchClz, "batch") - - val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init as batchIdx = 0 - val columnVectorClzs = vectorTypes.getOrElse( - Seq.fill(output.indices.size)(classOf[ColumnVector].getName)) - val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map { - case (columnVectorClz, i) => - val name = ctx.addMutableState(columnVectorClz, s"colInstance$i") - (name, s"$name = ($columnVectorClz) $batch.column($i);") - }.unzip - - val nextBatch = ctx.freshName("nextBatch") - val nextBatchFuncName = ctx.addNewFunction(nextBatch, - s""" - |private void $nextBatch() throws java.io.IOException { - | long getBatchStart = System.nanoTime(); - | if ($input.hasNext()) { - | $batch = ($columnarBatchClz)$input.next(); - | $numOutputRows.add($batch.numRows()); - | $idx = 0; - | ${columnAssigns.mkString("", "\n", "\n")} - | } - | $scanTimeTotalNs += System.nanoTime() - getBatchStart; - |}""".stripMargin) - - ctx.currentVars = null - val rowidx = ctx.freshName("rowIdx") - val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => - genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) - } - val localIdx = ctx.freshName("localIdx") - val localEnd = ctx.freshName("localEnd") - val numRows = ctx.freshName("numRows") - val shouldStop = if (parent.needStopCheck) { - s"if (shouldStop()) { $idx = $rowidx + 1; return; }" - } else { - "// shouldStop check is eliminated" - } - s""" - |if ($batch == null) { - | $nextBatchFuncName(); - |} - |while ($limitNotReachedCond $batch != null) { - | int $numRows = $batch.numRows(); - | int $localEnd = $numRows - $idx; - | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { - | int $rowidx = $idx + $localIdx; - | ${consume(ctx, columnsBatchInput).trim} - | $shouldStop - | } - | $idx = $numRows; - | $batch = null; - | $nextBatchFuncName(); - |} - |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); - |$scanTimeTotalNs = 0; - """.stripMargin - } - - private def produceRows(ctx: CodegenContext, input: String): String = { - val numOutputRows = metricTerm(ctx, "numOutputRows") - val row = ctx.freshName("row") - - ctx.INPUT_ROW = row - ctx.currentVars = null - s""" - |while ($limitNotReachedCond $input.hasNext()) { - | InternalRow $row = (InternalRow) $input.next(); - | $numOutputRows.add(1); - | ${consume(ctx, null, row).trim} - | if (shouldStop()) return; - |} - """.stripMargin - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 518460d98f05c..728ac3a466fbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -37,10 +37,11 @@ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.spark.util.collection.BitSet -trait DataSourceScanExec extends LeafExecNode with CodegenSupport { +trait DataSourceScanExec extends LeafExecNode { val relation: BaseRelation val tableIdentifier: Option[TableIdentifier] @@ -69,6 +70,12 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { private def redact(text: String): String = { Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text) } + + /** + * The data being read in. This is to provide input to the tests in a way compatible with + * [[InputRDDCodegen]] which all implementations used to extend. + */ + def inputRDDs(): Seq[RDD[InternalRow]] } /** Physical plan node for scanning data from a relation. */ @@ -141,11 +148,11 @@ case class FileSourceScanExec( optionalBucketSet: Option[BitSet], dataFilters: Seq[Expression], override val tableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec with ColumnarBatchScan { + extends DataSourceScanExec { // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsBatch: Boolean = { + override lazy val supportsColumnar: Boolean = { relation.fileFormat.supportBatch(relation.sparkSession, schema) } @@ -275,7 +282,7 @@ case class FileSourceScanExec( Map( "Format" -> relation.fileFormat.toString, "ReadSchema" -> requiredSchema.catalogString, - "Batched" -> supportsBatch.toString, + "Batched" -> supportsColumnar.toString, "PartitionFilters" -> seqToString(partitionFilters), "PushedFilters" -> seqToString(pushedDownFilters), "DataFilters" -> seqToString(dataFilters), @@ -302,7 +309,7 @@ case class FileSourceScanExec( withSelectedBucketsCount } - private lazy val inputRDD: RDD[InternalRow] = { + lazy val inputRDD: RDD[InternalRow] = { val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -334,29 +341,30 @@ case class FileSourceScanExec( "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - // in the case of fallback, this batched scan should never fail because of: - // 1) only primitive types are supported - // 2) the number of columns should be smaller than spark.sql.codegen.maxFields - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - val numOutputRows = longMetric("numOutputRows") - - if (needsUnsafeRowConversion) { - inputRDD.mapPartitionsWithIndexInternal { (index, iter) => - val proj = UnsafeProjection.create(schema) - proj.initialize(index) - iter.map( r => { - numOutputRows += 1 - proj(r) - }) - } - } else { - inputRDD.map { r => + val numOutputRows = longMetric("numOutputRows") + + if (needsUnsafeRowConversion) { + inputRDD.mapPartitionsWithIndexInternal { (index, iter) => + val proj = UnsafeProjection.create(schema) + proj.initialize(index) + iter.map( r => { numOutputRows += 1 - r - } + proj(r) + }) } + } else { + inputRDD.map { r => + numOutputRows += 1 + r + } + } + } + + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].map { batch => + numOutputRows += batch.numRows() + batch } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 981ecae80a724..1ab183fe843ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -69,9 +69,8 @@ case class ExternalRDDScanExec[T]( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val outputDataType = outputObjAttr.dataType rdd.mapPartitionsInternal { iter => - val outputObject = ObjectOperator.wrapObjectToRow(outputDataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) iter.map { value => numOutputRows += 1 outputObject(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 9fcffac53c999..c8531e9a046a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -76,7 +76,7 @@ class QueryExecution( lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { SparkSession.setActiveSession(sparkSession) // Runtime re-optimization requires a unique instance of every node in the logical plan. - val logicalPlan = if (sparkSession.sessionState.conf.runtimeReoptimizationEnabled) { + val logicalPlan = if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { optimizedPlan.clone() } else { optimizedPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c4d5a2b9b3f0a..5500941936442 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -703,7 +703,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { - ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil + ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), + planLater(child), canChangeNumPartitions = false) :: Nil } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } @@ -736,7 +737,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => - exchange.ShuffleExchangeExec(r.partitioning, planLater(r.child)) :: Nil + exchange.ShuffleExchangeExec( + r.partitioning, planLater(r.child), canChangeNumPartitions = false) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 94a5ede751456..a0afa9a26fd51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -709,11 +709,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " + s"for this plan (id=$codegenStageId). To avoid this, you can raise the limit " + s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString") - child match { - // The fallback solution of batch file source scan still uses WholeStageCodegenExec - case f: FileSourceScanExec if f.supportsBatch => // do nothing - case _ => return child.execute() - } + return child.execute() } val references = ctx.references.toArray 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 0708878ece46d..61dbc58297388 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 @@ -84,6 +84,8 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReduceNumShufflePartitions(conf), + ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, + session.sessionState.columnarRules), CollapseCodegenStages(conf) ) 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 a1b0e291c1b68..14ca2b41a442b 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 @@ -44,7 +44,7 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan override def apply(plan: SparkPlan): SparkPlan = plan match { case _: ExecutedCommandExec => plan - case _ if conf.runtimeReoptimizationEnabled && supportAdaptive(plan) => + case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) => try { // 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. @@ -57,13 +57,13 @@ case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache) } catch { case SubqueryAdaptiveNotSupportedException(subquery) => - logWarning(s"${SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key} is enabled " + + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + s"but is not supported for sub-query: $subquery.") plan } case _ => - if (conf.runtimeReoptimizationEnabled) { - logWarning(s"${SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key} is enabled " + + if (conf.adaptiveExecutionEnabled) { + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + s"but is not supported for query: $plan.") } plan 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 d93eb76b9fbc4..78923433eaab9 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 @@ -61,12 +61,18 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan // have same number of output partitions. + return plan + } + + val shuffleStages = plan.collect { + case stage: ShuffleQueryStageExec => stage + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage + } + // ShuffleExchanges introduced by repartition do not support changing the number of partitions. + // We change the number of partitions in the stage only if all the ShuffleExchanges support it. + if (!shuffleStages.forall(_.plan.canChangeNumPartitions)) { plan } else { - val shuffleStages = plan.collect { - case stage: ShuffleQueryStageExec => stage - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage - } val shuffleMetrics = shuffleStages.map { stage => val metricsFuture = stage.mapOutputStatisticsFuture assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") @@ -76,12 +82,7 @@ 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) - // We may get different pre-shuffle partition number if user calls repartition manually. - // We don't reduce shuffle partition number in that case. - val distinctNumPreShufflePartitions = - validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - - if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { + if (validMetrics.nonEmpty) { 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/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 25ff6584360e6..4a95f76381339 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator import org.apache.spark.util.Utils @@ -559,7 +560,7 @@ case class HashAggregateExec( private def enableTwoLevelHashMap(ctx: CodegenContext): Unit = { if (!checkIfFastHashMapSupported(ctx)) { if (modes.forall(mode => mode == Partial || mode == PartialMerge) && !Utils.isTesting) { - logInfo("spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but" + logInfo(s"${SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key} is set to true, but" + " current version of codegened fast hashmap does not support this aggregate.") } } else { @@ -567,8 +568,7 @@ case class HashAggregateExec( // This is for testing/benchmarking only. // We enforce to first level to be a vectorized hashmap, instead of the default row-based one. - isVectorizedHashMapEnabled = sqlContext.getConf( - "spark.sql.codegen.aggregate.map.vectorized.enable", "false") == "true" + isVectorizedHashMapEnabled = sqlContext.conf.enableVectorizedHashMap } } @@ -576,12 +576,8 @@ case class HashAggregateExec( val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg") if (sqlContext.conf.enableTwoLevelAggMap) { enableTwoLevelHashMap(ctx) - } else { - sqlContext.getConf("spark.sql.codegen.aggregate.map.vectorized.enable", null) match { - case "true" => - logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") - case _ => - } + } else if (sqlContext.conf.enableVectorizedHashMap) { + logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") } val bitMaxCapacity = sqlContext.conf.fastHashAggregateRowMaxCapacityBit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 06634c13ec439..3566ab1aa5a33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} @@ -34,7 +35,10 @@ case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) - extends LeafExecNode with ColumnarBatchScan { + extends LeafExecNode { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override val nodeName: String = { relation.cacheBuilder.tableName match { @@ -65,7 +69,7 @@ case class InMemoryTableScanExec( * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. * If false, get data from UnsafeRow build from CachedBatch */ - override val supportsBatch: Boolean = { + override val supportsColumnar: Boolean = { // In the initial implementation, for ease of review // support only primitive data types and # of fields is less than wholeStageMaxNumFields conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { @@ -75,9 +79,6 @@ case class InMemoryTableScanExec( }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) } - // TODO: revisit this. Shall we always turn off whole stage codegen if the output data are rows? - override def supportCodegen: Boolean = supportsBatch - private val columnIndices = attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray @@ -108,59 +109,61 @@ case class InMemoryTableScanExec( columnarBatch } + private lazy val columnarInputRDD: RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + val buffers = filteredCachedBatches() + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + buffers + .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) + .map(b => { + numOutputRows += b.numRows() + b + }) + } + private lazy val inputRDD: RDD[InternalRow] = { val buffers = filteredCachedBatches() val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - if (supportsBatch) { - // HACK ALERT: This is actually an RDD[ColumnarBatch]. - // We're taking advantage of Scala's type erasure here to pass these batches along. - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .asInstanceOf[RDD[InternalRow]] - } else { - val numOutputRows = longMetric("numOutputRows") + val numOutputRows = longMetric("numOutputRows") - if (enableAccumulatorsForTest) { - readPartitions.setValue(0) - readBatches.setValue(0) - } + if (enableAccumulatorsForTest) { + readPartitions.setValue(0) + readBatches.setValue(0) + } - // Using these variables here to avoid serialization of entire objects (if referenced - // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) - } - numOutputRows += batch.numRows - batch + // Using these variables here to avoid serialization of entire objects (if referenced + // directly) within the map Partitions closure. + val relOutput: AttributeSeq = relation.output + + filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + attributes.map { a => + relOutput.indexOf(a.exprId) -> a.dataType + }.unzip + + // update SQL metrics + val withMetrics = cachedBatchIterator.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) } + numOutputRows += batch.numRows + batch + } - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) - } - columnarIterator + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) + if (enableAccumulatorsForTest && columnarIterator.hasNext) { + readPartitions.add(1) } + columnarIterator } } - override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) - override def output: Seq[Attribute] = attributes private def updateAttribute(expr: Expression): Expression = { @@ -294,8 +297,7 @@ case class InMemoryTableScanExec( } } - lazy val enableAccumulatorsForTest: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes lazy val readPartitions = sparkContext.longAccumulator @@ -339,10 +341,10 @@ case class InMemoryTableScanExec( } protected override def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - inputRDD - } + inputRDD + } + + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + columnarInputRDD } } 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 26f7230c8fe8f..1b7bb169b36fd 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 @@ -26,31 +26,32 @@ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.CastSupport -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, 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} case class DataSourceResolution( conf: SQLConf, - findCatalog: String => CatalogPlugin) - extends Rule[LogicalPlan] with CastSupport with LookupCatalog { + lookup: LookupCatalog) + extends Rule[LogicalPlan] with CastSupport { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import lookup._ - override protected def lookupCatalog(name: String): CatalogPlugin = findCatalog(name) - - def defaultCatalog: Option[CatalogPlugin] = conf.defaultV2Catalog.map(findCatalog) + lazy val v2SessionCatalog: CatalogPlugin = lookup.sessionCatalog + .getOrElse(throw new AnalysisException("No v2 session catalog implementation is available")) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTableStatement( AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, V1WriteProvider(provider), options, location, comment, ifNotExists) => - + // the source is v1, the identifier has no catalog, and there is no default v2 catalog val tableDesc = buildCatalogTable(table, schema, partitionCols, bucketSpec, properties, provider, options, location, comment, ifNotExists) val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists @@ -58,18 +59,22 @@ case class DataSourceResolution( CreateTable(tableDesc, mode, None) case create: CreateTableStatement => - // the provider was not a v1 source, convert to a v2 plan + // the provider was not a v1 source or a v2 catalog is the default, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = create.tableName - val catalog = maybeCatalog.orElse(defaultCatalog) - .getOrElse(throw new AnalysisException( - s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) - .asTableCatalog - convertCreateTable(catalog, identifier, create) + maybeCatalog match { + case Some(catalog) => + // the identifier had a catalog, or there is a default v2 catalog + convertCreateTable(catalog.asTableCatalog, identifier, create) + case _ => + // the identifier had no catalog and no default catalog is set, but the source is v2. + // use the v2 session catalog, which delegates to the global v1 session catalog + convertCreateTable(v2SessionCatalog.asTableCatalog, identifier, create) + } case CreateTableAsSelectStatement( AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, V1WriteProvider(provider), options, location, comment, ifNotExists) => - + // the source is v1, the identifier has no catalog, and there is no default v2 catalog val tableDesc = buildCatalogTable(table, new StructType, partitionCols, bucketSpec, properties, provider, options, location, comment, ifNotExists) val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists @@ -77,13 +82,17 @@ case class DataSourceResolution( CreateTable(tableDesc, mode, Some(query)) case create: CreateTableAsSelectStatement => - // the provider was not a v1 source, convert to a v2 plan + // the provider was not a v1 source or a v2 catalog is the default, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = create.tableName - val catalog = maybeCatalog.orElse(defaultCatalog) - .getOrElse(throw new AnalysisException( - s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) - .asTableCatalog - convertCTAS(catalog, identifier, create) + maybeCatalog match { + case Some(catalog) => + // the identifier had a catalog, or there is a default v2 catalog + convertCTAS(catalog.asTableCatalog, identifier, create) + case _ => + // the identifier had no catalog and no default catalog is set, but the source is v2. + // use the v2 session catalog, which delegates to the global v1 session catalog + convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) + } case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) @@ -118,6 +127,9 @@ case class DataSourceResolution( if newColumns.forall(_.name.size == 1) => // only top-level adds are supported using AlterTableAddColumnsCommand AlterTableAddColumnsCommand(table, newColumns.map(convertToStructField)) + + case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) => + UnresolvedCatalogRelation(catalogTable) } object V1WriteProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 8508322f54e8b..b9b86adb438e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -613,7 +613,7 @@ class ParquetFilters( } override def keep(value: Binary): Boolean = { - UTF8String.fromBytes(value.getBytes).startsWith( + value != null && UTF8String.fromBytes(value.getBytes).startsWith( UTF8String.fromBytes(strToBinary.getBytes)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala new file mode 100644 index 0000000000000..a3fa82b12e938 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala @@ -0,0 +1,47 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for altering a table. + */ +case class AlterTableExec( + catalog: TableCatalog, + ident: Identifier, + changes: Seq[TableChange]) extends LeafExecNode { + + override def output: Seq[Attribute] = Seq.empty + + override protected def doExecute(): RDD[InternalRow] = { + try { + catalog.alterTable(ident, changes: _*) + } catch { + case e: IllegalArgumentException => + throw new SparkException(s"Unsupported table change: ${e.getMessage}", e) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index 3276ab5067500..c3cbb9d2af4ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -45,7 +45,7 @@ case class BatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - new DataSourceRDD(sparkContext, partitions, readerFactory, supportsBatch) + new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar) } override def doCanonicalize(): BatchScanExec = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala index 9ad683fbe1df6..c5c902ffc4104 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -23,11 +23,16 @@ import org.apache.spark.sql.catalyst.expressions.AttributeMap import org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils -trait DataSourceV2ScanExecBase extends LeafExecNode with ColumnarBatchScan { +trait DataSourceV2ScanExecBase extends LeafExecNode { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) def scan: Scan @@ -52,7 +57,7 @@ trait DataSourceV2ScanExecBase extends LeafExecNode with ColumnarBatchScan { case _ => super.outputPartitioning } - override def supportsBatch: Boolean = { + override def supportsColumnar: Boolean = { require(partitions.forall(readerFactory.supportColumnarReads) || !partitions.exists(readerFactory.supportColumnarReads), "Cannot mix row-based and columnar input partitions.") @@ -62,17 +67,22 @@ trait DataSourceV2ScanExecBase extends LeafExecNode with ColumnarBatchScan { def inputRDD: RDD[InternalRow] - override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) + def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) override def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - val numOutputRows = longMetric("numOutputRows") - inputRDD.map { r => - numOutputRows += 1 - r - } + val numOutputRows = longMetric("numOutputRows") + inputRDD.map { r => + numOutputRows += 1 + r + } + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].map { + b => + numOutputRows += b.numRows() + b } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 27d87960edb3f..4f8507da39240 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -202,6 +202,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil + case AlterTable(catalog, ident, _, changes) => + AlterTableExec(catalog, ident, changes) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala index d2e33d4fa77c1..a9b0f5bce1b09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala @@ -46,6 +46,6 @@ case class MicroBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = stream.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - new DataSourceRDD(sparkContext, partitions, readerFactory, supportsBatch) + new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar) } } 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 new file mode 100644 index 0000000000000..4cd0346b57e7e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -0,0 +1,255 @@ +/* + * 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.datasources.v2 + +import java.util +import java.util.Locale + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} +import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} +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.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * A [[TableCatalog]] that translates calls to the v1 SessionCatalog. + */ +class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { + def this() = { + this(SparkSession.active.sessionState) + } + + private lazy val catalog: SessionCatalog = sessionState.catalog + + private var _name: String = _ + + override def name: String = _name + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + this._name = name + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + namespace match { + case Array(db) => + catalog.listTables(db).map(ident => Identifier.of(Array(db), ident.table)).toArray + case _ => + throw new NoSuchNamespaceException(namespace) + } + } + + override def loadTable(ident: Identifier): Table = { + val catalogTable = try { + catalog.getTableMetadata(ident.asTableIdentifier) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + CatalogTableAsV2(catalogTable) + } + + override def invalidateTable(ident: Identifier): Unit = { + catalog.refreshTable(ident.asTableIdentifier) + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + val (partitionColumns, maybeBucketSpec) = V2SessionCatalog.convertTransforms(partitions) + val provider = properties.getOrDefault("provider", sessionState.conf.defaultDataSourceName) + val tableProperties = properties.asScala + val location = Option(properties.get("location")) + val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) + .copy(locationUri = location.map(CatalogUtils.stringToURI)) + + val tableDesc = CatalogTable( + identifier = ident.asTableIdentifier, + tableType = CatalogTableType.MANAGED, + storage = storage, + schema = schema, + provider = Some(provider), + partitionColumnNames = partitionColumns, + bucketSpec = maybeBucketSpec, + properties = tableProperties.toMap, + tracksPartitionsInCatalog = sessionState.conf.manageFilesourcePartitions, + comment = Option(properties.get("comment"))) + + try { + catalog.createTable(tableDesc, ignoreIfExists = false) + } catch { + case _: TableAlreadyExistsException => + throw new TableAlreadyExistsException(ident) + } + + loadTable(ident) + } + + override def alterTable( + ident: Identifier, + changes: TableChange*): Table = { + val catalogTable = try { + catalog.getTableMetadata(ident.asTableIdentifier) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes) + val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) + + try { + catalog.alterTable(catalogTable.copy(properties = properties, schema = schema)) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + loadTable(ident) + } + + override def dropTable(ident: Identifier): Boolean = { + try { + if (loadTable(ident) != null) { + catalog.dropTable( + ident.asTableIdentifier, + ignoreIfNotExists = true, + purge = true /* skip HDFS trash */) + true + } else { + false + } + } catch { + case _: NoSuchTableException => + false + } + } + + implicit class TableIdentifierHelper(ident: Identifier) { + def asTableIdentifier: TableIdentifier = { + ident.namespace match { + case Array(db) => + TableIdentifier(ident.name, Some(db)) + case Array() => + TableIdentifier(ident.name, Some(catalog.getCurrentDatabase)) + case _ => + throw new NoSuchTableException(ident) + } + } + } + + 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. + */ + private def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + val identityCols = new mutable.ArrayBuffer[String] + var bucketSpec = Option.empty[BucketSpec] + + partitions.map { + case IdentityTransform(FieldReference(Seq(col))) => + identityCols += col + + case BucketTransform(numBuckets, FieldReference(Seq(col))) => + bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil)) + + case transform => + throw new UnsupportedOperationException( + s"SessionCatalog does not support partition transform: $transform") + } + + (identityCols, bucketSpec) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index b7196d4578fb0..c56a5c015f32d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, - SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf /** @@ -37,7 +36,7 @@ import org.apache.spark.sql.internal.SQLConf */ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { private def defaultNumPreShufflePartitions: Int = - if (conf.runtimeReoptimizationEnabled) { + if (conf.adaptiveExecutionEnabled) { conf.maxNumPostShufflePartitions } else { conf.numShufflePartitions @@ -94,7 +93,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) child match { // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c) => ShuffleExchangeExec(defaultPartitioning, c) + case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) case _ => ShuffleExchangeExec(defaultPartitioning, child) } } @@ -117,25 +116,41 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { } private def reorder( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], + leftKeys: IndexedSeq[Expression], + rightKeys: IndexedSeq[Expression], expectedOrderOfKeys: Seq[Expression], currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - val leftKeysBuffer = ArrayBuffer[Expression]() - val rightKeysBuffer = ArrayBuffer[Expression]() - val pickedIndexes = mutable.Set[Int]() - val keysAndIndexes = currentOrderOfKeys.zipWithIndex - - expectedOrderOfKeys.foreach(expression => { - val index = keysAndIndexes.find { case (e, idx) => - // As we may have the same key used many times, we need to filter out its occurrence we - // have already used. - e.semanticEquals(expression) && !pickedIndexes.contains(idx) - }.map(_._2).get - pickedIndexes += index - leftKeysBuffer.append(leftKeys(index)) - rightKeysBuffer.append(rightKeys(index)) - }) + if (expectedOrderOfKeys.size != currentOrderOfKeys.size) { + return (leftKeys, rightKeys) + } + + // Build a lookup between an expression and the positions its holds in the current key seq. + val keyToIndexMap = mutable.Map.empty[Expression, mutable.BitSet] + currentOrderOfKeys.zipWithIndex.foreach { + case (key, index) => + keyToIndexMap.getOrElseUpdate(key.canonicalized, mutable.BitSet.empty).add(index) + } + + // Reorder the keys. + val leftKeysBuffer = new ArrayBuffer[Expression](leftKeys.size) + val rightKeysBuffer = new ArrayBuffer[Expression](rightKeys.size) + val iterator = expectedOrderOfKeys.iterator + while (iterator.hasNext) { + // Lookup the current index of this key. + keyToIndexMap.get(iterator.next().canonicalized) match { + case Some(indices) if indices.nonEmpty => + // Take the first available index from the map. + val index = indices.firstKey + indices.remove(index) + + // Add the keys for that index to the reordered keys. + leftKeysBuffer += leftKeys(index) + rightKeysBuffer += rightKeys(index) + case _ => + // The expression cannot be found, or we have exhausted all indices for that expression. + return (leftKeys, rightKeys) + } + } (leftKeysBuffer, rightKeysBuffer) } @@ -145,20 +160,13 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { leftPartitioning: Partitioning, rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = { if (leftKeys.forall(_.deterministic) && rightKeys.forall(_.deterministic)) { - leftPartitioning match { - case HashPartitioning(leftExpressions, _) - if leftExpressions.length == leftKeys.length && - leftKeys.forall(x => leftExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, leftExpressions, leftKeys) - - case _ => rightPartitioning match { - case HashPartitioning(rightExpressions, _) - if rightExpressions.length == rightKeys.length && - rightKeys.forall(x => rightExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, rightExpressions, rightKeys) - - case _ => (leftKeys, rightKeys) - } + (leftPartitioning, rightPartitioning) match { + case (HashPartitioning(leftExpressions, _), _) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leftExpressions, leftKeys) + case (_, HashPartitioning(rightExpressions, _)) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) + case _ => + (leftKeys, rightKeys) } } else { (leftKeys, rightKeys) @@ -191,7 +199,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = plan.transformUp { // TODO: remove this after we create a physical operator for `RepartitionByExpression`. - case operator @ ShuffleExchangeExec(upper: HashPartitioning, child) => + case operator @ ShuffleExchangeExec(upper: HashPartitioning, child, _) => child.outputPartitioning match { case lower: HashPartitioning if upper.semanticEquals(lower) => child case _ => operator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 5d0208f1ecc46..fec05a76b4516 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -43,7 +43,8 @@ import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordCo */ case class ShuffleExchangeExec( override val outputPartitioning: Partitioning, - child: SparkPlan) extends Exchange { + child: SparkPlan, + canChangeNumPartitions: Boolean = true) extends Exchange { // NOTE: coordinator can be null after serialization/deserialization, // e.g. it can be null on the Executor side diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 9a76e144b8850..d05113431df41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -191,7 +191,7 @@ case class MapPartitionsExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) func(iter.map(getObject)).map(outputObject) } } @@ -278,10 +278,10 @@ case class MapElementsExec( override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { val (funcClass, methodName) = func match { case m: MapFunction[_, _] => classOf[MapFunction[_, _]] -> "call" - case _ => FunctionUtils.getFunctionOneName(outputObjAttr.dataType, child.output(0).dataType) + case _ => FunctionUtils.getFunctionOneName(outputObjectType, child.output(0).dataType) } val funcObj = Literal.create(func, ObjectType(funcClass)) - val callFunc = Invoke(funcObj, methodName, outputObjAttr.dataType, child.output) + val callFunc = Invoke(funcObj, methodName, outputObjectType, child.output) val result = BindReferences.bindReference(callFunc, child.output).genCode(ctx) @@ -296,7 +296,7 @@ case class MapElementsExec( child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) iter.map(row => outputObject(callFunc(getObject(row)))) } } @@ -395,7 +395,7 @@ case class MapGroupsExec( val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) grouped.flatMap { case (key, rowIter) => val result = func( @@ -447,8 +447,6 @@ case class FlatMapGroupsInRExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def outputPartitioning: Partitioning = child.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = @@ -473,7 +471,7 @@ case class FlatMapGroupsInRExec( val grouped = GroupedIterator(iter, groupingAttributes, child.output) val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) val runner = new RRunner[(Array[Byte], Iterator[Array[Byte]]), Array[Byte]]( func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, isDataFrame = true, colNames = inputSchema.fieldNames, @@ -606,7 +604,7 @@ case class CoGroupExec( val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, leftGroup) val getLeft = ObjectOperator.deserializeRowToObject(leftDeserializer, leftAttr) val getRight = ObjectOperator.deserializeRowToObject(rightDeserializer, rightAttr) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { case (key, leftResult, rightResult) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index 4f352782067cf..02bfbc4949b37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -81,10 +81,6 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] outputIterator.flatMap { pickedResult => val unpickledBatch = unpickle.loads(pickedResult) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. Pyrolite - // doesn't clear it up, so we manually clear it. - unpickle.close() unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala }.map { result => if (udfs.length == 1) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 58fe7d5cbc0f4..fc4ded376bf1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -179,7 +179,7 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { validUdfs.forall(PythonUDF.isScalarPythonUDF), "Can only extract scalar vectorized udf or sql batch udf") - val resultAttrs = udfs.zipWithIndex.map { case (u, i) => + val resultAttrs = validUdfs.zipWithIndex.map { case (u, i) => AttributeReference(s"pythonUDF$i", u.dataType)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index f4c2d02ee9420..41521bfae1add 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -32,7 +32,7 @@ case class UserDefinedPythonFunction( pythonEvalType: Int, udfDeterministic: Boolean) { - def builder(e: Seq[Expression]): PythonUDF = { + def builder(e: Seq[Expression]): Expression = { PythonUDF(name, func, dataType, e, pythonEvalType, udfDeterministic) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala index 6b6eb78404e3d..fe91d24912222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala @@ -28,14 +28,14 @@ import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode} import org.apache.spark.util.CompletionIterator /** - * Physical operator for executing `FlatMapGroupsWithState.` + * Physical operator for executing `FlatMapGroupsWithState` * * @param func function called on each group * @param keyDeserializer used to extract the key object for each group. * @param valueDeserializer used to extract the items in the iterator from an input row. * @param groupingAttributes used to group the data * @param dataAttributes used to read the data - * @param outputObjAttr used to define the output object + * @param outputObjAttr Defines the output object * @param stateEncoder used to serialize/deserialize state before calling `func` * @param outputMode the output mode of `func` * @param timeoutConf used to timeout groups that have not received data in a while @@ -154,7 +154,7 @@ case class FlatMapGroupsWithStateExec( ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) private val getValueObj = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - private val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + private val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType) // Metrics private val numUpdatedStateRows = longMetric("numUpdatedStateRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index fd2638f30469c..e7eb2cb558cdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchSt import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock class MicroBatchExecution( @@ -51,7 +51,7 @@ class MicroBatchExecution( @volatile protected var sources: Seq[SparkDataStream] = Seq.empty private val triggerExecutor = trigger match { - case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) + case t: ProcessingTimeTrigger => ProcessingTimeExecutor(t, triggerClock) case OneTimeTrigger => OneTimeExecutor() case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index eb957192bcdc5..7c1f6ca42c1f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -320,7 +320,7 @@ abstract class StreamExecution( logicalPlan // Adaptive execution can change num shuffle partitions, disallow - sparkSessionForStream.conf.set(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key, "false") + sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") // Disable cost-based join optimization as we do not want stateful operations to be rearranged sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false") offsetSeqMetadata = OffsetSeqMetadata( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index d188566f822b4..088471053b6f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.util.{Clock, SystemClock} trait TriggerExecutor { @@ -43,10 +42,12 @@ case class OneTimeExecutor() extends TriggerExecutor { /** * A trigger executor that runs a batch every `intervalMs` milliseconds. */ -case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = new SystemClock()) +case class ProcessingTimeExecutor( + processingTimeTrigger: ProcessingTimeTrigger, + clock: Clock = new SystemClock()) extends TriggerExecutor with Logging { - private val intervalMs = processingTime.intervalMs + private val intervalMs = processingTimeTrigger.intervalMs require(intervalMs >= 0) override def execute(triggerHandler: () => Boolean): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 4c0db3cb42a82..aede08820503b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -17,8 +17,31 @@ package org.apache.spark.sql.execution.streaming +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration.Duration + import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.CalendarInterval + +private object Triggers { + def validate(intervalMs: Long): Unit = { + require(intervalMs >= 0, "the interval of trigger should not be negative") + } + + def convert(interval: String): Long = { + val cal = CalendarInterval.fromCaseInsensitiveString(interval) + if (cal.months > 0) { + throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") + } + TimeUnit.MICROSECONDS.toMillis(cal.microseconds) + } + + def convert(interval: Duration): Long = interval.toMillis + + def convert(interval: Long, unit: TimeUnit): Long = unit.toMillis(interval) +} /** * A [[Trigger]] that processes only one batch of data in a streaming query then terminates @@ -26,4 +49,62 @@ import org.apache.spark.sql.streaming.Trigger */ @Experimental @Evolving -case object OneTimeTrigger extends Trigger +private[sql] case object OneTimeTrigger extends Trigger + +/** + * A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0, + * the query will run as fast as possible. + */ +@Evolving +private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ProcessingTimeTrigger { + import Triggers._ + + def apply(interval: String): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def apply(interval: Duration): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def create(interval: String): ProcessingTimeTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval, unit)) + } +} + +/** + * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at + * the specified interval. + */ +@Evolving +private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ContinuousTrigger { + import Triggers._ + + def apply(interval: String): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def apply(interval: Duration): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def create(interval: String): ContinuousTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { + ContinuousTrigger(convert(interval, unit)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 509b103faa0dc..f6d156ded7663 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} import org.apache.spark.sql.sources.v2 import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability} import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock class ContinuousExecution( @@ -93,7 +93,7 @@ class ContinuousExecution( } private val triggerExecutor = trigger match { - case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTime(t), triggerClock) + case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTimeTrigger(t), triggerClock) case _ => throw new IllegalStateException(s"Unsupported type of trigger: $trigger") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index d55f71c7be830..e1b7a8fc283d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -134,8 +134,10 @@ class RateStreamContinuousPartitionReader( nextReadTime += readTimeIncrement try { - while (System.currentTimeMillis < nextReadTime) { - Thread.sleep(nextReadTime - System.currentTimeMillis) + var toWaitMs = nextReadTime - System.currentTimeMillis + while (toWaitMs > 0) { + Thread.sleep(toWaitMs) + toWaitMs = nextReadTime - System.currentTimeMillis } } catch { case _: InterruptedException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala deleted file mode 100644 index bd343f3806034..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala +++ /dev/null @@ -1,57 +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.execution.streaming.continuous - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.spark.annotation.Evolving -import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at - * the specified interval. - */ -@Evolving -case class ContinuousTrigger(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -private[sql] object ContinuousTrigger { - def apply(interval: String): ContinuousTrigger = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ContinuousTrigger(TimeUnit.MICROSECONDS.toMillis(cal.microseconds)) - } - - def apply(interval: Duration): ContinuousTrigger = { - ContinuousTrigger(interval.toMillis) - } - - def create(interval: String): ContinuousTrigger = { - apply(interval) - } - - def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { - ContinuousTrigger(unit.toMillis(interval)) - } -} 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 5fa3fd0a37a65..72a197bdbcfc5 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 @@ -3932,7 +3932,7 @@ object functions { val anyTypeArgs = (0 to i).map(_ => "Any").mkString(", ") val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Defines a Java UDF$i instance as user-defined function (UDF). @@ -3944,8 +3944,8 @@ object functions { | * @since 2.3.0 | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { - | val func = f$anyCast.call($anyParams) - | SparkUserDefinedFunction($funcCall, returnType, inputSchemas = Seq.fill($i)(None)) + | val func = $funcCall + | SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill($i)(None)) |}""".stripMargin) } @@ -4145,8 +4145,8 @@ object functions { * @since 2.3.0 */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { - val func = f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction(() => func, returnType, inputSchemas = Seq.fill(0)(None)) + val func = () => f.asInstanceOf[UDF0[Any]].call() + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(0)(None)) } /** 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 8dc30eaa3a318..b05a5dfea3fff 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 @@ -170,7 +170,7 @@ abstract class BaseSessionStateBuilder( new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - DataSourceResolution(conf, session.catalog(_)) +: + DataSourceResolution(conf, this) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index 29500cf2afbc0..805f73dee141b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -30,7 +30,11 @@ private object MsSqlServerDialect extends JdbcDialect { // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients Option(StringType) } else { - None + sqlType match { + case java.sql.Types.SMALLINT => Some(ShortType) + case java.sql.Types.REAL => Some(FloatType) + case _ => None + } } } @@ -39,6 +43,7 @@ private object MsSqlServerDialect extends JdbcDialect { case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR)) case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT)) case BinaryType => Some(JdbcType("VARBINARY(MAX)", java.sql.Types.VARBINARY)) + case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 5be45c973a5f2..2645e4c9d528b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -73,14 +73,13 @@ private object PostgresDialect extends JdbcDialect { case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) case FloatType => Some(JdbcType("FLOAT4", Types.FLOAT)) case DoubleType => Some(JdbcType("FLOAT8", Types.DOUBLE)) - case ShortType => Some(JdbcType("SMALLINT", Types.SMALLINT)) + case ShortType | ByteType => Some(JdbcType("SMALLINT", Types.SMALLINT)) case t: DecimalType => Some( JdbcType(s"NUMERIC(${t.precision},${t.scale})", java.sql.Types.NUMERIC)) case ArrayType(et, _) if et.isInstanceOf[AtomicType] => getJDBCType(et).map(_.databaseTypeDefinition) .orElse(JdbcUtils.getCommonJDBCType(et).map(_.databaseTypeDefinition)) .map(typeName => JdbcType(s"$typeName[]", java.sql.Types.ARRAY)) - case ByteType => throw new IllegalArgumentException(s"Unsupported type in postgresql: $dt"); case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d051cf9c1d4a1..36104d7a70443 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider} import org.apache.spark.sql.sources.v2.TableCapability._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala deleted file mode 100644 index 417d698bdbb0e..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala +++ /dev/null @@ -1,133 +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.streaming - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.spark.annotation.Evolving -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A trigger that runs a query periodically based on the processing time. If `interval` is 0, - * the query will run as fast as possible. - * - * Scala Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * Java Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - */ -@Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -case class ProcessingTime(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -/** - * Used to create [[ProcessingTime]] triggers for [[StreamingQuery]]s. - * - * @since 2.0.0 - */ -@Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -object ProcessingTime { - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: String): ProcessingTime = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ProcessingTime(TimeUnit.MICROSECONDS.toMillis(cal.microseconds)) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: Duration): ProcessingTime = { - new ProcessingTime(interval.toMillis) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def create(interval: String): ProcessingTime = { - apply(interval) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval, unit) - */ - @deprecated("use Trigger.ProcessingTime(interval, unit)", "2.2.0") - def create(interval: Long, unit: TimeUnit): ProcessingTime = { - new ProcessingTime(unit.toMillis(interval)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 0b472c509e76d..abee5f6017df8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, ContinuousTrigger} +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS @@ -255,8 +255,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo val operationCheckEnabled = sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled - if (sparkSession.sessionState.conf.runtimeReoptimizationEnabled) { - logWarning(s"${SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key} " + + if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " + "is not supported in streaming DataFrames/Datasets and will be disabled.") } diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql new file mode 100644 index 0000000000000..2f2606d44d910 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql @@ -0,0 +1,115 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); + +-- CTE legacy substitution +SET spark.sql.legacy.ctePrecedence.enabled=true; + +-- CTE in CTE definition +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t; + +-- CTE in subquery +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in subquery expression +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in CTE definition shadows outer +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 2 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 3 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2; + +-- CTE in subquery shadows outer +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery shadows outer 2 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery shadows outer 3 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery expression shadows outer 2 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 3 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +); + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql index a81eca20e335c..801a16cf41f54 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql @@ -3,7 +3,7 @@ -- -- -- AGGREGATES [Part 1] --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/aggregates.sql#L1-L143 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 -- avoid bit-exact output here because operations may not be bit-exact. -- SET extra_float_digits = 0; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql index c4613701ec747..47f9d2f373069 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql @@ -3,7 +3,7 @@ -- -- -- AGGREGATES [Part 2] --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/aggregates.sql#L145-L350 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 create temporary view int4_tbl as select * from values (0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql index 8ba6f979c2e9c..fd0d299d7b0ff 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql @@ -3,7 +3,7 @@ -- -- -- BOOLEAN --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/boolean.sql +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/boolean.sql -- -- sanity check - if this fails go insane! @@ -17,19 +17,20 @@ SELECT 1 AS one; SELECT true AS true; -SELECT false AS false; +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT false AS `false`; SELECT boolean('t') AS true; -- [SPARK-27931] Trim the string when cast string type to boolean type -SELECT boolean(' f ') AS false; +SELECT boolean(' f ') AS `false`; SELECT boolean('true') AS true; -- [SPARK-27923] PostgreSQL does not accept 'test' but Spark SQL accepts it and sets it to NULL SELECT boolean('test') AS error; -SELECT boolean('false') AS false; +SELECT boolean('false') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'foo' but Spark SQL accepts it and sets it to NULL SELECT boolean('foo') AS error; @@ -41,9 +42,9 @@ SELECT boolean('yes') AS true; -- [SPARK-27923] PostgreSQL does not accept 'yeah' but Spark SQL accepts it and sets it to NULL SELECT boolean('yeah') AS error; -SELECT boolean('n') AS false; +SELECT boolean('n') AS `false`; -SELECT boolean('no') AS false; +SELECT boolean('no') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL SELECT boolean('nay') AS error; @@ -51,10 +52,10 @@ SELECT boolean('nay') AS error; -- [SPARK-27931] Accept 'on' and 'off' as input for boolean data type SELECT boolean('on') AS true; -SELECT boolean('off') AS false; +SELECT boolean('off') AS `false`; -- [SPARK-27931] Accept unique prefixes thereof -SELECT boolean('of') AS false; +SELECT boolean('of') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL SELECT boolean('o') AS error; @@ -70,7 +71,7 @@ SELECT boolean('1') AS true; -- [SPARK-27923] PostgreSQL does not accept '11' but Spark SQL accepts it and sets it to NULL SELECT boolean('11') AS error; -SELECT boolean('0') AS false; +SELECT boolean('0') AS `false`; -- [SPARK-27923] PostgreSQL does not accept '000' but Spark SQL accepts it and sets it to NULL SELECT boolean('000') AS error; @@ -82,11 +83,11 @@ SELECT boolean('') AS error; SELECT boolean('t') or boolean('f') AS true; -SELECT boolean('t') and boolean('f') AS false; +SELECT boolean('t') and boolean('f') AS `false`; SELECT not boolean('f') AS true; -SELECT boolean('t') = boolean('f') AS false; +SELECT boolean('t') = boolean('f') AS `false`; SELECT boolean('t') <> boolean('f') AS true; @@ -99,11 +100,11 @@ SELECT boolean('f') < boolean('t') AS true; SELECT boolean('f') <= boolean('t') AS true; -- explicit casts to/from text -SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS false; +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; -- [SPARK-27931] Trim the string when cast to boolean type SELECT boolean(string(' true ')) AS true, - boolean(string(' FALSE')) AS false; -SELECT string(boolean(true)) AS true, string(boolean(false)) AS false; + boolean(string(' FALSE')) AS `false`; +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; -- [SPARK-27923] PostgreSQL does not accept ' tru e ' but Spark SQL accepts it and sets it to NULL SELECT boolean(string(' tru e ')) AS invalid; -- error diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql index 4d45537898c1e..6d9c44c67a96b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql @@ -3,12 +3,9 @@ -- -- -- CASE --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/case.sql +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql -- Test the CASE statement -- --- This test suite contains two Cartesian products without using explicit CROSS JOIN syntax. --- Thus, we set spark.sql.crossJoin.enabled to true. -set spark.sql.crossJoin.enabled=true; CREATE TABLE CASE_TBL ( i integer, f double @@ -264,4 +261,3 @@ SELECT CASE DROP TABLE CASE_TBL; DROP TABLE CASE2_TBL; -set spark.sql.crossJoin.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql index 9e684d13d7f65..3dad5cd56ba02 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql @@ -3,7 +3,7 @@ -- -- -- FLOAT4 --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/float4.sql +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float4.sql CREATE TABLE FLOAT4_TBL (f1 float) USING parquet; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql new file mode 100644 index 0000000000000..6f8e3b596e60e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql @@ -0,0 +1,500 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT8 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql + +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; + +INSERT INTO FLOAT8_TBL VALUES (' 0.0 '); +INSERT INTO FLOAT8_TBL VALUES ('1004.30 '); +INSERT INTO FLOAT8_TBL VALUES (' -34.84'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200'); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for underflow and overflow handling +SELECT double('10e400'); +SELECT double('-10e400'); +SELECT double('10e-400'); +SELECT double('-10e-400'); + +-- [SPARK-28061] Support for converting float to binary format +-- test smallest normalized input +-- SELECT float8send('2.2250738585072014E-308'::float8); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT8_TBL VALUES (''); +-- INSERT INTO FLOAT8_TBL VALUES (' '); +-- INSERT INTO FLOAT8_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT8_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT8_TBL VALUES (' - 3'); +-- INSERT INTO FLOAT8_TBL VALUES ('123 5'); + +-- special inputs +SELECT double('NaN'); +-- [SPARK-28060] Double type can not accept some special inputs +SELECT double('nan'); +SELECT double(' NAN '); +SELECT double('infinity'); +SELECT double(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT double('N A N'); +SELECT double('NaN x'); +SELECT double(' INFINITY x'); + +SELECT double('Infinity') + 100.0; +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +SELECT double('Infinity') / double('Infinity'); +SELECT double('NaN') / double('NaN'); +-- [SPARK-28315] Decimal can not accept NaN as input +SELECT double(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS one, f.f1 ^ '2.0' AS square_f1 +-- FROM FLOAT8_TBL f where f.f1 = '1004.3'; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- absolute value +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 +-- FROM FLOAT8_TBL f; + +-- [SPARK-23906] Support Truncate number +-- truncate +-- SELECT '' AS five, f.f1, trunc(f.f1) AS trunc_f1 +-- FROM FLOAT8_TBL f; + +-- round +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f; + +-- [SPARK-28135] ceil/ceiling/floor returns incorrect values +-- ceil / ceiling +select ceil(f1) as ceil_f1 from float8_tbl f; +select ceiling(f1) as ceiling_f1 from float8_tbl f; + +-- floor +select floor(f1) as floor_f1 from float8_tbl f; + +-- sign +select sign(f1) as sign_f1 from float8_tbl f; + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- square root +SELECT sqrt(double('64')) AS eight; + +-- [SPARK-28027] Spark SQL does not support prefix operator |/ +-- SELECT |/ float8 '64' AS eight; + +-- SELECT '' AS three, f.f1, |/f.f1 AS sqrt_f1 +-- FROM FLOAT8_TBL f +-- WHERE f.f1 > '0.0'; + +-- power +SELECT power(double('144'), double('0.5')); +SELECT power(double('NaN'), double('0.5')); +SELECT power(double('144'), double('NaN')); +SELECT power(double('NaN'), double('NaN')); +SELECT power(double('-1'), double('NaN')); +-- [SPARK-28135] power returns incorrect values +SELECT power(double('1'), double('NaN')); +SELECT power(double('NaN'), double('0')); + +-- take exp of ln(f.f1) +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-28027] Spark SQL does not support prefix operator ||/ +-- cube root +-- SELECT ||/ float8 '27' AS three; + +-- SELECT '' AS five, f.f1, ||/f.f1 AS cbrt_f1 FROM FLOAT8_TBL f; + + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- UPDATE FLOAT8_TBL +-- SET f1 = FLOAT8_TBL.f1 * '-1' +-- WHERE FLOAT8_TBL.f1 > '0.0'; +-- Update the FLOAT8_TBL to UPDATED_FLOAT8_TBL +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL; + +-- [SPARK-27923] Out of range, Spark SQL returns Infinity +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS bad, f.f1 ^ '1e200' from UPDATED_FLOAT8_TBL f; + +-- SELECT 0 ^ 0 + 0 ^ 1 + 0 ^ 0.0 + 0 ^ 0.5; + +-- [SPARK-27923] Cannot take logarithm of zero +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 = '0.0' ; + +-- [SPARK-27923] Cannot take logarithm of a negative number +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 < '0.0' ; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT '' AS bad, exp(f.f1) from UPDATED_FLOAT8_TBL f; + +-- [SPARK-27923] Divide by zero, Spark SQL returns NULL +-- SELECT '' AS bad, f.f1 / '0.0' from UPDATED_FLOAT8_TBL f; + +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL; + +-- hyperbolic functions +-- we run these with extra_float_digits = 0 too, since different platforms +-- tend to produce results that vary in the last place. +SELECT sinh(double('1')); +SELECT cosh(double('1')); +SELECT tanh(double('1')); +SELECT asinh(double('1')); +SELECT acosh(double('2')); +SELECT atanh(double('0.5')); +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +-- test Inf/NaN cases for hyperbolic functions +SELECT sinh(double('Infinity')); +SELECT sinh(double('-Infinity')); +SELECT sinh(double('NaN')); +SELECT cosh(double('Infinity')); +SELECT cosh(double('-Infinity')); +SELECT cosh(double('NaN')); +SELECT tanh(double('Infinity')); +SELECT tanh(double('-Infinity')); +SELECT tanh(double('NaN')); +SELECT asinh(double('Infinity')); +SELECT asinh(double('-Infinity')); +SELECT asinh(double('NaN')); +-- acosh(Inf) should be Inf, but some mingw versions produce NaN, so skip test +SELECT acosh(double('Infinity')); +SELECT acosh(double('-Infinity')); +SELECT acosh(double('NaN')); +SELECT atanh(double('Infinity')); +SELECT atanh(double('-Infinity')); +SELECT atanh(double('NaN')); + +-- RESET extra_float_digits; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over- and underflow +-- INSERT INTO FLOAT8_TBL VALUES ('10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('10e-400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e-400'); + +-- maintain external table consistency across platforms +-- delete all values and reinsert well-behaved ones + +TRUNCATE TABLE FLOAT8_TBL; + +INSERT INTO FLOAT8_TBL VALUES ('0.0'); + +INSERT INTO FLOAT8_TBL VALUES ('-34.84'); + +INSERT INTO FLOAT8_TBL VALUES ('-1004.30'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200'); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(double('32767.4')); +SELECT smallint(double('32767.6')); +SELECT smallint(double('-32768.4')); +SELECT smallint(double('-32768.6')); +SELECT int(double('2147483647.4')); +SELECT int(double('2147483647.6')); +SELECT int(double('-2147483648.4')); +SELECT int(double('-2147483648.6')); +SELECT bigint(double('9223372036854773760')); +SELECT bigint(double('9223372036854775807')); +SELECT bigint(double('-9223372036854775808.5')); +SELECT bigint(double('-9223372036854780000')); + +-- [SPARK-28134] Missing Trigonometric Functions +-- test exact cases for trigonometric functions in degrees + +-- SELECT x, +-- sind(x), +-- sind(x) IN (-1,-0.5,0,0.5,1) AS sind_exact +-- FROM (VALUES (0), (30), (90), (150), (180), +-- (210), (270), (330), (360)) AS t(x); + +-- SELECT x, +-- cosd(x), +-- cosd(x) IN (-1,-0.5,0,0.5,1) AS cosd_exact +-- FROM (VALUES (0), (60), (90), (120), (180), +-- (240), (270), (300), (360)) AS t(x); + +-- SELECT x, +-- tand(x), +-- tand(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS tand_exact, +-- cotd(x), +-- cotd(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS cotd_exact +-- FROM (VALUES (0), (45), (90), (135), (180), +-- (225), (270), (315), (360)) AS t(x); + +-- SELECT x, +-- asind(x), +-- asind(x) IN (-90,-30,0,30,90) AS asind_exact, +-- acosd(x), +-- acosd(x) IN (0,60,90,120,180) AS acosd_exact +-- FROM (VALUES (-1), (-0.5), (0), (0.5), (1)) AS t(x); + +-- SELECT x, +-- atand(x), +-- atand(x) IN (-90,-45,0,45,90) AS atand_exact +-- FROM (VALUES ('-Infinity'::float8), (-1), (0), (1), +-- ('Infinity'::float8)) AS t(x); + +-- SELECT x, y, +-- atan2d(y, x), +-- atan2d(y, x) IN (-90,0,90,180) AS atan2d_exact +-- FROM (SELECT 10*cosd(a), 10*sind(a) +-- FROM generate_series(0, 360, 90) AS t(a)) AS t(x,y); + +-- We do not support creating types, skip the test below +-- +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat8; +-- create function xfloat8in(cstring) returns xfloat8 immutable strict +-- language internal as 'int8in'; +-- create function xfloat8out(xfloat8) returns cstring immutable strict +-- language internal as 'int8out'; +-- create type xfloat8 (input = xfloat8in, output = xfloat8out, like = float8); +-- create cast (xfloat8 as float8) without function; +-- create cast (float8 as xfloat8) without function; +-- create cast (xfloat8 as bigint) without function; +-- create cast (bigint as xfloat8) without function; + +-- float8: seeeeeee eeeeeeee eeeeeeee mmmmmmmm mmmmmmmm(x4) + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'0000000000000001'), +-- (x'0000000000000002'), (x'0000000000000003'), +-- (x'0000000000001000'), (x'0000000100000000'), +-- (x'0000010000000000'), (x'0000010100000000'), +-- (x'0000400000000000'), (x'0000400100000000'), +-- (x'0000800000000000'), (x'0000800000000001'), +-- -- these values taken from upstream testsuite +-- (x'00000000000f4240'), +-- (x'00000000016e3600'), +-- (x'0000008cdcdea440'), +-- -- borderline between subnormal and normal +-- (x'000ffffffffffff0'), (x'000ffffffffffff1'), +-- (x'000ffffffffffffe'), (x'000fffffffffffff')) +-- select float8send(flt) as ibits, +-- flt +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- round-trip tests + +-- with testdata(bits) as (values +-- (x'0000000000000000'), +-- -- smallest normal values +-- (x'0010000000000000'), (x'0010000000000001'), +-- (x'0010000000000002'), (x'0018000000000000'), +-- -- +-- (x'3ddb7cdfd9d7bdba'), (x'3ddb7cdfd9d7bdbb'), (x'3ddb7cdfd9d7bdbc'), +-- (x'3e112e0be826d694'), (x'3e112e0be826d695'), (x'3e112e0be826d696'), +-- (x'3e45798ee2308c39'), (x'3e45798ee2308c3a'), (x'3e45798ee2308c3b'), +-- (x'3e7ad7f29abcaf47'), (x'3e7ad7f29abcaf48'), (x'3e7ad7f29abcaf49'), +-- (x'3eb0c6f7a0b5ed8c'), (x'3eb0c6f7a0b5ed8d'), (x'3eb0c6f7a0b5ed8e'), +-- (x'3ee4f8b588e368ef'), (x'3ee4f8b588e368f0'), (x'3ee4f8b588e368f1'), +-- (x'3f1a36e2eb1c432c'), (x'3f1a36e2eb1c432d'), (x'3f1a36e2eb1c432e'), +-- (x'3f50624dd2f1a9fb'), (x'3f50624dd2f1a9fc'), (x'3f50624dd2f1a9fd'), +-- (x'3f847ae147ae147a'), (x'3f847ae147ae147b'), (x'3f847ae147ae147c'), +-- (x'3fb9999999999999'), (x'3fb999999999999a'), (x'3fb999999999999b'), +-- -- values very close to 1 +-- (x'3feffffffffffff0'), (x'3feffffffffffff1'), (x'3feffffffffffff2'), +-- (x'3feffffffffffff3'), (x'3feffffffffffff4'), (x'3feffffffffffff5'), +-- (x'3feffffffffffff6'), (x'3feffffffffffff7'), (x'3feffffffffffff8'), +-- (x'3feffffffffffff9'), (x'3feffffffffffffa'), (x'3feffffffffffffb'), +-- (x'3feffffffffffffc'), (x'3feffffffffffffd'), (x'3feffffffffffffe'), +-- (x'3fefffffffffffff'), +-- (x'3ff0000000000000'), +-- (x'3ff0000000000001'), (x'3ff0000000000002'), (x'3ff0000000000003'), +-- (x'3ff0000000000004'), (x'3ff0000000000005'), (x'3ff0000000000006'), +-- (x'3ff0000000000007'), (x'3ff0000000000008'), (x'3ff0000000000009'), +-- -- +-- (x'3ff921fb54442d18'), +-- (x'4005bf0a8b14576a'), +-- (x'400921fb54442d18'), +-- -- +-- (x'4023ffffffffffff'), (x'4024000000000000'), (x'4024000000000001'), +-- (x'4058ffffffffffff'), (x'4059000000000000'), (x'4059000000000001'), +-- (x'408f3fffffffffff'), (x'408f400000000000'), (x'408f400000000001'), +-- (x'40c387ffffffffff'), (x'40c3880000000000'), (x'40c3880000000001'), +-- (x'40f869ffffffffff'), (x'40f86a0000000000'), (x'40f86a0000000001'), +-- (x'412e847fffffffff'), (x'412e848000000000'), (x'412e848000000001'), +-- (x'416312cfffffffff'), (x'416312d000000000'), (x'416312d000000001'), +-- (x'4197d783ffffffff'), (x'4197d78400000000'), (x'4197d78400000001'), +-- (x'41cdcd64ffffffff'), (x'41cdcd6500000000'), (x'41cdcd6500000001'), +-- (x'4202a05f1fffffff'), (x'4202a05f20000000'), (x'4202a05f20000001'), +-- (x'42374876e7ffffff'), (x'42374876e8000000'), (x'42374876e8000001'), +-- (x'426d1a94a1ffffff'), (x'426d1a94a2000000'), (x'426d1a94a2000001'), +-- (x'42a2309ce53fffff'), (x'42a2309ce5400000'), (x'42a2309ce5400001'), +-- (x'42d6bcc41e8fffff'), (x'42d6bcc41e900000'), (x'42d6bcc41e900001'), +-- (x'430c6bf52633ffff'), (x'430c6bf526340000'), (x'430c6bf526340001'), +-- (x'4341c37937e07fff'), (x'4341c37937e08000'), (x'4341c37937e08001'), +-- (x'4376345785d89fff'), (x'4376345785d8a000'), (x'4376345785d8a001'), +-- (x'43abc16d674ec7ff'), (x'43abc16d674ec800'), (x'43abc16d674ec801'), +-- (x'43e158e460913cff'), (x'43e158e460913d00'), (x'43e158e460913d01'), +-- (x'4415af1d78b58c3f'), (x'4415af1d78b58c40'), (x'4415af1d78b58c41'), +-- (x'444b1ae4d6e2ef4f'), (x'444b1ae4d6e2ef50'), (x'444b1ae4d6e2ef51'), +-- (x'4480f0cf064dd591'), (x'4480f0cf064dd592'), (x'4480f0cf064dd593'), +-- (x'44b52d02c7e14af5'), (x'44b52d02c7e14af6'), (x'44b52d02c7e14af7'), +-- (x'44ea784379d99db3'), (x'44ea784379d99db4'), (x'44ea784379d99db5'), +-- (x'45208b2a2c280290'), (x'45208b2a2c280291'), (x'45208b2a2c280292'), +-- -- +-- (x'7feffffffffffffe'), (x'7fefffffffffffff'), +-- -- round to even tests (+ve) +-- (x'4350000000000002'), +-- (x'4350000000002e06'), +-- (x'4352000000000003'), +-- (x'4352000000000004'), +-- (x'4358000000000003'), +-- (x'4358000000000004'), +-- (x'435f000000000020'), +-- -- round to even tests (-ve) +-- (x'c350000000000002'), +-- (x'c350000000002e06'), +-- (x'c352000000000003'), +-- (x'c352000000000004'), +-- (x'c358000000000003'), +-- (x'c358000000000004'), +-- (x'c35f000000000020'), +-- -- exercise fixed-point memmoves +-- (x'42dc12218377de66'), +-- (x'42a674e79c5fe51f'), +-- (x'4271f71fb04cb74c'), +-- (x'423cbe991a145879'), +-- (x'4206fee0e1a9e061'), +-- (x'41d26580b487e6b4'), +-- (x'419d6f34540ca453'), +-- (x'41678c29dcd6e9dc'), +-- (x'4132d687e3df217d'), +-- (x'40fe240c9fcb68c8'), +-- (x'40c81cd6e63c53d3'), +-- (x'40934a4584fd0fdc'), +-- (x'405edd3c07fb4c93'), +-- (x'4028b0fcd32f7076'), +-- (x'3ff3c0ca428c59f8'), +-- -- these cases come from the upstream's testsuite +-- -- LotsOfTrailingZeros) +-- (x'3e60000000000000'), +-- -- Regression +-- (x'c352bd2668e077c4'), +-- (x'434018601510c000'), +-- (x'43d055dc36f24000'), +-- (x'43e052961c6f8000'), +-- (x'3ff3c0ca2a5b1d5d'), +-- -- LooksLikePow5 +-- (x'4830f0cf064dd592'), +-- (x'4840f0cf064dd592'), +-- (x'4850f0cf064dd592'), +-- -- OutputLength +-- (x'3ff3333333333333'), +-- (x'3ff3ae147ae147ae'), +-- (x'3ff3be76c8b43958'), +-- (x'3ff3c083126e978d'), +-- (x'3ff3c0c1fc8f3238'), +-- (x'3ff3c0c9539b8887'), +-- (x'3ff3c0ca2a5b1d5d'), +-- (x'3ff3c0ca4283de1b'), +-- (x'3ff3c0ca43db770a'), +-- (x'3ff3c0ca428abd53'), +-- (x'3ff3c0ca428c1d2b'), +-- (x'3ff3c0ca428c51f2'), +-- (x'3ff3c0ca428c58fc'), +-- (x'3ff3c0ca428c59dd'), +-- (x'3ff3c0ca428c59f8'), +-- (x'3ff3c0ca428c59fb'), +-- -- 32-bit chunking +-- (x'40112e0be8047a7d'), +-- (x'40112e0be815a889'), +-- (x'40112e0be826d695'), +-- (x'40112e0be83804a1'), +-- (x'40112e0be84932ad'), +-- -- MinMaxShift +-- (x'0040000000000000'), +-- (x'007fffffffffffff'), +-- (x'0290000000000000'), +-- (x'029fffffffffffff'), +-- (x'4350000000000000'), +-- (x'435fffffffffffff'), +-- (x'1330000000000000'), +-- (x'133fffffffffffff'), +-- (x'3a6fa7161a4d6e0c') +-- ) +-- select float8send(flt) as ibits, +-- flt, +-- flt::text::float8 as r_flt, +-- float8send(flt::text::float8) as obits, +-- float8send(flt::text::float8) = float8send(flt) as correct +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat8 cascade; +DROP TABLE FLOAT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql new file mode 100644 index 0000000000000..f64ec5d75afcf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql @@ -0,0 +1,127 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT2 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int2.sql + +CREATE TABLE INT2_TBL(f1 smallint) USING parquet; + +-- [SPARK-28023] Trim the string when cast string type to other types +INSERT INTO INT2_TBL VALUES (trim('0 ')); + +INSERT INTO INT2_TBL VALUES (trim(' 1234 ')); + +INSERT INTO INT2_TBL VALUES (trim(' -1234')); + +-- [SPARK-27923] Invalid input syntax for type short throws exception at PostgreSQL +-- INSERT INTO INT2_TBL VALUES ('34.5'); + +-- largest and smallest values +INSERT INTO INT2_TBL VALUES ('32767'); + +INSERT INTO INT2_TBL VALUES ('-32767'); + +-- bad input values -- should give errors +-- INSERT INTO INT2_TBL VALUES ('100000'); +-- INSERT INTO INT2_TBL VALUES ('asdf'); +-- INSERT INTO INT2_TBL VALUES (' '); +-- INSERT INTO INT2_TBL VALUES ('- 1234'); +-- INSERT INTO INT2_TBL VALUES ('4 444'); +-- INSERT INTO INT2_TBL VALUES ('123 dt'); +-- INSERT INTO INT2_TBL VALUES (''); + + +SELECT '' AS five, * FROM INT2_TBL; + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0'); + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0'); + +-- positive odds +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1'); + +-- any evens +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0'); + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384; + +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766; + +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767; + +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; + +-- corner cases +SELECT string(shiftleft(smallint(-1), 15)); +SELECT string(smallint(shiftleft(smallint(-1), 15))+1); + +-- check sane handling of INT16_MIN overflow cases +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT smallint((-32768)) * smallint(-1); +-- SELECT smallint(-32768) / smallint(-1); +SELECT smallint(-32768) % smallint(-1); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from numeric +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT2_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql index 89cac00228f7f..86432a845b6e1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql @@ -3,7 +3,7 @@ -- -- -- INT4 --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/int4.sql +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql -- CREATE TABLE INT4_TBL(f1 int) USING parquet; @@ -125,7 +125,8 @@ SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true; SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true; -SELECT int('1000') < int('999') AS false; +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT int('1000') < int('999') AS `false`; -- [SPARK-28027] Our ! and !! has different meanings -- SELECT 4! AS twenty_four; @@ -134,7 +135,6 @@ SELECT int('1000') < int('999') AS false; SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; --- [SPARK-2659] HiveQL: Division operator should always perform fractional division SELECT 2 + 2 / 2 AS three; SELECT (2 + 2) / 2 AS two; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql new file mode 100644 index 0000000000000..d29bf3bfad4ca --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql @@ -0,0 +1,239 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT8 +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql +-- +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; + +INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')); +INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789'); +INSERT INTO INT8_TBL VALUES('4567890123456789','123'); +INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789'); +INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789'); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad inputs +-- INSERT INTO INT8_TBL(q1) VALUES (' '); +-- INSERT INTO INT8_TBL(q1) VALUES ('xxx'); +-- INSERT INTO INT8_TBL(q1) VALUES ('3908203590239580293850293850329485'); +-- INSERT INTO INT8_TBL(q1) VALUES ('-1204982019841029840928340329840934'); +-- INSERT INTO INT8_TBL(q1) VALUES ('- 123'); +-- INSERT INTO INT8_TBL(q1) VALUES (' 345 5'); +-- INSERT INTO INT8_TBL(q1) VALUES (''); + +SELECT * FROM INT8_TBL; + +-- int8/int8 cmp +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789; + +-- int8/int4 cmp +SELECT * FROM INT8_TBL WHERE q2 = 456; +SELECT * FROM INT8_TBL WHERE q2 <> 456; +SELECT * FROM INT8_TBL WHERE q2 < 456; +SELECT * FROM INT8_TBL WHERE q2 > 456; +SELECT * FROM INT8_TBL WHERE q2 <= 456; +SELECT * FROM INT8_TBL WHERE q2 >= 456; + +-- int4/int8 cmp +SELECT * FROM INT8_TBL WHERE 123 = q1; +SELECT * FROM INT8_TBL WHERE 123 <> q1; +SELECT * FROM INT8_TBL WHERE 123 < q1; +SELECT * FROM INT8_TBL WHERE 123 > q1; +SELECT * FROM INT8_TBL WHERE 123 <= q1; +SELECT * FROM INT8_TBL WHERE 123 >= q1; + +-- int8/int2 cmp +SELECT * FROM INT8_TBL WHERE q2 = smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 < smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 > smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456'); + +-- int2/int8 cmp +SELECT * FROM INT8_TBL WHERE smallint('123') = q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1; +SELECT * FROM INT8_TBL WHERE smallint('123') < q1; +SELECT * FROM INT8_TBL WHERE smallint('123') > q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1; +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1; + + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL; + +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL; +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000); +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL; + +SELECT '' AS five, q1, double(q1) FROM INT8_TBL; +SELECT '' AS five, q2, double(q2) FROM INT8_TBL; + +SELECT 37 + q1 AS plus4 FROM INT8_TBL; +SELECT 37 - q1 AS minus4 FROM INT8_TBL; +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; + +-- int8 op int4 +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; +-- int4 op int8 +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL; + +-- int8 op int2 +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL; +-- int2 op int8 +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL; + +SELECT q2, abs(q2) FROM INT8_TBL; +SELECT min(q1), min(q2) FROM INT8_TBL; +SELECT max(q1), max(q2) FROM INT8_TBL; + +-- [SPARK-28137] Missing Data Type Formatting Functions +-- TO_CHAR() +-- +-- SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), to_char(q2, '9,999,999,999,999,999.999,999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_6, to_char(q2, 'FMS9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_7, to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL; +-- SELECT '' AS to_char_8, to_char(q2, 'SG9999999999999999th') FROM INT8_TBL; +-- SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM INT8_TBL; +-- SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9') FROM INT8_TBL; +-- SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM INT8_TBL; +-- SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999') FROM INT8_TBL; + +-- [SPARK-26218] Throw exception on overflow for integers +-- check min/max values and overflow behavior + +-- select bigint('-9223372036854775808'); +-- select bigint('-9223372036854775809'); +-- select bigint('9223372036854775807'); +-- select bigint('9223372036854775808'); + +-- select bigint('9223372036854775808'); + +-- select -(bigint('-9223372036854775807')); +-- select -(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + bigint('9223372036854775800'); +-- select bigint('-9223372036854775800') + bigint('-9223372036854775800'); + +-- select bigint('9223372036854775800') - bigint('-9223372036854775800'); +-- select bigint('-9223372036854775800') - bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') * bigint('9223372036854775800'); + +select bigint('9223372036854775800') / bigint('0'); +-- select bigint('9223372036854775800') % bigint('0'); + +-- select abs(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + int('100'); +-- select bigint('-9223372036854775800') - int('100'); +-- select bigint('9223372036854775800') * int('100'); + +-- select int('100') + bigint('9223372036854775800'); +-- select int('-100') - bigint('9223372036854775800'); +-- select int('100') * bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') + smallint('100'); +-- select bigint('-9223372036854775800') - smallint('100'); +-- select bigint('9223372036854775800') * smallint('100'); +select bigint('-9223372036854775808') / smallint('0'); + +-- select smallint('100') + bigint('9223372036854775800'); +-- select smallint('-100') - bigint('9223372036854775800'); +-- select smallint('100') * bigint('9223372036854775800'); +select smallint('100') / bigint('0'); + +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint); + +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL; +SELECT CAST(float('36854775807.0') AS bigint); +SELECT CAST(double('922337203685477580700.0') AS bigint); + + +-- [SPARK-28027] Missing some mathematical operators +-- bit operations + +-- SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, q1 # q2 AS `xor`, ~q1 AS `not` FROM INT8_TBL; +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL; +-- SELECT q1, q1 << 2 AS `shl`, q1 >> 3 AS `shr` FROM INT8_TBL; + + +-- generate_series + +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2); + +-- corner case +SELECT string(shiftleft(bigint(-1), 63)); +SELECT string(int(shiftleft(bigint(-1), 63))+1); + +-- [SPARK-26218] Throw exception on overflow for integers +-- check sane handling of INT64_MIN overflow cases +SELECT bigint((-9223372036854775808)) * bigint((-1)); +SELECT bigint((-9223372036854775808)) / bigint((-1)); +SELECT bigint((-9223372036854775808)) % bigint((-1)); +SELECT bigint((-9223372036854775808)) * int((-1)); +SELECT bigint((-9223372036854775808)) / int((-1)); +SELECT bigint((-9223372036854775808)) % int((-1)); +SELECT bigint((-9223372036854775808)) * smallint((-1)); +SELECT bigint((-9223372036854775808)) / smallint((-1)); +SELECT bigint((-9223372036854775808)) % smallint((-1)); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x); + +-- check rounding when casting from numeric +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql new file mode 100644 index 0000000000000..0fe47a4d583ef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql @@ -0,0 +1,1096 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- NUMERIC +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/numeric.sql +-- + +-- [SPARK-28318] Decimal can only support precision up to 38. We rewrite numeric(210,10) to decimal(38,10). +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet; +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet; + +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet; + + +-- ****************************** +-- * The following EXPECTED results are computed by bc(1) +-- * with a scale of 200 +-- ****************************** + +-- BEGIN TRANSACTION; +INSERT INTO num_exp_add VALUES (0,0,'0'); +INSERT INTO num_exp_sub VALUES (0,0,'0'); +INSERT INTO num_exp_mul VALUES (0,0,'0'); +INSERT INTO num_exp_div VALUES (0,0,'NaN'); +INSERT INTO num_exp_add VALUES (0,1,'0'); +INSERT INTO num_exp_sub VALUES (0,1,'0'); +INSERT INTO num_exp_mul VALUES (0,1,'0'); +INSERT INTO num_exp_div VALUES (0,1,'NaN'); +INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047'); +INSERT INTO num_exp_mul VALUES (0,2,'0'); +INSERT INTO num_exp_div VALUES (0,2,'0'); +INSERT INTO num_exp_add VALUES (0,3,'4.31'); +INSERT INTO num_exp_sub VALUES (0,3,'-4.31'); +INSERT INTO num_exp_mul VALUES (0,3,'0'); +INSERT INTO num_exp_div VALUES (0,3,'0'); +INSERT INTO num_exp_add VALUES (0,4,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119'); +INSERT INTO num_exp_mul VALUES (0,4,'0'); +INSERT INTO num_exp_div VALUES (0,4,'0'); +INSERT INTO num_exp_add VALUES (0,5,'16397.038491'); +INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491'); +INSERT INTO num_exp_mul VALUES (0,5,'0'); +INSERT INTO num_exp_div VALUES (0,5,'0'); +INSERT INTO num_exp_add VALUES (0,6,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026'); +INSERT INTO num_exp_mul VALUES (0,6,'0'); +INSERT INTO num_exp_div VALUES (0,6,'0'); +INSERT INTO num_exp_add VALUES (0,7,'-83028485'); +INSERT INTO num_exp_sub VALUES (0,7,'83028485'); +INSERT INTO num_exp_mul VALUES (0,7,'0'); +INSERT INTO num_exp_div VALUES (0,7,'0'); +INSERT INTO num_exp_add VALUES (0,8,'74881'); +INSERT INTO num_exp_sub VALUES (0,8,'-74881'); +INSERT INTO num_exp_mul VALUES (0,8,'0'); +INSERT INTO num_exp_div VALUES (0,8,'0'); +INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420'); +INSERT INTO num_exp_mul VALUES (0,9,'0'); +INSERT INTO num_exp_div VALUES (0,9,'0'); +INSERT INTO num_exp_add VALUES (1,0,'0'); +INSERT INTO num_exp_sub VALUES (1,0,'0'); +INSERT INTO num_exp_mul VALUES (1,0,'0'); +INSERT INTO num_exp_div VALUES (1,0,'NaN'); +INSERT INTO num_exp_add VALUES (1,1,'0'); +INSERT INTO num_exp_sub VALUES (1,1,'0'); +INSERT INTO num_exp_mul VALUES (1,1,'0'); +INSERT INTO num_exp_div VALUES (1,1,'NaN'); +INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047'); +INSERT INTO num_exp_mul VALUES (1,2,'0'); +INSERT INTO num_exp_div VALUES (1,2,'0'); +INSERT INTO num_exp_add VALUES (1,3,'4.31'); +INSERT INTO num_exp_sub VALUES (1,3,'-4.31'); +INSERT INTO num_exp_mul VALUES (1,3,'0'); +INSERT INTO num_exp_div VALUES (1,3,'0'); +INSERT INTO num_exp_add VALUES (1,4,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119'); +INSERT INTO num_exp_mul VALUES (1,4,'0'); +INSERT INTO num_exp_div VALUES (1,4,'0'); +INSERT INTO num_exp_add VALUES (1,5,'16397.038491'); +INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491'); +INSERT INTO num_exp_mul VALUES (1,5,'0'); +INSERT INTO num_exp_div VALUES (1,5,'0'); +INSERT INTO num_exp_add VALUES (1,6,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026'); +INSERT INTO num_exp_mul VALUES (1,6,'0'); +INSERT INTO num_exp_div VALUES (1,6,'0'); +INSERT INTO num_exp_add VALUES (1,7,'-83028485'); +INSERT INTO num_exp_sub VALUES (1,7,'83028485'); +INSERT INTO num_exp_mul VALUES (1,7,'0'); +INSERT INTO num_exp_div VALUES (1,7,'0'); +INSERT INTO num_exp_add VALUES (1,8,'74881'); +INSERT INTO num_exp_sub VALUES (1,8,'-74881'); +INSERT INTO num_exp_mul VALUES (1,8,'0'); +INSERT INTO num_exp_div VALUES (1,8,'0'); +INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420'); +INSERT INTO num_exp_mul VALUES (1,9,'0'); +INSERT INTO num_exp_div VALUES (1,9,'0'); +INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047'); +INSERT INTO num_exp_mul VALUES (2,0,'0'); +INSERT INTO num_exp_div VALUES (2,0,'NaN'); +INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047'); +INSERT INTO num_exp_mul VALUES (2,1,'0'); +INSERT INTO num_exp_div VALUES (2,1,'NaN'); +INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094'); +INSERT INTO num_exp_sub VALUES (2,2,'0'); +INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209'); +INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047'); +INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047'); +INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257'); +INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266'); +INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047'); +INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047'); +INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593'); +INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685'); +INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047'); +INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047'); +INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077'); +INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429'); +INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787'); +INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307'); +INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222'); +INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940'); +INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047'); +INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953'); +INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795'); +INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518'); +INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047'); +INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047'); +INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407'); +INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476'); +INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467'); +INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627'); +INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740'); +INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811'); +INSERT INTO num_exp_add VALUES (3,0,'4.31'); +INSERT INTO num_exp_sub VALUES (3,0,'4.31'); +INSERT INTO num_exp_mul VALUES (3,0,'0'); +INSERT INTO num_exp_div VALUES (3,0,'NaN'); +INSERT INTO num_exp_add VALUES (3,1,'4.31'); +INSERT INTO num_exp_sub VALUES (3,1,'4.31'); +INSERT INTO num_exp_mul VALUES (3,1,'0'); +INSERT INTO num_exp_div VALUES (3,1,'NaN'); +INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047'); +INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047'); +INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257'); +INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352'); +INSERT INTO num_exp_add VALUES (3,3,'8.62'); +INSERT INTO num_exp_sub VALUES (3,3,'0'); +INSERT INTO num_exp_mul VALUES (3,3,'18.5761'); +INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (3,4,'7799465.7219'); +INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019'); +INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289'); +INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552'); +INSERT INTO num_exp_add VALUES (3,5,'16401.348491'); +INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491'); +INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621'); +INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504'); +INSERT INTO num_exp_add VALUES (3,6,'93905.88763026'); +INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026'); +INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206'); +INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595'); +INSERT INTO num_exp_add VALUES (3,7,'-83028480.69'); +INSERT INTO num_exp_sub VALUES (3,7,'83028489.31'); +INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35'); +INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240'); +INSERT INTO num_exp_add VALUES (3,8,'74885.31'); +INSERT INTO num_exp_sub VALUES (3,8,'-74876.69'); +INSERT INTO num_exp_mul VALUES (3,8,'322737.11'); +INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553'); +INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420'); +INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420'); +INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020'); +INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854'); +INSERT INTO num_exp_add VALUES (4,0,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119'); +INSERT INTO num_exp_mul VALUES (4,0,'0'); +INSERT INTO num_exp_div VALUES (4,0,'NaN'); +INSERT INTO num_exp_add VALUES (4,1,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119'); +INSERT INTO num_exp_mul VALUES (4,1,'0'); +INSERT INTO num_exp_div VALUES (4,1,'NaN'); +INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047'); +INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047'); +INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593'); +INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385'); +INSERT INTO num_exp_add VALUES (4,3,'7799465.7219'); +INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019'); +INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289'); +INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883'); +INSERT INTO num_exp_add VALUES (4,4,'15598922.8238'); +INSERT INTO num_exp_sub VALUES (4,4,'0'); +INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161'); +INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (4,5,'7815858.450391'); +INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409'); +INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429'); +INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061'); +INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026'); +INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974'); +INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094'); +INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606'); +INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881'); +INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119'); +INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715'); +INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637'); +INSERT INTO num_exp_add VALUES (4,8,'7874342.4119'); +INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119'); +INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839'); +INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143'); +INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420'); +INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420'); +INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980'); +INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409'); +INSERT INTO num_exp_add VALUES (5,0,'16397.038491'); +INSERT INTO num_exp_sub VALUES (5,0,'16397.038491'); +INSERT INTO num_exp_mul VALUES (5,0,'0'); +INSERT INTO num_exp_div VALUES (5,0,'NaN'); +INSERT INTO num_exp_add VALUES (5,1,'16397.038491'); +INSERT INTO num_exp_sub VALUES (5,1,'16397.038491'); +INSERT INTO num_exp_mul VALUES (5,1,'0'); +INSERT INTO num_exp_div VALUES (5,1,'NaN'); +INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047'); +INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047'); +INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077'); +INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446'); +INSERT INTO num_exp_add VALUES (5,3,'16401.348491'); +INSERT INTO num_exp_sub VALUES (5,3,'16392.728491'); +INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621'); +INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584'); +INSERT INTO num_exp_add VALUES (5,4,'7815858.450391'); +INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409'); +INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429'); +INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192'); +INSERT INTO num_exp_add VALUES (5,5,'32794.076982'); +INSERT INTO num_exp_sub VALUES (5,5,'0'); +INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081'); +INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (5,6,'110298.61612126'); +INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926'); +INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766'); +INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689'); +INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509'); +INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491'); +INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135'); +INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710'); +INSERT INTO num_exp_add VALUES (5,8,'91278.038491'); +INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509'); +INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571'); +INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228'); +INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420'); +INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420'); +INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220'); +INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427'); +INSERT INTO num_exp_add VALUES (6,0,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026'); +INSERT INTO num_exp_mul VALUES (6,0,'0'); +INSERT INTO num_exp_div VALUES (6,0,'NaN'); +INSERT INTO num_exp_add VALUES (6,1,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026'); +INSERT INTO num_exp_mul VALUES (6,1,'0'); +INSERT INTO num_exp_div VALUES (6,1,'NaN'); +INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787'); +INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307'); +INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222'); +INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823'); +INSERT INTO num_exp_add VALUES (6,3,'93905.88763026'); +INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026'); +INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206'); +INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907'); +INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026'); +INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974'); +INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094'); +INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469'); +INSERT INTO num_exp_add VALUES (6,5,'110298.61612126'); +INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926'); +INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766'); +INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679'); +INSERT INTO num_exp_add VALUES (6,6,'187803.15526052'); +INSERT INTO num_exp_sub VALUES (6,6,'0'); +INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676'); +INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974'); +INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026'); +INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610'); +INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980'); +INSERT INTO num_exp_add VALUES (6,8,'168782.57763026'); +INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026'); +INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906'); +INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184'); +INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160'); +INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680'); +INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920'); +INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789'); +INSERT INTO num_exp_add VALUES (7,0,'-83028485'); +INSERT INTO num_exp_sub VALUES (7,0,'-83028485'); +INSERT INTO num_exp_mul VALUES (7,0,'0'); +INSERT INTO num_exp_div VALUES (7,0,'NaN'); +INSERT INTO num_exp_add VALUES (7,1,'-83028485'); +INSERT INTO num_exp_sub VALUES (7,1,'-83028485'); +INSERT INTO num_exp_mul VALUES (7,1,'0'); +INSERT INTO num_exp_div VALUES (7,1,'NaN'); +INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047'); +INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953'); +INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795'); +INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700'); +INSERT INTO num_exp_add VALUES (7,3,'-83028480.69'); +INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31'); +INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35'); +INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974'); +INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881'); +INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119'); +INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715'); +INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686'); +INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509'); +INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491'); +INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135'); +INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574'); +INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974'); +INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026'); +INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610'); +INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294'); +INSERT INTO num_exp_add VALUES (7,7,'-166056970'); +INSERT INTO num_exp_sub VALUES (7,7,'0'); +INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225'); +INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (7,8,'-82953604'); +INSERT INTO num_exp_sub VALUES (7,8,'-83103366'); +INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285'); +INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118'); +INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420'); +INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580'); +INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700'); +INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382'); +INSERT INTO num_exp_add VALUES (8,0,'74881'); +INSERT INTO num_exp_sub VALUES (8,0,'74881'); +INSERT INTO num_exp_mul VALUES (8,0,'0'); +INSERT INTO num_exp_div VALUES (8,0,'NaN'); +INSERT INTO num_exp_add VALUES (8,1,'74881'); +INSERT INTO num_exp_sub VALUES (8,1,'74881'); +INSERT INTO num_exp_mul VALUES (8,1,'0'); +INSERT INTO num_exp_div VALUES (8,1,'NaN'); +INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047'); +INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047'); +INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407'); +INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615'); +INSERT INTO num_exp_add VALUES (8,3,'74885.31'); +INSERT INTO num_exp_sub VALUES (8,3,'74876.69'); +INSERT INTO num_exp_mul VALUES (8,3,'322737.11'); +INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410'); +INSERT INTO num_exp_add VALUES (8,4,'7874342.4119'); +INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119'); +INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839'); +INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956'); +INSERT INTO num_exp_add VALUES (8,5,'91278.038491'); +INSERT INTO num_exp_sub VALUES (8,5,'58483.961509'); +INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571'); +INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456'); +INSERT INTO num_exp_add VALUES (8,6,'168782.57763026'); +INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026'); +INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906'); +INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424'); +INSERT INTO num_exp_add VALUES (8,7,'-82953604'); +INSERT INTO num_exp_sub VALUES (8,7,'83103366'); +INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285'); +INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172'); +INSERT INTO num_exp_add VALUES (8,8,'149762'); +INSERT INTO num_exp_sub VALUES (8,8,'0'); +INSERT INTO num_exp_mul VALUES (8,8,'5607164161'); +INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420'); +INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420'); +INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020'); +INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735'); +INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420'); +INSERT INTO num_exp_mul VALUES (9,0,'0'); +INSERT INTO num_exp_div VALUES (9,0,'NaN'); +INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420'); +INSERT INTO num_exp_mul VALUES (9,1,'0'); +INSERT INTO num_exp_div VALUES (9,1,'NaN'); +INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467'); +INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627'); +INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740'); +INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526'); +INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420'); +INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420'); +INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020'); +INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677'); +INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420'); +INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420'); +INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980'); +INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484'); +INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420'); +INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420'); +INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220'); +INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807'); +INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160'); +INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680'); +INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920'); +INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280'); +INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420'); +INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580'); +INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700'); +INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689'); +INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420'); +INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420'); +INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020'); +INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748'); +INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840'); +INSERT INTO num_exp_sub VALUES (9,9,'0'); +INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400'); +INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_sqrt VALUES (0,'0'); +INSERT INTO num_exp_sqrt VALUES (1,'0'); +INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505'); +INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396'); +INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923'); +INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473'); +INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406'); +INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230'); +INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542'); +INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_ln VALUES (0,'NaN'); +INSERT INTO num_exp_ln VALUES (1,'NaN'); +INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514'); +INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971'); +INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464'); +INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038'); +INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127'); +INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991'); +INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668'); +INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_log10 VALUES (0,'NaN'); +INSERT INTO num_exp_log10 VALUES (1,'NaN'); +INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459'); +INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075'); +INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345'); +INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626'); +INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671'); +INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914'); +INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138'); +INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN'); +INSERT INTO num_exp_power_10_ln VALUES (1,'NaN'); +INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184'); +INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393'); +INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636'); +INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098'); +INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067'); +INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627'); +INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952'); +INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_data VALUES (0, '0'); +INSERT INTO num_data VALUES (1, '0'); +INSERT INTO num_data VALUES (2, '-34338492.215397047'); +INSERT INTO num_data VALUES (3, '4.31'); +INSERT INTO num_data VALUES (4, '7799461.4119'); +INSERT INTO num_data VALUES (5, '16397.038491'); +INSERT INTO num_data VALUES (6, '93901.57763026'); +INSERT INTO num_data VALUES (7, '-83028485'); +INSERT INTO num_data VALUES (8, '74881'); +INSERT INTO num_data VALUES (9, '-24926804.045047420'); +-- COMMIT TRANSACTION; + +SELECT * FROM num_data; + +-- ****************************** +-- * Create indices for faster checks +-- ****************************** + +-- CREATE UNIQUE INDEX num_exp_add_idx ON num_exp_add (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sub_idx ON num_exp_sub (id1, id2); +-- CREATE UNIQUE INDEX num_exp_div_idx ON num_exp_div (id1, id2); +-- CREATE UNIQUE INDEX num_exp_mul_idx ON num_exp_mul (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sqrt_idx ON num_exp_sqrt (id); +-- CREATE UNIQUE INDEX num_exp_ln_idx ON num_exp_ln (id); +-- CREATE UNIQUE INDEX num_exp_log10_idx ON num_exp_log10 (id); +-- CREATE UNIQUE INDEX num_exp_power_10_ln_idx ON num_exp_power_10_ln (id); + +-- VACUUM ANALYZE num_exp_add; +-- VACUUM ANALYZE num_exp_sub; +-- VACUUM ANALYZE num_exp_div; +-- VACUUM ANALYZE num_exp_mul; +-- VACUUM ANALYZE num_exp_sqrt; +-- VACUUM ANALYZE num_exp_ln; +-- VACUUM ANALYZE num_exp_log10; +-- VACUUM ANALYZE num_exp_power_10_ln; + +-- ****************************** +-- * Now check the behaviour of the NUMERIC type +-- ****************************** + +-- ****************************** +-- * Addition check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10); + +-- ****************************** +-- * Subtraction check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40); + +-- ****************************** +-- * Multiply check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30); + +-- ****************************** +-- * Division check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80); + +-- ****************************** +-- * Square root check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Natural logarithm check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Logarithm base 10 check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * POWER(10, LN(value)) check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * miscellaneous checks for things that have been broken in the past... +-- ****************************** +-- numeric AVG used to fail on some platforms +SELECT AVG(val) FROM num_data; +-- [SPARK-28316] STDDEV and VARIANCE returns double type +-- Skip it because: Expected "2.779120328758835[]E7", but got "2.779120328758835[4]E7" +-- SELECT STDDEV(val) FROM num_data; +-- Skip it because: Expected "7.72350980172061[8]E14", but got "7.72350980172061[6]E14" +-- SELECT VARIANCE(val) FROM num_data; + +-- Check for appropriate rounding and overflow +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet; +INSERT INTO fract_only VALUES (1, '0.0'); +INSERT INTO fract_only VALUES (2, '0.1'); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (3, '1.0'); -- should fail +INSERT INTO fract_only VALUES (4, '-0.9999'); +INSERT INTO fract_only VALUES (5, '0.99994'); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (6, '0.99995'); -- should fail +INSERT INTO fract_only VALUES (7, '0.00001'); +INSERT INTO fract_only VALUES (8, '0.00017'); +SELECT * FROM fract_only; +DROP TABLE fract_only; + +-- [SPARK-28315] Decimal can not accept NaN as input +-- [SPARK-27923] Decimal type can not accept Infinity and -Infinity +-- Check inf/nan conversion behavior +SELECT decimal(double('NaN')); +SELECT decimal(double('Infinity')); +SELECT decimal(double('-Infinity')); +SELECT decimal(float('NaN')); +SELECT decimal(float('Infinity')); +SELECT decimal(float('-Infinity')); + +-- Simple check that ceil(), floor(), and round() work correctly +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet; +INSERT INTO ceil_floor_round VALUES ('-5.5'); +INSERT INTO ceil_floor_round VALUES ('-5.499999'); +INSERT INTO ceil_floor_round VALUES ('9.5'); +INSERT INTO ceil_floor_round VALUES ('9.4999999'); +INSERT INTO ceil_floor_round VALUES ('0.0'); +INSERT INTO ceil_floor_round VALUES ('0.0000001'); +INSERT INTO ceil_floor_round VALUES ('-0.000001'); +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round; +DROP TABLE ceil_floor_round; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Check rounding, it should round ties away from zero. +-- SELECT i as pow, +-- round((-2.5 * 10 ^ i)::numeric, -i), +-- round((-1.5 * 10 ^ i)::numeric, -i), +-- round((-0.5 * 10 ^ i)::numeric, -i), +-- round((0.5 * 10 ^ i)::numeric, -i), +-- round((1.5 * 10 ^ i)::numeric, -i), +-- round((2.5 * 10 ^ i)::numeric, -i) +-- FROM generate_series(-5,5) AS t(i); + +-- [SPARK-21117] Built-in SQL Function Support - WIDTH_BUCKET +-- Testing for width_bucket(). For convenience, we test both the +-- numeric and float8 versions of the function in this file. + +-- errors +-- SELECT width_bucket(5.0, 3.0, 4.0, 0); +-- SELECT width_bucket(5.0, 3.0, 4.0, -5); +-- SELECT width_bucket(3.5, 3.0, 3.0, 888); +-- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, 0); +-- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, -5); +-- SELECT width_bucket(3.5::float8, 3.0::float8, 3.0::float8, 888); +-- SELECT width_bucket('NaN', 3.0, 4.0, 888); +-- SELECT width_bucket(0::float8, 'NaN', 4.0::float8, 888); + +-- normal operation +-- CREATE TABLE width_bucket_test (operand_num numeric, operand_f8 float8); + +-- COPY width_bucket_test (operand_num) FROM stdin; +-- -5.2 +-- -0.0000000001 +-- 0.000000000001 +-- 1 +-- 1.99999999999999 +-- 2 +-- 2.00000000000001 +-- 3 +-- 4 +-- 4.5 +-- 5 +-- 5.5 +-- 6 +-- 7 +-- 8 +-- 9 +-- 9.99999999999999 +-- 10 +-- 10.0000000000001 +-- \. + +-- UPDATE width_bucket_test SET operand_f8 = operand_num::float8; + +-- SELECT +-- operand_num, +-- width_bucket(operand_num, 0, 10, 5) AS wb_1, +-- width_bucket(operand_f8, 0, 10, 5) AS wb_1f, +-- width_bucket(operand_num, 10, 0, 5) AS wb_2, +-- width_bucket(operand_f8, 10, 0, 5) AS wb_2f, +-- width_bucket(operand_num, 2, 8, 4) AS wb_3, +-- width_bucket(operand_f8, 2, 8, 4) AS wb_3f, +-- width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, +-- width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, +-- width_bucket(operand_num, -25, 25, 10) AS wb_5, +-- width_bucket(operand_f8, -25, 25, 10) AS wb_5f +-- FROM width_bucket_test; + +-- for float8 only, check positive and negative infinity: we require +-- finite bucket bounds, but allow an infinite operand +-- SELECT width_bucket(0.0::float8, 'Infinity'::float8, 5, 10); -- error +-- SELECT width_bucket(0.0::float8, 5, '-Infinity'::float8, 20); -- error +-- SELECT width_bucket('Infinity'::float8, 1, 10, 10), +-- width_bucket('-Infinity'::float8, 1, 10, 10); + +-- DROP TABLE width_bucket_test; + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR +-- TO_CHAR() +-- +-- SELECT '' AS to_char_1, to_char(val, '9G999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_2, to_char(val, '9G999G999G999G999G999D999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR') +-- FROM num_data; + +-- SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S') +-- FROM num_data; + +-- SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR') FROM num_data; +-- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th') FROM num_data; +-- SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999') FROM num_data; +-- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999') FROM num_data; +-- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999') FROM num_data; +-- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data; +-- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data; +-- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM num_data; +-- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999') FROM num_data; +-- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_23, to_char(val, '9.999EEEE') FROM num_data; + +-- SELECT '' AS to_char_24, to_char('100'::numeric, 'FM999.9'); +-- SELECT '' AS to_char_25, to_char('100'::numeric, 'FM999.'); +-- SELECT '' AS to_char_26, to_char('100'::numeric, 'FM999'); + +-- Check parsing of literal text in a format string +-- SELECT '' AS to_char_27, to_char('100'::numeric, 'foo999'); +-- SELECT '' AS to_char_28, to_char('100'::numeric, 'f\oo999'); +-- SELECT '' AS to_char_29, to_char('100'::numeric, 'f\\oo999'); +-- SELECT '' AS to_char_30, to_char('100'::numeric, 'f\"oo999'); +-- SELECT '' AS to_char_31, to_char('100'::numeric, 'f\\"oo999'); +-- SELECT '' AS to_char_32, to_char('100'::numeric, 'f"ool"999'); +-- SELECT '' AS to_char_33, to_char('100'::numeric, 'f"\ool"999'); +-- SELECT '' AS to_char_34, to_char('100'::numeric, 'f"\\ool"999'); +-- SELECT '' AS to_char_35, to_char('100'::numeric, 'f"ool\"999'); +-- SELECT '' AS to_char_36, to_char('100'::numeric, 'f"ool\\"999'); + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_NUMBER +-- TO_NUMBER() +-- +-- SET lc_numeric = 'C'; +-- SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999'); +-- SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999'); +-- SELECT '' AS to_number_3, to_number('<564646.654564>', '999999.999999PR'); +-- SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999MI'); +-- SELECT '' AS to_number_7, to_number('5 4 4 4 4 8 . 7 8', '9 9 9 9 9 9 . 9 9'); +-- SELECT '' AS to_number_8, to_number('.01', 'FM9.99'); +-- SELECT '' AS to_number_9, to_number('.0', '99999999.99999999'); +-- SELECT '' AS to_number_10, to_number('0', '99.99'); +-- SELECT '' AS to_number_11, to_number('.-01', 'S99.99'); +-- SELECT '' AS to_number_12, to_number('.01-', '99.99S'); +-- SELECT '' AS to_number_13, to_number(' . 0 1-', ' 9 9 . 9 9 S'); +-- SELECT '' AS to_number_14, to_number('34,50','999,99'); +-- SELECT '' AS to_number_15, to_number('123,000','999G'); +-- SELECT '' AS to_number_16, to_number('123456','999G999'); +-- SELECT '' AS to_number_17, to_number('$1234.56','L9,999.99'); +-- SELECT '' AS to_number_18, to_number('$1234.56','L99,999.99'); +-- SELECT '' AS to_number_19, to_number('$1,234.56','L99,999.99'); +-- SELECT '' AS to_number_20, to_number('1234.56','L99,999.99'); +-- SELECT '' AS to_number_21, to_number('1,234.56','L99,999.99'); +-- SELECT '' AS to_number_22, to_number('42nd', '99th'); +-- RESET lc_numeric; + +-- +-- Input syntax +-- + +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet; + +-- good inputs +INSERT INTO num_input_test VALUES (trim(' 123')); +INSERT INTO num_input_test VALUES (trim(' 3245874 ')); +INSERT INTO num_input_test VALUES (trim(' -93853')); +INSERT INTO num_input_test VALUES ('555.50'); +INSERT INTO num_input_test VALUES ('-555.50'); +-- [SPARK-28315] Decimal can not accept NaN as input +-- INSERT INTO num_input_test VALUES (trim('NaN ')); +-- INSERT INTO num_input_test VALUES (trim(' nan')); + +-- [SPARK-27923] Spark SQL accept bad inputs to NULL +-- bad inputs +-- INSERT INTO num_input_test VALUES (' '); +-- INSERT INTO num_input_test VALUES (' 1234 %'); +-- INSERT INTO num_input_test VALUES ('xyz'); +-- INSERT INTO num_input_test VALUES ('- 1234'); +-- INSERT INTO num_input_test VALUES ('5 . 0'); +-- INSERT INTO num_input_test VALUES ('5. 0 '); +-- INSERT INTO num_input_test VALUES (''); +-- INSERT INTO num_input_test VALUES (' N aN '); + +SELECT * FROM num_input_test; + +-- [SPARK-28318] Decimal can only support precision up to 38 +-- +-- Test some corner cases for multiplication +-- + +-- select 4790999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4789999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4770999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4769999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- +-- Test some corner cases for division +-- +-- 999999999999999999999 is overflow for SYSTEM_DEFAULT(decimal(38, 18)), we use BigIntDecimal(decimal(38, 0)). +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000; +-- [SPARK-28322] DIV support decimal type +-- select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +-- select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +-- select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod (70.0,70) ; +-- select div (70.0,70) ; +select 70.0 / 70 ; +select 12345678901234567890 % 123; +-- [SPARK-2659] HiveQL: Division operator should always perform fractional division +-- select 12345678901234567890 DIV 123; +-- select div(12345678901234567890, 123); +-- select div(12345678901234567890, 123) * 123 + 12345678901234567890 % 123; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- +-- Test code path for raising to integer powers +-- + +-- select 10.0 ^ -2147483648 as rounds_to_zero; +-- select 10.0 ^ -2147483647 as rounds_to_zero; +-- select 10.0 ^ 2147483647 as overflows; +-- select 117743296169.0 ^ 1000000000 as overflows; + +-- cases that used to return inaccurate results +-- select 3.789 ^ 21; +-- select 3.789 ^ 35; +-- select 1.2 ^ 345; +-- select 0.12 ^ (-20); + +-- cases that used to error out +-- select 0.12 ^ (-25); +-- select 0.5678 ^ (-85); + +-- +-- Tests for raising to non-integer powers +-- + +-- special cases +-- select 0.0 ^ 0.0; +-- select (-12.34) ^ 0.0; +-- select 12.34 ^ 0.0; +-- select 0.0 ^ 12.34; + +-- NaNs +-- select 'NaN'::numeric ^ 'NaN'::numeric; +-- select 'NaN'::numeric ^ 0; +-- select 'NaN'::numeric ^ 1; +-- select 0 ^ 'NaN'::numeric; +-- select 1 ^ 'NaN'::numeric; + +-- invalid inputs +-- select 0.0 ^ (-12.34); +-- select (-12.34) ^ 1.2; + +-- cases that used to generate inaccurate results +-- select 32.1 ^ 9.8; +-- select 32.1 ^ (-9.8); +-- select 12.3 ^ 45.6; +-- select 12.3 ^ (-45.6); + +-- big test +-- select 1.234 ^ 5678; + +-- +-- Tests for EXP() +-- + +-- special cases +select exp(0.0); +select exp(1.0); +-- [SPARK-28316] EXP returns double type for decimal input +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select exp(1.0::numeric(71,70)); + +-- cases that used to generate inaccurate results +select exp(32.999); +select exp(-32.999); +select exp(123.456); +select exp(-123.456); + +-- big test +select exp(1234.5678); + +-- +-- Tests for generate_series +-- +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))); +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))); +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))); +-- Trigger errors +-- select * from generate_series(-100::numeric, 100::numeric, 0::numeric); +-- select * from generate_series(-100::numeric, 100::numeric, 'nan'::numeric); +-- select * from generate_series('nan'::numeric, 100::numeric, 10::numeric); +-- select * from generate_series(0::numeric, 'nan'::numeric, 10::numeric); +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Checks maximum, output is truncated +-- select (i / (10::numeric ^ 131071))::numeric(1,0) +-- from generate_series(6 * (10::numeric ^ 131071), +-- 9 * (10::numeric ^ 131071), +-- 10::numeric ^ 131071) as a(i); +-- Check usage with variables +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(i,3) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,i) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,5,i) j; + +-- +-- Tests for LN() +-- + +-- [SPARK-27923] Invalid inputs for LN throws exception at PostgreSQL +-- Invalid inputs +-- select ln(-12.34); +-- select ln(0.0); + +-- Some random tests +select ln(1.2345678e-28); +select ln(0.0456789); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select ln(0.349873948359354029493948309745709580730482050975); +select ln(0.99949452); +select ln(1.00049687395); +select ln(1234.567890123456789); +select ln(5.80397490724e5); +select ln(9.342536355e34); + +-- +-- Tests for LOG() (base 10) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34); +-- select log(0.0); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.234567e-89); +-- [SPARK-28324] The LOG function using 10 as the base, but Spark using E +select log(3.4634998359873254962349856073435545); +select log(9.999999999999999999); +select log(10.00000000000000000); +select log(10.00000000000000001); +select log(590489.45235237); + +-- +-- Tests for LOG() (arbitrary base) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34, 56.78); +-- select log(-12.34, -56.78); +-- select log(12.34, -56.78); +-- select log(0.0, 12.34); +-- select log(12.34, 0.0); +-- select log(1.0, 12.34); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.23e-89, 6.4689e45); +select log(0.99923, 4.58934e34); +select log(1.000016, 8.452010e18); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(3.1954752e47, 9.4792021e-73); + +-- [SPARK-28317] Built-in Mathematical Functions: SCALE +-- +-- Tests for scale() +-- + +-- select scale(numeric 'NaN'); +-- select scale(NULL::numeric); +-- select scale(1.12); +-- select scale(0); +-- select scale(0.00); +-- select scale(1.12345); +-- select scale(110123.12475871856128); +-- select scale(-1123.12471856128); +-- select scale(-13.000000000000000); + +-- +-- Tests for SUM() +-- + +-- cases that need carry propagation +SELECT SUM(decimal(9999)) FROM range(1, 100001); +SELECT SUM(decimal(-9999)) FROM range(1, 100001); + +DROP TABLE num_data; +DROP TABLE num_exp_add; +DROP TABLE num_exp_sub; +DROP TABLE num_exp_div; +DROP TABLE num_exp_mul; +DROP TABLE num_exp_sqrt; +DROP TABLE num_exp_ln; +DROP TABLE num_exp_log10; +DROP TABLE num_exp_power_10_ln; +DROP TABLE num_result; +DROP TABLE num_input_test; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql new file mode 100644 index 0000000000000..1f83d6c41661f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql @@ -0,0 +1,285 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select.sql +-- +create or replace temporary view onek2 as select * from onek; +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2); + +-- btree index +-- awk '{if($1<10){print;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1d -2 +0nr -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1dr -2 +0n -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0nr -1 +1d -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0n -1 +1dr -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC; + +-- +-- test partial btree indexes +-- +-- As of 7.2, planner probably won't pick an indexscan without stats, +-- so ANALYZE first. Also, we want to prevent it from picking a bitmapscan +-- followed by sort, because that could hide index ordering problems. +-- +-- ANALYZE onek2; + +-- SET enable_seqscan TO off; +-- SET enable_bitmapscan TO off; +-- SET enable_sort TO off; + +-- +-- awk '{if($1<10){print $0;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10; + +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980; + +-- RESET enable_seqscan; +-- RESET enable_bitmapscan; +-- RESET enable_sort; + +-- [SPARK-28329] SELECT INTO syntax +-- SELECT two, stringu1, ten, string4 +-- INTO TABLE tmp +-- FROM onek; +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=2){print $4,$5;}else{print;}}' - stud_emp.data +-- +-- SELECT name, age FROM person*; ??? check if different +-- SELECT p.name, p.age FROM person* p; + +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $4,$5;}else{print;}}' - stud_emp.data | +-- sort +1nr -2 +-- +-- SELECT p.name, p.age FROM person* p ORDER BY age DESC, name; + +-- [SPARK-28330] Enhance query limit +-- +-- Test some cases involving whole-row Var referencing a subquery +-- +select foo.* from (select 1) as foo; +select foo.* from (select null) as foo; +select foo.* from (select 'xyzzy',1,null) as foo; + +-- +-- Test VALUES lists +-- +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j; + +-- [SPARK-28296] Improved VALUES support +-- a more complex case +-- looks like we're coding lisp :-) +-- select * from onek, +-- (values ((select i from +-- (values(10000), (2), (389), (1000), (2000), ((select 10029))) as foo(i) +-- order by i asc limit 1))) bar (i) +-- where onek.unique1 = bar.i; + +-- try VALUES in a subquery +-- select * from onek +-- where (unique1,ten) in (values (1,1), (20,0), (99,9), (17,99)) +-- order by unique1; + +-- VALUES is also legal as a standalone query or a set-operation member +VALUES (1,2), (3,4+4), (7,77.7); + +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl; + +-- +-- Test ORDER BY options +-- + +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1); + +-- [SPARK-28333] NULLS FIRST for DESC and NULLS LAST for ASC +SELECT * FROM foo ORDER BY f1; +SELECT * FROM foo ORDER BY f1 ASC; -- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST; +SELECT * FROM foo ORDER BY f1 DESC; +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- check if indexscans do the right things +-- CREATE INDEX fooi ON foo (f1); +-- SET enable_sort = false; + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC NULLS LAST); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- +-- Test planning of some cases with partial indexes +-- + +-- partial index is usable +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- actually run the query with an analyze to use the partial index +-- explain (costs off, analyze on, timing off, summary off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- partial index predicate implies clause, so no need for retest +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 < 'B'; +select * from onek2 where unique2 = 11 and stringu1 < 'B'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- but if it's an update target, must retest anyway +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- partial index is not applicable +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +-- partial index implies clause, but bitmap scan must recheck predicate anyway +-- SET enable_indexscan TO off; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- RESET enable_indexscan; +-- check multi-index cases too +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; + +-- +-- Test some corner cases that have been known to confuse the planner +-- + +-- ORDER BY on a constant doesn't really need any sorting +SELECT 1 AS x ORDER BY x; + +-- But ORDER BY on a set-valued expression does +-- create function sillysrf(int) returns setof int as +-- 'values (1),(10),(2),($1)' language sql immutable; + +-- select sillysrf(42); +-- select sillysrf(-1) order by 1; + +-- drop function sillysrf(int); + +-- X = X isn't a no-op, it's effectively X IS NOT NULL assuming = is strict +-- (see bug #5084) +select * from (values (2),(null),(1)) v(k) where k = k order by k; +select * from (values (2),(null),(1)) v(k) where k = k; + +-- Test partitioned tables with no partitions, which should be handled the +-- same as the non-inheritance case when expanding its RTE. +-- create table list_parted_tbl (a int,b int) partition by list (a); +-- create table list_parted_tbl1 partition of list_parted_tbl +-- for values in (1) partition by list(b); +-- explain (costs off) select * from list_parted_tbl; +-- drop table list_parted_tbl; +drop table tmp; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql new file mode 100644 index 0000000000000..5306028e5bd7b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql @@ -0,0 +1,86 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_DISTINCT +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_distinct.sql +-- + +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- +-- awk '{print $3;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT two FROM tmp ORDER BY 1; + +-- +-- awk '{print $5;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT ten FROM tmp ORDER BY 1; + +-- +-- awk '{print $16;}' onek.data | sort -d | uniq +-- +SELECT DISTINCT string4 FROM tmp ORDER BY 1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{print $3,$16,$5;}' onek.data | sort -d | uniq | +-- sort +0n -1 +1d -2 +2n -3 +-- +-- SELECT DISTINCT two, string4, ten +-- FROM tmp +-- ORDER BY two using <, string4 using <, ten using <; +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $2;}' person.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - emp.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $5;}else{print;}}' - stud_emp.data | +-- sort -n -r | uniq +-- +-- SELECT DISTINCT p.age FROM person* p ORDER BY age using >; + +-- +-- Check mentioning same column more than once +-- + +-- EXPLAIN (VERBOSE, COSTS OFF) +-- SELECT count(*) FROM +-- (SELECT DISTINCT two, four, two FROM tenk1) ss; + +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss; + +-- +-- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its +-- very own regression file. +-- + +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1); + +-- basic cases +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable; + +-- check that optimizer constant-folds it properly +SELECT 1 IS DISTINCT FROM 2 as `yes`; +SELECT 2 IS DISTINCT FROM 2 as `no`; +SELECT 2 IS DISTINCT FROM null as `yes`; +SELECT null IS DISTINCT FROM null as `no`; + +-- negated form +SELECT 1 IS NOT DISTINCT FROM 2 as `no`; +SELECT 2 IS NOT DISTINCT FROM 2 as `yes`; +SELECT 2 IS NOT DISTINCT FROM null as `no`; +SELECT null IS NOT DISTINCT FROM null as `yes`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql new file mode 100644 index 0000000000000..2edde8df08047 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql @@ -0,0 +1,55 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_HAVING +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +-- + +-- load test data +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b'); +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c'); +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j'); + +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c; + +-- HAVING is effectively equivalent to WHERE in this case +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c; + +-- [SPARK-28386] Cannot resolve ORDER BY columns with GROUP BY and HAVING +-- SELECT lower(c), count(c) FROM test_having +-- GROUP BY lower(c) HAVING count(*) > 2 OR min(a) = max(a) +-- ORDER BY lower(c); + +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c; + +-- test degenerate cases involving HAVING without GROUP BY +-- Per SQL spec, these should generate 0 or 1 row, even without aggregates + +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a); +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a); + +-- errors: ungrouped column references +SELECT a FROM test_having HAVING min(a) < max(a); +SELECT 1 AS one FROM test_having HAVING a > 1; + +-- the really degenerate case: need not scan table at all +SELECT 1 AS one FROM test_having HAVING 1 > 2; +SELECT 1 AS one FROM test_having HAVING 1 < 2; + +-- and just to prove that we aren't scanning the table: +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2; + +DROP TABLE test_having; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql new file mode 100644 index 0000000000000..54b3083a9f4a2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql @@ -0,0 +1,160 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_IMPLICIT +-- Test cases for queries with ordering terms missing from the target list. +-- This used to be called "junkfilter.sql". +-- The parser uses the term "resjunk" to handle these cases. +-- - thomas 1998-07-09 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql +-- + +-- load test data +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet; +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b'); +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c'); +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j'); + + +-- w/ existing GROUP BY target +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/ existing GROUP BY target using a relation name in target +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/o existing GROUP BY target +SELECT c FROM test_missing_target ORDER BY a; + +-- w/o existing ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc; + +-- group using reference number +SELECT count(*) FROM test_missing_target ORDER BY 1 desc; + +-- order using reference number +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1; + +-- group using reference number out of range +-- failure expected +SELECT c, count(*) FROM test_missing_target GROUP BY 3; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b; + +-- order w/ target under ambiguous condition +-- failure NOT expected +SELECT a, a FROM test_missing_target + ORDER BY a; + +-- order expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2; + +-- group expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- group w/o existing GROUP BY target under ambiguous condition +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(*) INTO TABLE test_missing_target2 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b ORDER BY x.b; +-- SELECT * FROM test_missing_target2; + + +-- Functions and expressions + +-- w/ existing GROUP BY target +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c); + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2; + +-- w/ existing GROUP BY target using a relation name in target +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c); + +-- w/o existing GROUP BY target +SELECT a FROM test_missing_target ORDER BY upper(d); + +-- w/o existing ORDER BY target +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2; + +-- group w/o existing GROUP BY target under ambiguous condition +-- failure expected due to ambiguous b in count(b) +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(x.b) INTO TABLE test_missing_target3 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b/2 ORDER BY x.b/2; +-- SELECT * FROM test_missing_target3; + +-- Cleanup +DROP TABLE test_missing_target; +-- DROP TABLE test_missing_target2; +-- DROP TABLE test_missing_target3; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql new file mode 100644 index 0000000000000..02af15a879c84 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -0,0 +1,247 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- TIMESTAMP +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/timestamp.sql + +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; + +-- [SPARK-28141] Timestamp type can not accept special values +-- Test shorthand input values +-- We can't just "select" the results since they aren't constants; test for +-- equality instead. We can do that by running the test inside a transaction +-- block, within which the value of 'now' shouldn't change. We also check +-- that 'now' *does* change over a reasonable interval such as 100 msec. +-- NOTE: it is possible for this part of the test to fail if the transaction +-- block is entered exactly at local midnight; then 'now' and 'today' have +-- the same values and the counts will come out different. + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); + +-- BEGIN; + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('today'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +-- time zone should be ignored by this data type +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); + +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; +-- SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; + +-- COMMIT; + +-- DELETE FROM TIMESTAMP_TBL; + +-- verify uniform transaction time within transaction block +-- BEGIN; +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; +-- COMMIT; + +-- TRUNCATE TIMESTAMP_TBL; + +-- Special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- [SPARK-27923] Spark SQL insert there obsolete special values to NULL +-- Obsolete special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('undefined'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('current'); + +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- Postgres v6.0 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); + +-- Variations on Postgres v6.1 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); + +-- ISO 8601 format +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); + +-- POSIX format (note that the timezone abbrev is just decoration here) +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); + +-- Variations for acceptable input formats +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); +-- set datestyle to ymd; +-- INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); +-- reset datestyle; +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +-- this fails (even though TZ is a no-op, we still look it up) +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/Does_not_exist'); + +-- Check date conversion and date arithmetic +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 11 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 12 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 13 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 14 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 15 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0597'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1697'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1797'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1897'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 2097'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1999'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); + +-- Currently unsupported syntax and ranges +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC'); + +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL; + +-- [SPARK-28253] Date/Timestamp type have different low value and high value with Spark +-- Check behavior at the lower boundary of the timestamp range +-- SELECT '4714-11-24 00:00:00 BC'::timestamp; +-- SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range +-- The upper boundary differs between integer and float timestamps, so no check + +-- Demonstrate functions and operators +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02'; + +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02'; + +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02'; + +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02'; + +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02'; + +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02'; + +-- [SPARK-28425] Add more Date/Time Operators +-- SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc; + +-- [SPARK-28425] Add more Date/Time Operators +-- Test casting within a BETWEEN qualifier +-- SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff +-- FROM TIMESTAMP_TBL +-- WHERE d1 BETWEEN timestamp '1902-01-01' +-- AND timestamp '2038-01-01'; + +-- [SPARK-28420] Date/Time Functions: date_part +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'year', d1) AS year, date_part( 'month', d1) AS month, +-- date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, +-- date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, +-- date_part( 'usec', d1) AS usec +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, +-- date_part( 'dow', d1) AS dow +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- [SPARK-28137] Data Type Formatting Functions +-- TO_CHAR() +-- SELECT '' AS to_char_1, to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_2, to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_3, to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_4, to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_5, to_char(d1, 'HH HH12 HH24 MI SS SSSS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_6, to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_7, to_char(d1, 'HH24--text--MI--text--SS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_8, to_char(d1, 'YYYYTH YYYYth Jth') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_9, to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_10, to_char(d1, 'IYYY IYY IY I IW IDDD ID') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_11, to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') +-- FROM TIMESTAMP_TBL; + + +--[SPARK-28432] Missing Date/Time Functions: make_timestamp +-- timestamp numeric fields constructor +-- SELECT make_timestamp(2014,12,28,6,30,45.887); + +DROP TABLE TIMESTAMP_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql new file mode 100644 index 0000000000000..83c6fd8cbac91 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql @@ -0,0 +1,1208 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- WITH +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/with.sql +-- +-- This test uses the generate_series(...) function which is rewritten to EXPLODE(SEQUENCE(...)) as +-- it's feature tracking ticket SPARK-27767 is closed as Won't Do. + +-- +-- Tests for common table expressions (WITH query, ... SELECT ...) +-- + +-- Basic WITH +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2; + +-- Multiple uses are evaluated only once +-- [SPARK-28299] Evaluation of multiple CTE uses +-- [ORIGINAL SQL] +--SELECT count(*) FROM ( +-- WITH q1(x) AS (SELECT random() FROM generate_series(1, 5)) +-- SELECT * FROM q1 +-- UNION +-- SELECT * FROM q1 +--) ss; +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- WITH RECURSIVE + +-- sum of 1..100 +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT (VALUES(1)) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 5 +--) +--SELECT * FROM t; + +-- recursive view +-- [SPARK-24497] Support recursive SQL query +--CREATE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 5; +-- +--SELECT * FROM nums; + +-- [SPARK-24497] Support recursive SQL query +--CREATE OR REPLACE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 6; +-- +--SELECT * FROM nums; + +-- This is an infinite loop with UNION ALL, but not with UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT 10-n FROM t) +--SELECT * FROM t; + +-- This'd be an infinite loop, but outside query reads only as much as needed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- UNION case should have same property +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- Test behavior with an unknown-type literal in the WITH +-- [SPARK-28146] Support IS OF type predicate +--WITH q AS (SELECT 'foo' AS x) +--SELECT x, x IS OF (text) AS is_text FROM q; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT 'foo' +--UNION ALL +-- SELECT n || ' bar' FROM t WHERE length(n) < 20 +--) +--SELECT n, n IS OF (text) AS is_text FROM t; + +-- In a perfect world, this would work and resolve the literal as int ... +-- but for now, we have to be content with resolving to text too soon. +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT '7' +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 10 +--) +--SELECT n, n IS OF (int) AS is_int FROM t; + +-- +-- Some examples with a tree +-- +-- department structure represented here is as follows: +-- +-- ROOT-+->A-+->B-+->C +-- | | +-- | +->D-+->F +-- +->E-+->G + + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE department ( +-- id INTEGER PRIMARY KEY, -- department ID +-- parent_department INTEGER REFERENCES department, -- upper department ID +-- name string -- department name +--); +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet; + +INSERT INTO department VALUES (0, NULL, 'ROOT'); +INSERT INTO department VALUES (1, 0, 'A'); +INSERT INTO department VALUES (2, 1, 'B'); +INSERT INTO department VALUES (3, 2, 'C'); +INSERT INTO department VALUES (4, 2, 'D'); +INSERT INTO department VALUES (5, 0, 'E'); +INSERT INTO department VALUES (6, 4, 'F'); +INSERT INTO department VALUES (7, 5, 'G'); + + +-- extract all departments under 'A'. Result should be A, B, C, D and F +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- non recursive term +-- SELECT name as root_name, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number. +-- Only shows level 2 or more +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; + +-- "RECURSIVE" is ignored if the query has no self-reference +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- note lack of recursive UNION structure +-- SELECT * FROM department WHERE name = 'A' +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- inside subqueries +-- [SPARK-24497] Support recursive SQL query +--SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 +-- ) +-- SELECT * FROM t) AS t WHERE n < ( +-- SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 +-- ) +-- SELECT * FROM t WHERE n < 50000 +-- ) AS t WHERE n < 100); + +-- use same CTE twice at different subquery levels +-- [SPARK-24497] Support recursive SQL query +--WITH q1(x,y) AS ( +-- SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred +-- ) +--SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); + +-- via a VIEW +-- [SPARK-24497] Support recursive SQL query +--CREATE TEMPORARY VIEW vsubdepartment AS +-- WITH RECURSIVE subdepartment AS +-- ( +-- -- non recursive term +-- SELECT * FROM department WHERE name = 'A' +-- UNION ALL +-- -- recursive term +-- SELECT d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +-- ) +-- SELECT * FROM subdepartment; +-- +--SELECT * FROM vsubdepartment ORDER BY name; +-- +---- Check reverse listing +--SELECT pg_get_viewdef('vsubdepartment'::regclass); +--SELECT pg_get_viewdef('vsubdepartment'::regclass, true); + +-- Another reverse-listing example +-- [SPARK-24497] Support recursive SQL query +--CREATE VIEW sums_1_100 AS +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; +-- +--\d+ sums_1_100 + +-- corner case in which sub-WITH gets initialized first +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with x as (select * from q) +-- select * from x) +-- ) +--select * from q limit 24; + +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with recursive x as ( +-- select * from department +-- union all +-- (select * from q union all select * from x) +-- ) +-- select * from x) +-- ) +--select * from q limit 32; + +-- recursive term has sub-UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(i,j) AS ( +-- VALUES (1,2) +-- UNION ALL +-- SELECT t2.i, t.j+1 FROM +-- (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 +-- JOIN t ON (t2.i = t.i+1)) +-- +-- SELECT * FROM t; + +-- +-- different tree example +-- +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE tree( +-- id INTEGER PRIMARY KEY, +-- parent_id INTEGER REFERENCES tree(id) +--); +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet; + +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11); + +-- +-- get all paths from "second level" nodes to leaf nodes +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- ORDER BY t1.id, t2.id; + +-- just count 'em +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, count(t2.*) FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- GROUP BY t1.id +-- ORDER BY t1.id; + +-- this variant tickled a whole-row-variable bug in 8.4devel +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, t2.path, t2 FROM t AS t1 JOIN t AS t2 ON +--(t1.id=t2.id); + +-- +-- test cycle detection +-- +-- [ORIGINAL SQL] +--create temp table graph( f int, t int, label text ); +create table graph( f int, t int, label string ) USING parquet; + +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1'); + +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph; + +-- ordering by the path column has same effect as SEARCH DEPTH FIRST +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph order by path; + +-- +-- test multiple WITH queries +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- y (id) AS (VALUES (1)), +-- x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- forward reference OK +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS (values (1)) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- +-- Test WITH attached to a data-modifying statement +-- + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- SELECT a FROM y +--) +--INSERT INTO y +--SELECT a+20 FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- SELECT a FROM y +--) +--UPDATE y SET a = y.a-10 FROM t WHERE y.a > 20 AND t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH RECURSIVE t(a) AS ( +-- SELECT 11 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a < 50 +--) +--DELETE FROM y USING t WHERE t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +DROP TABLE y; + +-- +-- error cases +-- + +-- INTERSECT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- EXCEPT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- no non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x) +-- SELECT * FROM x; + +-- recursive term in the left hand side (strictly speaking, should allow this) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) +-- SELECT * FROM x; + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- LEFT JOIN + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- RIGHT JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- FULL JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- subquery +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x +-- WHERE n IN (SELECT * FROM x)) +-- SELECT * FROM x; + +-- aggregate functions +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) +-- SELECT * FROM x; + +-- ORDER BY +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) +-- SELECT * FROM x; + +-- LIMIT/OFFSET +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) +-- SELECT * FROM x; + +-- FOR UPDATE +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) +-- SELECT * FROM x; + +-- target list has a recursive query name +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(id) AS (values (1) +-- UNION ALL +-- SELECT (SELECT * FROM x) FROM x WHERE id < 5 +--) SELECT * FROM x; + +-- mutual recursive query (not implemented) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), +-- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- non-linear recursion is not allowed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- SELECT * FROM +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) AS t +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- EXCEPT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- INTERSECT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- Wrong type induced from non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- rejects different typmod, too (should we allow this?) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- [NOTE] Spark SQL doesn't support RULEs +-- disallow OLD/NEW reference in CTE +--CREATE TABLE x (n integer) USING parquet; +--CREATE RULE r2 AS ON UPDATE TO x DO INSTEAD +-- WITH t AS (SELECT OLD.*) UPDATE y SET a = t.n FROM t; + +-- +-- test for bug #4902 +-- +-- [SPARK-28296] Improved VALUES support +--with cte(foo) as ( values(42) ) values((select foo from cte)); +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q; + +-- test CTE referencing an outer-level variable (to see that changed-parameter +-- signaling still works properly after fixing this bug) +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- select (select foo from cte) ) +--from int4_tbl; + +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- values((select foo from cte)) ) +--from int4_tbl; + +-- +-- test for nested-recursive-WITH bug +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(j) AS ( +-- WITH RECURSIVE s(i) AS ( +-- VALUES (1) +-- UNION ALL +-- SELECT i+1 FROM s WHERE i < 10 +-- ) +-- SELECT i FROM s +-- UNION ALL +-- SELECT j+1 FROM t WHERE j < 10 +--) +--SELECT * FROM t; + +-- +-- test WITH attached to intermediate-level set operation +-- + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1; + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- SELECT 1 +-- UNION (WITH innermost as (SELECT 2) +-- SELECT * FROM outermost +-- UNION SELECT * FROM innermost) +--) +--SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- WITH innermost as (SELECT 2 FROM outermost) -- fail +-- SELECT * FROM innermost +-- UNION SELECT * from outermost +--) +--SELECT * FROM outermost ORDER BY 1; + +-- +-- This test will fail with the old implementation of PARAM_EXEC parameter +-- assignment, because the "q1" Var passed down to A's targetlist subselect +-- looks exactly like the "A.id" Var passed down to C's subselect, causing +-- the old code to give them the same runtime PARAM_EXEC slot. But the +-- lifespans of the two parameters overlap, thanks to B also reading A. +-- + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +--with +--A as ( select q2 as id, (select q1) as x from int8_tbl ), +--B as ( select id, row_number() over (partition by id) as r from A ), +--C as ( select A.id, array(select B.id from B where B.id = A.id) from A ) +--select * from C; + +-- +-- Test CTEs read in non-initialization orders +-- + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ALL ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- +-- Data-modifying statements in WITH +-- + +-- INSERT ... RETURNING +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (11), +-- (12), +-- (13), +-- (14), +-- (15), +-- (16), +-- (17), +-- (18), +-- (19), +-- (20) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- UPDATE ... RETURNING +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y +-- SET a=a+1 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- DELETE ... RETURNING +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- forward reference +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT a+5 FROM t2 WHERE a > 5 +-- RETURNING * +--), t2 AS ( +-- UPDATE y SET a=a-11 RETURNING * +--) +--SELECT * FROM t +--UNION ALL +--SELECT * FROM t2; +-- +--SELECT * FROM y; + +-- unconditional DO INSTEAD rule +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON DELETE TO y DO INSTEAD +-- INSERT INTO y VALUES(42) RETURNING *; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +--DROP RULE y_rule ON y; + +-- check merging of outer CTE with CTE in a rule action +--CREATE TEMP TABLE bug6051 AS +-- select i from generate_series(1,3) as t(i); + +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE TEMP TABLE bug6051_2 (i int); +-- +--CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD +-- INSERT INTO bug6051_2 +-- SELECT NEW.i; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; +--SELECT * FROM bug6051_2; + +-- a truly recursive CTE in the same list +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(a) AS ( +-- SELECT 0 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a+1 < 5 +--), t2 as ( +-- INSERT INTO y +-- SELECT * FROM t RETURNING * +--) +--SELECT * FROM t2 JOIN y USING (a) ORDER BY a; +-- +--SELECT * FROM y; + +-- data-modifying WITH in a modifying statement +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--INSERT INTO y SELECT -a FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- check that WITH query is run to completion even if outer query isn't +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y SET a = a * 100 RETURNING * +--) +--SELECT * FROM t LIMIT 10; +-- +--SELECT * FROM y; + +-- data-modifying WITH containing INSERT...ON CONFLICT DO UPDATE +-- [ORIGINAL SQL] +--CREATE TABLE withz AS SELECT i AS k, (i || ' v')::text v FROM generate_series(1, 16, 3) i; +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i); +-- [NOTE] Spark SQL doesn't support UNIQUE constraints +--ALTER TABLE withz ADD UNIQUE (k); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- INSERT INTO withz SELECT i, 'insert' +-- FROM generate_series(0, 16) i +-- ON CONFLICT (k) DO UPDATE SET v = withz.v || ', now update' +-- RETURNING * +--) +--SELECT * FROM t JOIN y ON t.k = y.a ORDER BY a, k; + +-- Test EXCLUDED.* reference within CTE +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS ( +-- INSERT INTO withz VALUES(1, 5) ON CONFLICT (k) DO UPDATE SET v = EXCLUDED.v +-- WHERE withz.k != EXCLUDED.k +-- RETURNING * +--) +--SELECT * FROM aa; + +-- New query/snapshot demonstrates side-effects of previous query. +SELECT * FROM withz ORDER BY k; + +-- +-- Ensure subqueries within the update clause work, even if they +-- reference outside values +-- +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = ' update' WHERE withz.k = (SELECT a FROM aa); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 'a' a, 'b' b UNION ALL SELECT 'a' a, 'b' b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 'a' LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, (SELECT b || ' insert' FROM aa WHERE a = 1 )) +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); + +-- Update a row more than once, in different parts of a wCTE. That is +-- an allowed, presumably very rare, edge case, but since it was +-- broken in the past, having a test seems worthwhile. +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH simpletup AS ( +-- SELECT 2 k, 'Green' v), +--upsert_cte AS ( +-- INSERT INTO withz VALUES(2, 'Blue') ON CONFLICT (k) DO +-- UPDATE SET (k, v) = (SELECT k, v FROM simpletup WHERE simpletup.k = withz.k) +-- RETURNING k, v) +--INSERT INTO withz VALUES(2, 'Red') ON CONFLICT (k) DO +--UPDATE SET (k, v) = (SELECT k, v FROM upsert_cte WHERE upsert_cte.k = withz.k) +--RETURNING k, v; + +DROP TABLE withz; + +-- check that run to completion happens in proper ordering + +TRUNCATE TABLE y; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 3); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)); +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE yy (a INTEGER); +CREATE TABLE yy (a INTEGER) USING parquet; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--), t2 AS ( +-- INSERT INTO yy SELECT * FROM t1 RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO yy SELECT * FROM t2 RETURNING * +--), t2 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [NOTE] Spark SQL doesn't support TRIGGERs +-- triggers +-- +--TRUNCATE TABLE y; +--INSERT INTO y SELECT generate_series(1, 10); +-- +--CREATE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger: a = %', new.a; +-- return new; +--end; +--$$ LANGUAGE plpgsql; +-- +-- +--CREATE TRIGGER y_trig BEFORE INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (21), +-- (22), +-- (23) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (31), +-- (32), +-- (33) +-- RETURNING * +--) +--SELECT * FROM t LIMIT 1; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE OR REPLACE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger'; +-- return null; +--end; +--$$ LANGUAGE plpgsql; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH STATEMENT +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (41), +-- (42), +-- (43) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +--DROP FUNCTION y_trigger(); + +-- WITH attached to inherited UPDATE or DELETE + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE parent ( id int, val text ); +CREATE TABLE parent ( id int, val string ) USING parquet; +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child1 ( ) INHERITS ( parent ); +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child2 ( ) INHERITS ( parent ); + +INSERT INTO parent VALUES ( 1, 'p1' ); +--INSERT INTO child1 VALUES ( 11, 'c11' ),( 12, 'c12' ); +--INSERT INTO child2 VALUES ( 23, 'c21' ),( 24, 'c22' ); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH rcte AS ( SELECT sum(id) AS totalid FROM parent ) +--UPDATE parent SET id = id + totalid FROM rcte; + +SELECT * FROM parent; + +-- [SPARK-28147] Support RETURNING clause +--WITH wcte AS ( INSERT INTO child1 VALUES ( 42, 'new' ) RETURNING id AS newid ) +--UPDATE parent SET id = id + newid FROM wcte; +-- +--SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH rcte AS ( SELECT max(id) AS maxid FROM parent ) +--DELETE FROM parent USING rcte WHERE id = maxid; + +SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH wcte AS ( INSERT INTO child2 VALUES ( 42, 'new2' ) RETURNING id AS newid ) +--DELETE FROM parent USING wcte WHERE id = newid; +-- +--SELECT * FROM parent; + +-- check EXPLAIN VERBOSE for a wCTE with RETURNING + +-- [NOTE] Spark SQL doesn't support DELETE statement +--EXPLAIN (VERBOSE, COSTS OFF) +--WITH wcte AS ( INSERT INTO int8_tbl VALUES ( 42, 47 ) RETURNING q2 ) +--DELETE FROM a USING wcte WHERE aa = q2; + +-- error cases + +-- data-modifying WITH tries to use its own output +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT * FROM t +--) +--VALUES(FALSE); + +-- no RETURNING in a referenced data-modifying WITH +-- [SPARK-24497] Support recursive SQL query +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--SELECT * FROM t; + +-- data-modifying WITH allowed only at the top level +-- [SPARK-28147] Support RETURNING clause +--SELECT * FROM ( +-- WITH t AS (UPDATE y SET a=a+1 RETURNING *) +-- SELECT * FROM t +--) ss; + +-- most variants of rules aren't allowed +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON INSERT TO y WHERE a=0 DO INSTEAD DELETE FROM y; +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--VALUES(FALSE); +--DROP RULE y_rule ON y; + +-- check that parser lookahead for WITH doesn't cause any odd behavior +create table foo (with baz); -- fail, WITH is a reserved word +create table foo (with ordinality); -- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality; + +-- check sane response to attempt to modify CTE relation +WITH test AS (SELECT 42) INSERT INTO test VALUES (1); + +-- check response to attempt to modify table with same name as a CTE (perhaps +-- surprisingly it works, because CTEs don't hide tables from data-modifying +-- statements) +-- [ORIGINAL SQL] +--create temp table test (i int); +create table test (i int) USING parquet; +with test as (select 42) insert into test select * from test; +select * from test; +drop table test; + +-- +-- Clean up +-- + +DROP TABLE department; +DROP TABLE tree; +DROP TABLE graph; +DROP TABLE y; +DROP TABLE yy; +DROP TABLE parent; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql new file mode 100644 index 0000000000000..d829a5c1159fd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql @@ -0,0 +1,154 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 1] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- This test file was converted from pgSQL/aggregates_part1.sql. + +SELECT avg(udf(four)) AS avg_1 FROM onek; + +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100; + +-- In 7.1, avg(float4) is computed using float8 arithmetic. +-- Round the result to 3 digits to avoid platform-specific results. + +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; + +SELECT sum(udf(four)) AS sum_1500 FROM onek; +SELECT udf(sum(a)) AS sum_198 FROM aggtest; +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; + +SELECT udf(max(four)) AS max_3 FROM onek; +SELECT max(udf(a)) AS max_100 FROM aggtest; +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT max(student.gpa) AS max_3_7 FROM student; + +SELECT stddev_pop(udf(b)) FROM aggtest; +SELECT udf(stddev_samp(b)) FROM aggtest; +SELECT var_pop(udf(b)) FROM aggtest; +SELECT udf(var_samp(b)) FROM aggtest; + +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest; +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest; + +-- population variance is defined for a single tuple, sample variance +-- is not +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)); +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))); + + +-- verify correct results for null and NaN inputs +select sum(udf(CAST(null AS int))) from range(1,4); +select sum(udf(CAST(null AS long))) from range(1,4); +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select sum(udf(CAST(null AS DOUBLE))) from range(1,4); +select avg(udf(CAST(null AS int))) from range(1,4); +select avg(udf(CAST(null AS long))) from range(1,4); +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select avg(udf(CAST(null AS DOUBLE))) from range(1,4); +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4); +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4); + +-- [SPARK-27768] verify correct results for infinite inputs +-- [SPARK-28291] UDFs cannot be evaluated within inline table definition +-- SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +-- FROM (VALUES (CAST(udf('1') AS DOUBLE)), (CAST(udf('Infinity') AS DOUBLE))) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x); + + +-- test accuracy with a large input offset +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x); + +-- SQL2003 binary aggregates [SPARK-23907] +-- SELECT regr_count(b, a) FROM aggtest; +-- SELECT regr_sxx(b, a) FROM aggtest; +-- SELECT regr_syy(b, a) FROM aggtest; +-- SELECT regr_sxy(b, a) FROM aggtest; +-- SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; +-- SELECT regr_r2(b, a) FROM aggtest; +-- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest; +SELECT corr(b, udf(a)) FROM aggtest; + + +-- test accum and combine functions directly [SPARK-23907] +-- CREATE TABLE regr_test (x float8, y float8); +-- INSERT INTO regr_test VALUES (10,150),(20,250),(30,350),(80,540),(100,200); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30,80); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test; +-- SELECT float8_accum('{4,140,2900}'::float8[], 100); +-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (80,100); +-- SELECT float8_combine('{3,60,200}'::float8[],ELECT CAST(udf(covar_pop(b, udf(a))) AS '{0,0,0}'::float8[]); +-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{0,0,0,0,0,0}'::float8[]); +-- SELECT float8_regr_combine('{0,0,0,0,0,0}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- DROP TABLE regr_test; + + +-- test count, distinct +SELECT count(udf(four)) AS cnt_1000 FROM onek; +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek; + +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten; + +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten; + +-- user-defined aggregates +-- SELECT newavg(four) AS avg_1 FROM onek; +-- SELECT newsum(four) AS sum_1500 FROM onek; +-- SELECT newcnt(four) AS cnt_1000 FROM onek; +-- SELECT newcnt(*) AS cnt_1000 FROM onek; +-- SELECT oldcnt(*) AS cnt_1000 FROM onek; +-- SELECT sum2(q1,q2) FROM int8_tbl; + +-- test for outer-level aggregates + +-- this should work +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four); + +-- this should fail because subquery has an agg of its own in WHERE +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)); + +-- [SPARK-27769] Test handling of sublinks within outer-level aggregates. +-- Per bug report from Daniel Grace. +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql new file mode 100644 index 0000000000000..5636537398a86 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql @@ -0,0 +1,230 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 2] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 +-- +-- This test file was converted from pgSQL/aggregates_part2.sql. + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- Test handling of Params within aggregate arguments in hashed aggregation. +-- Per bug report from Jeevan Chalke. +-- [SPARK-27877] Implement SQL-standard LATERAL subqueries +-- explain (verbose, costs off) +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- explain (verbose, costs off) +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; + +-- [SPARK-27879] Implement bitwise integer aggregates(BIT_AND and BIT_OR) +-- +-- test for bitwise integer aggregates +-- +-- CREATE TEMPORARY TABLE bitwise_test( +-- i2 INT2, +-- i4 INT4, +-- i8 INT8, +-- i INTEGER, +-- x INT2, +-- y BIT(4) +-- ); + +-- empty case +-- SELECT +-- BIT_AND(i2) AS "?", +-- BIT_OR(i4) AS "?" +-- FROM bitwise_test; + +-- COPY bitwise_test FROM STDIN NULL 'null'; +-- 1 1 1 1 1 B0101 +-- 3 3 3 null 2 B0100 +-- 7 7 7 3 4 B1100 +-- \. + +-- SELECT +-- BIT_AND(i2) AS "1", +-- BIT_AND(i4) AS "1", +-- BIT_AND(i8) AS "1", +-- BIT_AND(i) AS "?", +-- BIT_AND(x) AS "0", +-- BIT_AND(y) AS "0100", +-- +-- BIT_OR(i2) AS "7", +-- BIT_OR(i4) AS "7", +-- BIT_OR(i8) AS "7", +-- BIT_OR(i) AS "?", +-- BIT_OR(x) AS "7", +-- BIT_OR(y) AS "1101" +-- FROM bitwise_test; + +-- +-- test boolean aggregates +-- +-- first test all possible transition and final states + +-- The result is inconsistent with PostgreSQL because our AND does not have strict mode +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t`; + +-- The result is inconsistent with PostgreSQL because our OR does not have strict mode +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t`; + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- CREATE TEMPORARY TABLE bool_test( +-- b1 BOOL, +-- b2 BOOL, +-- b3 BOOL, +-- b4 BOOL); + +-- empty case +-- SELECT +-- BOOL_AND(b1) AS "n", +-- BOOL_OR(b3) AS "n" +-- FROM bool_test; + +-- COPY bool_test FROM STDIN NULL 'null'; +-- TRUE null FALSE null +-- FALSE TRUE null null +-- null TRUE FALSE null +-- \. + +-- SELECT +-- BOOL_AND(b1) AS "f", +-- BOOL_AND(b2) AS "t", +-- BOOL_AND(b3) AS "f", +-- BOOL_AND(b4) AS "n", +-- BOOL_AND(NOT b2) AS "f", +-- BOOL_AND(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- EVERY(b1) AS "f", +-- EVERY(b2) AS "t", +-- EVERY(b3) AS "f", +-- EVERY(b4) AS "n", +-- EVERY(NOT b2) AS "f", +-- EVERY(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- BOOL_OR(b1) AS "t", +-- BOOL_OR(b2) AS "t", +-- BOOL_OR(b3) AS "f", +-- BOOL_OR(b4) AS "n", +-- BOOL_OR(NOT b2) AS "f", +-- BOOL_OR(NOT b3) AS "t" +-- FROM bool_test; + +-- +-- Test cases that should be optimized into indexscans instead of +-- the generic aggregate implementation. +-- + +-- Basic cases +-- explain +-- select min(unique1) from tenk1; +select min(udf(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1; +select udf(max(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1 where unique1 < 42; +select max(unique1) from tenk1 where udf(unique1) < 42; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42; +select max(unique1) from tenk1 where unique1 > udf(42); + +-- the planner may choose a generic aggregate here if parallel query is +-- enabled, since that plan will be parallel safe and the "optimized" +-- plan, which has almost identical cost, will not be. we want to test +-- the optimized plan, so temporarily disable parallel query. +-- begin; +-- set local max_parallel_workers_per_gather = 0; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42000; +select max(unique1) from tenk1 where udf(unique1) > 42000; +-- rollback; + +-- multi-column index (uses tenk1_thous_tenthous) +-- explain +-- select max(tenthous) from tenk1 where thousand = 33; +select max(tenthous) from tenk1 where udf(thousand) = 33; +-- explain +-- select min(tenthous) from tenk1 where thousand = 33; +select min(tenthous) from tenk1 where udf(thousand) = 33; + +-- [SPARK-17348] Correlated column is not allowed in a non-equality predicate +-- check parameter propagation into an indexscan subquery +-- explain +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; + +-- check some cases that were handled incorrectly in 8.3.0 +-- explain +-- select distinct max(unique2) from tenk1; +select distinct max(udf(unique2)) from tenk1; +-- explain +-- select max(unique2) from tenk1 order by 1; +select max(unique2) from tenk1 order by udf(1); +-- explain +-- select max(unique2) from tenk1 order by max(unique2); +select max(unique2) from tenk1 order by max(udf(unique2)); +-- explain +-- select max(unique2) from tenk1 order by max(unique2)+1; +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1; +-- explain +-- select max(unique2), generate_series(1,3) as g from tenk1 order by g desc; +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc; + +-- interesting corner case: constant gets optimized into a seqscan +-- explain +-- select max(100) from tenk1; +select udf(max(100)) from tenk1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql index 5ec4cb1d6326f..1865ee94ec1f9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql @@ -3,17 +3,11 @@ -- -- -- CASE --- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/case.sql +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql -- Test the CASE statement -- --- This test suite contains two Cartesian products without using explicit CROSS JOIN syntax. --- Thus, we set spark.sql.crossJoin.enabled to true. - -- This test file was converted from pgSQL/case.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. -set spark.sql.crossJoin.enabled=true; CREATE TABLE CASE_TBL ( i integer, f double @@ -42,7 +36,7 @@ INSERT INTO CASE2_TBL VALUES (NULL, -6); SELECT '3' AS `One`, CASE - WHEN CAST(udf(1 < 2) AS boolean) THEN 3 + WHEN udf(1 < 2) THEN 3 END AS `Simple WHEN`; SELECT '' AS `One`, @@ -64,7 +58,7 @@ SELECT udf('4') AS `One`, SELECT udf('6') AS `One`, CASE - WHEN CAST(udf(1 > 2) AS boolean) THEN 3 + WHEN udf(1 > 2) THEN 3 WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default`; @@ -74,7 +68,7 @@ SELECT '7' AS `None`, END AS `NULL on no matches`; -- Constant-expression folding shouldn't evaluate unreachable subexpressions -SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END; -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL @@ -146,7 +140,7 @@ SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b - WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean); + WHERE udf(COALESCE(f,b.i) = 2); -- We don't support update now. -- @@ -269,4 +263,3 @@ SELECT CASE DROP TABLE CASE_TBL; DROP TABLE CASE2_TBL; -set spark.sql.crossJoin.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql new file mode 100644 index 0000000000000..6ab207b93f561 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql @@ -0,0 +1,28 @@ +-- This test file was converted from count.sql +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData; + +-- distinct count with single expression +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData; + +-- count with multiple expressions +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData; + +-- distinct count with multiple expressions +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) +FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql new file mode 100644 index 0000000000000..9aa6de7e92b8e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql @@ -0,0 +1,37 @@ +-- Cross join detection and error checking is done in JoinSuite since explain output is +-- used in the error message and the ids are not stable. Only positive cases are checked here. +-- This test file was converted from cross-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +-- Cross joins with and without predicates +SELECT * FROM nt1 cross join nt2; +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k); +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)); +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22"; + +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b; + +-- Join reordering +create temporary view A(a, va) as select * from nt1; +create temporary view B(b, vb) as select * from nt1; +create temporary view C(c, vc) as select * from nt1; +create temporary view D(d, vd) as select * from nt1; + +-- Allowed since cross join with C is explicit +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)); +-- Cross joins with non-equal predicates +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql new file mode 100644 index 0000000000000..71e05e67698c8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from except-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1); +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v); +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v); + +-- Basic EXCEPT ALL +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- MINUS ALL (synonym for EXCEPT) +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2; + +-- EXCEPT ALL same table in both branches +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL; + +-- Empty left relation +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- Empty right relation +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)); + +-- Type Coerced ExceptAll +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1); + +-- Basic +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4; + +-- Basic +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3; + +-- EXCEPT ALL + INTERSECT +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- EXCEPT ALL + EXCEPT +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Mismatch on number of columns across both branches +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4; + +-- Chain of set operations +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Using MINUS ALL +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4; + +-- Join under except all. Should produce empty resultset since both left and right sets +-- are same. +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)); + +-- Join under except all (2) +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)); + +-- Group by under ExceptAll +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; +DROP VIEW IF EXISTS tab3; +DROP VIEW IF EXISTS tab4; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql new file mode 100644 index 0000000000000..ff8573ad7e562 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql @@ -0,0 +1,20 @@ +-- This test file was converted from having.sql. + +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2; + +-- having condition contains grouping column +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2); + +-- SPARK-11032: resolve having correctly +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0); + +-- SPARK-20329: make sure we handle timezones correctly +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql new file mode 100644 index 0000000000000..028d4c7695995 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from intersect-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v); + +-- Basic INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2; + +-- INTERSECT ALL same table in both branches +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1; + +-- Empty left relation +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Empty right relation +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3; + +-- Type Coerced INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2); + +-- Mismatch on number of columns across both branches +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Basic +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Chain of different `set operations +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +; + +-- Chain of different `set operations +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +; + +-- test use parenthesis to control order of evaluation +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +; + +-- Join under intersect all +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))); + +-- Join under intersect all (2) +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)); + +-- Group by under intersect all +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k; + +-- Test pre spark2.4 behaviour of set operation precedence +-- All the set operators are given equal precedence and are evaluated +-- from left to right as they appear in the query. + +-- Set the property +SET spark.sql.legacy.setopsPrecedence.enabled= true; + +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2; + +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2; + +-- Restore the property +SET spark.sql.legacy.setopsPrecedence.enabled = false; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql new file mode 100644 index 0000000000000..e5eb812d69a1c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql @@ -0,0 +1,27 @@ +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- This test file was converted from natural-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + + +SELECT * FROM nt1 natural join nt2 where udf(k) = "one"; + +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2; + +SELECT udf(count(*)) FROM nt1 natural full outer join nt2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql new file mode 100644 index 0000000000000..93937930de7f5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql @@ -0,0 +1,307 @@ +-- This test file was converted from pivot.sql. + +-- Note some test cases have been commented as the current integrated UDFs cannot handle complex types + +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings); + +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s); + +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s); + +-- pivot courses +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot years with no subquery +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot courses with multiple aggregations +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column and with multiple aggregations on different columns +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple group by columns +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +); + +-- pivot on join query with multiple aggregations on different columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple columns in one aggregation +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with aliases and projection +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +); + +-- pivot with projection and value aliases +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +); + +-- pivot years with non-aggregate function +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with one of the expressions as non-aggregate function +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +); + +-- pivot with unresolvable columns +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with complex aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +); + +-- pivot with invalid arguments in aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on multiple pivot columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +); + +-- pivot on multiple pivot columns with aliased values +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +); + +-- pivot on multiple pivot columns with values of wrong data types +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +); + +-- pivot with unresolvable values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +); + +-- pivot with non-literal values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on join query with columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(min(a)) +-- FOR course IN ('dotNET', 'Java') +--); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on multiple pivot columns with agg columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, y, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(max(a)) +-- FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +--); + +-- pivot on pivot column of array type +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +); + +-- pivot on multiple pivot columns containing array type +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +); + +-- pivot on pivot column of struct type +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +); + +-- pivot on multiple pivot columns containing struct type +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +); + +-- pivot on pivot column of map type +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +); + +-- pivot on multiple pivot columns containing map type +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +); + +-- grouping columns output in the same order as input +-- correctly handle pivot columns with different cases +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql new file mode 100644 index 0000000000000..9cd15369bb164 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql @@ -0,0 +1,8 @@ +-- This file tests special values such as NaN, Infinity and NULL. + +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x); diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out new file mode 100644 index 0000000000000..5193e2536c0cc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SET spark.sql.legacy.ctePrecedence.enabled=true +-- !query 2 schema +struct +-- !query 2 output +spark.sql.legacy.ctePrecedence.enabled true + + +-- !query 3 +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query 3 schema +struct<1:int> +-- !query 3 output +1 + + +-- !query 4 +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query 5 schema +struct +-- !query 5 output +1 + + +-- !query 6 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query 6 schema +struct<1:int> +-- !query 6 output +1 + + +-- !query 7 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query 8 schema +struct<2:int> +-- !query 8 output +2 + + +-- !query 9 +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query 9 schema +struct +-- !query 9 output +2 + + +-- !query 10 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 10 schema +struct +-- !query 10 output +2 + + +-- !query 11 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 11 schema +struct +-- !query 11 output +3 + + +-- !query 12 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +DROP VIEW IF EXISTS t +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +DROP VIEW IF EXISTS t2 +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 9e90908d92faf..b7dd76c725209 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -204,9 +204,9 @@ WITH ) SELECT * FROM t2 -- !query 16 schema -struct<1:int> +struct<2:int> -- !query 16 output -1 +2 -- !query 17 @@ -224,7 +224,7 @@ SELECT * FROM t2 -- !query 17 schema struct -- !query 17 output -1 +2 -- !query 18 @@ -240,9 +240,9 @@ WITH ) SELECT * FROM t2 -- !query 18 schema -struct<2:int> +struct<3:int> -- !query 18 output -2 +3 -- !query 19 @@ -295,7 +295,7 @@ SELECT ( -- !query 22 schema struct -- !query 22 output -1 +2 -- !query 23 @@ -309,7 +309,7 @@ SELECT ( -- !query 23 schema struct -- !query 23 output -1 +2 -- !query 24 @@ -324,7 +324,7 @@ SELECT ( -- !query 24 schema struct -- !query 24 output -1 +3 -- !query 25 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out index 99c42ec2eb6ca..b7cf3a9f1ad84 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out @@ -19,7 +19,7 @@ true -- !query 2 -SELECT false AS false +SELECT false AS `false` -- !query 2 schema struct -- !query 2 output @@ -35,7 +35,7 @@ true -- !query 4 -SELECT boolean(' f ') AS false +SELECT boolean(' f ') AS `false` -- !query 4 schema struct -- !query 4 output @@ -59,7 +59,7 @@ NULL -- !query 7 -SELECT boolean('false') AS false +SELECT boolean('false') AS `false` -- !query 7 schema struct -- !query 7 output @@ -99,7 +99,7 @@ NULL -- !query 12 -SELECT boolean('n') AS false +SELECT boolean('n') AS `false` -- !query 12 schema struct -- !query 12 output @@ -107,7 +107,7 @@ false -- !query 13 -SELECT boolean('no') AS false +SELECT boolean('no') AS `false` -- !query 13 schema struct -- !query 13 output @@ -131,7 +131,7 @@ NULL -- !query 16 -SELECT boolean('off') AS false +SELECT boolean('off') AS `false` -- !query 16 schema struct -- !query 16 output @@ -139,7 +139,7 @@ NULL -- !query 17 -SELECT boolean('of') AS false +SELECT boolean('of') AS `false` -- !query 17 schema struct -- !query 17 output @@ -187,7 +187,7 @@ NULL -- !query 23 -SELECT boolean('0') AS false +SELECT boolean('0') AS `false` -- !query 23 schema struct -- !query 23 output @@ -219,7 +219,7 @@ true -- !query 27 -SELECT boolean('t') and boolean('f') AS false +SELECT boolean('t') and boolean('f') AS `false` -- !query 27 schema struct -- !query 27 output @@ -235,7 +235,7 @@ true -- !query 29 -SELECT boolean('t') = boolean('f') AS false +SELECT boolean('t') = boolean('f') AS `false` -- !query 29 schema struct -- !query 29 output @@ -283,7 +283,7 @@ true -- !query 35 -SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS false +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` -- !query 35 schema struct -- !query 35 output @@ -292,7 +292,7 @@ true false -- !query 36 SELECT boolean(string(' true ')) AS true, - boolean(string(' FALSE')) AS false + boolean(string(' FALSE')) AS `false` -- !query 36 schema struct -- !query 36 output @@ -300,7 +300,7 @@ NULL NULL -- !query 37 -SELECT string(boolean(true)) AS true, string(boolean(false)) AS false +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` -- !query 37 schema struct -- !query 37 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out index dbd775e5ebba9..f95adcde81b3f 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out @@ -1,19 +1,22 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 35 -- !query 0 -set spark.sql.crossJoin.enabled=true +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet -- !query 0 schema -struct +struct<> -- !query 0 output -spark.sql.crossJoin.enabled true + -- !query 1 -CREATE TABLE CASE_TBL ( +CREATE TABLE CASE2_TBL ( i integer, - f double + j integer ) USING parquet -- !query 1 schema struct<> @@ -22,10 +25,7 @@ struct<> -- !query 2 -CREATE TABLE CASE2_TBL ( - i integer, - j integer -) USING parquet +INSERT INTO CASE_TBL VALUES (1, 10.1) -- !query 2 schema struct<> -- !query 2 output @@ -33,7 +33,7 @@ struct<> -- !query 3 -INSERT INTO CASE_TBL VALUES (1, 10.1) +INSERT INTO CASE_TBL VALUES (2, 20.2) -- !query 3 schema struct<> -- !query 3 output @@ -41,7 +41,7 @@ struct<> -- !query 4 -INSERT INTO CASE_TBL VALUES (2, 20.2) +INSERT INTO CASE_TBL VALUES (3, -30.3) -- !query 4 schema struct<> -- !query 4 output @@ -49,7 +49,7 @@ struct<> -- !query 5 -INSERT INTO CASE_TBL VALUES (3, -30.3) +INSERT INTO CASE_TBL VALUES (4, NULL) -- !query 5 schema struct<> -- !query 5 output @@ -57,7 +57,7 @@ struct<> -- !query 6 -INSERT INTO CASE_TBL VALUES (4, NULL) +INSERT INTO CASE2_TBL VALUES (1, -1) -- !query 6 schema struct<> -- !query 6 output @@ -65,7 +65,7 @@ struct<> -- !query 7 -INSERT INTO CASE2_TBL VALUES (1, -1) +INSERT INTO CASE2_TBL VALUES (2, -2) -- !query 7 schema struct<> -- !query 7 output @@ -73,7 +73,7 @@ struct<> -- !query 8 -INSERT INTO CASE2_TBL VALUES (2, -2) +INSERT INTO CASE2_TBL VALUES (3, -3) -- !query 8 schema struct<> -- !query 8 output @@ -81,7 +81,7 @@ struct<> -- !query 9 -INSERT INTO CASE2_TBL VALUES (3, -3) +INSERT INTO CASE2_TBL VALUES (2, -4) -- !query 9 schema struct<> -- !query 9 output @@ -89,7 +89,7 @@ struct<> -- !query 10 -INSERT INTO CASE2_TBL VALUES (2, -4) +INSERT INTO CASE2_TBL VALUES (1, NULL) -- !query 10 schema struct<> -- !query 10 output @@ -97,7 +97,7 @@ struct<> -- !query 11 -INSERT INTO CASE2_TBL VALUES (1, NULL) +INSERT INTO CASE2_TBL VALUES (NULL, -6) -- !query 11 schema struct<> -- !query 11 output @@ -105,148 +105,140 @@ struct<> -- !query 12 -INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 12 schema -struct<> --- !query 12 output - - - --- !query 13 SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 END AS `Simple WHEN` --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output 3 3 --- !query 14 +-- !query 13 SELECT '' AS `One`, CASE WHEN 1 > 2 THEN 3 END AS `Simple default` --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output NULL --- !query 15 +-- !query 14 SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 ELSE 4 END AS `Simple ELSE` --- !query 15 schema +-- !query 14 schema struct --- !query 15 output +-- !query 14 output 3 3 --- !query 16 +-- !query 15 SELECT '4' AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 16 schema +-- !query 15 schema struct --- !query 16 output +-- !query 15 output 4 4 --- !query 17 +-- !query 16 SELECT '6' AS `One`, CASE WHEN 1 > 2 THEN 3 WHEN 4 < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output 6 6 --- !query 18 +-- !query 17 SELECT '7' AS `None`, CASE WHEN rand() < 0 THEN 1 END AS `NULL on no matches` --- !query 18 schema +-- !query 17 schema struct --- !query 18 output +-- !query 17 output 7 NULL --- !query 19 +-- !query 18 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + +-- !query 19 +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1.0 +1 -- !query 20 -SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl -- !query 20 schema -struct +struct 100) THEN (1 div 0) ELSE 0 END:int> -- !query 20 output -1.0 +0 +0 +0 +0 -- !query 21 -SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl --- !query 21 schema -struct 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> --- !query 21 output -0.0 -0.0 -0.0 -0.0 - - --- !query 22 SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END --- !query 22 schema +-- !query 21 schema struct --- !query 22 output +-- !query 21 output 1 --- !query 23 +-- !query 22 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 23 schema +-- !query 22 schema struct= 3 or Null:int> --- !query 23 output +-- !query 22 output 3 4 NULL NULL --- !query 24 +-- !query 23 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output 1 2 6 8 --- !query 25 +-- !query 24 SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -255,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query 24 schema struct --- !query 25 output +-- !query 24 output 1 one 2 two 3 big 4 big --- !query 26 +-- !query 25 SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -273,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 26 schema +-- !query 25 schema struct --- !query 26 output +-- !query 25 output big big one two --- !query 27 +-- !query 26 SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 --- !query 27 schema +-- !query 26 schema struct --- !query 27 output +-- !query 26 output 4 NULL --- !query 28 +-- !query 27 SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 --- !query 28 schema +-- !query 27 schema struct --- !query 28 output +-- !query 27 output --- !query 29 +-- !query 28 SELECT COALESCE(a.f, b.i, b.j) FROM CASE_TBL a, CASE2_TBL b --- !query 29 schema +-- !query 28 schema struct --- !query 29 output +-- !query 28 output -30.3 -30.3 -30.3 @@ -330,24 +322,24 @@ struct 3.0 --- !query 30 +-- !query 29 SELECT * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(a.f, b.i, b.j) = 2 --- !query 30 schema +-- !query 29 schema struct --- !query 30 output +-- !query 29 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 31 +-- !query 30 SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 31 schema +-- !query 30 schema struct --- !query 31 output +-- !query 30 output 1 2 1 2 1 3 @@ -374,18 +366,18 @@ struct NULL 3 --- !query 32 +-- !query 31 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(f,b.i) = 2 --- !query 32 schema +-- !query 31 schema struct --- !query 32 output +-- !query 31 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 33 +-- !query 32 SELECT CASE (CASE vol('bar') WHEN 'foo' THEN 'it was foo!' @@ -395,31 +387,23 @@ SELECT CASE WHEN 'it was foo!' THEN 'foo recognized' WHEN 'it was bar!' THEN 'bar recognized' ELSE 'unrecognized' END --- !query 33 schema +-- !query 32 schema struct --- !query 33 output +-- !query 32 output bar recognized --- !query 34 +-- !query 33 DROP TABLE CASE_TBL --- !query 34 schema +-- !query 33 schema struct<> --- !query 34 output +-- !query 33 output --- !query 35 +-- !query 34 DROP TABLE CASE2_TBL --- !query 35 schema +-- !query 34 schema struct<> --- !query 35 output - - +-- !query 34 output --- !query 36 -set spark.sql.crossJoin.enabled=false --- !query 36 schema -struct --- !query 36 output -spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index a0630b9f3f826..0d669ae7ce5b5 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -198,17 +198,29 @@ struct -- !query 21 SELECT date '1999 Jan 08' -- !query 21 schema -struct +struct<> -- !query 21 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ -- !query 22 SELECT date '1999 08 Jan' -- !query 22 schema -struct +struct<> -- !query 22 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ -- !query 23 @@ -230,17 +242,29 @@ struct -- !query 25 SELECT date '1999 01 08' -- !query 25 schema -struct +struct<> -- !query 25 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ -- !query 26 SELECT date '1999 08 01' -- !query 26 schema -struct +struct<> -- !query 26 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ -- !query 27 @@ -254,17 +278,29 @@ struct -- !query 28 SELECT date '1999 Jan 08' -- !query 28 schema -struct +struct<> -- !query 28 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ -- !query 29 SELECT date '1999 08 Jan' -- !query 29 schema -struct +struct<> -- !query 29 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ -- !query 30 @@ -286,17 +322,29 @@ struct -- !query 32 SELECT date '1999 01 08' -- !query 32 schema -struct +struct<> -- !query 32 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ -- !query 33 SELECT date '1999 08 01' -- !query 33 schema -struct +struct<> -- !query 33 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ -- !query 34 @@ -318,17 +366,29 @@ struct -- !query 36 SELECT date '1999 Jan 08' -- !query 36 schema -struct +struct<> -- !query 36 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ -- !query 37 SELECT date '1999 08 Jan' -- !query 37 schema -struct +struct<> -- !query 37 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ -- !query 38 @@ -350,17 +410,29 @@ struct -- !query 40 SELECT date '1999 01 08' -- !query 40 schema -struct +struct<> -- !query 40 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ -- !query 41 SELECT date '1999 08 01' -- !query 41 schema -struct +struct<> -- !query 41 output -1999-01-01 +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ -- !query 42 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out new file mode 100644 index 0000000000000..3e3f24d603ff0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -0,0 +1,839 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 95 + + +-- !query 0 +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO FLOAT8_TBL VALUES (' 0.0 ') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO FLOAT8_TBL VALUES ('1004.30 ') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO FLOAT8_TBL VALUES (' -34.84') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT double('10e400') +-- !query 6 schema +struct +-- !query 6 output +Infinity + + +-- !query 7 +SELECT double('-10e400') +-- !query 7 schema +struct +-- !query 7 output +-Infinity + + +-- !query 8 +SELECT double('10e-400') +-- !query 8 schema +struct +-- !query 8 output +0.0 + + +-- !query 9 +SELECT double('-10e-400') +-- !query 9 schema +struct +-- !query 9 output +-0.0 + + +-- !query 10 +SELECT double('NaN') +-- !query 10 schema +struct +-- !query 10 output +NaN + + +-- !query 11 +SELECT double('nan') +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT double(' NAN ') +-- !query 12 schema +struct +-- !query 12 output +NULL + + +-- !query 13 +SELECT double('infinity') +-- !query 13 schema +struct +-- !query 13 output +NULL + + +-- !query 14 +SELECT double(' -INFINiTY ') +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +SELECT double('N A N') +-- !query 15 schema +struct +-- !query 15 output +NULL + + +-- !query 16 +SELECT double('NaN x') +-- !query 16 schema +struct +-- !query 16 output +NULL + + +-- !query 17 +SELECT double(' INFINITY x') +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT double('Infinity') + 100.0 +-- !query 18 schema +struct<(CAST(Infinity AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> +-- !query 18 output +Infinity + + +-- !query 19 +SELECT double('Infinity') / double('Infinity') +-- !query 19 schema +struct<(CAST(Infinity AS DOUBLE) / CAST(Infinity AS DOUBLE)):double> +-- !query 19 output +NaN + + +-- !query 20 +SELECT double('NaN') / double('NaN') +-- !query 20 schema +struct<(CAST(NaN AS DOUBLE) / CAST(NaN AS DOUBLE)):double> +-- !query 20 output +NaN + + +-- !query 21 +SELECT double(decimal('nan')) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 22 schema +struct +-- !query 22 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 23 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' +-- !query 23 schema +struct +-- !query 23 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + + +-- !query 24 +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' +-- !query 24 schema +struct +-- !query 24 output +1004.3 + + +-- !query 25 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 +-- !query 25 schema +struct +-- !query 25 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 26 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' +-- !query 26 schema +struct +-- !query 26 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 27 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 +-- !query 27 schema +struct +-- !query 27 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 28 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' +-- !query 28 schema +struct +-- !query 28 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 29 +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 29 schema +struct +-- !query 29 output +1.2345678901234E-200 -1.2345678901234E-199 + 1.2345678901234E200 -1.2345678901234E201 + 1004.3 -10043.0 + + +-- !query 30 +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 30 schema +struct +-- !query 30 output +1.2345678901234E-200 -10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 994.3 + + +-- !query 31 +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 31 schema +struct +-- !query 31 output +1.2345678901234E-200 -1.2345678901234E-201 + 1.2345678901234E200 -1.2345678901234E199 + 1004.3 -100.42999999999999 + + +-- !query 32 +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 32 schema +struct +-- !query 32 output +1.2345678901234E-200 10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1014.3 + + +-- !query 33 +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f +-- !query 33 schema +struct +-- !query 33 output +-34.84 -35.0 + 0.0 0.0 + 1.2345678901234E-200 0.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1004.0 + + +-- !query 34 +select ceil(f1) as ceil_f1 from float8_tbl f +-- !query 34 schema +struct +-- !query 34 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 35 +select ceiling(f1) as ceiling_f1 from float8_tbl f +-- !query 35 schema +struct +-- !query 35 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 36 +select floor(f1) as floor_f1 from float8_tbl f +-- !query 36 schema +struct +-- !query 36 output +-35 +0 +0 +1004 +9223372036854775807 + + +-- !query 37 +select sign(f1) as sign_f1 from float8_tbl f +-- !query 37 schema +struct +-- !query 37 output +-1.0 +0.0 +1.0 +1.0 +1.0 + + +-- !query 38 +SELECT sqrt(double('64')) AS eight +-- !query 38 schema +struct +-- !query 38 output +8.0 + + +-- !query 39 +SELECT power(double('144'), double('0.5')) +-- !query 39 schema +struct +-- !query 39 output +12.0 + + +-- !query 40 +SELECT power(double('NaN'), double('0.5')) +-- !query 40 schema +struct +-- !query 40 output +NaN + + +-- !query 41 +SELECT power(double('144'), double('NaN')) +-- !query 41 schema +struct +-- !query 41 output +NaN + + +-- !query 42 +SELECT power(double('NaN'), double('NaN')) +-- !query 42 schema +struct +-- !query 42 output +NaN + + +-- !query 43 +SELECT power(double('-1'), double('NaN')) +-- !query 43 schema +struct +-- !query 43 output +NaN + + +-- !query 44 +SELECT power(double('1'), double('NaN')) +-- !query 44 schema +struct +-- !query 44 output +NaN + + +-- !query 45 +SELECT power(double('NaN'), double('0')) +-- !query 45 schema +struct +-- !query 45 output +1.0 + + +-- !query 46 +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 46 schema +struct +-- !query 46 output +1.2345678901234E-200 1.2345678901233948E-200 + 1.2345678901234E200 1.234567890123379E200 + 1004.3 1004.3000000000004 + + +-- !query 47 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 47 schema +struct +-- !query 47 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 48 +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f +-- !query 49 schema +struct +-- !query 49 output +-1.0042999999999999E203 + -1.2345678901234 + -3.484E201 + -Infinity + 0.0 + + +-- !query 50 +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL +-- !query 50 schema +struct +-- !query 50 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 51 +SELECT sinh(double('1')) +-- !query 51 schema +struct +-- !query 51 output +1.1752011936438014 + + +-- !query 52 +SELECT cosh(double('1')) +-- !query 52 schema +struct +-- !query 52 output +1.543080634815244 + + +-- !query 53 +SELECT tanh(double('1')) +-- !query 53 schema +struct +-- !query 53 output +0.7615941559557649 + + +-- !query 54 +SELECT asinh(double('1')) +-- !query 54 schema +struct +-- !query 54 output +0.8813735870195429 + + +-- !query 55 +SELECT acosh(double('2')) +-- !query 55 schema +struct +-- !query 55 output +1.3169578969248166 + + +-- !query 56 +SELECT atanh(double('0.5')) +-- !query 56 schema +struct +-- !query 56 output +0.5493061443340549 + + +-- !query 57 +SELECT sinh(double('Infinity')) +-- !query 57 schema +struct +-- !query 57 output +Infinity + + +-- !query 58 +SELECT sinh(double('-Infinity')) +-- !query 58 schema +struct +-- !query 58 output +-Infinity + + +-- !query 59 +SELECT sinh(double('NaN')) +-- !query 59 schema +struct +-- !query 59 output +NaN + + +-- !query 60 +SELECT cosh(double('Infinity')) +-- !query 60 schema +struct +-- !query 60 output +Infinity + + +-- !query 61 +SELECT cosh(double('-Infinity')) +-- !query 61 schema +struct +-- !query 61 output +Infinity + + +-- !query 62 +SELECT cosh(double('NaN')) +-- !query 62 schema +struct +-- !query 62 output +NaN + + +-- !query 63 +SELECT tanh(double('Infinity')) +-- !query 63 schema +struct +-- !query 63 output +1.0 + + +-- !query 64 +SELECT tanh(double('-Infinity')) +-- !query 64 schema +struct +-- !query 64 output +-1.0 + + +-- !query 65 +SELECT tanh(double('NaN')) +-- !query 65 schema +struct +-- !query 65 output +NaN + + +-- !query 66 +SELECT asinh(double('Infinity')) +-- !query 66 schema +struct +-- !query 66 output +Infinity + + +-- !query 67 +SELECT asinh(double('-Infinity')) +-- !query 67 schema +struct +-- !query 67 output +-Infinity + + +-- !query 68 +SELECT asinh(double('NaN')) +-- !query 68 schema +struct +-- !query 68 output +NaN + + +-- !query 69 +SELECT acosh(double('Infinity')) +-- !query 69 schema +struct +-- !query 69 output +Infinity + + +-- !query 70 +SELECT acosh(double('-Infinity')) +-- !query 70 schema +struct +-- !query 70 output +NaN + + +-- !query 71 +SELECT acosh(double('NaN')) +-- !query 71 schema +struct +-- !query 71 output +NaN + + +-- !query 72 +SELECT atanh(double('Infinity')) +-- !query 72 schema +struct +-- !query 72 output +NaN + + +-- !query 73 +SELECT atanh(double('-Infinity')) +-- !query 73 schema +struct +-- !query 73 output +NaN + + +-- !query 74 +SELECT atanh(double('NaN')) +-- !query 74 schema +struct +-- !query 74 output +NaN + + +-- !query 75 +TRUNCATE TABLE FLOAT8_TBL +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO FLOAT8_TBL VALUES ('0.0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO FLOAT8_TBL VALUES ('-34.84') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO FLOAT8_TBL VALUES ('-1004.30') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 81 schema +struct +-- !query 81 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 82 +SELECT smallint(double('32767.4')) +-- !query 82 schema +struct +-- !query 82 output +32767 + + +-- !query 83 +SELECT smallint(double('32767.6')) +-- !query 83 schema +struct +-- !query 83 output +32767 + + +-- !query 84 +SELECT smallint(double('-32768.4')) +-- !query 84 schema +struct +-- !query 84 output +-32768 + + +-- !query 85 +SELECT smallint(double('-32768.6')) +-- !query 85 schema +struct +-- !query 85 output +-32768 + + +-- !query 86 +SELECT int(double('2147483647.4')) +-- !query 86 schema +struct +-- !query 86 output +2147483647 + + +-- !query 87 +SELECT int(double('2147483647.6')) +-- !query 87 schema +struct +-- !query 87 output +2147483647 + + +-- !query 88 +SELECT int(double('-2147483648.4')) +-- !query 88 schema +struct +-- !query 88 output +-2147483648 + + +-- !query 89 +SELECT int(double('-2147483648.6')) +-- !query 89 schema +struct +-- !query 89 output +-2147483648 + + +-- !query 90 +SELECT bigint(double('9223372036854773760')) +-- !query 90 schema +struct +-- !query 90 output +9223372036854773760 + + +-- !query 91 +SELECT bigint(double('9223372036854775807')) +-- !query 91 schema +struct +-- !query 91 output +9223372036854775807 + + +-- !query 92 +SELECT bigint(double('-9223372036854775808.5')) +-- !query 92 schema +struct +-- !query 92 output +-9223372036854775808 + + +-- !query 93 +SELECT bigint(double('-9223372036854780000')) +-- !query 93 schema +struct +-- !query 93 output +-9223372036854775808 + + +-- !query 94 +DROP TABLE FLOAT8_TBL +-- !query 94 schema +struct<> +-- !query 94 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out new file mode 100644 index 0000000000000..7a7ce5f37dea4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out @@ -0,0 +1,363 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 35 + + +-- !query 0 +CREATE TABLE INT2_TBL(f1 smallint) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO INT2_TBL VALUES (trim('0 ')) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO INT2_TBL VALUES (trim(' 1234 ')) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO INT2_TBL VALUES (trim(' -1234')) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO INT2_TBL VALUES ('32767') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO INT2_TBL VALUES ('-32767') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT '' AS five, * FROM INT2_TBL +-- !query 6 schema +struct +-- !query 6 output +-1234 + -32767 + 0 + 1234 + 32767 + + +-- !query 7 +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0') +-- !query 7 schema +struct +-- !query 7 output +-1234 + -32767 + 1234 + 32767 + + +-- !query 8 +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0') +-- !query 8 schema +struct +-- !query 8 output +-1234 + -32767 + 1234 + 32767 + + +-- !query 9 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0') +-- !query 9 schema +struct +-- !query 9 output +0 + + +-- !query 10 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0') +-- !query 10 schema +struct +-- !query 10 output +0 + + +-- !query 11 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0') +-- !query 11 schema +struct +-- !query 11 output +-1234 + -32767 + + +-- !query 12 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0') +-- !query 12 schema +struct +-- !query 12 output +-1234 + -32767 + + +-- !query 13 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0') +-- !query 13 schema +struct +-- !query 13 output +-1234 + -32767 + 0 + + +-- !query 14 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0') +-- !query 14 schema +struct +-- !query 14 output +-1234 + -32767 + 0 + + +-- !query 15 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0') +-- !query 15 schema +struct +-- !query 15 output +1234 + 32767 + + +-- !query 16 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0') +-- !query 16 schema +struct +-- !query 16 output +1234 + 32767 + + +-- !query 17 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0') +-- !query 17 schema +struct +-- !query 17 output +0 + 1234 + 32767 + + +-- !query 18 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0') +-- !query 18 schema +struct +-- !query 18 output +0 + 1234 + 32767 + + +-- !query 19 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query 19 schema +struct +-- !query 19 output +32767 + + +-- !query 20 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query 20 schema +struct +-- !query 20 output +-1234 + 0 + 1234 + + +-- !query 21 +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384 +-- !query 21 schema +struct +-- !query 21 output +-1234 -2468 + 0 0 + 1234 2468 + + +-- !query 22 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i +-- !query 22 schema +struct +-- !query 22 output +-1234 -2468 + -32767 -65534 + 0 0 + 1234 2468 + 32767 65534 + + +-- !query 23 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766 +-- !query 23 schema +struct +-- !query 23 output +-1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + + +-- !query 24 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i +-- !query 24 schema +struct +-- !query 24 output +-1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + 32767 32769 + + +-- !query 25 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767 +-- !query 25 schema +struct +-- !query 25 output +-1234 -1236 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query 26 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i +-- !query 26 schema +struct +-- !query 26 output +-1234 -1236 + -32767 -32769 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query 27 +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i +-- !query 27 schema +struct +-- !query 27 output +-1234 -617 + -32767 -16383 + 0 0 + 1234 617 + 32767 16383 + + +-- !query 28 +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i +-- !query 28 schema +struct +-- !query 28 output +-1234 -617 + -32767 -16383 + 0 0 + 1234 617 + 32767 16383 + + +-- !query 29 +SELECT string(shiftleft(smallint(-1), 15)) +-- !query 29 schema +struct +-- !query 29 output +-32768 + + +-- !query 30 +SELECT string(smallint(shiftleft(smallint(-1), 15))+1) +-- !query 30 schema +struct +-- !query 30 output +-32767 + + +-- !query 31 +SELECT smallint(-32768) % smallint(-1) +-- !query 31 schema +struct<(CAST(-32768 AS SMALLINT) % CAST(-1 AS SMALLINT)):smallint> +-- !query 31 output +0 + + +-- !query 32 +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x) +-- !query 32 schema +struct +-- !query 32 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 33 +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query 33 schema +struct +-- !query 33 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 34 +DROP TABLE INT2_TBL +-- !query 34 schema +struct<> +-- !query 34 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out index 9c17e9a1a197b..456b1ef962d46 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out @@ -339,25 +339,25 @@ struct -- !query 33 SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema -struct +struct -- !query 33 output --123456 -61728.0 - -2147483647 -1.0737418235E9 - 0 0.0 - 123456 61728.0 - 2147483647 1.0737418235E9 +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 34 SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema -struct +struct -- !query 34 output --123456 -61728.0 - -2147483647 -1.0737418235E9 - 0 0.0 - 123456 61728.0 - 2147483647 1.0737418235E9 +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 35 @@ -417,7 +417,7 @@ true -- !query 42 -SELECT int('1000') < int('999') AS false +SELECT int('1000') < int('999') AS `false` -- !query 42 schema struct -- !query 42 output @@ -435,17 +435,17 @@ struct -- !query 44 SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -3.0 +3 -- !query 45 SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -2.0 +2 -- !query 46 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out new file mode 100644 index 0000000000000..6d7fae19aa7e4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out @@ -0,0 +1,855 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 85 + + +-- !query 0 +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO INT8_TBL VALUES('4567890123456789','123') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT * FROM INT8_TBL +-- !query 6 schema +struct +-- !query 6 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 7 +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 +-- !query 7 schema +struct +-- !query 7 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 8 +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 +-- !query 8 schema +struct +-- !query 8 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 9 +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 +-- !query 9 schema +struct +-- !query 9 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 10 +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 +-- !query 10 schema +struct +-- !query 10 output + + + +-- !query 11 +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 +-- !query 11 schema +struct +-- !query 11 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 12 +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 +-- !query 12 schema +struct +-- !query 12 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 13 +SELECT * FROM INT8_TBL WHERE q2 = 456 +-- !query 13 schema +struct +-- !query 13 output +123 456 + + +-- !query 14 +SELECT * FROM INT8_TBL WHERE q2 <> 456 +-- !query 14 schema +struct +-- !query 14 output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 15 +SELECT * FROM INT8_TBL WHERE q2 < 456 +-- !query 15 schema +struct +-- !query 15 output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 16 +SELECT * FROM INT8_TBL WHERE q2 > 456 +-- !query 16 schema +struct +-- !query 16 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 17 +SELECT * FROM INT8_TBL WHERE q2 <= 456 +-- !query 17 schema +struct +-- !query 17 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 18 +SELECT * FROM INT8_TBL WHERE q2 >= 456 +-- !query 18 schema +struct +-- !query 18 output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 19 +SELECT * FROM INT8_TBL WHERE 123 = q1 +-- !query 19 schema +struct +-- !query 19 output +123 456 +123 4567890123456789 + + +-- !query 20 +SELECT * FROM INT8_TBL WHERE 123 <> q1 +-- !query 20 schema +struct +-- !query 20 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 21 +SELECT * FROM INT8_TBL WHERE 123 < q1 +-- !query 21 schema +struct +-- !query 21 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 22 +SELECT * FROM INT8_TBL WHERE 123 > q1 +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT * FROM INT8_TBL WHERE 123 <= q1 +-- !query 23 schema +struct +-- !query 23 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 24 +SELECT * FROM INT8_TBL WHERE 123 >= q1 +-- !query 24 schema +struct +-- !query 24 output +123 456 +123 4567890123456789 + + +-- !query 25 +SELECT * FROM INT8_TBL WHERE q2 = smallint('456') +-- !query 25 schema +struct +-- !query 25 output +123 456 + + +-- !query 26 +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') +-- !query 26 schema +struct +-- !query 26 output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 27 +SELECT * FROM INT8_TBL WHERE q2 < smallint('456') +-- !query 27 schema +struct +-- !query 27 output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 28 +SELECT * FROM INT8_TBL WHERE q2 > smallint('456') +-- !query 28 schema +struct +-- !query 28 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 29 +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') +-- !query 29 schema +struct +-- !query 29 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 30 +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') +-- !query 30 schema +struct +-- !query 30 output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 31 +SELECT * FROM INT8_TBL WHERE smallint('123') = q1 +-- !query 31 schema +struct +-- !query 31 output +123 456 +123 4567890123456789 + + +-- !query 32 +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 +-- !query 32 schema +struct +-- !query 32 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 33 +SELECT * FROM INT8_TBL WHERE smallint('123') < q1 +-- !query 33 schema +struct +-- !query 33 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 34 +SELECT * FROM INT8_TBL WHERE smallint('123') > q1 +-- !query 34 schema +struct +-- !query 34 output + + + +-- !query 35 +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 +-- !query 35 schema +struct +-- !query 35 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 36 +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 +-- !query 36 schema +struct +-- !query 36 output +123 456 +123 4567890123456789 + + +-- !query 37 +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL +-- !query 37 schema +struct +-- !query 37 output +123 -123 + 123 -123 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + + +-- !query 38 +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL +-- !query 38 schema +struct +-- !query 38 output +123 456 579 + 123 4567890123456789 4567890123456912 + 4567890123456789 -4567890123456789 0 + 4567890123456789 123 4567890123456912 + 4567890123456789 4567890123456789 9135780246913578 + + +-- !query 39 +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL +-- !query 39 schema +struct +-- !query 39 output +123 456 -333 + 123 4567890123456789 -4567890123456666 + 4567890123456789 -4567890123456789 9135780246913578 + 4567890123456789 123 4567890123456666 + 4567890123456789 4567890123456789 0 + + +-- !query 40 +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL +-- !query 40 schema +struct +-- !query 40 output +123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 -4567890123456789 -4868582358072306617 + 4567890123456789 123 561850485185185047 + 4567890123456789 4567890123456789 4868582358072306617 + + +-- !query 41 +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) +-- !query 41 schema +struct +-- !query 41 output +123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 123 561850485185185047 + + +-- !query 42 +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL +-- !query 42 schema +struct +-- !query 42 output +123 456 0 123 + 123 4567890123456789 0 123 + 4567890123456789 -4567890123456789 -1 0 + 4567890123456789 123 37137318076884 57 + 4567890123456789 4567890123456789 1 0 + + +-- !query 43 +SELECT '' AS five, q1, double(q1) FROM INT8_TBL +-- !query 43 schema +struct +-- !query 43 output +123 123.0 + 123 123.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query 44 +SELECT '' AS five, q2, double(q2) FROM INT8_TBL +-- !query 44 schema +struct +-- !query 44 output +-4567890123456789 -4.567890123456789E15 + 123 123.0 + 456 456.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query 45 +SELECT 37 + q1 AS plus4 FROM INT8_TBL +-- !query 45 schema +struct +-- !query 45 output +160 +160 +4567890123456826 +4567890123456826 +4567890123456826 + + +-- !query 46 +SELECT 37 - q1 AS minus4 FROM INT8_TBL +-- !query 46 schema +struct +-- !query 46 output +-4567890123456752 +-4567890123456752 +-4567890123456752 +-86 +-86 + + +-- !query 47 +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL +-- !query 47 schema +struct +-- !query 47 output +246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query 48 +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL +-- !query 48 schema +struct +-- !query 48 output +246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query 49 +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL +-- !query 49 schema +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:bigint> +-- !query 49 output +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 + + +-- !query 50 +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL +-- !query 50 schema +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:bigint> +-- !query 50 output +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 + + +-- !query 51 +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL +-- !query 51 schema +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:bigint> +-- !query 51 output +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 + + +-- !query 52 +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL +-- !query 52 schema +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:bigint> +-- !query 52 output +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 + + +-- !query 53 +SELECT q2, abs(q2) FROM INT8_TBL +-- !query 53 schema +struct +-- !query 53 output +-4567890123456789 4567890123456789 +123 123 +456 456 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 54 +SELECT min(q1), min(q2) FROM INT8_TBL +-- !query 54 schema +struct +-- !query 54 output +123 -4567890123456789 + + +-- !query 55 +SELECT max(q1), max(q2) FROM INT8_TBL +-- !query 55 schema +struct +-- !query 55 output +4567890123456789 4567890123456789 + + +-- !query 56 +select bigint('9223372036854775800') / bigint('0') +-- !query 56 schema +struct<(CAST(9223372036854775800 AS BIGINT) div CAST(0 AS BIGINT)):bigint> +-- !query 56 output +NULL + + +-- !query 57 +select bigint('-9223372036854775808') / smallint('0') +-- !query 57 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(0 AS SMALLINT) AS BIGINT)):bigint> +-- !query 57 output +NULL + + +-- !query 58 +select smallint('100') / bigint('0') +-- !query 58 schema +struct<(CAST(CAST(100 AS SMALLINT) AS BIGINT) div CAST(0 AS BIGINT)):bigint> +-- !query 58 output +NULL + + +-- !query 59 +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 +-- !query 59 schema +struct +-- !query 59 output +123 + + +-- !query 60 +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 +-- !query 60 schema +struct +-- !query 60 output +-869367531 +-869367531 +-869367531 +123 + + +-- !query 61 +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 +-- !query 61 schema +struct +-- !query 61 output +123 + + +-- !query 62 +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 +-- !query 62 schema +struct +-- !query 62 output +-32491 +-32491 +-32491 +123 + + +-- !query 63 +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) +-- !query 63 schema +struct +-- !query 63 output +42 -37 + + +-- !query 64 +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL +-- !query 64 schema +struct +-- !query 64 output +123.0 4.567890123456789E15 +123.0 456.0 +4.5678899E15 -4.567890123456789E15 +4.5678899E15 123.0 +4.5678899E15 4.567890123456789E15 + + +-- !query 65 +SELECT CAST(float('36854775807.0') AS bigint) +-- !query 65 schema +struct +-- !query 65 output +36854775808 + + +-- !query 66 +SELECT CAST(double('922337203685477580700.0') AS bigint) +-- !query 66 schema +struct +-- !query 66 output +9223372036854775807 + + +-- !query 67 +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL +-- !query 67 schema +struct +-- !query 67 output +123 456 72 507 -124 +123 4567890123456789 17 4567890123456895 -124 +4567890123456789 -4567890123456789 1 -1 -4567890123456790 +4567890123456789 123 17 4567890123456895 -4567890123456790 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 -4567890123456790 + + +-- !query 68 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) +-- !query 68 schema +struct +-- !query 68 output +4567890123456789 +4567890123456790 +4567890123456791 +4567890123456792 +4567890123456793 +4567890123456794 +4567890123456795 +4567890123456796 +4567890123456797 +4567890123456798 + + +-- !query 69 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) +-- !query 69 schema +struct<> +-- !query 69 output +java.lang.IllegalArgumentException +requirement failed: step (0) cannot be 0 + + +-- !query 70 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) +-- !query 70 schema +struct +-- !query 70 output +4567890123456789 +4567890123456791 +4567890123456793 +4567890123456795 +4567890123456797 + + +-- !query 71 +SELECT string(shiftleft(bigint(-1), 63)) +-- !query 71 schema +struct +-- !query 71 output +-9223372036854775808 + + +-- !query 72 +SELECT string(int(shiftleft(bigint(-1), 63))+1) +-- !query 72 schema +struct +-- !query 72 output +1 + + +-- !query 73 +SELECT bigint((-9223372036854775808)) * bigint((-1)) +-- !query 73 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(-1 AS BIGINT)):bigint> +-- !query 73 output +-9223372036854775808 + + +-- !query 74 +SELECT bigint((-9223372036854775808)) / bigint((-1)) +-- !query 74 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(-1 AS BIGINT)):bigint> +-- !query 74 output +-9223372036854775808 + + +-- !query 75 +SELECT bigint((-9223372036854775808)) % bigint((-1)) +-- !query 75 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(-1 AS BIGINT)):bigint> +-- !query 75 output +0 + + +-- !query 76 +SELECT bigint((-9223372036854775808)) * int((-1)) +-- !query 76 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 76 output +-9223372036854775808 + + +-- !query 77 +SELECT bigint((-9223372036854775808)) / int((-1)) +-- !query 77 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 77 output +-9223372036854775808 + + +-- !query 78 +SELECT bigint((-9223372036854775808)) % int((-1)) +-- !query 78 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 78 output +0 + + +-- !query 79 +SELECT bigint((-9223372036854775808)) * smallint((-1)) +-- !query 79 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 79 output +-9223372036854775808 + + +-- !query 80 +SELECT bigint((-9223372036854775808)) / smallint((-1)) +-- !query 80 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 80 output +-9223372036854775808 + + +-- !query 81 +SELECT bigint((-9223372036854775808)) % smallint((-1)) +-- !query 81 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 81 output +0 + + +-- !query 82 +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x) +-- !query 82 schema +struct +-- !query 82 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 83 +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query 83 schema +struct +-- !query 83 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 84 +DROP TABLE INT8_TBL +-- !query 84 schema +struct<> +-- !query 84 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out new file mode 100644 index 0000000000000..04425fd4b5444 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out @@ -0,0 +1,4832 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 573 + + +-- !query 0 +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO num_exp_add VALUES (0,0,'0') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO num_exp_sub VALUES (0,0,'0') +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO num_exp_mul VALUES (0,0,'0') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO num_exp_div VALUES (0,0,'NaN') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO num_exp_add VALUES (0,1,'0') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO num_exp_sub VALUES (0,1,'0') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO num_exp_mul VALUES (0,1,'0') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +INSERT INTO num_exp_div VALUES (0,1,'NaN') +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047') +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047') +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +INSERT INTO num_exp_mul VALUES (0,2,'0') +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +INSERT INTO num_exp_div VALUES (0,2,'0') +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +INSERT INTO num_exp_add VALUES (0,3,'4.31') +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +INSERT INTO num_exp_sub VALUES (0,3,'-4.31') +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +INSERT INTO num_exp_mul VALUES (0,3,'0') +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +INSERT INTO num_exp_div VALUES (0,3,'0') +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +INSERT INTO num_exp_add VALUES (0,4,'7799461.4119') +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119') +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +INSERT INTO num_exp_mul VALUES (0,4,'0') +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +INSERT INTO num_exp_div VALUES (0,4,'0') +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +INSERT INTO num_exp_add VALUES (0,5,'16397.038491') +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491') +-- !query 31 schema +struct<> +-- !query 31 output + + + +-- !query 32 +INSERT INTO num_exp_mul VALUES (0,5,'0') +-- !query 32 schema +struct<> +-- !query 32 output + + + +-- !query 33 +INSERT INTO num_exp_div VALUES (0,5,'0') +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO num_exp_add VALUES (0,6,'93901.57763026') +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026') +-- !query 35 schema +struct<> +-- !query 35 output + + + +-- !query 36 +INSERT INTO num_exp_mul VALUES (0,6,'0') +-- !query 36 schema +struct<> +-- !query 36 output + + + +-- !query 37 +INSERT INTO num_exp_div VALUES (0,6,'0') +-- !query 37 schema +struct<> +-- !query 37 output + + + +-- !query 38 +INSERT INTO num_exp_add VALUES (0,7,'-83028485') +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +INSERT INTO num_exp_sub VALUES (0,7,'83028485') +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +INSERT INTO num_exp_mul VALUES (0,7,'0') +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +INSERT INTO num_exp_div VALUES (0,7,'0') +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +INSERT INTO num_exp_add VALUES (0,8,'74881') +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +INSERT INTO num_exp_sub VALUES (0,8,'-74881') +-- !query 43 schema +struct<> +-- !query 43 output + + + +-- !query 44 +INSERT INTO num_exp_mul VALUES (0,8,'0') +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +INSERT INTO num_exp_div VALUES (0,8,'0') +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420') +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420') +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +INSERT INTO num_exp_mul VALUES (0,9,'0') +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +INSERT INTO num_exp_div VALUES (0,9,'0') +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +INSERT INTO num_exp_add VALUES (1,0,'0') +-- !query 50 schema +struct<> +-- !query 50 output + + + +-- !query 51 +INSERT INTO num_exp_sub VALUES (1,0,'0') +-- !query 51 schema +struct<> +-- !query 51 output + + + +-- !query 52 +INSERT INTO num_exp_mul VALUES (1,0,'0') +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +INSERT INTO num_exp_div VALUES (1,0,'NaN') +-- !query 53 schema +struct<> +-- !query 53 output + + + +-- !query 54 +INSERT INTO num_exp_add VALUES (1,1,'0') +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +INSERT INTO num_exp_sub VALUES (1,1,'0') +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +INSERT INTO num_exp_mul VALUES (1,1,'0') +-- !query 56 schema +struct<> +-- !query 56 output + + + +-- !query 57 +INSERT INTO num_exp_div VALUES (1,1,'NaN') +-- !query 57 schema +struct<> +-- !query 57 output + + + +-- !query 58 +INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047') +-- !query 58 schema +struct<> +-- !query 58 output + + + +-- !query 59 +INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047') +-- !query 59 schema +struct<> +-- !query 59 output + + + +-- !query 60 +INSERT INTO num_exp_mul VALUES (1,2,'0') +-- !query 60 schema +struct<> +-- !query 60 output + + + +-- !query 61 +INSERT INTO num_exp_div VALUES (1,2,'0') +-- !query 61 schema +struct<> +-- !query 61 output + + + +-- !query 62 +INSERT INTO num_exp_add VALUES (1,3,'4.31') +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +INSERT INTO num_exp_sub VALUES (1,3,'-4.31') +-- !query 63 schema +struct<> +-- !query 63 output + + + +-- !query 64 +INSERT INTO num_exp_mul VALUES (1,3,'0') +-- !query 64 schema +struct<> +-- !query 64 output + + + +-- !query 65 +INSERT INTO num_exp_div VALUES (1,3,'0') +-- !query 65 schema +struct<> +-- !query 65 output + + + +-- !query 66 +INSERT INTO num_exp_add VALUES (1,4,'7799461.4119') +-- !query 66 schema +struct<> +-- !query 66 output + + + +-- !query 67 +INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119') +-- !query 67 schema +struct<> +-- !query 67 output + + + +-- !query 68 +INSERT INTO num_exp_mul VALUES (1,4,'0') +-- !query 68 schema +struct<> +-- !query 68 output + + + +-- !query 69 +INSERT INTO num_exp_div VALUES (1,4,'0') +-- !query 69 schema +struct<> +-- !query 69 output + + + +-- !query 70 +INSERT INTO num_exp_add VALUES (1,5,'16397.038491') +-- !query 70 schema +struct<> +-- !query 70 output + + + +-- !query 71 +INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491') +-- !query 71 schema +struct<> +-- !query 71 output + + + +-- !query 72 +INSERT INTO num_exp_mul VALUES (1,5,'0') +-- !query 72 schema +struct<> +-- !query 72 output + + + +-- !query 73 +INSERT INTO num_exp_div VALUES (1,5,'0') +-- !query 73 schema +struct<> +-- !query 73 output + + + +-- !query 74 +INSERT INTO num_exp_add VALUES (1,6,'93901.57763026') +-- !query 74 schema +struct<> +-- !query 74 output + + + +-- !query 75 +INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026') +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO num_exp_mul VALUES (1,6,'0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO num_exp_div VALUES (1,6,'0') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO num_exp_add VALUES (1,7,'-83028485') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO num_exp_sub VALUES (1,7,'83028485') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO num_exp_mul VALUES (1,7,'0') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +INSERT INTO num_exp_div VALUES (1,7,'0') +-- !query 81 schema +struct<> +-- !query 81 output + + + +-- !query 82 +INSERT INTO num_exp_add VALUES (1,8,'74881') +-- !query 82 schema +struct<> +-- !query 82 output + + + +-- !query 83 +INSERT INTO num_exp_sub VALUES (1,8,'-74881') +-- !query 83 schema +struct<> +-- !query 83 output + + + +-- !query 84 +INSERT INTO num_exp_mul VALUES (1,8,'0') +-- !query 84 schema +struct<> +-- !query 84 output + + + +-- !query 85 +INSERT INTO num_exp_div VALUES (1,8,'0') +-- !query 85 schema +struct<> +-- !query 85 output + + + +-- !query 86 +INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420') +-- !query 86 schema +struct<> +-- !query 86 output + + + +-- !query 87 +INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420') +-- !query 87 schema +struct<> +-- !query 87 output + + + +-- !query 88 +INSERT INTO num_exp_mul VALUES (1,9,'0') +-- !query 88 schema +struct<> +-- !query 88 output + + + +-- !query 89 +INSERT INTO num_exp_div VALUES (1,9,'0') +-- !query 89 schema +struct<> +-- !query 89 output + + + +-- !query 90 +INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047') +-- !query 90 schema +struct<> +-- !query 90 output + + + +-- !query 91 +INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047') +-- !query 91 schema +struct<> +-- !query 91 output + + + +-- !query 92 +INSERT INTO num_exp_mul VALUES (2,0,'0') +-- !query 92 schema +struct<> +-- !query 92 output + + + +-- !query 93 +INSERT INTO num_exp_div VALUES (2,0,'NaN') +-- !query 93 schema +struct<> +-- !query 93 output + + + +-- !query 94 +INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047') +-- !query 94 schema +struct<> +-- !query 94 output + + + +-- !query 95 +INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047') +-- !query 95 schema +struct<> +-- !query 95 output + + + +-- !query 96 +INSERT INTO num_exp_mul VALUES (2,1,'0') +-- !query 96 schema +struct<> +-- !query 96 output + + + +-- !query 97 +INSERT INTO num_exp_div VALUES (2,1,'NaN') +-- !query 97 schema +struct<> +-- !query 97 output + + + +-- !query 98 +INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094') +-- !query 98 schema +struct<> +-- !query 98 output + + + +-- !query 99 +INSERT INTO num_exp_sub VALUES (2,2,'0') +-- !query 99 schema +struct<> +-- !query 99 output + + + +-- !query 100 +INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209') +-- !query 100 schema +struct<> +-- !query 100 output + + + +-- !query 101 +INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000') +-- !query 101 schema +struct<> +-- !query 101 output + + + +-- !query 102 +INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047') +-- !query 102 schema +struct<> +-- !query 102 output + + + +-- !query 103 +INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047') +-- !query 103 schema +struct<> +-- !query 103 output + + + +-- !query 104 +INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257') +-- !query 104 schema +struct<> +-- !query 104 output + + + +-- !query 105 +INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266') +-- !query 105 schema +struct<> +-- !query 105 output + + + +-- !query 106 +INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047') +-- !query 106 schema +struct<> +-- !query 106 output + + + +-- !query 107 +INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047') +-- !query 107 schema +struct<> +-- !query 107 output + + + +-- !query 108 +INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593') +-- !query 108 schema +struct<> +-- !query 108 output + + + +-- !query 109 +INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685') +-- !query 109 schema +struct<> +-- !query 109 output + + + +-- !query 110 +INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047') +-- !query 110 schema +struct<> +-- !query 110 output + + + +-- !query 111 +INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047') +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077') +-- !query 112 schema +struct<> +-- !query 112 output + + + +-- !query 113 +INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429') +-- !query 113 schema +struct<> +-- !query 113 output + + + +-- !query 114 +INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787') +-- !query 114 schema +struct<> +-- !query 114 output + + + +-- !query 115 +INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307') +-- !query 115 schema +struct<> +-- !query 115 output + + + +-- !query 116 +INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222') +-- !query 116 schema +struct<> +-- !query 116 output + + + +-- !query 117 +INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940') +-- !query 117 schema +struct<> +-- !query 117 output + + + +-- !query 118 +INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047') +-- !query 118 schema +struct<> +-- !query 118 output + + + +-- !query 119 +INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953') +-- !query 119 schema +struct<> +-- !query 119 output + + + +-- !query 120 +INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795') +-- !query 120 schema +struct<> +-- !query 120 output + + + +-- !query 121 +INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518') +-- !query 121 schema +struct<> +-- !query 121 output + + + +-- !query 122 +INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047') +-- !query 122 schema +struct<> +-- !query 122 output + + + +-- !query 123 +INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047') +-- !query 123 schema +struct<> +-- !query 123 output + + + +-- !query 124 +INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407') +-- !query 124 schema +struct<> +-- !query 124 output + + + +-- !query 125 +INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476') +-- !query 125 schema +struct<> +-- !query 125 output + + + +-- !query 126 +INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467') +-- !query 126 schema +struct<> +-- !query 126 output + + + +-- !query 127 +INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627') +-- !query 127 schema +struct<> +-- !query 127 output + + + +-- !query 128 +INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740') +-- !query 128 schema +struct<> +-- !query 128 output + + + +-- !query 129 +INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811') +-- !query 129 schema +struct<> +-- !query 129 output + + + +-- !query 130 +INSERT INTO num_exp_add VALUES (3,0,'4.31') +-- !query 130 schema +struct<> +-- !query 130 output + + + +-- !query 131 +INSERT INTO num_exp_sub VALUES (3,0,'4.31') +-- !query 131 schema +struct<> +-- !query 131 output + + + +-- !query 132 +INSERT INTO num_exp_mul VALUES (3,0,'0') +-- !query 132 schema +struct<> +-- !query 132 output + + + +-- !query 133 +INSERT INTO num_exp_div VALUES (3,0,'NaN') +-- !query 133 schema +struct<> +-- !query 133 output + + + +-- !query 134 +INSERT INTO num_exp_add VALUES (3,1,'4.31') +-- !query 134 schema +struct<> +-- !query 134 output + + + +-- !query 135 +INSERT INTO num_exp_sub VALUES (3,1,'4.31') +-- !query 135 schema +struct<> +-- !query 135 output + + + +-- !query 136 +INSERT INTO num_exp_mul VALUES (3,1,'0') +-- !query 136 schema +struct<> +-- !query 136 output + + + +-- !query 137 +INSERT INTO num_exp_div VALUES (3,1,'NaN') +-- !query 137 schema +struct<> +-- !query 137 output + + + +-- !query 138 +INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047') +-- !query 138 schema +struct<> +-- !query 138 output + + + +-- !query 139 +INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047') +-- !query 139 schema +struct<> +-- !query 139 output + + + +-- !query 140 +INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257') +-- !query 140 schema +struct<> +-- !query 140 output + + + +-- !query 141 +INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352') +-- !query 141 schema +struct<> +-- !query 141 output + + + +-- !query 142 +INSERT INTO num_exp_add VALUES (3,3,'8.62') +-- !query 142 schema +struct<> +-- !query 142 output + + + +-- !query 143 +INSERT INTO num_exp_sub VALUES (3,3,'0') +-- !query 143 schema +struct<> +-- !query 143 output + + + +-- !query 144 +INSERT INTO num_exp_mul VALUES (3,3,'18.5761') +-- !query 144 schema +struct<> +-- !query 144 output + + + +-- !query 145 +INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000') +-- !query 145 schema +struct<> +-- !query 145 output + + + +-- !query 146 +INSERT INTO num_exp_add VALUES (3,4,'7799465.7219') +-- !query 146 schema +struct<> +-- !query 146 output + + + +-- !query 147 +INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019') +-- !query 147 schema +struct<> +-- !query 147 output + + + +-- !query 148 +INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289') +-- !query 148 schema +struct<> +-- !query 148 output + + + +-- !query 149 +INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552') +-- !query 149 schema +struct<> +-- !query 149 output + + + +-- !query 150 +INSERT INTO num_exp_add VALUES (3,5,'16401.348491') +-- !query 150 schema +struct<> +-- !query 150 output + + + +-- !query 151 +INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491') +-- !query 151 schema +struct<> +-- !query 151 output + + + +-- !query 152 +INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621') +-- !query 152 schema +struct<> +-- !query 152 output + + + +-- !query 153 +INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504') +-- !query 153 schema +struct<> +-- !query 153 output + + + +-- !query 154 +INSERT INTO num_exp_add VALUES (3,6,'93905.88763026') +-- !query 154 schema +struct<> +-- !query 154 output + + + +-- !query 155 +INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026') +-- !query 155 schema +struct<> +-- !query 155 output + + + +-- !query 156 +INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206') +-- !query 156 schema +struct<> +-- !query 156 output + + + +-- !query 157 +INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595') +-- !query 157 schema +struct<> +-- !query 157 output + + + +-- !query 158 +INSERT INTO num_exp_add VALUES (3,7,'-83028480.69') +-- !query 158 schema +struct<> +-- !query 158 output + + + +-- !query 159 +INSERT INTO num_exp_sub VALUES (3,7,'83028489.31') +-- !query 159 schema +struct<> +-- !query 159 output + + + +-- !query 160 +INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35') +-- !query 160 schema +struct<> +-- !query 160 output + + + +-- !query 161 +INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240') +-- !query 161 schema +struct<> +-- !query 161 output + + + +-- !query 162 +INSERT INTO num_exp_add VALUES (3,8,'74885.31') +-- !query 162 schema +struct<> +-- !query 162 output + + + +-- !query 163 +INSERT INTO num_exp_sub VALUES (3,8,'-74876.69') +-- !query 163 schema +struct<> +-- !query 163 output + + + +-- !query 164 +INSERT INTO num_exp_mul VALUES (3,8,'322737.11') +-- !query 164 schema +struct<> +-- !query 164 output + + + +-- !query 165 +INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553') +-- !query 165 schema +struct<> +-- !query 165 output + + + +-- !query 166 +INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420') +-- !query 166 schema +struct<> +-- !query 166 output + + + +-- !query 167 +INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420') +-- !query 167 schema +struct<> +-- !query 167 output + + + +-- !query 168 +INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020') +-- !query 168 schema +struct<> +-- !query 168 output + + + +-- !query 169 +INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854') +-- !query 169 schema +struct<> +-- !query 169 output + + + +-- !query 170 +INSERT INTO num_exp_add VALUES (4,0,'7799461.4119') +-- !query 170 schema +struct<> +-- !query 170 output + + + +-- !query 171 +INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119') +-- !query 171 schema +struct<> +-- !query 171 output + + + +-- !query 172 +INSERT INTO num_exp_mul VALUES (4,0,'0') +-- !query 172 schema +struct<> +-- !query 172 output + + + +-- !query 173 +INSERT INTO num_exp_div VALUES (4,0,'NaN') +-- !query 173 schema +struct<> +-- !query 173 output + + + +-- !query 174 +INSERT INTO num_exp_add VALUES (4,1,'7799461.4119') +-- !query 174 schema +struct<> +-- !query 174 output + + + +-- !query 175 +INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119') +-- !query 175 schema +struct<> +-- !query 175 output + + + +-- !query 176 +INSERT INTO num_exp_mul VALUES (4,1,'0') +-- !query 176 schema +struct<> +-- !query 176 output + + + +-- !query 177 +INSERT INTO num_exp_div VALUES (4,1,'NaN') +-- !query 177 schema +struct<> +-- !query 177 output + + + +-- !query 178 +INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047') +-- !query 178 schema +struct<> +-- !query 178 output + + + +-- !query 179 +INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047') +-- !query 179 schema +struct<> +-- !query 179 output + + + +-- !query 180 +INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593') +-- !query 180 schema +struct<> +-- !query 180 output + + + +-- !query 181 +INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385') +-- !query 181 schema +struct<> +-- !query 181 output + + + +-- !query 182 +INSERT INTO num_exp_add VALUES (4,3,'7799465.7219') +-- !query 182 schema +struct<> +-- !query 182 output + + + +-- !query 183 +INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019') +-- !query 183 schema +struct<> +-- !query 183 output + + + +-- !query 184 +INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289') +-- !query 184 schema +struct<> +-- !query 184 output + + + +-- !query 185 +INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883') +-- !query 185 schema +struct<> +-- !query 185 output + + + +-- !query 186 +INSERT INTO num_exp_add VALUES (4,4,'15598922.8238') +-- !query 186 schema +struct<> +-- !query 186 output + + + +-- !query 187 +INSERT INTO num_exp_sub VALUES (4,4,'0') +-- !query 187 schema +struct<> +-- !query 187 output + + + +-- !query 188 +INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161') +-- !query 188 schema +struct<> +-- !query 188 output + + + +-- !query 189 +INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000') +-- !query 189 schema +struct<> +-- !query 189 output + + + +-- !query 190 +INSERT INTO num_exp_add VALUES (4,5,'7815858.450391') +-- !query 190 schema +struct<> +-- !query 190 output + + + +-- !query 191 +INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409') +-- !query 191 schema +struct<> +-- !query 191 output + + + +-- !query 192 +INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429') +-- !query 192 schema +struct<> +-- !query 192 output + + + +-- !query 193 +INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061') +-- !query 193 schema +struct<> +-- !query 193 output + + + +-- !query 194 +INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026') +-- !query 194 schema +struct<> +-- !query 194 output + + + +-- !query 195 +INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974') +-- !query 195 schema +struct<> +-- !query 195 output + + + +-- !query 196 +INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094') +-- !query 196 schema +struct<> +-- !query 196 output + + + +-- !query 197 +INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606') +-- !query 197 schema +struct<> +-- !query 197 output + + + +-- !query 198 +INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881') +-- !query 198 schema +struct<> +-- !query 198 output + + + +-- !query 199 +INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119') +-- !query 199 schema +struct<> +-- !query 199 output + + + +-- !query 200 +INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715') +-- !query 200 schema +struct<> +-- !query 200 output + + + +-- !query 201 +INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637') +-- !query 201 schema +struct<> +-- !query 201 output + + + +-- !query 202 +INSERT INTO num_exp_add VALUES (4,8,'7874342.4119') +-- !query 202 schema +struct<> +-- !query 202 output + + + +-- !query 203 +INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119') +-- !query 203 schema +struct<> +-- !query 203 output + + + +-- !query 204 +INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839') +-- !query 204 schema +struct<> +-- !query 204 output + + + +-- !query 205 +INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143') +-- !query 205 schema +struct<> +-- !query 205 output + + + +-- !query 206 +INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420') +-- !query 206 schema +struct<> +-- !query 206 output + + + +-- !query 207 +INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420') +-- !query 207 schema +struct<> +-- !query 207 output + + + +-- !query 208 +INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980') +-- !query 208 schema +struct<> +-- !query 208 output + + + +-- !query 209 +INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409') +-- !query 209 schema +struct<> +-- !query 209 output + + + +-- !query 210 +INSERT INTO num_exp_add VALUES (5,0,'16397.038491') +-- !query 210 schema +struct<> +-- !query 210 output + + + +-- !query 211 +INSERT INTO num_exp_sub VALUES (5,0,'16397.038491') +-- !query 211 schema +struct<> +-- !query 211 output + + + +-- !query 212 +INSERT INTO num_exp_mul VALUES (5,0,'0') +-- !query 212 schema +struct<> +-- !query 212 output + + + +-- !query 213 +INSERT INTO num_exp_div VALUES (5,0,'NaN') +-- !query 213 schema +struct<> +-- !query 213 output + + + +-- !query 214 +INSERT INTO num_exp_add VALUES (5,1,'16397.038491') +-- !query 214 schema +struct<> +-- !query 214 output + + + +-- !query 215 +INSERT INTO num_exp_sub VALUES (5,1,'16397.038491') +-- !query 215 schema +struct<> +-- !query 215 output + + + +-- !query 216 +INSERT INTO num_exp_mul VALUES (5,1,'0') +-- !query 216 schema +struct<> +-- !query 216 output + + + +-- !query 217 +INSERT INTO num_exp_div VALUES (5,1,'NaN') +-- !query 217 schema +struct<> +-- !query 217 output + + + +-- !query 218 +INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047') +-- !query 218 schema +struct<> +-- !query 218 output + + + +-- !query 219 +INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047') +-- !query 219 schema +struct<> +-- !query 219 output + + + +-- !query 220 +INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077') +-- !query 220 schema +struct<> +-- !query 220 output + + + +-- !query 221 +INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446') +-- !query 221 schema +struct<> +-- !query 221 output + + + +-- !query 222 +INSERT INTO num_exp_add VALUES (5,3,'16401.348491') +-- !query 222 schema +struct<> +-- !query 222 output + + + +-- !query 223 +INSERT INTO num_exp_sub VALUES (5,3,'16392.728491') +-- !query 223 schema +struct<> +-- !query 223 output + + + +-- !query 224 +INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621') +-- !query 224 schema +struct<> +-- !query 224 output + + + +-- !query 225 +INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584') +-- !query 225 schema +struct<> +-- !query 225 output + + + +-- !query 226 +INSERT INTO num_exp_add VALUES (5,4,'7815858.450391') +-- !query 226 schema +struct<> +-- !query 226 output + + + +-- !query 227 +INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409') +-- !query 227 schema +struct<> +-- !query 227 output + + + +-- !query 228 +INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429') +-- !query 228 schema +struct<> +-- !query 228 output + + + +-- !query 229 +INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192') +-- !query 229 schema +struct<> +-- !query 229 output + + + +-- !query 230 +INSERT INTO num_exp_add VALUES (5,5,'32794.076982') +-- !query 230 schema +struct<> +-- !query 230 output + + + +-- !query 231 +INSERT INTO num_exp_sub VALUES (5,5,'0') +-- !query 231 schema +struct<> +-- !query 231 output + + + +-- !query 232 +INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081') +-- !query 232 schema +struct<> +-- !query 232 output + + + +-- !query 233 +INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000') +-- !query 233 schema +struct<> +-- !query 233 output + + + +-- !query 234 +INSERT INTO num_exp_add VALUES (5,6,'110298.61612126') +-- !query 234 schema +struct<> +-- !query 234 output + + + +-- !query 235 +INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926') +-- !query 235 schema +struct<> +-- !query 235 output + + + +-- !query 236 +INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766') +-- !query 236 schema +struct<> +-- !query 236 output + + + +-- !query 237 +INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689') +-- !query 237 schema +struct<> +-- !query 237 output + + + +-- !query 238 +INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509') +-- !query 238 schema +struct<> +-- !query 238 output + + + +-- !query 239 +INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491') +-- !query 239 schema +struct<> +-- !query 239 output + + + +-- !query 240 +INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135') +-- !query 240 schema +struct<> +-- !query 240 output + + + +-- !query 241 +INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710') +-- !query 241 schema +struct<> +-- !query 241 output + + + +-- !query 242 +INSERT INTO num_exp_add VALUES (5,8,'91278.038491') +-- !query 242 schema +struct<> +-- !query 242 output + + + +-- !query 243 +INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509') +-- !query 243 schema +struct<> +-- !query 243 output + + + +-- !query 244 +INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571') +-- !query 244 schema +struct<> +-- !query 244 output + + + +-- !query 245 +INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228') +-- !query 245 schema +struct<> +-- !query 245 output + + + +-- !query 246 +INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420') +-- !query 246 schema +struct<> +-- !query 246 output + + + +-- !query 247 +INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420') +-- !query 247 schema +struct<> +-- !query 247 output + + + +-- !query 248 +INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220') +-- !query 248 schema +struct<> +-- !query 248 output + + + +-- !query 249 +INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427') +-- !query 249 schema +struct<> +-- !query 249 output + + + +-- !query 250 +INSERT INTO num_exp_add VALUES (6,0,'93901.57763026') +-- !query 250 schema +struct<> +-- !query 250 output + + + +-- !query 251 +INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026') +-- !query 251 schema +struct<> +-- !query 251 output + + + +-- !query 252 +INSERT INTO num_exp_mul VALUES (6,0,'0') +-- !query 252 schema +struct<> +-- !query 252 output + + + +-- !query 253 +INSERT INTO num_exp_div VALUES (6,0,'NaN') +-- !query 253 schema +struct<> +-- !query 253 output + + + +-- !query 254 +INSERT INTO num_exp_add VALUES (6,1,'93901.57763026') +-- !query 254 schema +struct<> +-- !query 254 output + + + +-- !query 255 +INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026') +-- !query 255 schema +struct<> +-- !query 255 output + + + +-- !query 256 +INSERT INTO num_exp_mul VALUES (6,1,'0') +-- !query 256 schema +struct<> +-- !query 256 output + + + +-- !query 257 +INSERT INTO num_exp_div VALUES (6,1,'NaN') +-- !query 257 schema +struct<> +-- !query 257 output + + + +-- !query 258 +INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787') +-- !query 258 schema +struct<> +-- !query 258 output + + + +-- !query 259 +INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307') +-- !query 259 schema +struct<> +-- !query 259 output + + + +-- !query 260 +INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222') +-- !query 260 schema +struct<> +-- !query 260 output + + + +-- !query 261 +INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823') +-- !query 261 schema +struct<> +-- !query 261 output + + + +-- !query 262 +INSERT INTO num_exp_add VALUES (6,3,'93905.88763026') +-- !query 262 schema +struct<> +-- !query 262 output + + + +-- !query 263 +INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026') +-- !query 263 schema +struct<> +-- !query 263 output + + + +-- !query 264 +INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206') +-- !query 264 schema +struct<> +-- !query 264 output + + + +-- !query 265 +INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907') +-- !query 265 schema +struct<> +-- !query 265 output + + + +-- !query 266 +INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026') +-- !query 266 schema +struct<> +-- !query 266 output + + + +-- !query 267 +INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974') +-- !query 267 schema +struct<> +-- !query 267 output + + + +-- !query 268 +INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094') +-- !query 268 schema +struct<> +-- !query 268 output + + + +-- !query 269 +INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469') +-- !query 269 schema +struct<> +-- !query 269 output + + + +-- !query 270 +INSERT INTO num_exp_add VALUES (6,5,'110298.61612126') +-- !query 270 schema +struct<> +-- !query 270 output + + + +-- !query 271 +INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926') +-- !query 271 schema +struct<> +-- !query 271 output + + + +-- !query 272 +INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766') +-- !query 272 schema +struct<> +-- !query 272 output + + + +-- !query 273 +INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679') +-- !query 273 schema +struct<> +-- !query 273 output + + + +-- !query 274 +INSERT INTO num_exp_add VALUES (6,6,'187803.15526052') +-- !query 274 schema +struct<> +-- !query 274 output + + + +-- !query 275 +INSERT INTO num_exp_sub VALUES (6,6,'0') +-- !query 275 schema +struct<> +-- !query 275 output + + + +-- !query 276 +INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676') +-- !query 276 schema +struct<> +-- !query 276 output + + + +-- !query 277 +INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000') +-- !query 277 schema +struct<> +-- !query 277 output + + + +-- !query 278 +INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974') +-- !query 278 schema +struct<> +-- !query 278 output + + + +-- !query 279 +INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026') +-- !query 279 schema +struct<> +-- !query 279 output + + + +-- !query 280 +INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610') +-- !query 280 schema +struct<> +-- !query 280 output + + + +-- !query 281 +INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980') +-- !query 281 schema +struct<> +-- !query 281 output + + + +-- !query 282 +INSERT INTO num_exp_add VALUES (6,8,'168782.57763026') +-- !query 282 schema +struct<> +-- !query 282 output + + + +-- !query 283 +INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026') +-- !query 283 schema +struct<> +-- !query 283 output + + + +-- !query 284 +INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906') +-- !query 284 schema +struct<> +-- !query 284 output + + + +-- !query 285 +INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184') +-- !query 285 schema +struct<> +-- !query 285 output + + + +-- !query 286 +INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160') +-- !query 286 schema +struct<> +-- !query 286 output + + + +-- !query 287 +INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680') +-- !query 287 schema +struct<> +-- !query 287 output + + + +-- !query 288 +INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920') +-- !query 288 schema +struct<> +-- !query 288 output + + + +-- !query 289 +INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789') +-- !query 289 schema +struct<> +-- !query 289 output + + + +-- !query 290 +INSERT INTO num_exp_add VALUES (7,0,'-83028485') +-- !query 290 schema +struct<> +-- !query 290 output + + + +-- !query 291 +INSERT INTO num_exp_sub VALUES (7,0,'-83028485') +-- !query 291 schema +struct<> +-- !query 291 output + + + +-- !query 292 +INSERT INTO num_exp_mul VALUES (7,0,'0') +-- !query 292 schema +struct<> +-- !query 292 output + + + +-- !query 293 +INSERT INTO num_exp_div VALUES (7,0,'NaN') +-- !query 293 schema +struct<> +-- !query 293 output + + + +-- !query 294 +INSERT INTO num_exp_add VALUES (7,1,'-83028485') +-- !query 294 schema +struct<> +-- !query 294 output + + + +-- !query 295 +INSERT INTO num_exp_sub VALUES (7,1,'-83028485') +-- !query 295 schema +struct<> +-- !query 295 output + + + +-- !query 296 +INSERT INTO num_exp_mul VALUES (7,1,'0') +-- !query 296 schema +struct<> +-- !query 296 output + + + +-- !query 297 +INSERT INTO num_exp_div VALUES (7,1,'NaN') +-- !query 297 schema +struct<> +-- !query 297 output + + + +-- !query 298 +INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047') +-- !query 298 schema +struct<> +-- !query 298 output + + + +-- !query 299 +INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953') +-- !query 299 schema +struct<> +-- !query 299 output + + + +-- !query 300 +INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795') +-- !query 300 schema +struct<> +-- !query 300 output + + + +-- !query 301 +INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700') +-- !query 301 schema +struct<> +-- !query 301 output + + + +-- !query 302 +INSERT INTO num_exp_add VALUES (7,3,'-83028480.69') +-- !query 302 schema +struct<> +-- !query 302 output + + + +-- !query 303 +INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31') +-- !query 303 schema +struct<> +-- !query 303 output + + + +-- !query 304 +INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35') +-- !query 304 schema +struct<> +-- !query 304 output + + + +-- !query 305 +INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974') +-- !query 305 schema +struct<> +-- !query 305 output + + + +-- !query 306 +INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881') +-- !query 306 schema +struct<> +-- !query 306 output + + + +-- !query 307 +INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119') +-- !query 307 schema +struct<> +-- !query 307 output + + + +-- !query 308 +INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715') +-- !query 308 schema +struct<> +-- !query 308 output + + + +-- !query 309 +INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686') +-- !query 309 schema +struct<> +-- !query 309 output + + + +-- !query 310 +INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509') +-- !query 310 schema +struct<> +-- !query 310 output + + + +-- !query 311 +INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491') +-- !query 311 schema +struct<> +-- !query 311 output + + + +-- !query 312 +INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135') +-- !query 312 schema +struct<> +-- !query 312 output + + + +-- !query 313 +INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574') +-- !query 313 schema +struct<> +-- !query 313 output + + + +-- !query 314 +INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974') +-- !query 314 schema +struct<> +-- !query 314 output + + + +-- !query 315 +INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026') +-- !query 315 schema +struct<> +-- !query 315 output + + + +-- !query 316 +INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610') +-- !query 316 schema +struct<> +-- !query 316 output + + + +-- !query 317 +INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294') +-- !query 317 schema +struct<> +-- !query 317 output + + + +-- !query 318 +INSERT INTO num_exp_add VALUES (7,7,'-166056970') +-- !query 318 schema +struct<> +-- !query 318 output + + + +-- !query 319 +INSERT INTO num_exp_sub VALUES (7,7,'0') +-- !query 319 schema +struct<> +-- !query 319 output + + + +-- !query 320 +INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225') +-- !query 320 schema +struct<> +-- !query 320 output + + + +-- !query 321 +INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000') +-- !query 321 schema +struct<> +-- !query 321 output + + + +-- !query 322 +INSERT INTO num_exp_add VALUES (7,8,'-82953604') +-- !query 322 schema +struct<> +-- !query 322 output + + + +-- !query 323 +INSERT INTO num_exp_sub VALUES (7,8,'-83103366') +-- !query 323 schema +struct<> +-- !query 323 output + + + +-- !query 324 +INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285') +-- !query 324 schema +struct<> +-- !query 324 output + + + +-- !query 325 +INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118') +-- !query 325 schema +struct<> +-- !query 325 output + + + +-- !query 326 +INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420') +-- !query 326 schema +struct<> +-- !query 326 output + + + +-- !query 327 +INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580') +-- !query 327 schema +struct<> +-- !query 327 output + + + +-- !query 328 +INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700') +-- !query 328 schema +struct<> +-- !query 328 output + + + +-- !query 329 +INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382') +-- !query 329 schema +struct<> +-- !query 329 output + + + +-- !query 330 +INSERT INTO num_exp_add VALUES (8,0,'74881') +-- !query 330 schema +struct<> +-- !query 330 output + + + +-- !query 331 +INSERT INTO num_exp_sub VALUES (8,0,'74881') +-- !query 331 schema +struct<> +-- !query 331 output + + + +-- !query 332 +INSERT INTO num_exp_mul VALUES (8,0,'0') +-- !query 332 schema +struct<> +-- !query 332 output + + + +-- !query 333 +INSERT INTO num_exp_div VALUES (8,0,'NaN') +-- !query 333 schema +struct<> +-- !query 333 output + + + +-- !query 334 +INSERT INTO num_exp_add VALUES (8,1,'74881') +-- !query 334 schema +struct<> +-- !query 334 output + + + +-- !query 335 +INSERT INTO num_exp_sub VALUES (8,1,'74881') +-- !query 335 schema +struct<> +-- !query 335 output + + + +-- !query 336 +INSERT INTO num_exp_mul VALUES (8,1,'0') +-- !query 336 schema +struct<> +-- !query 336 output + + + +-- !query 337 +INSERT INTO num_exp_div VALUES (8,1,'NaN') +-- !query 337 schema +struct<> +-- !query 337 output + + + +-- !query 338 +INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047') +-- !query 338 schema +struct<> +-- !query 338 output + + + +-- !query 339 +INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047') +-- !query 339 schema +struct<> +-- !query 339 output + + + +-- !query 340 +INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407') +-- !query 340 schema +struct<> +-- !query 340 output + + + +-- !query 341 +INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615') +-- !query 341 schema +struct<> +-- !query 341 output + + + +-- !query 342 +INSERT INTO num_exp_add VALUES (8,3,'74885.31') +-- !query 342 schema +struct<> +-- !query 342 output + + + +-- !query 343 +INSERT INTO num_exp_sub VALUES (8,3,'74876.69') +-- !query 343 schema +struct<> +-- !query 343 output + + + +-- !query 344 +INSERT INTO num_exp_mul VALUES (8,3,'322737.11') +-- !query 344 schema +struct<> +-- !query 344 output + + + +-- !query 345 +INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410') +-- !query 345 schema +struct<> +-- !query 345 output + + + +-- !query 346 +INSERT INTO num_exp_add VALUES (8,4,'7874342.4119') +-- !query 346 schema +struct<> +-- !query 346 output + + + +-- !query 347 +INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119') +-- !query 347 schema +struct<> +-- !query 347 output + + + +-- !query 348 +INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839') +-- !query 348 schema +struct<> +-- !query 348 output + + + +-- !query 349 +INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956') +-- !query 349 schema +struct<> +-- !query 349 output + + + +-- !query 350 +INSERT INTO num_exp_add VALUES (8,5,'91278.038491') +-- !query 350 schema +struct<> +-- !query 350 output + + + +-- !query 351 +INSERT INTO num_exp_sub VALUES (8,5,'58483.961509') +-- !query 351 schema +struct<> +-- !query 351 output + + + +-- !query 352 +INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571') +-- !query 352 schema +struct<> +-- !query 352 output + + + +-- !query 353 +INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456') +-- !query 353 schema +struct<> +-- !query 353 output + + + +-- !query 354 +INSERT INTO num_exp_add VALUES (8,6,'168782.57763026') +-- !query 354 schema +struct<> +-- !query 354 output + + + +-- !query 355 +INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026') +-- !query 355 schema +struct<> +-- !query 355 output + + + +-- !query 356 +INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906') +-- !query 356 schema +struct<> +-- !query 356 output + + + +-- !query 357 +INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424') +-- !query 357 schema +struct<> +-- !query 357 output + + + +-- !query 358 +INSERT INTO num_exp_add VALUES (8,7,'-82953604') +-- !query 358 schema +struct<> +-- !query 358 output + + + +-- !query 359 +INSERT INTO num_exp_sub VALUES (8,7,'83103366') +-- !query 359 schema +struct<> +-- !query 359 output + + + +-- !query 360 +INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285') +-- !query 360 schema +struct<> +-- !query 360 output + + + +-- !query 361 +INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172') +-- !query 361 schema +struct<> +-- !query 361 output + + + +-- !query 362 +INSERT INTO num_exp_add VALUES (8,8,'149762') +-- !query 362 schema +struct<> +-- !query 362 output + + + +-- !query 363 +INSERT INTO num_exp_sub VALUES (8,8,'0') +-- !query 363 schema +struct<> +-- !query 363 output + + + +-- !query 364 +INSERT INTO num_exp_mul VALUES (8,8,'5607164161') +-- !query 364 schema +struct<> +-- !query 364 output + + + +-- !query 365 +INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000') +-- !query 365 schema +struct<> +-- !query 365 output + + + +-- !query 366 +INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420') +-- !query 366 schema +struct<> +-- !query 366 output + + + +-- !query 367 +INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420') +-- !query 367 schema +struct<> +-- !query 367 output + + + +-- !query 368 +INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020') +-- !query 368 schema +struct<> +-- !query 368 output + + + +-- !query 369 +INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735') +-- !query 369 schema +struct<> +-- !query 369 output + + + +-- !query 370 +INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420') +-- !query 370 schema +struct<> +-- !query 370 output + + + +-- !query 371 +INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420') +-- !query 371 schema +struct<> +-- !query 371 output + + + +-- !query 372 +INSERT INTO num_exp_mul VALUES (9,0,'0') +-- !query 372 schema +struct<> +-- !query 372 output + + + +-- !query 373 +INSERT INTO num_exp_div VALUES (9,0,'NaN') +-- !query 373 schema +struct<> +-- !query 373 output + + + +-- !query 374 +INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420') +-- !query 374 schema +struct<> +-- !query 374 output + + + +-- !query 375 +INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420') +-- !query 375 schema +struct<> +-- !query 375 output + + + +-- !query 376 +INSERT INTO num_exp_mul VALUES (9,1,'0') +-- !query 376 schema +struct<> +-- !query 376 output + + + +-- !query 377 +INSERT INTO num_exp_div VALUES (9,1,'NaN') +-- !query 377 schema +struct<> +-- !query 377 output + + + +-- !query 378 +INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467') +-- !query 378 schema +struct<> +-- !query 378 output + + + +-- !query 379 +INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627') +-- !query 379 schema +struct<> +-- !query 379 output + + + +-- !query 380 +INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740') +-- !query 380 schema +struct<> +-- !query 380 output + + + +-- !query 381 +INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526') +-- !query 381 schema +struct<> +-- !query 381 output + + + +-- !query 382 +INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420') +-- !query 382 schema +struct<> +-- !query 382 output + + + +-- !query 383 +INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420') +-- !query 383 schema +struct<> +-- !query 383 output + + + +-- !query 384 +INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020') +-- !query 384 schema +struct<> +-- !query 384 output + + + +-- !query 385 +INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677') +-- !query 385 schema +struct<> +-- !query 385 output + + + +-- !query 386 +INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420') +-- !query 386 schema +struct<> +-- !query 386 output + + + +-- !query 387 +INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420') +-- !query 387 schema +struct<> +-- !query 387 output + + + +-- !query 388 +INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980') +-- !query 388 schema +struct<> +-- !query 388 output + + + +-- !query 389 +INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484') +-- !query 389 schema +struct<> +-- !query 389 output + + + +-- !query 390 +INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420') +-- !query 390 schema +struct<> +-- !query 390 output + + + +-- !query 391 +INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420') +-- !query 391 schema +struct<> +-- !query 391 output + + + +-- !query 392 +INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220') +-- !query 392 schema +struct<> +-- !query 392 output + + + +-- !query 393 +INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807') +-- !query 393 schema +struct<> +-- !query 393 output + + + +-- !query 394 +INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160') +-- !query 394 schema +struct<> +-- !query 394 output + + + +-- !query 395 +INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680') +-- !query 395 schema +struct<> +-- !query 395 output + + + +-- !query 396 +INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920') +-- !query 396 schema +struct<> +-- !query 396 output + + + +-- !query 397 +INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280') +-- !query 397 schema +struct<> +-- !query 397 output + + + +-- !query 398 +INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420') +-- !query 398 schema +struct<> +-- !query 398 output + + + +-- !query 399 +INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580') +-- !query 399 schema +struct<> +-- !query 399 output + + + +-- !query 400 +INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700') +-- !query 400 schema +struct<> +-- !query 400 output + + + +-- !query 401 +INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689') +-- !query 401 schema +struct<> +-- !query 401 output + + + +-- !query 402 +INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420') +-- !query 402 schema +struct<> +-- !query 402 output + + + +-- !query 403 +INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420') +-- !query 403 schema +struct<> +-- !query 403 output + + + +-- !query 404 +INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020') +-- !query 404 schema +struct<> +-- !query 404 output + + + +-- !query 405 +INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748') +-- !query 405 schema +struct<> +-- !query 405 output + + + +-- !query 406 +INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840') +-- !query 406 schema +struct<> +-- !query 406 output + + + +-- !query 407 +INSERT INTO num_exp_sub VALUES (9,9,'0') +-- !query 407 schema +struct<> +-- !query 407 output + + + +-- !query 408 +INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400') +-- !query 408 schema +struct<> +-- !query 408 output + + + +-- !query 409 +INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000') +-- !query 409 schema +struct<> +-- !query 409 output + + + +-- !query 410 +INSERT INTO num_exp_sqrt VALUES (0,'0') +-- !query 410 schema +struct<> +-- !query 410 output + + + +-- !query 411 +INSERT INTO num_exp_sqrt VALUES (1,'0') +-- !query 411 schema +struct<> +-- !query 411 output + + + +-- !query 412 +INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505') +-- !query 412 schema +struct<> +-- !query 412 output + + + +-- !query 413 +INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396') +-- !query 413 schema +struct<> +-- !query 413 output + + + +-- !query 414 +INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923') +-- !query 414 schema +struct<> +-- !query 414 output + + + +-- !query 415 +INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473') +-- !query 415 schema +struct<> +-- !query 415 output + + + +-- !query 416 +INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406') +-- !query 416 schema +struct<> +-- !query 416 output + + + +-- !query 417 +INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230') +-- !query 417 schema +struct<> +-- !query 417 output + + + +-- !query 418 +INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542') +-- !query 418 schema +struct<> +-- !query 418 output + + + +-- !query 419 +INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766') +-- !query 419 schema +struct<> +-- !query 419 output + + + +-- !query 420 +INSERT INTO num_exp_ln VALUES (0,'NaN') +-- !query 420 schema +struct<> +-- !query 420 output + + + +-- !query 421 +INSERT INTO num_exp_ln VALUES (1,'NaN') +-- !query 421 schema +struct<> +-- !query 421 output + + + +-- !query 422 +INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514') +-- !query 422 schema +struct<> +-- !query 422 output + + + +-- !query 423 +INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971') +-- !query 423 schema +struct<> +-- !query 423 output + + + +-- !query 424 +INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464') +-- !query 424 schema +struct<> +-- !query 424 output + + + +-- !query 425 +INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038') +-- !query 425 schema +struct<> +-- !query 425 output + + + +-- !query 426 +INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127') +-- !query 426 schema +struct<> +-- !query 426 output + + + +-- !query 427 +INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991') +-- !query 427 schema +struct<> +-- !query 427 output + + + +-- !query 428 +INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668') +-- !query 428 schema +struct<> +-- !query 428 output + + + +-- !query 429 +INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962') +-- !query 429 schema +struct<> +-- !query 429 output + + + +-- !query 430 +INSERT INTO num_exp_log10 VALUES (0,'NaN') +-- !query 430 schema +struct<> +-- !query 430 output + + + +-- !query 431 +INSERT INTO num_exp_log10 VALUES (1,'NaN') +-- !query 431 schema +struct<> +-- !query 431 output + + + +-- !query 432 +INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459') +-- !query 432 schema +struct<> +-- !query 432 output + + + +-- !query 433 +INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075') +-- !query 433 schema +struct<> +-- !query 433 output + + + +-- !query 434 +INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345') +-- !query 434 schema +struct<> +-- !query 434 output + + + +-- !query 435 +INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626') +-- !query 435 schema +struct<> +-- !query 435 output + + + +-- !query 436 +INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671') +-- !query 436 schema +struct<> +-- !query 436 output + + + +-- !query 437 +INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914') +-- !query 437 schema +struct<> +-- !query 437 output + + + +-- !query 438 +INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138') +-- !query 438 schema +struct<> +-- !query 438 output + + + +-- !query 439 +INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059') +-- !query 439 schema +struct<> +-- !query 439 output + + + +-- !query 440 +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN') +-- !query 440 schema +struct<> +-- !query 440 output + + + +-- !query 441 +INSERT INTO num_exp_power_10_ln VALUES (1,'NaN') +-- !query 441 schema +struct<> +-- !query 441 output + + + +-- !query 442 +INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184') +-- !query 442 schema +struct<> +-- !query 442 output + + + +-- !query 443 +INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393') +-- !query 443 schema +struct<> +-- !query 443 output + + + +-- !query 444 +INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636') +-- !query 444 schema +struct<> +-- !query 444 output + + + +-- !query 445 +INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098') +-- !query 445 schema +struct<> +-- !query 445 output + + + +-- !query 446 +INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067') +-- !query 446 schema +struct<> +-- !query 446 output + + + +-- !query 447 +INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627') +-- !query 447 schema +struct<> +-- !query 447 output + + + +-- !query 448 +INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952') +-- !query 448 schema +struct<> +-- !query 448 output + + + +-- !query 449 +INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457') +-- !query 449 schema +struct<> +-- !query 449 output + + + +-- !query 450 +INSERT INTO num_data VALUES (0, '0') +-- !query 450 schema +struct<> +-- !query 450 output + + + +-- !query 451 +INSERT INTO num_data VALUES (1, '0') +-- !query 451 schema +struct<> +-- !query 451 output + + + +-- !query 452 +INSERT INTO num_data VALUES (2, '-34338492.215397047') +-- !query 452 schema +struct<> +-- !query 452 output + + + +-- !query 453 +INSERT INTO num_data VALUES (3, '4.31') +-- !query 453 schema +struct<> +-- !query 453 output + + + +-- !query 454 +INSERT INTO num_data VALUES (4, '7799461.4119') +-- !query 454 schema +struct<> +-- !query 454 output + + + +-- !query 455 +INSERT INTO num_data VALUES (5, '16397.038491') +-- !query 455 schema +struct<> +-- !query 455 output + + + +-- !query 456 +INSERT INTO num_data VALUES (6, '93901.57763026') +-- !query 456 schema +struct<> +-- !query 456 output + + + +-- !query 457 +INSERT INTO num_data VALUES (7, '-83028485') +-- !query 457 schema +struct<> +-- !query 457 output + + + +-- !query 458 +INSERT INTO num_data VALUES (8, '74881') +-- !query 458 schema +struct<> +-- !query 458 output + + + +-- !query 459 +INSERT INTO num_data VALUES (9, '-24926804.045047420') +-- !query 459 schema +struct<> +-- !query 459 output + + + +-- !query 460 +SELECT * FROM num_data +-- !query 460 schema +struct +-- !query 460 output +0 0 +1 0 +2 -34338492.215397047 +3 4.31 +4 7799461.4119 +5 16397.038491 +6 93901.57763026 +7 -83028485 +8 74881 +9 -24926804.04504742 + + +-- !query 461 +TRUNCATE TABLE num_result +-- !query 461 schema +struct<> +-- !query 461 output + + + +-- !query 462 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query 462 schema +struct<> +-- !query 462 output + + + +-- !query 463 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 463 schema +struct +-- !query 463 output + + + +-- !query 464 +TRUNCATE TABLE num_result +-- !query 464 schema +struct<> +-- !query 464 output + + + +-- !query 465 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query 465 schema +struct<> +-- !query 465 output + + + +-- !query 466 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10) +-- !query 466 schema +struct +-- !query 466 output + + + +-- !query 467 +TRUNCATE TABLE num_result +-- !query 467 schema +struct<> +-- !query 467 output + + + +-- !query 468 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query 468 schema +struct<> +-- !query 468 output + + + +-- !query 469 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 469 schema +struct +-- !query 469 output + + + +-- !query 470 +TRUNCATE TABLE num_result +-- !query 470 schema +struct<> +-- !query 470 output + + + +-- !query 471 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query 471 schema +struct<> +-- !query 471 output + + + +-- !query 472 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40) +-- !query 472 schema +struct +-- !query 472 output + + + +-- !query 473 +TRUNCATE TABLE num_result +-- !query 473 schema +struct<> +-- !query 473 output + + + +-- !query 474 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2 +-- !query 474 schema +struct<> +-- !query 474 output +org.apache.spark.sql.AnalysisException +`default`.`num_result` requires that the data to be inserted have the same number of columns as the target table: target table has 3 column(s) but the inserted data has 5 column(s), including 0 partition column(s) having constant value(s).; + + +-- !query 475 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 475 schema +struct +-- !query 475 output + + + +-- !query 476 +TRUNCATE TABLE num_result +-- !query 476 schema +struct<> +-- !query 476 output + + + +-- !query 477 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query 477 schema +struct<> +-- !query 477 output + + + +-- !query 478 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30) +-- !query 478 schema +struct +-- !query 478 output +2 2 1179132047626883.596862 1179132047626883.5968621359 +2 3 -147998901.448361 -147998901.4483612726 +2 4 -267821744976817.811114 -267821744976817.8111137107 +2 5 -563049578578.769243 -563049578578.7692425067 +2 6 -3224438592470.184498 -3224438592470.1844981193 +2 7 2851072985828710.485884 2851072985828710.485883795 +2 8 -2571300635581.146276 -2571300635581.146276407 +2 9 855948866655588.453742 855948866655588.4537415092 +3 2 -147998901.448361 -147998901.4483612726 +3 5 70671.235896 70671.23589621 +3 6 404715.799586 404715.7995864206 +3 9 -107434525.434154 -107434525.4341543802 +4 2 -267821744976817.811114 -267821744976817.8111137107 +4 4 60831598315717.141462 60831598315717.14146161 +4 5 127888068979.993505 127888068979.9935054429 +4 6 732381731243.745116 732381731243.7451157641 +4 9 -194415646271340.181596 -194415646271340.1815956523 +5 2 -563049578578.769243 -563049578578.7692425067 +5 3 70671.235896 70671.23589621 +5 4 127888068979.993505 127888068979.9935054429 +5 5 268862871.275336 268862871.2753355571 +5 6 1539707782.768998 1539707782.7689977863 +5 9 -408725765384.257044 -408725765384.2570436602 +6 2 -3224438592470.184498 -3224438592470.1844981193 +6 3 404715.799586 404715.7995864206 +6 4 732381731243.745116 732381731243.7451157641 +6 5 1539707782.768998 1539707782.7689977863 +6 6 8817506281.451745 8817506281.4517452373 +6 7 -7796505729750.377956 -7796505729750.3779561 +6 8 7031444034.531499 7031444034.53149906 +6 9 -2340666225110.299295 -2340666225110.2992952129 +7 2 2851072985828710.485884 2851072985828710.485883795 +7 6 -7796505729750.377956 -7796505729750.3779561 +7 9 2069634775752159.035759 2069634775752159.0357587 +8 2 -2571300635581.146276 -2571300635581.146276407 +8 6 7031444034.531499 7031444034.53149906 +8 9 -1866544013697.195857 -1866544013697.19585702 +9 2 855948866655588.453742 855948866655588.4537415092 +9 3 -107434525.434154 -107434525.4341543802 +9 4 -194415646271340.181596 -194415646271340.1815956523 +9 5 -408725765384.257044 -408725765384.2570436602 +9 6 -2340666225110.299295 -2340666225110.2992952129 +9 7 2069634775752159.035759 2069634775752159.0357587 +9 8 -1866544013697.195857 -1866544013697.19585702 +9 9 621345559900192.420121 621345559900192.42012063 + + +-- !query 479 +TRUNCATE TABLE num_result +-- !query 479 schema +struct<> +-- !query 479 output + + + +-- !query 480 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query 480 schema +struct<> +-- !query 480 output + + + +-- !query 481 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 481 schema +struct +-- !query 481 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 482 +TRUNCATE TABLE num_result +-- !query 482 schema +struct<> +-- !query 482 output + + + +-- !query 483 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query 483 schema +struct<> +-- !query 483 output + + + +-- !query 484 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80) +-- !query 484 schema +struct +-- !query 484 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 485 +TRUNCATE TABLE num_result +-- !query 485 schema +struct<> +-- !query 485 output + + + +-- !query 486 +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query 486 schema +struct<> +-- !query 486 output + + + +-- !query 487 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 487 schema +struct +-- !query 487 output + + + +-- !query 488 +TRUNCATE TABLE num_result +-- !query 488 schema +struct<> +-- !query 488 output + + + +-- !query 489 +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 489 schema +struct<> +-- !query 489 output + + + +-- !query 490 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 490 schema +struct +-- !query 490 output + + + +-- !query 491 +TRUNCATE TABLE num_result +-- !query 491 schema +struct<> +-- !query 491 output + + + +-- !query 492 +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 492 schema +struct<> +-- !query 492 output + + + +-- !query 493 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 493 schema +struct +-- !query 493 output + + + +-- !query 494 +TRUNCATE TABLE num_result +-- !query 494 schema +struct<> +-- !query 494 output + + + +-- !query 495 +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0' +-- !query 495 schema +struct<> +-- !query 495 output + + + +-- !query 496 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 496 schema +struct +-- !query 496 output +2 224790267919917472 224790267919917955.1326161858 +4 7405685069595001 7405685069594999.0773399947 +5 5068226527.321263 5068226527.3212726541 +6 281839893606.99365 281839893606.9937234336 +7 1716699575118595840 1716699575118597095.4233081991 +8 167361463828.0749 167361463828.0749132007 +9 107511333880051856 107511333880052007.0414112467 + + +-- !query 497 +SELECT AVG(val) FROM num_data +-- !query 497 schema +struct +-- !query 497 output +-13430913.5922423207 + + +-- !query 498 +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query 498 schema +struct<> +-- !query 498 output + + + +-- !query 499 +INSERT INTO fract_only VALUES (1, '0.0') +-- !query 499 schema +struct<> +-- !query 499 output + + + +-- !query 500 +INSERT INTO fract_only VALUES (2, '0.1') +-- !query 500 schema +struct<> +-- !query 500 output + + + +-- !query 501 +INSERT INTO fract_only VALUES (4, '-0.9999') +-- !query 501 schema +struct<> +-- !query 501 output + + + +-- !query 502 +INSERT INTO fract_only VALUES (5, '0.99994') +-- !query 502 schema +struct<> +-- !query 502 output + + + +-- !query 503 +INSERT INTO fract_only VALUES (7, '0.00001') +-- !query 503 schema +struct<> +-- !query 503 output + + + +-- !query 504 +INSERT INTO fract_only VALUES (8, '0.00017') +-- !query 504 schema +struct<> +-- !query 504 output + + + +-- !query 505 +SELECT * FROM fract_only +-- !query 505 schema +struct +-- !query 505 output +1 0 +2 0.1 +4 -0.9999 +5 0.9999 +7 0 +8 0.0002 + + +-- !query 506 +DROP TABLE fract_only +-- !query 506 schema +struct<> +-- !query 506 output + + + +-- !query 507 +SELECT decimal(double('NaN')) +-- !query 507 schema +struct +-- !query 507 output +NULL + + +-- !query 508 +SELECT decimal(double('Infinity')) +-- !query 508 schema +struct +-- !query 508 output +NULL + + +-- !query 509 +SELECT decimal(double('-Infinity')) +-- !query 509 schema +struct +-- !query 509 output +NULL + + +-- !query 510 +SELECT decimal(float('NaN')) +-- !query 510 schema +struct +-- !query 510 output +NULL + + +-- !query 511 +SELECT decimal(float('Infinity')) +-- !query 511 schema +struct +-- !query 511 output +NULL + + +-- !query 512 +SELECT decimal(float('-Infinity')) +-- !query 512 schema +struct +-- !query 512 output +NULL + + +-- !query 513 +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query 513 schema +struct<> +-- !query 513 output + + + +-- !query 514 +INSERT INTO ceil_floor_round VALUES ('-5.5') +-- !query 514 schema +struct<> +-- !query 514 output + + + +-- !query 515 +INSERT INTO ceil_floor_round VALUES ('-5.499999') +-- !query 515 schema +struct<> +-- !query 515 output + + + +-- !query 516 +INSERT INTO ceil_floor_round VALUES ('9.5') +-- !query 516 schema +struct<> +-- !query 516 output + + + +-- !query 517 +INSERT INTO ceil_floor_round VALUES ('9.4999999') +-- !query 517 schema +struct<> +-- !query 517 output + + + +-- !query 518 +INSERT INTO ceil_floor_round VALUES ('0.0') +-- !query 518 schema +struct<> +-- !query 518 output + + + +-- !query 519 +INSERT INTO ceil_floor_round VALUES ('0.0000001') +-- !query 519 schema +struct<> +-- !query 519 output + + + +-- !query 520 +INSERT INTO ceil_floor_round VALUES ('-0.000001') +-- !query 520 schema +struct<> +-- !query 520 output + + + +-- !query 521 +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query 521 schema +struct +-- !query 521 output +-0.000001 0 0 -1 0 +-5.499999 -5 -5 -6 -5 +-5.5 -5 -5 -6 -6 +0 0 0 0 0 +0.0000001 1 1 0 0 +9.4999999 10 10 9 9 +9.5 10 10 9 10 + + +-- !query 522 +DROP TABLE ceil_floor_round +-- !query 522 schema +struct<> +-- !query 522 output + + + +-- !query 523 +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query 523 schema +struct<> +-- !query 523 output + + + +-- !query 524 +INSERT INTO num_input_test VALUES (trim(' 123')) +-- !query 524 schema +struct<> +-- !query 524 output + + + +-- !query 525 +INSERT INTO num_input_test VALUES (trim(' 3245874 ')) +-- !query 525 schema +struct<> +-- !query 525 output + + + +-- !query 526 +INSERT INTO num_input_test VALUES (trim(' -93853')) +-- !query 526 schema +struct<> +-- !query 526 output + + + +-- !query 527 +INSERT INTO num_input_test VALUES ('555.50') +-- !query 527 schema +struct<> +-- !query 527 output + + + +-- !query 528 +INSERT INTO num_input_test VALUES ('-555.50') +-- !query 528 schema +struct<> +-- !query 528 output + + + +-- !query 529 +SELECT * FROM num_input_test +-- !query 529 schema +struct +-- !query 529 output +-555.5 +-93853 +123 +3245874 +555.5 + + +-- !query 530 +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query 530 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> +-- !query 530 output +1 + + +-- !query 531 +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 531 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 531 output +999999999999999999999 + + +-- !query 532 +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 532 schema +struct<(CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 532 output +-999999999999999999999 + + +-- !query 533 +select mod (70.0,70) +-- !query 533 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> +-- !query 533 output +0 + + +-- !query 534 +select 70.0 / 70 +-- !query 534 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> +-- !query 534 output +1 + + +-- !query 535 +select 12345678901234567890 % 123 +-- !query 535 schema +struct<(CAST(12345678901234567890 AS DECIMAL(20,0)) % CAST(CAST(123 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 535 output +78 + + +-- !query 536 +select exp(0.0) +-- !query 536 schema +struct +-- !query 536 output +1.0 + + +-- !query 537 +select exp(1.0) +-- !query 537 schema +struct +-- !query 537 output +2.718281828459045 + + +-- !query 538 +select exp(32.999) +-- !query 538 schema +struct +-- !query 538 output +2.1442904349215556E14 + + +-- !query 539 +select exp(-32.999) +-- !query 539 schema +struct +-- !query 539 output +4.663547361468238E-15 + + +-- !query 540 +select exp(123.456) +-- !query 540 schema +struct +-- !query 540 output +4.132944352778106E53 + + +-- !query 541 +select exp(-123.456) +-- !query 541 schema +struct +-- !query 541 output +2.4195825412645934E-54 + + +-- !query 542 +select exp(1234.5678) +-- !query 542 schema +struct +-- !query 542 output +Infinity + + +-- !query 543 +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query 543 schema +struct +-- !query 543 output +0 +1 +2 +3 + + +-- !query 544 +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) +-- !query 544 schema +struct +-- !query 544 output +0 +1 +2 +3 + + +-- !query 545 +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) +-- !query 545 schema +struct +-- !query 545 output +0 +2 +4 + + +-- !query 546 +select ln(1.2345678e-28) +-- !query 546 schema +struct +-- !query 546 output +-64.26166165451762 + + +-- !query 547 +select ln(0.0456789) +-- !query 547 schema +struct +-- !query 547 output +-3.0861187944847437 + + +-- !query 548 +select ln(0.99949452) +-- !query 548 schema +struct +-- !query 548 output +-5.056077980832118E-4 + + +-- !query 549 +select ln(1.00049687395) +-- !query 549 schema +struct +-- !query 549 output +4.967505490136803E-4 + + +-- !query 550 +select ln(1234.567890123456789) +-- !query 550 schema +struct +-- !query 550 output +7.11847630129779 + + +-- !query 551 +select ln(5.80397490724e5) +-- !query 551 schema +struct +-- !query 551 output +13.271468476626518 + + +-- !query 552 +select ln(9.342536355e34) +-- !query 552 schema +struct +-- !query 552 output +80.52247093552418 + + +-- !query 553 +select log(3.4634998359873254962349856073435545) +-- !query 553 schema +struct +-- !query 553 output +1.2422795911259166 + + +-- !query 554 +select log(9.999999999999999999) +-- !query 554 schema +struct +-- !query 554 output +2.302585092994046 + + +-- !query 555 +select log(10.00000000000000000) +-- !query 555 schema +struct +-- !query 555 output +2.302585092994046 + + +-- !query 556 +select log(10.00000000000000001) +-- !query 556 schema +struct +-- !query 556 output +2.302585092994046 + + +-- !query 557 +select log(590489.45235237) +-- !query 557 schema +struct +-- !query 557 output +13.288707052228641 + + +-- !query 558 +select log(0.99923, 4.58934e34) +-- !query 558 schema +struct +-- !query 558 output +-103611.55579543479 + + +-- !query 559 +select log(1.000016, 8.452010e18) +-- !query 559 schema +struct +-- !query 559 output +2723830.287707013 + + +-- !query 560 +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query 560 schema +struct +-- !query 560 output +999900000 + + +-- !query 561 +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query 561 schema +struct +-- !query 561 output +-999900000 + + +-- !query 562 +DROP TABLE num_data +-- !query 562 schema +struct<> +-- !query 562 output + + + +-- !query 563 +DROP TABLE num_exp_add +-- !query 563 schema +struct<> +-- !query 563 output + + + +-- !query 564 +DROP TABLE num_exp_sub +-- !query 564 schema +struct<> +-- !query 564 output + + + +-- !query 565 +DROP TABLE num_exp_div +-- !query 565 schema +struct<> +-- !query 565 output + + + +-- !query 566 +DROP TABLE num_exp_mul +-- !query 566 schema +struct<> +-- !query 566 output + + + +-- !query 567 +DROP TABLE num_exp_sqrt +-- !query 567 schema +struct<> +-- !query 567 output + + + +-- !query 568 +DROP TABLE num_exp_ln +-- !query 568 schema +struct<> +-- !query 568 output + + + +-- !query 569 +DROP TABLE num_exp_log10 +-- !query 569 schema +struct<> +-- !query 569 output + + + +-- !query 570 +DROP TABLE num_exp_power_10_ln +-- !query 570 schema +struct<> +-- !query 570 output + + + +-- !query 571 +DROP TABLE num_result +-- !query 571 schema +struct<> +-- !query 571 output + + + +-- !query 572 +DROP TABLE num_input_test +-- !query 572 schema +struct<> +-- !query 572 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out new file mode 100644 index 0000000000000..797f808dad11e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out @@ -0,0 +1,543 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 37 + + +-- !query 0 +create or replace temporary view onek2 as select * from onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1 +-- !query 2 schema +struct +-- !query 2 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 3 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC +-- !query 3 schema +struct +-- !query 3 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 4 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC +-- !query 4 schema +struct +-- !query 4 output +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA + + +-- !query 5 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC +-- !query 5 schema +struct +-- !query 5 output +999 AAAAxx +995 AAAAxx +983 AAAAxx +982 AAAAxx +981 AAAAxx +998 HHHHxx +997 HHHHxx +993 HHHHxx +990 HHHHxx +986 HHHHxx +996 OOOOxx +991 OOOOxx +988 OOOOxx +987 OOOOxx +985 OOOOxx +994 VVVVxx +992 VVVVxx +989 VVVVxx +984 VVVVxx + + +-- !query 6 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC +-- !query 6 schema +struct +-- !query 6 output +984 VVVVxx +989 VVVVxx +992 VVVVxx +994 VVVVxx +985 OOOOxx +987 OOOOxx +988 OOOOxx +991 OOOOxx +996 OOOOxx +986 HHHHxx +990 HHHHxx +993 HHHHxx +997 HHHHxx +998 HHHHxx +981 AAAAxx +982 AAAAxx +983 AAAAxx +995 AAAAxx +999 AAAAxx + + +-- !query 7 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC +-- !query 7 schema +struct +-- !query 7 output +19 OOOOxx +18 VVVVxx +17 HHHHxx +16 OOOOxx +15 VVVVxx +14 AAAAxx +13 OOOOxx +12 AAAAxx +11 OOOOxx +10 AAAAxx +9 HHHHxx +8 HHHHxx +7 VVVVxx +6 OOOOxx +5 HHHHxx +4 HHHHxx +3 VVVVxx +2 OOOOxx +1 OOOOxx +0 OOOOxx + + +-- !query 8 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC +-- !query 8 schema +struct +-- !query 8 output +0 OOOOxx +1 OOOOxx +2 OOOOxx +3 VVVVxx +4 HHHHxx +5 HHHHxx +6 OOOOxx +7 VVVVxx +8 HHHHxx +9 HHHHxx +10 AAAAxx +11 OOOOxx +12 AAAAxx +13 OOOOxx +14 AAAAxx +15 VVVVxx +16 OOOOxx +17 HHHHxx +18 VVVVxx +19 OOOOxx + + +-- !query 9 +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 +-- !query 9 schema +struct +-- !query 9 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 10 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC +-- !query 10 schema +struct +-- !query 10 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 11 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980 +-- !query 11 schema +struct +-- !query 11 output +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA + + +-- !query 12 +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select foo.* from (select 1) as foo +-- !query 13 schema +struct<1:int> +-- !query 13 output +1 + + +-- !query 14 +select foo.* from (select null) as foo +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +select foo.* from (select 'xyzzy',1,null) as foo +-- !query 15 schema +struct +-- !query 15 output +xyzzy 1 NULL + + +-- !query 16 +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j +-- !query 16 schema +struct +-- !query 16 output +147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx 147 RFAAAA +931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx 931 VJAAAA + + +-- !query 17 +VALUES (1,2), (3,4+4), (7,77.7) +-- !query 17 schema +struct +-- !query 17 output +1 2 +3 8 +7 77.7 + + +-- !query 18 +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl +-- !query 18 schema +struct +-- !query 18 output +1 2 +123 456 +123 4567890123456789 +3 8 +4 57 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 +7 77.7 + + +-- !query 19 +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +SELECT * FROM foo ORDER BY f1 +-- !query 20 schema +struct +-- !query 20 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 21 +SELECT * FROM foo ORDER BY f1 ASC +-- !query 21 schema +struct +-- !query 21 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 22 +-- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST +-- !query 22 schema +struct +-- !query 22 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 23 +SELECT * FROM foo ORDER BY f1 DESC +-- !query 23 schema +struct +-- !query 23 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 24 +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST +-- !query 24 schema +struct +-- !query 24 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 25 +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 25 schema +struct +-- !query 25 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 26 +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 26 schema +struct +-- !query 26 output +11 + + +-- !query 27 +select * from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 27 schema +struct +-- !query 27 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 28 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 28 schema +struct +-- !query 28 output +11 + + +-- !query 29 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' +-- !query 29 schema +struct +-- !query 29 output +11 + + +-- !query 30 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 30 schema +struct +-- !query 30 output +11 + + +-- !query 31 +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' +-- !query 31 schema +struct +-- !query 31 output +0 998 +494 11 + + +-- !query 32 +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 +-- !query 32 schema +struct +-- !query 32 output +0 998 +494 11 + + +-- !query 33 +SELECT 1 AS x ORDER BY x +-- !query 33 schema +struct +-- !query 33 output +1 + + +-- !query 34 +select * from (values (2),(null),(1)) v(k) where k = k order by k +-- !query 34 schema +struct +-- !query 34 output +1 +2 + + +-- !query 35 +select * from (values (2),(null),(1)) v(k) where k = k +-- !query 35 schema +struct +-- !query 35 output +1 +2 + + +-- !query 36 +drop table tmp +-- !query 36 schema +struct<> +-- !query 36 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out new file mode 100644 index 0000000000000..38eae1739f553 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out @@ -0,0 +1,225 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 19 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT DISTINCT two FROM tmp ORDER BY 1 +-- !query 1 schema +struct +-- !query 1 output +0 +1 + + +-- !query 2 +SELECT DISTINCT ten FROM tmp ORDER BY 1 +-- !query 2 schema +struct +-- !query 2 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 3 +SELECT DISTINCT string4 FROM tmp ORDER BY 1 +-- !query 3 schema +struct +-- !query 3 output +AAAAxx +HHHHxx +OOOOxx +VVVVxx + + +-- !query 4 +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC +-- !query 4 schema +struct +-- !query 4 output +0 AAAAxx 0 +0 AAAAxx 2 +0 AAAAxx 4 +0 AAAAxx 6 +0 AAAAxx 8 +0 HHHHxx 0 +0 HHHHxx 2 +0 HHHHxx 4 +0 HHHHxx 6 +0 HHHHxx 8 +0 OOOOxx 0 +0 OOOOxx 2 +0 OOOOxx 4 +0 OOOOxx 6 +0 OOOOxx 8 +0 VVVVxx 0 +0 VVVVxx 2 +0 VVVVxx 4 +0 VVVVxx 6 +0 VVVVxx 8 +1 AAAAxx 1 +1 AAAAxx 3 +1 AAAAxx 5 +1 AAAAxx 7 +1 AAAAxx 9 +1 HHHHxx 1 +1 HHHHxx 3 +1 HHHHxx 5 +1 HHHHxx 7 +1 HHHHxx 9 +1 OOOOxx 1 +1 OOOOxx 3 +1 OOOOxx 5 +1 OOOOxx 7 +1 OOOOxx 9 +1 VVVVxx 1 +1 VVVVxx 3 +1 VVVVxx 5 +1 VVVVxx 7 +1 VVVVxx 9 + + +-- !query 5 +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss +-- !query 5 schema +struct +-- !query 5 output +4 + + +-- !query 6 +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable +-- !query 7 schema +struct +-- !query 7 output +1 true +2 false +3 true +NULL true + + +-- !query 8 +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable +-- !query 8 schema +struct +-- !query 8 output +1 true +2 true +3 true +NULL false + + +-- !query 9 +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable +-- !query 9 schema +struct +-- !query 9 output +1 false +2 false +3 false +NULL false + + +-- !query 10 +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable +-- !query 10 schema +struct +-- !query 10 output +1 true +2 true +3 true +NULL false + + +-- !query 11 +SELECT 1 IS DISTINCT FROM 2 as `yes` +-- !query 11 schema +struct +-- !query 11 output +true + + +-- !query 12 +SELECT 2 IS DISTINCT FROM 2 as `no` +-- !query 12 schema +struct +-- !query 12 output +false + + +-- !query 13 +SELECT 2 IS DISTINCT FROM null as `yes` +-- !query 13 schema +struct +-- !query 13 output +true + + +-- !query 14 +SELECT null IS DISTINCT FROM null as `no` +-- !query 14 schema +struct +-- !query 14 output +false + + +-- !query 15 +SELECT 1 IS NOT DISTINCT FROM 2 as `no` +-- !query 15 schema +struct +-- !query 15 output +false + + +-- !query 16 +SELECT 2 IS NOT DISTINCT FROM 2 as `yes` +-- !query 16 schema +struct +-- !query 16 output +true + + +-- !query 17 +SELECT 2 IS NOT DISTINCT FROM null as `no` +-- !query 17 schema +struct +-- !query 17 output +false + + +-- !query 18 +SELECT null IS NOT DISTINCT FROM null as `yes` +-- !query 18 schema +struct +-- !query 18 output +true diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out new file mode 100644 index 0000000000000..02536ebd8ebea --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out @@ -0,0 +1,187 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c +-- !query 11 schema +struct +-- !query 11 output +1 XXXX +3 bbbb + + +-- !query 12 +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c +-- !query 12 schema +struct +-- !query 12 output +3 BBBB +3 bbbb + + +-- !query 13 +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c +-- !query 13 schema +struct +-- !query 13 output +XXXX 0 +bbbb 5 + + +-- !query 14 +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) +-- !query 15 schema +struct +-- !query 15 output +0 9 + + +-- !query 16 +SELECT a FROM test_having HAVING min(a) < max(a) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'default.test_having.`a`' is not an aggregate function. Wrap '(min(default.test_having.`a`) AS `min(a#x)`, max(default.test_having.`a`) AS `max(a#x)`)' in windowing function(s) or wrap 'default.test_having.`a`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 17 +SELECT 1 AS one FROM test_having HAVING a > 1 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a`' given input columns: [one]; line 1 pos 40 + + +-- !query 18 +SELECT 1 AS one FROM test_having HAVING 1 > 2 +-- !query 18 schema +struct +-- !query 18 output + + + +-- !query 19 +SELECT 1 AS one FROM test_having HAVING 1 < 2 +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 +-- !query 20 schema +struct +-- !query 20 output +1 + + +-- !query 21 +DROP TABLE test_having +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out new file mode 100644 index 0000000000000..0675820b381da --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out @@ -0,0 +1,416 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 38 + + +-- !query 0 +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 11 schema +struct +-- !query 11 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 12 +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 12 schema +struct +-- !query 12 output +2 +2 +2 +1 +1 +2 + + +-- !query 13 +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61 + + +-- !query 14 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b +-- !query 14 schema +struct +-- !query 14 output +1 +2 +3 +4 + + +-- !query 15 +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b +-- !query 15 schema +struct +-- !query 15 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 16 +SELECT c FROM test_missing_target ORDER BY a +-- !query 16 schema +struct +-- !query 16 output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query 17 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc +-- !query 17 schema +struct +-- !query 17 output +4 +3 +2 +1 + + +-- !query 18 +SELECT count(*) FROM test_missing_target ORDER BY 1 desc +-- !query 18 schema +struct +-- !query 18 output +10 + + +-- !query 19 +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query 19 schema +struct +-- !query 19 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 20 +SELECT c, count(*) FROM test_missing_target GROUP BY 3 +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53 + + +-- !query 21 +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 22 +SELECT a, a FROM test_missing_target + ORDER BY a +-- !query 22 schema +struct +-- !query 22 output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query 23 +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2 +-- !query 23 schema +struct<(a div 2):int,(a div 2):int> +-- !query 23 output +0 0 +0 0 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 + + +-- !query 24 +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2 +-- !query 24 schema +struct<(a div 2):int,(a div 2):int> +-- !query 24 output +0 0 +1 1 +2 2 +3 3 +4 4 + + +-- !query 25 +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query 25 schema +struct +-- !query 25 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 26 +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query 26 schema +struct +-- !query 26 output +1 +2 +3 +4 + + +-- !query 27 +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2 +-- !query 27 schema +struct<(a % 2):int,count(b):bigint> +-- !query 27 output +0 5 +1 5 + + +-- !query 28 +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c) +-- !query 28 schema +struct +-- !query 28 output +2 +3 +4 +1 + + +-- !query 29 +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61 + + +-- !query 30 +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 +-- !query 30 schema +struct +-- !query 30 output +1 +5 +4 + + +-- !query 31 +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) +-- !query 31 schema +struct +-- !query 31 output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query 32 +SELECT a FROM test_missing_target ORDER BY upper(d) +-- !query 32 schema +struct +-- !query 32 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 33 +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc +-- !query 33 schema +struct +-- !query 33 output +7 +3 + + +-- !query 34 +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2 +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 35 +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2 +-- !query 35 schema +struct<(b div 2):int,count(b):bigint> +-- !query 35 output +0 1 +1 5 +2 4 + + +-- !query 36 +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13 + + +-- !query 37 +DROP TABLE test_missing_target +-- !query 37 schema +struct<> +-- !query 37 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out new file mode 100644 index 0000000000000..200fecce17bc0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query 5 schema +struct<64:string,d1:timestamp> +-- !query 5 output +1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 6 +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02' +-- !query 6 schema +struct<48:string,d1:timestamp> +-- !query 6 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 7 +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02' +-- !query 7 schema +struct<15:string,d1:timestamp> +-- !query 7 output + + + +-- !query 8 +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02' +-- !query 8 schema +struct +-- !query 8 output +1997-01-02 00:00:00 + + +-- !query 9 +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02' +-- !query 9 schema +struct<63:string,d1:timestamp> +-- !query 9 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 10 +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02' +-- !query 10 schema +struct<16:string,d1:timestamp> +-- !query 10 output +1997-01-02 00:00:00 + + +-- !query 11 +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02' +-- !query 11 schema +struct<49:string,d1:timestamp> +-- !query 11 output +1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 12 +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query 12 schema +struct +-- !query 12 output +2004-02-23 00:00:00 + + +-- !query 13 +DROP TABLE TIMESTAMP_TBL +-- !query 13 schema +struct<> +-- !query 13 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out new file mode 100644 index 0000000000000..366b65f3659cd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out @@ -0,0 +1,471 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 51 + + +-- !query 0 +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2 +-- !query 0 schema +struct +-- !query 0 output +1 2 1 2 + + +-- !query 1 +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query 1 schema +struct +-- !query 1 output +10 + + +-- !query 2 +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO department VALUES (1, 0, 'A') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO department VALUES (2, 1, 'B') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO department VALUES (3, 2, 'C') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO department VALUES (4, 2, 'D') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO department VALUES (5, 0, 'E') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO department VALUES (6, 4, 'F') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO department VALUES (7, 5, 'G') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +create table graph( f int, t int, label string ) USING parquet +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +DROP TABLE y +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q +-- !query 20 schema +struct +-- !query 20 output +42 + + +-- !query 21 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 21 schema +struct +-- !query 21 output +1 +2 +3 + + +-- !query 22 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Table or view not found: outermost; line 4 pos 23 + + +-- !query 23 +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT * FROM withz ORDER BY k +-- !query 24 schema +struct +-- !query 24 output +1 1 v +4 4 v +7 7 v +10 10 v +13 13 v +16 16 v + + +-- !query 25 +DROP TABLE withz +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +TRUNCATE TABLE y +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +CREATE TABLE yy (a INTEGER) USING parquet +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +SELECT * FROM y +-- !query 29 schema +struct +-- !query 29 output +1 +2 +3 + + +-- !query 30 +SELECT * FROM yy +-- !query 30 schema +struct +-- !query 30 output + + + +-- !query 31 +SELECT * FROM y +-- !query 31 schema +struct +-- !query 31 output +1 +2 +3 + + +-- !query 32 +SELECT * FROM yy +-- !query 32 schema +struct +-- !query 32 output + + + +-- !query 33 +CREATE TABLE parent ( id int, val string ) USING parquet +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO parent VALUES ( 1, 'p1' ) +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +SELECT * FROM parent +-- !query 35 schema +struct +-- !query 35 output +1 p1 + + +-- !query 36 +SELECT * FROM parent +-- !query 36 schema +struct +-- !query 36 output +1 p1 + + +-- !query 37 +create table foo (with baz) +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 1, pos 18) + +== SQL == +create table foo (with baz) +------------------^^^ + + +-- !query 38 +-- fail, WITH is a reserved word +create table foo (with ordinality) +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 2, pos 18) + +== SQL == +-- fail, WITH is a reserved word +create table foo (with ordinality) +------------------^^^ + + +-- !query 39 +-- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +WITH test AS (SELECT 42) INSERT INTO test VALUES (1) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +Table not found: test; + + +-- !query 41 +create table test (i int) USING parquet +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +with test as (select 42) insert into test select * from test +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +select * from test +-- !query 43 schema +struct +-- !query 43 output +42 + + +-- !query 44 +drop table test +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +DROP TABLE department +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +DROP TABLE tree +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +DROP TABLE graph +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +DROP TABLE y +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +DROP TABLE yy +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +DROP TABLE parent +-- !query 50 schema +struct<> +-- !query 50 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out new file mode 100644 index 0000000000000..a2f64717d73a1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out @@ -0,0 +1,387 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +SELECT avg(udf(four)) AS avg_1 FROM onek +-- !query 0 schema +struct +-- !query 0 output +1.5 + + +-- !query 1 +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 +-- !query 1 schema +struct +-- !query 1 output +32.666666666666664 + + +-- !query 2 +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query 2 schema +struct +-- !query 2 output +107.943 + + +-- !query 3 +SELECT sum(udf(four)) AS sum_1500 FROM onek +-- !query 3 schema +struct +-- !query 3 output +1500 + + +-- !query 4 +SELECT udf(sum(a)) AS sum_198 FROM aggtest +-- !query 4 schema +struct +-- !query 4 output +198 + + +-- !query 5 +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest +-- !query 5 schema +struct +-- !query 5 output +431.77260909229517 + + +-- !query 6 +SELECT udf(max(four)) AS max_3 FROM onek +-- !query 6 schema +struct +-- !query 6 output +3 + + +-- !query 7 +SELECT max(udf(a)) AS max_100 FROM aggtest +-- !query 7 schema +struct +-- !query 7 output +100 + + +-- !query 8 +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest +-- !query 8 schema +struct +-- !query 8 output +324.78 + + +-- !query 9 +SELECT stddev_pop(udf(b)) FROM aggtest +-- !query 9 schema +struct +-- !query 9 output +131.10703231895047 + + +-- !query 10 +SELECT udf(stddev_samp(b)) FROM aggtest +-- !query 10 schema +struct +-- !query 10 output +151.38936080399804 + + +-- !query 11 +SELECT var_pop(udf(b)) FROM aggtest +-- !query 11 schema +struct +-- !query 11 output +17189.053923482323 + + +-- !query 12 +SELECT udf(var_samp(b)) FROM aggtest +-- !query 12 schema +struct +-- !query 12 output +22918.738564643096 + + +-- !query 13 +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 13 schema +struct +-- !query 13 output +131.18117242958306 + + +-- !query 14 +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest +-- !query 14 schema +struct +-- !query 14 output +151.47497042966097 + + +-- !query 15 +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 15 schema +struct +-- !query 15 output +17208.5 + + +-- !query 16 +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 16 schema +struct +-- !query 16 output +22944.666666666668 + + +-- !query 17 +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) +-- !query 17 schema +struct +-- !query 17 output +0.0 NaN + + +-- !query 18 +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) +-- !query 18 schema +struct +-- !query 18 output +0.0 NaN + + +-- !query 19 +select sum(udf(CAST(null AS int))) from range(1,4) +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +select sum(udf(CAST(null AS long))) from range(1,4) +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +select sum(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +select avg(udf(CAST(null AS int))) from range(1,4) +-- !query 23 schema +struct +-- !query 23 output +NULL + + +-- !query 24 +select avg(udf(CAST(null AS long))) from range(1,4) +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query 25 schema +struct +-- !query 25 output +NULL + + +-- !query 26 +select avg(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query 26 schema +struct +-- !query 26 output +NULL + + +-- !query 27 +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query 27 schema +struct +-- !query 27 output +NaN + + +-- !query 28 +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query 28 schema +struct +-- !query 28 output +NaN + + +-- !query 29 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x) +-- !query 29 schema +struct +-- !query 29 output +Infinity NaN + + +-- !query 30 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x) +-- !query 30 schema +struct +-- !query 30 output +Infinity NaN + + +-- !query 31 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +-- !query 31 schema +struct +-- !query 31 output +NaN NaN + + +-- !query 32 +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query 32 schema +struct +-- !query 32 output +1.00000005E8 2.5 + + +-- !query 33 +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query 33 schema +struct +-- !query 33 output +7.000000000006E12 1.0 + + +-- !query 34 +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest +-- !query 34 schema +struct +-- !query 34 output +653.6289553875104 871.5052738500139 + + +-- !query 35 +SELECT corr(b, udf(a)) FROM aggtest +-- !query 35 schema +struct +-- !query 35 output +0.1396345165178734 + + +-- !query 36 +SELECT count(udf(four)) AS cnt_1000 FROM onek +-- !query 36 schema +struct +-- !query 36 output +1000 + + +-- !query 37 +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek +-- !query 37 schema +struct +-- !query 37 output +4 + + +-- !query 38 +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten +-- !query 38 schema +struct +-- !query 38 output +0 100 100 +1 100 200 +2 100 100 +3 100 200 +4 100 100 +5 100 200 +6 100 100 +7 100 200 +8 100 100 +9 100 200 + + +-- !query 39 +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten +-- !query 39 schema +struct +-- !query 39 output +0 100 2 +1 100 4 +2 100 2 +3 100 4 +4 100 2 +5 100 4 +6 100 2 +7 100 4 +8 100 2 +9 100 4 + + +-- !query 40 +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four) +-- !query 40 schema +struct +-- !query 40 output +0 2 +2 2 +4 2 +6 2 +8 2 + + +-- !query 41 +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)) +-- !query 41 schema +struct<> +-- !query 41 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(CAST(udf(cast(four as string)) AS INT) AS BIGINT))] +Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; + + +-- !query 42 +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o +-- !query 42 schema +struct<> +-- !query 42 output +org.apache.spark.sql.AnalysisException +cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out new file mode 100644 index 0000000000000..9fe943874c3e5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out @@ -0,0 +1,162 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query 1 schema +struct +-- !query 1 output +true true false true false true true true true + + +-- !query 2 +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query 2 schema +struct +-- !query 2 output +true false true false true true true true true + + +-- !query 3 +select min(udf(unique1)) from tenk1 +-- !query 3 schema +struct +-- !query 3 output +0 + + +-- !query 4 +select udf(max(unique1)) from tenk1 +-- !query 4 schema +struct +-- !query 4 output +9999 + + +-- !query 5 +select max(unique1) from tenk1 where udf(unique1) < 42 +-- !query 5 schema +struct +-- !query 5 output +41 + + +-- !query 6 +select max(unique1) from tenk1 where unique1 > udf(42) +-- !query 6 schema +struct +-- !query 6 output +9999 + + +-- !query 7 +select max(unique1) from tenk1 where udf(unique1) > 42000 +-- !query 7 schema +struct +-- !query 7 output +NULL + + +-- !query 8 +select max(tenthous) from tenk1 where udf(thousand) = 33 +-- !query 8 schema +struct +-- !query 8 output +9033 + + +-- !query 9 +select min(tenthous) from tenk1 where udf(thousand) = 33 +-- !query 9 schema +struct +-- !query 9 output +33 + + +-- !query 10 +select distinct max(udf(unique2)) from tenk1 +-- !query 10 schema +struct +-- !query 10 output +9999 + + +-- !query 11 +select max(unique2) from tenk1 order by udf(1) +-- !query 11 schema +struct +-- !query 11 output +9999 + + +-- !query 12 +select max(unique2) from tenk1 order by max(udf(unique2)) +-- !query 12 schema +struct +-- !query 12 output +9999 + + +-- !query 13 +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +-- !query 13 schema +struct +-- !query 13 output +9999 + + +-- !query 14 +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query 14 schema +struct +-- !query 14 output +9999 3 +9999 2 +9999 1 + + +-- !query 15 +select udf(max(100)) from tenk1 +-- !query 15 schema +struct +-- !query 15 output +100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out index 55bef64338f4e..d9a8ca86361fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out @@ -1,19 +1,22 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 35 -- !query 0 -set spark.sql.crossJoin.enabled=true +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet -- !query 0 schema -struct +struct<> -- !query 0 output -spark.sql.crossJoin.enabled true + -- !query 1 -CREATE TABLE CASE_TBL ( +CREATE TABLE CASE2_TBL ( i integer, - f double + j integer ) USING parquet -- !query 1 schema struct<> @@ -22,10 +25,7 @@ struct<> -- !query 2 -CREATE TABLE CASE2_TBL ( - i integer, - j integer -) USING parquet +INSERT INTO CASE_TBL VALUES (1, 10.1) -- !query 2 schema struct<> -- !query 2 output @@ -33,7 +33,7 @@ struct<> -- !query 3 -INSERT INTO CASE_TBL VALUES (1, 10.1) +INSERT INTO CASE_TBL VALUES (2, 20.2) -- !query 3 schema struct<> -- !query 3 output @@ -41,7 +41,7 @@ struct<> -- !query 4 -INSERT INTO CASE_TBL VALUES (2, 20.2) +INSERT INTO CASE_TBL VALUES (3, -30.3) -- !query 4 schema struct<> -- !query 4 output @@ -49,7 +49,7 @@ struct<> -- !query 5 -INSERT INTO CASE_TBL VALUES (3, -30.3) +INSERT INTO CASE_TBL VALUES (4, NULL) -- !query 5 schema struct<> -- !query 5 output @@ -57,7 +57,7 @@ struct<> -- !query 6 -INSERT INTO CASE_TBL VALUES (4, NULL) +INSERT INTO CASE2_TBL VALUES (1, -1) -- !query 6 schema struct<> -- !query 6 output @@ -65,7 +65,7 @@ struct<> -- !query 7 -INSERT INTO CASE2_TBL VALUES (1, -1) +INSERT INTO CASE2_TBL VALUES (2, -2) -- !query 7 schema struct<> -- !query 7 output @@ -73,7 +73,7 @@ struct<> -- !query 8 -INSERT INTO CASE2_TBL VALUES (2, -2) +INSERT INTO CASE2_TBL VALUES (3, -3) -- !query 8 schema struct<> -- !query 8 output @@ -81,7 +81,7 @@ struct<> -- !query 9 -INSERT INTO CASE2_TBL VALUES (3, -3) +INSERT INTO CASE2_TBL VALUES (2, -4) -- !query 9 schema struct<> -- !query 9 output @@ -89,7 +89,7 @@ struct<> -- !query 10 -INSERT INTO CASE2_TBL VALUES (2, -4) +INSERT INTO CASE2_TBL VALUES (1, NULL) -- !query 10 schema struct<> -- !query 10 output @@ -97,7 +97,7 @@ struct<> -- !query 11 -INSERT INTO CASE2_TBL VALUES (1, NULL) +INSERT INTO CASE2_TBL VALUES (NULL, -6) -- !query 11 schema struct<> -- !query 11 output @@ -105,148 +105,140 @@ struct<> -- !query 12 -INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 12 schema -struct<> --- !query 12 output - - - --- !query 13 SELECT '3' AS `One`, CASE - WHEN CAST(udf(1 < 2) AS boolean) THEN 3 + WHEN udf(1 < 2) THEN 3 END AS `Simple WHEN` --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output 3 3 --- !query 14 +-- !query 13 SELECT '' AS `One`, CASE WHEN 1 > 2 THEN udf(3) END AS `Simple default` --- !query 14 schema -struct --- !query 14 output +-- !query 13 schema +struct +-- !query 13 output NULL --- !query 15 +-- !query 14 SELECT '3' AS `One`, CASE WHEN udf(1) < 2 THEN udf(3) ELSE udf(4) END AS `Simple ELSE` --- !query 15 schema -struct --- !query 15 output +-- !query 14 schema +struct +-- !query 14 output 3 3 --- !query 16 +-- !query 15 SELECT udf('4') AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 16 schema +-- !query 15 schema struct --- !query 16 output +-- !query 15 output 4 4 --- !query 17 +-- !query 16 SELECT udf('6') AS `One`, CASE - WHEN CAST(udf(1 > 2) AS boolean) THEN 3 + WHEN udf(1 > 2) THEN 3 WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output 6 6 --- !query 18 +-- !query 17 SELECT '7' AS `None`, CASE WHEN rand() < udf(0) THEN 1 END AS `NULL on no matches` --- !query 18 schema +-- !query 17 schema struct --- !query 18 output +-- !query 17 output 7 NULL +-- !query 18 +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + -- !query 19 -SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1.0 +1 -- !query 20 -SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END +SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 20 schema -struct +struct 100) THEN CAST(udf(cast((1 div 0) as string)) AS INT) ELSE CAST(udf(cast(0 as string)) AS INT) END:int> -- !query 20 output -1.0 - - --- !query 21 -SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl --- !query 21 schema -struct 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string> --- !query 21 output 0 0 0 0 --- !query 22 +-- !query 21 SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END --- !query 22 schema -struct --- !query 22 output +-- !query 21 schema +struct +-- !query 21 output 1 --- !query 23 +-- !query 22 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 23 schema +-- !query 22 schema struct= 3 or Null:int> --- !query 23 output +-- !query 22 output 3 4 NULL NULL --- !query 24 +-- !query 23 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output 1 2 6 8 --- !query 25 +-- !query 24 SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -255,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query 24 schema struct --- !query 25 output +-- !query 24 output 1 one 2 two 3 big 4 big --- !query 26 +-- !query 25 SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -273,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 26 schema +-- !query 25 schema struct --- !query 26 output +-- !query 25 output big big one two --- !query 27 +-- !query 26 SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 --- !query 27 schema +-- !query 26 schema struct --- !query 27 output +-- !query 26 output 4 NULL --- !query 28 +-- !query 27 SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 --- !query 28 schema +-- !query 27 schema struct --- !query 28 output +-- !query 27 output --- !query 29 +-- !query 28 SELECT udf(COALESCE(a.f, b.i, b.j)) FROM CASE_TBL a, CASE2_TBL b --- !query 29 schema -struct --- !query 29 output +-- !query 28 schema +struct +-- !query 28 output -30.3 -30.3 -30.3 @@ -330,24 +322,24 @@ struct 3.0 --- !query 30 +-- !query 29 SELECT * FROM CASE_TBL a, CASE2_TBL b WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 --- !query 30 schema +-- !query 29 schema struct --- !query 30 output +-- !query 29 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 31 +-- !query 30 SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 31 schema +-- !query 30 schema struct --- !query 31 output +-- !query 30 output 1 2 1 2 1 3 @@ -374,18 +366,18 @@ struct NULL 3 --- !query 32 +-- !query 31 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b - WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean) --- !query 32 schema + WHERE udf(COALESCE(f,b.i) = 2) +-- !query 31 schema struct --- !query 32 output +-- !query 31 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 33 +-- !query 32 SELECT CASE (CASE vol('bar') WHEN udf('foo') THEN 'it was foo!' @@ -395,31 +387,23 @@ SELECT CASE WHEN udf('it was foo!') THEN 'foo recognized' WHEN 'it was bar!' THEN udf('bar recognized') ELSE 'unrecognized' END AS col --- !query 33 schema +-- !query 32 schema struct --- !query 33 output +-- !query 32 output bar recognized --- !query 34 +-- !query 33 DROP TABLE CASE_TBL --- !query 34 schema +-- !query 33 schema struct<> --- !query 34 output +-- !query 33 output --- !query 35 +-- !query 34 DROP TABLE CASE2_TBL --- !query 35 schema +-- !query 34 schema struct<> --- !query 35 output - - +-- !query 34 output --- !query 36 -set spark.sql.crossJoin.enabled=false --- !query 36 schema -struct --- !query 36 output -spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out new file mode 100644 index 0000000000000..3d7c64054a6ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData +-- !query 1 schema +struct +-- !query 1 output +7 7 0 5 5 4 7 + + +-- !query 2 +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData +-- !query 2 schema +struct +-- !query 2 output +1 0 2 2 2 6 + + +-- !query 3 +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData +-- !query 3 schema +struct +-- !query 3 output +4 4 4 + + +-- !query 4 +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) +FROM testData +-- !query 4 schema +struct +-- !query 4 output +3 3 3 3 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out new file mode 100644 index 0000000000000..98d3ad37a8dfa --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 cross join nt2 +-- !query 2 schema +struct +-- !query 2 output +one 1 one 1 +one 1 one 5 +one 1 two 22 +three 3 one 1 +three 3 one 5 +three 3 two 22 +two 2 one 1 +two 2 one 5 +two 2 two 22 + + +-- !query 3 +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) +-- !query 3 schema +struct +-- !query 3 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 4 +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) +-- !query 4 schema +struct +-- !query 4 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 5 +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" +-- !query 5 schema +struct +-- !query 5 output +one 1 two 22 + + +-- !query 6 +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b +-- !query 6 schema +struct +-- !query 6 output +one two + + +-- !query 7 +create temporary view A(a, va) as select * from nt1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +create temporary view B(b, vb) as select * from nt1 +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +create temporary view C(c, vc) as select * from nt1 +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +create temporary view D(d, vd) as select * from nt1 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) +-- !query 11 schema +struct +-- !query 11 output +one 1 one 1 one 1 one 1 +one 1 one 1 three 3 one 1 +one 1 one 1 two 2 one 1 +three 3 three 3 one 1 three 3 +three 3 three 3 three 3 three 3 +three 3 three 3 two 2 three 3 +two 2 two 2 one 1 two 2 +two 2 two 2 three 3 two 2 +two 2 two 2 two 2 two 2 + + +-- !query 12 +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) +-- !query 12 schema +struct +-- !query 12 output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out new file mode 100644 index 0000000000000..b7bfad0e538ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out @@ -0,0 +1,346 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 27 + + +-- !query 0 +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output +0 +2 +2 +NULL + + +-- !query 5 +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2 +-- !query 5 schema +struct +-- !query 5 output +0 +2 +2 +NULL + + +-- !query 6 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL +-- !query 6 schema +struct +-- !query 6 output +0 +2 +2 +NULL +NULL + + +-- !query 7 +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 7 schema +struct +-- !query 7 output + + + +-- !query 8 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) +-- !query 8 schema +struct +-- !query 8 output +0 +1 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 9 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT) +-- !query 9 schema +struct +-- !query 9 output +0 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 10 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 11 +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4 +-- !query 11 schema +struct +-- !query 11 output +1 2 +1 3 + + +-- !query 12 +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3 +-- !query 12 schema +struct +-- !query 12 output +2 2 +2 20 + + +-- !query 13 +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 13 schema +struct +-- !query 13 output +2 2 +2 20 + + +-- !query 14 +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query 15 schema +struct +-- !query 15 output +1 3 + + +-- !query 16 +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 17 +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 17 schema +struct +-- !query 17 output +1 3 + + +-- !query 18 +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query 18 schema +struct +-- !query 18 output +1 3 + + +-- !query 19 +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4 +-- !query 19 schema +struct +-- !query 19 output + + + +-- !query 20 +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)) +-- !query 20 schema +struct +-- !query 20 output + + + +-- !query 21 +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)) +-- !query 21 schema +struct +-- !query 21 output +1 2 +1 2 +1 2 +2 20 +2 20 +2 3 +2 3 + + +-- !query 22 +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k +-- !query 22 schema +struct +-- !query 22 output +3 + + +-- !query 23 +DROP VIEW IF EXISTS tab1 +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP VIEW IF EXISTS tab2 +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +DROP VIEW IF EXISTS tab3 +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +DROP VIEW IF EXISTS tab4 +-- !query 26 schema +struct<> +-- !query 26 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out new file mode 100644 index 0000000000000..1effcc8470e19 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out @@ -0,0 +1,49 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 +-- !query 1 schema +struct +-- !query 1 output +one 6 +three 3 + + +-- !query 2 +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) +-- !query 2 schema +struct +-- !query 2 output +1 + + +-- !query 3 +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) +-- !query 4 schema +struct +-- !query 4 output +3 +7 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out index 10952cb21e4f3..120f2d39f73dc 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out @@ -59,7 +59,7 @@ struct<> -- !query 6 SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag -- !query 6 schema -struct +struct -- !query 6 output 1 a 1 a diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out new file mode 100644 index 0000000000000..0cb82be2da228 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out @@ -0,0 +1,307 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2 +-- !query 2 schema +struct +-- !query 2 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 3 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 +-- !query 3 schema +struct +-- !query 3 output +1 2 +1 2 +1 3 +1 3 + + +-- !query 4 +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 +-- !query 5 schema +struct +-- !query 5 output + + + +-- !query 6 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) +-- !query 6 schema +struct +-- !query 6 output +1 2 + + +-- !query 7 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2) +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 8 +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 9 +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 9 schema +struct +-- !query 9 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 10 +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +-- !query 10 schema +struct +-- !query 10 output +1 2 +1 2 +1 3 +2 3 +NULL NULL +NULL NULL + + +-- !query 11 +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query 11 schema +struct +-- !query 11 output +1 3 + + +-- !query 12 +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +-- !query 12 schema +struct +-- !query 12 output + + + +-- !query 13 +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))) +-- !query 13 schema +struct +-- !query 13 output +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +2 3 + + +-- !query 14 +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)) +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k +-- !query 15 schema +struct +-- !query 15 output +2 +3 +NULL + + +-- !query 16 +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query 16 schema +struct +-- !query 16 output +spark.sql.legacy.setopsPrecedence.enabled true + + +-- !query 17 +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2 +-- !query 17 schema +struct +-- !query 17 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 18 +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query 18 schema +struct +-- !query 18 output +1 2 +2 3 +NULL NULL + + +-- !query 19 +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.legacy.setopsPrecedence.enabled false + + +-- !query 20 +DROP VIEW IF EXISTS tab1 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW IF EXISTS tab2 +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out new file mode 100644 index 0000000000000..950809ddcaf25 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 natural join nt2 where udf(k) = "one" +-- !query 2 schema +struct +-- !query 2 output +one 1 1 +one 1 5 + + +-- !query 3 +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2 +-- !query 3 schema +struct +-- !query 3 output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query 4 +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2 +-- !query 4 schema +struct +-- !query 4 output +one 1 1 +one 1 5 +two 2 22 + + +-- !query 5 +SELECT udf(count(*)) FROM nt1 natural full outer join nt2 +-- !query 5 schema +struct +-- !query 5 output +4 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out new file mode 100644 index 0000000000000..cb9e4d736c9a0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -0,0 +1,460 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 30 + + +-- !query 0 +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 3 schema +struct +-- !query 3 output +2012 15000 20000 +2013 48000 30000 + + +-- !query 4 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 4 schema +struct +-- !query 4 output +Java 20000 30000 +dotNET 15000 48000 + + +-- !query 5 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 5 schema +struct +-- !query 5 output +2012 15000 7500.0 20000 20000.0 +2013 48000 48000.0 30000 30000.0 + + +-- !query 6 +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 6 schema +struct +-- !query 6 output +63000 50000 + + +-- !query 7 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +) +-- !query 7 schema +struct +-- !query 7 output +63000 2012 50000 2012 + + +-- !query 8 +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +) +-- !query 8 schema +struct +-- !query 8 output +Java 2012 20000 NULL +Java 2013 NULL 30000 +dotNET 2012 15000 NULL +dotNET 2013 NULL 48000 + + +-- !query 9 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +) +-- !query 9 schema +struct +-- !query 9 output +2012 15000 1 20000 1 +2013 48000 2 30000 2 + + +-- !query 10 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +) +-- !query 10 schema +struct +-- !query 10 output +2012 15000 20000 +2013 96000 60000 + + +-- !query 11 +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +) +-- !query 11 schema +struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> +-- !query 11 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 12 +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query 12 schema +struct +-- !query 12 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 13 +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but 'coursesales.`earnings`' did not appear in any aggregate function.; + + +-- !query 14 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but '__auto_generated_subquery_name.`year`' did not appear in any aggregate function.; + + +-- !query 15 +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 + + +-- !query 16 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query 16 schema +struct +-- !query 16 output +2012 15000 7501.0 20000 20001.0 +2013 48000 48001.0 30000 30001.0 + + +-- !query 17 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +) +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; + + +-- !query 18 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query 18 schema +struct +-- !query 18 output +1 15000 NULL +2 NULL 30000 + + +-- !query 19 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query 19 schema +struct +-- !query 19 output +2012 NULL 20000 +2013 48000 NULL + + +-- !query 20 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct; + + +-- !query 21 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +) +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 + + +-- !query 22 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +) +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Literal expressions required for pivot values, found 'course#x'; + + +-- !query 23 +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query 23 schema +struct +-- !query 23 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 24 +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query 24 schema +struct +-- !query 24 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 25 +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query 25 schema +struct +-- !query 25 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 26 +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query 26 schema +struct +-- !query 26 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 27 +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'm#x'. Pivot columns must be comparable.; + + +-- !query 28 +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; + + +-- !query 29 +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +) +-- !query 29 schema +struct +-- !query 29 output +a z b y c x d w 63000 50000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out new file mode 100644 index 0000000000000..7b2b5dbe578cc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) +-- !query 0 schema +struct +-- !query 0 output +1 +2 +NULL + + +-- !query 1 +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) +-- !query 1 schema +struct +-- !query 1 output +A +B +NULL + + +-- !query 2 +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) +-- !query 2 schema +struct +-- !query 2 output +1 +2 +NaN + + +-- !query 3 +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +Infinity + + +-- !query 4 +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) +-- !query 4 schema +struct +-- !query 4 output +-Infinity +1 +2 + + +-- !query 5 +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) +-- !query 5 schema +struct +-- !query 5 output +0.00000001 +0.00000002 +0.00000003 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala index 938d76c9f0837..b253c4a70bbf9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -20,33 +20,34 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "false") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "false", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "false", "configuration parameter changed in test body") } } class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") } } @@ -56,18 +57,18 @@ class TwoLevelAggregateHashMapWithVectorizedMapSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - .set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") + .set(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.vectorized.enable") == "true", + assert(sparkConf.get(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key) == "true", "configuration parameter changed in test body") } } 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 6049e89c93cf6..267f255a11e86 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 @@ -832,7 +832,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext val df = spark.range(10).cache() df.queryExecution.executedPlan.foreach { case i: InMemoryTableScanExec => - assert(i.supportsBatch == vectorized && i.supportCodegen == vectorized) + assert(i.supportsColumnar == vectorized) case _ => } } 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 98936702a013d..e8ddd4e1fd974 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 @@ -1672,7 +1672,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("reuse exchange") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "2") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { val df = spark.range(100).toDF() val join = df.join(df, "id") val plan = join.queryExecution.executedPlan 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 efd5db1c5b6c4..ff6143162ff2f 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 @@ -603,6 +603,70 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", 30L, 32L, 2L, 15.0), ("b", 3L, 5L, 2L, 1.5), ("c", 1L, 2L, 1L, 1.0)) } + test("typed aggregation: expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L), ("b", 3L, 5L, 2L, 1.5, 2L), ("c", 1L, 2L, 1L, 1.0, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long], + mean("_2").as[Double]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L, 15.0), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L, 1.5), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L, 1.0)) + } + test("cogroup") { val ds1 = Seq(1 -> "a", 3 -> "abc", 5 -> "hello", 3 -> "foo").toDS() val ds2 = Seq(2 -> "q", 3 -> "w", 5 -> "e", 5 -> "r").toDS() @@ -1365,7 +1429,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val agg = cp.groupBy('id % 2).agg(count('id)) agg.queryExecution.executedPlan.collectFirst { - case ShuffleExchangeExec(_, _: RDDScanExec) => + case ShuffleExchangeExec(_, _: RDDScanExec, _) => case BroadcastExchangeExec(_, _: RDDScanExec) => }.foreach { _ => fail( 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 3f91b91850e8e..ff48ac8d7a6c4 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 @@ -301,11 +301,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( df.selectExpr(s"d - $i"), - Seq(Row(Date.valueOf("2015-07-30")), Row(Date.valueOf("2015-12-30")))) + Seq(Row(Date.valueOf("2015-07-29")), Row(Date.valueOf("2015-12-28")))) checkAnswer( df.selectExpr(s"t - $i"), Seq(Row(Timestamp.valueOf("2015-07-31 23:59:59")), - Row(Timestamp.valueOf("2015-12-31 00:00:00")))) + Row(Timestamp.valueOf("2015-12-29 00:00:00")))) } test("function add_months") { @@ -314,10 +314,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, d1), (2, d2)).toDF("n", "d") checkAnswer( df.select(add_months(col("d"), 1)), - Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-31")))) + Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-28")))) checkAnswer( df.selectExpr("add_months(d, -1)"), - Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-31")))) + Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-28")))) } test("function months_between") { 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 fffe52d52dec0..89195284a5b5f 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 @@ -671,8 +671,8 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo test("SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { Seq(1.0, 0.5).foreach { compressionFactor => - withSQLConf("spark.sql.sources.fileCompressionFactor" -> compressionFactor.toString, - "spark.sql.autoBroadcastJoinThreshold" -> "250") { + withSQLConf(SQLConf.FILE_COMRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "250") { withTempPath { workDir => // the file size is 486 bytes val workDirPath = workDir.getAbsolutePath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 7caf6241bb984..d62fe961117a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -26,6 +26,7 @@ import org.apache.spark.TestUtils import org.apache.spark.api.python.{PythonBroadcast, PythonEvalType, PythonFunction, PythonUtils} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.config.Tests +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.execution.python.UserDefinedPythonFunction import org.apache.spark.sql.expressions.SparkUserDefinedFunction @@ -35,8 +36,12 @@ import org.apache.spark.sql.types.StringType * This object targets to integrate various UDF test cases so that Scalar UDF, Python UDF and * Scalar Pandas UDFs can be tested in SBT & Maven tests. * - * The available UDFs cast input to strings, which take one column as input and return a string - * type column as output. + * The available UDFs are special. It defines an UDF wrapped by cast. So, the input column is + * casted into string, UDF returns strings as are, and then output column is casted back to + * the input column. In this way, UDF is virtually no-op. + * + * Note that, due to this implementation limitation, complex types such as map, array and struct + * types do not work with this UDFs because they cannot be same after the cast roundtrip. * * To register Scala UDF in SQL: * {{{ @@ -59,8 +64,9 @@ import org.apache.spark.sql.types.StringType * To use it in Scala API and SQL: * {{{ * sql("SELECT udf_name(1)") - * spark.range(10).select(expr("udf_name(id)") - * spark.range(10).select(pandasTestUDF($"id")) + * val df = spark.range(10) + * df.select(expr("udf_name(id)") + * df.select(pandasTestUDF(df("id"))) * }}} */ object IntegratedUDFTestUtils extends SQLHelper { @@ -137,7 +143,8 @@ object IntegratedUDFTestUtils extends SQLHelper { "from pyspark.sql.types import StringType; " + "from pyspark.serializers import CloudPickleSerializer; " + s"f = open('$path', 'wb');" + - s"f.write(CloudPickleSerializer().dumps((lambda x: str(x), StringType())))"), + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: None if x is None else str(x), StringType())))"), None, "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! binaryPythonFunc = Files.readAllBytes(path.toPath) @@ -158,7 +165,9 @@ object IntegratedUDFTestUtils extends SQLHelper { "from pyspark.sql.types import StringType; " + "from pyspark.serializers import CloudPickleSerializer; " + s"f = open('$path', 'wb');" + - s"f.write(CloudPickleSerializer().dumps((lambda x: x.apply(str), StringType())))"), + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: x.apply(" + + "lambda v: None if v is None else str(v)), StringType())))"), None, "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! binaryPandasFunc = Files.readAllBytes(path.toPath) @@ -198,11 +207,22 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Python UDF that takes one column and returns a string column. - * Equivalent to `udf(lambda x: str(x), "string")` + * A Python UDF that takes one column, casts into string, executes the Python native function, + * and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import udf + * + * df = spark.range(3).toDF("col") + * python_udf = udf(lambda x: str(x), "string") + * casted_col = python_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} */ case class TestPythonUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = UserDefinedPythonFunction( + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( name = name, func = PythonFunction( command = pythonFunc, @@ -214,7 +234,16 @@ object IntegratedUDFTestUtils extends SQLHelper { accumulator = null), dataType = StringType, pythonEvalType = PythonEvalType.SQL_BATCHED_UDF, - udfDeterministic = true) + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) @@ -222,11 +251,22 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Scalar Pandas UDF that takes one column and returns a string column. - * Equivalent to `pandas_udf(lambda x: x.apply(str), "string", PandasUDFType.SCALAR)`. + * A Scalar Pandas UDF that takes one column, casts into string, executes the + * Python native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import pandas_udf + * + * df = spark.range(3).toDF("col") + * scalar_udf = pandas_udf(lambda x: x.apply(lambda v: str(v)), "string") + * casted_col = scalar_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} */ case class TestScalarPandasUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = UserDefinedPythonFunction( + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( name = name, func = PythonFunction( command = pandasFunc, @@ -238,7 +278,16 @@ object IntegratedUDFTestUtils extends SQLHelper { accumulator = null), dataType = StringType, pythonEvalType = PythonEvalType.SQL_SCALAR_PANDAS_UDF, - udfDeterministic = true) + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) @@ -246,15 +295,39 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Scala UDF that takes one column and returns a string column. - * Equivalent to `udf((input: Any) => input.toString)`. + * A Scala UDF that takes one column, casts into string, executes the + * Scala native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * import org.apache.spark.sql.functions.udf + * + * val df = spark.range(3).toDF("col") + * val scala_udf = udf((input: Any) => input.toString) + * val casted_col = scala_udf(df.col("col").cast("string")) + * casted_col.cast(df.schema("col").dataType) + * }}} */ case class TestScalaUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = SparkUserDefinedFunction( - (input: Any) => String.valueOf(input), + private[IntegratedUDFTestUtils] lazy val udf = new SparkUserDefinedFunction( + (input: Any) => if (input == null) { + null + } else { + input.toString + }, StringType, inputSchemas = Seq.fill(1)(None), - name = Some(name)) + name = Some(name)) { + + override def apply(exprs: Column*): Column = { + assert(exprs.length == 1, "Defined UDF only has one column") + val expr = exprs.head.expr + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Column(Cast(createScalaUDF(Cast(expr, StringType) :: Nil), expr.dataType)) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) 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 38c634e1107bb..059dbf892c653 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 @@ -26,8 +26,10 @@ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} -import org.apache.spark.sql.execution.{BinaryExecNode, SortExec} +import org.apache.spark.sql.catalyst.plans.logical.Filter +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.types.StructType @@ -71,7 +73,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("join operator selection") { spark.sharedState.cacheManager.clearCache() - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", @@ -650,7 +652,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (without spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> Int.MaxValue.toString) { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> Int.MaxValue.toString) { assertNotSpilled(sparkContext, "inner join") { checkAnswer( @@ -707,8 +709,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (with spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.in.memory.threshold" -> "0", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> "1") { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { assertSpilled(sparkContext, "inner join") { checkAnswer( @@ -896,6 +898,26 @@ class JoinSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-27485: EnsureRequirements should not fail join with duplicate keys") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val tbl_a = spark.range(40) + .select($"id" as "x", $"id" % 10 as "y") + .repartition(2, $"x", $"y", $"x") + .as("tbl_a") + + val tbl_b = spark.range(20) + .select($"id" as "x", $"id" % 2 as "y1", $"id" % 20 as "y2") + .as("tbl_b") + + val res = tbl_a + .join(tbl_b, + $"tbl_a.x" === $"tbl_b.x" && $"tbl_a.y" === $"tbl_b.y1" && $"tbl_a.y" === $"tbl_b.y2") + .select($"tbl_a.x") + checkAnswer(res, Row(0L) :: Row(1L) :: Nil) + } + } + test("SPARK-26352: join reordering should not change the order of columns") { withTable("tab1", "tab2", "tab3") { spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1") @@ -969,4 +991,50 @@ class JoinSuite extends QueryTest with SharedSQLContext { Seq(Row(0.0d, 0.0/0.0))))) } } + + test("SPARK-28323: PythonUDF should be able to use in join condition") { + import IntegratedUDFTestUtils._ + + assume(shouldTestPythonUDFs) + + val pythonTestUDF = TestPythonUDF(name = "udf") + + val left = Seq((1, 2), (2, 3)).toDF("a", "b") + val right = Seq((1, 2), (3, 4)).toDF("c", "d") + val df = left.join(right, pythonTestUDF(left("a")) === pythonTestUDF(right.col("c"))) + + val joinNode = df.queryExecution.executedPlan.find(_.isInstanceOf[BroadcastHashJoinExec]) + assert(joinNode.isDefined) + + // There are two PythonUDFs which use attribute from left and right of join, individually. + // So two PythonUDFs should be evaluated before the join operator, at left and right side. + val pythonEvals = joinNode.get.collect { + case p: BatchEvalPythonExec => p + } + assert(pythonEvals.size == 2) + + checkAnswer(df, Row(1, 2, 1, 2) :: Nil) + } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + import IntegratedUDFTestUtils._ + + assume(shouldTestPythonUDFs) + + val pythonTestUDF = TestPythonUDF(name = "udf") + + val left = Seq((1, 2), (2, 3)).toDF("a", "b") + val right = Seq((1, 2), (3, 4)).toDF("c", "d") + val df = left.crossJoin(right).where(pythonTestUDF(left("a")) === right.col("c")) + + // Before optimization, there is a logical Filter operator. + val filterInAnalysis = df.queryExecution.analyzed.find(_.isInstanceOf[Filter]) + assert(filterInAnalysis.isDefined) + + // Filter predicate was pushdown as join condition. So there is no Filter exec operator. + val filterExec = df.queryExecution.executedPlan.find(_.isInstanceOf[FilterExec]) + assert(filterExec.isEmpty) + + checkAnswer(df, Row(1, 2, 1, 2) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala index 623a1b6f854cf..e33870d4e1afc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala @@ -22,12 +22,15 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger +import org.apache.spark.sql.streaming.Trigger class ProcessingTimeSuite extends SparkFunSuite { test("create") { - def getIntervalMs(trigger: Trigger): Long = trigger.asInstanceOf[ProcessingTime].intervalMs + def getIntervalMs(trigger: Trigger): Long = { + trigger.asInstanceOf[ProcessingTimeTrigger].intervalMs + } assert(getIntervalMs(Trigger.ProcessingTime(10.seconds)) === 10 * 1000) assert(getIntervalMs(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) === 10 * 1000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index 3284231606966..720d570ca8384 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite import org.apache.spark.internal.config +import org.apache.spark.sql.internal.SQLConf.CHECKPOINT_LOCATION +import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD class RuntimeConfigSuite extends SparkFunSuite { @@ -60,8 +62,8 @@ class RuntimeConfigSuite extends SparkFunSuite { val conf = newConf() // SQL configs - assert(!conf.isModifiable("spark.sql.sources.schemaStringLengthThreshold")) - assert(conf.isModifiable("spark.sql.streaming.checkpointLocation")) + assert(!conf.isModifiable(SCHEMA_STRING_LENGTH_THRESHOLD.key)) + assert(conf.isModifiable(CHECKPOINT_LOCATION.key)) // Core configs assert(!conf.isModifiable(config.CPUS_PER_TASK.key)) assert(!conf.isModifiable("spark.executor.cores")) 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 2cc1be9fdda2f..972950669198c 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 @@ -1896,7 +1896,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("Star Expansion - group by") { - withSQLConf("spark.sql.retainGroupColumns" -> "false") { + withSQLConf(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS.key -> "false") { checkAnswer( testData2.groupBy($"a", $"b").agg($"*"), sql("SELECT * FROM testData2 group by a, b")) @@ -1936,7 +1936,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Common subexpression elimination") { // TODO: support subexpression elimination in whole stage codegen - withSQLConf("spark.sql.codegen.wholeStage" -> "false") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { // select from a table to prevent constant folding. val df = sql("SELECT a, b from testData2 limit 1") checkAnswer(df, Row(1, 1)) @@ -1985,9 +1985,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) // Try disabling it via configuration. - spark.conf.set("spark.sql.subexpressionElimination.enabled", "false") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 2) - spark.conf.set("spark.sql.subexpressionElimination.enabled", "true") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "true") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1) } } 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 c8a187b57a610..e4052b7ed3fff 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 @@ -151,17 +151,37 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val resultFile: String } + /** + * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, + * PgSQL tests require to register some UDF functions. + */ + private trait PgSQLTest + + private trait UDFTest { + val udf: TestUDF + } + /** A regular test case. */ private case class RegularTestCase( name: String, inputFile: String, resultFile: String) extends TestCase /** A PostgreSQL test case. */ private case class PgSQLTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase + name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest /** A UDF test case. */ private case class UDFTestCase( - name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest + + /** A UDF PostgreSQL test case. */ + private case class UDFPgSQLTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest with PgSQLTest private def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => @@ -169,12 +189,14 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } } else testCase match { - case UDFTestCase(_, _, _, udf: TestPythonUDF) if !shouldTestPythonUDFs => + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && !shouldTestPythonUDFs => ignore(s"${testCase.name} is skipped because " + s"[$pythonExec] and/or pyspark were not available.") { /* Do nothing */ } - case UDFTestCase(_, _, _, udf: TestScalarPandasUDF) if !shouldTestScalarPandasUDFs => + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && !shouldTestScalarPandasUDFs => ignore(s"${testCase.name} is skipped because pyspark," + s"pandas and/or pyarrow were not available in [$pythonExec].") { /* Do nothing */ @@ -254,18 +276,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // This does not isolate catalog changes. val localSparkSession = spark.newSession() loadTestData(localSparkSession) + testCase match { - case udfTestCase: UDFTestCase => - // vol used by udf-case.sql. - localSparkSession.udf.register("vol", (s: String) => s) + case udfTestCase: UDFTest => registerTestUDF(udfTestCase.udf, localSparkSession) - case _: PgSQLTestCase => + case _ => + } + + testCase match { + case _: PgSQLTest => // booleq/boolne used by boolean.sql localSparkSession.udf.register("booleq", (b1: Boolean, b2: Boolean) => b1 == b2) localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - case _ => // Don't add UDFs in Regular tests. + // PostgreSQL enabled cartesian product by default. + localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) + localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) + case _ => } if (configSet.isDefined) { @@ -385,13 +414,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val absPath = file.getAbsolutePath val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) - if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { + if (file.getAbsolutePath.startsWith( + s"$inputFilePath${File.separator}udf${File.separator}pgSQL")) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => + UDFPgSQLTestCase( + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => UDFTestCase( - s"$testCaseName - ${udf.prettyName}", - absPath, - resultFile, - udf) + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) } } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 2e2e61b438963..74341f93dd5ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS import org.apache.spark.sql.types.{DataType, Decimal, IntegerType, LongType, Metadata, StructType} import org.apache.spark.sql.vectorized.{ColumnarArray, ColumnarBatch, ColumnarMap, ColumnVector} import org.apache.spark.unsafe.types.UTF8String @@ -152,7 +153,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use custom class for extensions") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", classOf[MyExtensions].getCanonicalName) + .config(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) .getOrCreate() try { assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) @@ -173,7 +174,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use multiple custom class for extensions in the specified order") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions2].getCanonicalName, classOf[MyExtensions].getCanonicalName).mkString(",")) .getOrCreate() @@ -201,7 +202,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("allow an extension to be duplicated") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions].getCanonicalName, classOf[MyExtensions].getCanonicalName).mkString(",")) .getOrCreate() @@ -228,7 +229,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use the last registered function name when there are duplicates") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions2].getCanonicalName, classOf[MyExtensions2Duplicate].getCanonicalName).mkString(",")) .getOrCreate() 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 fddc4f6bb3505..b2c38684071dc 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 @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} -import org.apache.spark.sql.execution.{ExecSubqueryExpression, FileSourceScanExec, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} +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 @@ -1293,7 +1293,8 @@ class SubquerySuite extends QueryTest with SharedSQLContext { checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) // need to execute the query before we can examine fs.inputRDDs() assert(df.queryExecution.executedPlan match { - case WholeStageCodegenExec(fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _)) => + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _), _))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( 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 f155b5dc80cf1..058c5ba7e50b9 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 @@ -514,4 +514,13 @@ class UDFSuite extends QueryTest with SharedSQLContext { assert(df.collect().toSeq === Seq(Row(expected))) } } + + test("SPARK-28321 0-args Java UDF should not be called only once") { + val nonDeterministicJavaUDF = udf( + new UDF0[Int] { + override def call(): Int = scala.util.Random.nextInt() + }, IntegerType).asNondeterministic() + + assert(spark.range(2).select(nonDeterministicJavaUDF()).distinct().count() == 2) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index b35348b4ea3b5..b1143484a85e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -44,9 +44,14 @@ class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { } // A scan plan tree is a plan tree that has a leaf node under zero or more Project/Filter nodes. - private def isScanPlanTree(plan: SparkPlan): Boolean = plan match { - case p: ProjectExec => isScanPlanTree(p.child) - case f: FilterExec => isScanPlanTree(f.child) + // Because of how codegen and columnar to row transitions work, we may have InputAdaptors + // and ColumnarToRow transformations in the middle of it, but they will not have the tag + // we want, so skip them if they are the first thing we see + private def isScanPlanTree(plan: SparkPlan, first: Boolean): Boolean = plan match { + case i: InputAdapter if !first => isScanPlanTree(i.child, false) + case c: ColumnarToRowExec if !first => isScanPlanTree(c.child, false) + case p: ProjectExec => isScanPlanTree(p.child, false) + case f: FilterExec => isScanPlanTree(f.child, false) case _: LeafExecNode => true case _ => false } @@ -87,7 +92,7 @@ class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { case _: SubqueryExec | _: ReusedSubqueryExec => assert(plan.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isEmpty) - case _ if isScanPlanTree(plan) => + case _ if isScanPlanTree(plan, true) => // The strategies for planning scan can remove or add FilterExec/ProjectExec nodes, // so it's not simple to check. Instead, we only check that the origin LogicalPlan // contains the corresponding leaf node of the SparkPlan. 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 c2d9e54981928..e30fb13d10df1 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 @@ -696,6 +696,32 @@ class PlannerSuite extends SharedSQLContext { } } + test("SPARK-27485: EnsureRequirements.reorder should handle duplicate expressions") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 5)) + val plan2 = DummySparkPlan() + val smjExec = SortMergeJoinExec( + leftKeys = exprA :: exprB :: exprB :: Nil, + rightKeys = exprA :: exprC :: exprC :: Nil, + joinType = Inner, + condition = None, + left = plan1, + right = plan2) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(smjExec) + outputPlan match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _, _), _), + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _), _, _), _)) => + assert(leftKeys === smjExec.leftKeys) + assert(rightKeys === smjExec.rightKeys) + assert(leftKeys === leftPartitioningExpressions) + assert(rightKeys === rightPartitioningExpressions) + case _ => fail(outputPlan.toString) + } + } + test("SPARK-24500: create union with stream of children") { val df = Union(Stream( Range(1, 1, 1, 1), 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 fea77c78980c7..35c33a7157d38 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 @@ -275,7 +275,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA .setAppName("test") .set(UI_ENABLED, false) .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") - .set(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key, "true") + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, @@ -574,22 +574,17 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA withSparkSession(test, 4, None) } - test("Union two datasets with different pre-shuffle partition number") { + test("Do not reduce the number of shuffle partition for repartition") { val test: SparkSession => Unit = { spark: SparkSession => - val dataset1 = spark.range(3) - val dataset2 = spark.range(3) - - val resultDf = dataset1.repartition(2, dataset1.col("id")) - .union(dataset2.repartition(3, dataset2.col("id"))).toDF() + val ds = spark.range(3) + val resultDf = ds.repartition(2, ds.col("id")).toDF() checkAnswer(resultDf, - Seq((0), (0), (1), (1), (2), (2)).map(i => Row(i))) + Seq(0, 1, 2).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) + withSparkSession(test, 200, 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 8269d4d3a285d..64e305cd5c371 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution 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} class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext @@ -665,17 +666,17 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql(s"CREATE VIEW view${idx + 1} AS SELECT * FROM view$idx") } - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "10") { + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "10") { val e = intercept[AnalysisException] { sql("SELECT * FROM view10") }.getMessage assert(e.contains("The depth of view `default`.`view0` exceeds the maximum view " + "resolution depth (10). Analysis is aborted to avoid errors. Increase the value " + - "of spark.sql.view.maxNestedViewDepth to work around this.")) + s"of ${MAX_NESTED_VIEW_DEPTH.key} to work around this.")) } val e = intercept[IllegalArgumentException] { - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "0") {} + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "0") {} }.getMessage assert(e.contains("The maximum depth of a view reference in a nested view must be " + "positive.")) 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 1c6fc3530cbe1..971fd842f046a 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 @@ -19,6 +19,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 case class WindowData(month: Int, area: String, product: Int) @@ -477,8 +478,8 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDiNG AND CURRENT RoW) """.stripMargin) - withSQLConf("spark.sql.windowExec.buffer.in.memory.threshold" -> "1", - "spark.sql.windowExec.buffer.spill.threshold" -> "2") { + withSQLConf(WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "2") { assertSpilled(sparkContext, "test with low buffer spill threshold") { checkAnswer(actual, expected) } 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 9462ee190a312..483a046103383 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 @@ -17,11 +17,9 @@ package org.apache.spark.sql.execution -import org.apache.spark.metrics.source.CodegenMetrics -import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.expressions.scalalang.typed @@ -121,29 +119,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) } - test("cache for primitive type should be in WholeStageCodegen with InMemoryTableScanExec") { - import testImplicits._ - - val dsInt = spark.range(3).cache() - dsInt.count() - val dsIntFilter = dsInt.filter(_ > 0) - val planInt = dsIntFilter.queryExecution.executedPlan - assert(planInt.collect { - case WholeStageCodegenExec(FilterExec(_, i: InMemoryTableScanExec)) if i.supportsBatch => () - }.length == 1) - assert(dsIntFilter.collect() === Array(1, 2)) - - // cache for string type is not supported for InMemoryTableScanExec - val dsString = spark.range(3).map(_.toString).cache() - dsString.count() - val dsStringFilter = dsString.filter(_ == "1") - val planString = dsStringFilter.queryExecution.executedPlan - assert(planString.collect { - case i: InMemoryTableScanExec if !i.supportsBatch => () - }.length == 1) - assert(dsStringFilter.collect() === Array("1")) - } - test("SPARK-19512 codegen for comparing structs is incorrect") { // this would raise CompileException before the fix spark.range(10) @@ -168,10 +143,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { .select("int") val plan = df.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].child.children(0) - .isInstanceOf[SortMergeJoinExec]).isDefined) + .isInstanceOf[SortMergeJoinExec]).isEmpty) assert(df.collect() === Array(Row(1), Row(2))) } } @@ -204,6 +179,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._2 } + def genCode(ds: Dataset[_]): Seq[CodeAndComment] = { + val plan = ds.queryExecution.executedPlan + val wholeStageCodeGenExecs = plan.collect { case p: WholeStageCodegenExec => p } + assert(wholeStageCodeGenExecs.nonEmpty, "WholeStageCodegenExec is expected") + wholeStageCodeGenExecs.map(_.doCodeGen()._2) + } + ignore("SPARK-21871 check if we can get large code size when compiling too long functions") { val codeWithShortFunctions = genGroupByCode(3) val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) @@ -213,25 +195,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } - ignore("bytecode of batch file scan exceeds the limit of WHOLESTAGE_HUGE_METHOD_LIMIT") { - import testImplicits._ - withTempPath { dir => - val path = dir.getCanonicalPath - val df = spark.range(10).select(Seq.tabulate(201) {i => ('id + i).as(s"c$i")} : _*) - df.write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "202", - SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "2000") { - // wide table batch scan causes the byte code of codegen exceeds the limit of - // WHOLESTAGE_HUGE_METHOD_LIMIT - val df2 = spark.read.parquet(path) - val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(fileScan2.asInstanceOf[FileSourceScanExec].supportsBatch) - checkAnswer(df2, df) - } - } - } - test("Control splitting consume function by operators with config") { import testImplicits._ val df = spark.range(10).select(Seq.tabulate(2) {i => ('id + i).as(s"c$i")} : _*) @@ -283,9 +246,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { val df = spark.range(100) val join = df.join(df, "id") val plan = join.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isDefined, + p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isEmpty, "codegen stage IDs should be preserved through ReuseExchange") checkAnswer(join, df.toDF) } @@ -295,18 +258,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { import testImplicits._ withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME.key -> "true") { - val bytecodeSizeHisto = CodegenMetrics.METRIC_COMPILATION_TIME - - // the same query run twice should hit the codegen cache - spark.range(3).select('id + 2).collect - val after1 = bytecodeSizeHisto.getCount - spark.range(3).select('id + 2).collect - val after2 = bytecodeSizeHisto.getCount // same query shape as above, deliberately - // bytecodeSizeHisto's count is always monotonically increasing if new compilation to - // bytecode had occurred. If the count stayed the same that means we've got a cache hit. - assert(after1 == after2, "Should hit codegen cache. No new compilation to bytecode expected") - - // a different query can result in codegen cache miss, that's by design + // the same query run twice should produce identical code, which would imply a hit in + // the generated code cache. + val ds1 = spark.range(3).select('id + 2) + val code1 = genCode(ds1) + val ds2 = spark.range(3).select('id + 2) + val code2 = genCode(ds2) // same query shape as above, deliberately + assert(code1 == code2, "Should produce same code") } } 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 0a0973ad2111a..d8efca323d519 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 @@ -35,7 +35,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { val planBefore = dfAdaptive.queryExecution.executedPlan assert(planBefore.toString.startsWith("AdaptiveSparkPlan(isFinalPlan=false)")) val result = dfAdaptive.collect() - withSQLConf(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "false") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { val df = sql(query) QueryTest.sameRows(result.toSeq, df.collect().toSeq) } @@ -82,7 +82,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Change merge join to broadcast join") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM testData join testData2 ON key = a where value = '1'") @@ -95,7 +95,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Change merge join to broadcast join and reduce number of shuffle partitions") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "150") { @@ -119,7 +119,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Scalar subquery") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM testData join testData2 ON key = a " + @@ -133,7 +133,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Scalar subquery in later stages") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT * FROM testData join testData2 ON key = a " + @@ -147,7 +147,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("multiple joins") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ @@ -168,7 +168,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("multiple joins with aggregate") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ @@ -191,7 +191,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("multiple joins with aggregate 2") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ @@ -214,7 +214,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Exchange reuse") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT value FROM testData join testData2 ON key = a " + @@ -230,7 +230,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Exchange reuse with subqueries") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT a FROM testData join testData2 ON key = a " + @@ -246,7 +246,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Exchange reuse across subqueries") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( @@ -266,7 +266,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Subquery reuse") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT a FROM testData join testData2 ON key = a " + @@ -285,7 +285,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { test("Broadcast exchange reuse across subqueries") { withSQLConf( - SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( @@ -307,7 +307,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { } test("Union/Except/Intersect queries") { - withSQLConf(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { runAdaptiveAndVerifyResult( """ |SELECT * FROM testData @@ -322,7 +322,7 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { } test("Subquery de-correlation in Union queries") { - withSQLConf(SQLConf.RUNTIME_REOPTIMIZATION_ENABLED.key -> "true") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { withTempView("a", "b") { Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("a") Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("b") 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 86874b9817c20..67c3fa0d3bf59 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 @@ -1191,7 +1191,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { test("max records in batch conf") { val totalRecords = 10 val maxRecordsPerBatch = 3 - spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", maxRecordsPerBatch) + spark.conf.set(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key, maxRecordsPerBatch) val df = spark.sparkContext.parallelize(1 to totalRecords, 2).toDF("i") val arrowBatches = df.toArrowBatchRdd.collect() assert(arrowBatches.length >= 4) @@ -1206,7 +1206,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { } assert(recordCount == totalRecords) allocator.close() - spark.conf.unset("spark.sql.execution.arrow.maxRecordsPerBatch") + spark.conf.unset(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key) } testQuietly("unsupported types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 81158d9e54246..2776bc310fefe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -83,7 +83,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -92,7 +92,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -119,7 +119,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -128,7 +128,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -154,7 +154,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -163,7 +163,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -189,7 +189,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -198,7 +198,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -234,7 +234,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -243,7 +243,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala index cd97324c997f5..6925bdd72674f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.benchmark +import org.apache.spark.sql.internal.SQLConf + /** * Benchmark to measure built-in data sources write performance. * To run this benchmark: @@ -45,8 +47,8 @@ object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark { mainArgs } - spark.conf.set("spark.sql.parquet.compression.codec", "snappy") - spark.conf.set("spark.sql.orc.compression.codec", "snappy") + spark.conf.set(SQLConf.PARQUET_COMPRESSION.key, "snappy") + spark.conf.set(SQLConf.ORC_COMPRESSION.key, "snappy") formats.foreach { format => runBenchmark(s"$format writer benchmark") { 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 d31e49cf8cd4c..711ecf1738ab1 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{FilterExec, LocalTableScanExec, WholeStageCodegenExec} +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 @@ -437,7 +437,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-20356: pruned InMemoryTableScanExec should have correct ordering and partitioning") { - withSQLConf("spark.sql.shuffle.partitions" -> "200") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "200") { val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group") val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id") val df3 = df1.join(df2, Seq("item")).select($"id", $"group".as("item")).distinct() @@ -486,15 +486,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val df2 = df1.where("y = 3") val planBeforeFilter = df2.queryExecution.executedPlan.collect { - case f: FilterExec => f.child + case FilterExec(_, c: ColumnarToRowExec) => c.child + case WholeStageCodegenExec(FilterExec(_, ColumnarToRowExec(i: InputAdapter))) => i.child } assert(planBeforeFilter.head.isInstanceOf[InMemoryTableScanExec]) - val execPlan = if (codegenEnabled == "true") { - WholeStageCodegenExec(planBeforeFilter.head)(codegenStageId = 0) - } else { - planBeforeFilter.head - } + val execPlan = planBeforeFilter.head assert(execPlan.executeCollectPublic().length == 0) } 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 b3a5c687f775d..e74099202a1df 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 @@ -50,7 +50,7 @@ class PartitionBatchPruningSuite // Enable in-memory partition pruning spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, true) // Enable in-memory table scan accumulators - spark.conf.set("spark.sql.inMemoryTableScanStatistics.enable", "true") + spark.conf.set(SQLConf.IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED.key, "true") } override protected def afterAll(): 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 727160dafc5de..ce209666024d0 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 @@ -21,7 +21,7 @@ import java.net.URI import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, TableCatalog, TestTableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog, 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} @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -43,17 +44,43 @@ class PlanResolutionSuite extends AnalysisTest { newCatalog } - private val lookupCatalog: String => CatalogPlugin = { - case "testcat" => - testCat - case name => - throw new CatalogNotFoundException(s"No such catalog: $name") + private val v2SessionCatalog = { + val newCatalog = new TestTableCatalog + newCatalog.initialize("session", CaseInsensitiveStringMap.empty()) + 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 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") + } } - def parseAndResolve(query: String): LogicalPlan = { + def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = { val newConf = conf.copy() - newConf.setConfString("spark.sql.default.catalog", "testcat") - DataSourceResolution(newConf, lookupCatalog).apply(parsePlan(query)) + newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat") + DataSourceResolution(newConf, if (withDefault) lookupWithDefault else lookupWithoutDefault) + .apply(parsePlan(query)) } private def parseResolveCompare(query: String, expected: LogicalPlan): Unit = @@ -338,7 +365,46 @@ class PlanResolutionSuite extends AnalysisTest { } } - test("Test v2 CreateTable with data source v2 provider") { + test("Test v2 CreateTable with default catalog") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.table_name ( + | id bigint, + | description string, + | point struct) + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql, withDefault = true) match { + case create: CreateV2Table => + assert(create.catalog.name == "testcat") + assert(create.tableName == Identifier.of(Array("mydb"), "table_name")) + assert(create.tableSchema == new StructType() + .add("id", LongType) + .add("description", StringType) + .add("point", new StructType().add("x", DoubleType).add("y", DoubleType))) + assert(create.partitioning.isEmpty) + assert(create.properties == expectedProperties) + assert(create.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateV2Table].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CreateTable with data source v2 provider and no default") { val sql = s""" |CREATE TABLE IF NOT EXISTS mydb.page_view ( @@ -360,7 +426,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case create: CreateV2Table => - assert(create.catalog.name == "testcat") + assert(create.catalog.name == "session") assert(create.tableName == Identifier.of(Array("mydb"), "page_view")) assert(create.tableSchema == new StructType() .add("id", LongType) @@ -410,7 +476,41 @@ class PlanResolutionSuite extends AnalysisTest { } } - test("Test v2 CTAS with data source v2 provider") { + test("Test v2 CTAS with default catalog") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.table_name + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + |AS SELECT * FROM src + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql, withDefault = true) match { + case ctas: CreateTableAsSelect => + assert(ctas.catalog.name == "testcat") + assert(ctas.tableName == Identifier.of(Array("mydb"), "table_name")) + assert(ctas.properties == expectedProperties) + assert(ctas.writeOptions == Map("other" -> "20")) + assert(ctas.partitioning.isEmpty) + assert(ctas.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableAsSelect].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CTAS with data source v2 provider and no default") { val sql = s""" |CREATE TABLE IF NOT EXISTS mydb.page_view @@ -430,7 +530,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case ctas: CreateTableAsSelect => - assert(ctas.catalog.name == "testcat") + assert(ctas.catalog.name == "session") assert(ctas.tableName == Identifier.of(Array("mydb"), "page_view")) assert(ctas.properties == expectedProperties) assert(ctas.writeOptions.isEmpty) 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 af524c7ca0255..eaff5a2352a0c 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 @@ -201,7 +201,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("partitioned table - case insensitive") { - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val table = createTable( files = Seq( @@ -437,7 +437,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("[SPARK-16818] exchange reuse respects differences in partition pruning") { - spark.conf.set("spark.sql.exchange.reuse", true) + spark.conf.set(SQLConf.EXCHANGE_REUSE_ENABLED.key, true) withTempPath { path => val tempDir = path.getCanonicalPath spark.range(10) 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 2e7d682a3bbca..fdb50a6dd929c 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 @@ -1399,8 +1399,8 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) 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 6316e89537ca1..34b44be576897 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 @@ -2041,8 +2041,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) 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 c5d12d618e05f..577d1bc8d6a4f 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 @@ -1208,6 +1208,14 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } + // SPARK-28371: make sure filter is null-safe. + withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => + checkFilterPredicate( + '_1.startsWith("blah").asInstanceOf[Predicate], + classOf[UserDefinedByInstance[_, _]], + Seq.empty[Row]) + } + import testImplicits._ // Test canDrop() has taken effect testStringStartsWith(spark.range(1024).map(_.toString).toDF(), "value like 'a%'") 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 6b05b9c0f7207..6f2218ba82dc8 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 @@ -475,7 +475,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { val extraOptions = Map( SQLConf.OUTPUT_COMMITTER_CLASS.key -> classOf[ParquetOutputCommitter].getCanonicalName, - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[JobCommitFailureParquetOutputCommitter].getCanonicalName ) withTempPath { dir => @@ -505,7 +505,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { // Using a output committer that always fail when committing a task, so that both // `commitTask()` and `abortTask()` are invoked. val extraOptions = Map[String, String]( - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[TaskCommitFailureParquetOutputCommitter].getCanonicalName ) 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 6f3ed3d85e937..04ace0a236e6d 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 @@ -953,7 +953,7 @@ abstract class ParquetPartitionDiscoverySuite withSQLConf( ParquetOutputFormat.JOB_SUMMARY_LEVEL -> "ALL", - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { spark.range(3).write.parquet(s"$path/p0=0/p1=0") } 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 7aa0ba7f4e0c9..a6429bfc52924 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 @@ -924,14 +924,14 @@ class ParquetV1QuerySuite extends ParquetQuerySuite { // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) val df2 = spark.read.parquet(path) val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(!fileScan2.asInstanceOf[FileSourceScanExec].supportsBatch) + assert(!fileScan2.asInstanceOf[FileSourceScanExec].supportsColumnar) checkAnswer(df2, df) // return batch val columns = Seq.tabulate(9) {i => s"c$i"} val df3 = df2.selectExpr(columns : _*) val fileScan3 = df3.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(fileScan3.asInstanceOf[FileSourceScanExec].supportsBatch) + assert(fileScan3.asInstanceOf[FileSourceScanExec].supportsColumnar) checkAnswer(df3, df.selectExpr(columns : _*)) } } 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 new file mode 100644 index 0000000000000..3822882cc91cb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -0,0 +1,683 @@ +/* + * 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.datasources.v2 + +import java.util +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, TableCatalog, TableChange} +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.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 { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] + private val schema: StructType = new StructType() + .add("id", IntegerType) + .add("data", StringType) + + override protected def beforeAll(): Unit = { + super.beforeAll() + spark.sql("""CREATE DATABASE IF NOT EXISTS db""") + spark.sql("""CREATE DATABASE IF NOT EXISTS ns""") + spark.sql("""CREATE DATABASE IF NOT EXISTS ns2""") + } + + override protected def afterAll(): Unit = { + spark.sql("""DROP TABLE IF EXISTS db.test_table""") + spark.sql("""DROP DATABASE IF EXISTS db""") + spark.sql("""DROP DATABASE IF EXISTS ns""") + spark.sql("""DROP DATABASE IF EXISTS ns2""") + super.afterAll() + } + + after { + newCatalog().dropTable(testIdent) + } + + private def newCatalog(): TableCatalog = { + val newCatalog = new V2SessionCatalog(spark.sessionState) + newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) + newCatalog + } + + private val testIdent = Identifier.of(Array("db"), "test_table") + + test("Catalogs can load the catalog") { + val catalog = newCatalog() + + val conf = new SQLConf + conf.setConfString("spark.sql.catalog.test", catalog.getClass.getName) + + val loaded = Catalogs.load("test", conf) + assert(loaded.getClass == catalog.getClass) + } + + test("listTables") { + val catalog = newCatalog() + val ident1 = Identifier.of(Array("ns"), "test_table_1") + val ident2 = Identifier.of(Array("ns"), "test_table_2") + val ident3 = Identifier.of(Array("ns2"), "test_table_1") + + assert(catalog.listTables(Array("ns")).isEmpty) + + catalog.createTable(ident1, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) + assert(catalog.listTables(Array("ns2")).isEmpty) + + catalog.createTable(ident3, schema, Array.empty, emptyProps) + catalog.createTable(ident2, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident1) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident2)) + + catalog.dropTable(ident2) + + assert(catalog.listTables(Array("ns")).isEmpty) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident3) + } + + test("createTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("db", "test_table")) + assert(table.schema == schema) + assert(table.properties.asScala == Map()) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: with properties") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("property", "value") + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("db", "test_table")) + assert(table.schema == schema) + assert(table.properties == properties) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: table already exists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val exc = intercept[TableAlreadyExistsException] { + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + } + + assert(exc.message.contains(table.name())) + assert(exc.message.contains("already exists")) + + assert(catalog.tableExists(testIdent)) + } + + test("tableExists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + catalog.dropTable(testIdent) + + assert(!catalog.tableExists(testIdent)) + } + + test("loadTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("loadTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.loadTable(testIdent) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("invalidateTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.invalidateTable(testIdent) + + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("invalidateTable: table does not exist") { + val catalog = newCatalog() + + assert(catalog.tableExists(testIdent) === false) + + catalog.invalidateTable(testIdent) + } + + test("alterTable: add property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-1", "1")) + assert(updated.properties.asScala == Map("prop-1" -> "1")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1")) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add property to existing") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-2", "2")) + assert(updated.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove existing property") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove missing property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType)) + + assert(updated.schema == schema.add("ts", TimestampType)) + } + + test("alterTable: add required column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false)) + + assert(updated.schema == schema.add("ts", TimestampType, nullable = false)) + } + + test("alterTable: add column with comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false, "comment text")) + + val field = StructField("ts", TimestampType, nullable = false).withComment("comment text") + assert(updated.schema == schema.add(field)) + } + + test("alterTable: add nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("point", "z"), DoubleType)) + + val expectedSchema = schema.add("point", pointStruct.add("z", DoubleType)) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add column to primitive field fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) + } + + assert(exc.getMessage.contains("Not a struct")) + assert(exc.getMessage.contains("data")) + + // the table has not changed + assert(catalog.loadTable(testIdent).schema == schema) + } + + test("alterTable: add field to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.addColumn(Array("missing_col", "new_field"), StringType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: update column data type") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update column data type and nullability") { + val catalog = newCatalog() + + val originalSchema = new StructType() + .add("id", IntegerType, nullable = false) + .add("data", StringType) + val table = catalog.createTable(testIdent, originalSchema, Array.empty, emptyProps) + + assert(table.schema == originalSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("id"), LongType, true)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update optional column to required fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType, false)) + } + + assert(exc.getMessage.contains("Cannot change optional column to required")) + assert(exc.getMessage.contains("id")) + } + + test("alterTable: update missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("missing_col"), LongType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: add comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "comment text") + .add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: replace comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "replacement comment") + .add("data", StringType) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "replacement comment")) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add comment to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("missing_col"), "comment")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: rename top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("id"), "some_id")) + + val expectedSchema = new StructType().add("some_id", IntegerType).add("data", StringType) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first")) + + val newPointStruct = new StructType().add("first", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename struct column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point"), "p")) + + val newPointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("p", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.renameColumn(Array("missing_col"), "new_name")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: multiple changes") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first"), + TableChange.renameColumn(Array("point", "y"), "second")) + + val newPointStruct = new StructType().add("first", DoubleType).add("second", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("id"))) + + val expectedSchema = new StructType().add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("point", "y"))) + + val newPointStruct = new StructType().add("x", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"))) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: delete missing nested column fails") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"))) + } + + assert(exc.getMessage.contains("z")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("dropTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(wasDropped) + assert(!catalog.tableExists(testIdent)) + } + + test("dropTable: table does not exist") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(!wasDropped) + assert(!catalog.tableExists(testIdent)) + } +} 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 a8d230870aeb2..dc4a2998a908f 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 @@ -277,9 +277,9 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } test("ShuffledHashJoin metrics") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "40", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key", "value") // Assume the execution plan is @@ -584,19 +584,19 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared sql("CREATE TEMPORARY VIEW inMemoryTable AS SELECT 1 AS c1") sql("CACHE TABLE inMemoryTable") testSparkPlanMetrics(spark.table("inMemoryTable"), 1, - Map(0L -> (("Scan In-memory table `inMemoryTable`", Map.empty))) + Map(1L -> (("Scan In-memory table `inMemoryTable`", Map.empty))) ) sql("CREATE TEMPORARY VIEW ```a``b``` AS SELECT 2 AS c1") sql("CACHE TABLE ```a``b```") testSparkPlanMetrics(spark.table("```a``b```"), 1, - Map(0L -> (("Scan In-memory table ```a``b```", Map.empty))) + Map(1L -> (("Scan In-memory table ```a``b```", Map.empty))) ) } // Show InMemoryTableScan on UI testSparkPlanMetrics(spark.range(1).cache().select("id"), 1, - Map(0L -> (("InMemoryTableScan", Map.empty))) + Map(1L -> (("InMemoryTableScan", Map.empty))) ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index f12eeaa580642..8f26c04307adc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SQLAppStatusStore} +import org.apache.spark.sql.internal.SQLConf.WHOLESTAGE_CODEGEN_ENABLED import org.apache.spark.sql.test.SQLTestUtils @@ -154,7 +155,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { expectedNodeIds: Set[Long], enableWholeStage: Boolean = false): Option[Map[Long, (String, Map[String, Any])]] = { val previousExecutionIds = currentExecutionIds() - withSQLConf("spark.sql.codegen.wholeStage" -> enableWholeStage.toString) { + withSQLConf(WHOLESTAGE_CODEGEN_ENABLED.key -> enableWholeStage.toString) { df.collect() } sparkContext.listenerBus.waitUntilEmpty(10000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index 723764c777271..c0fd3fe3ef7b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.sql.streaming.util.StreamManualClock class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @@ -35,7 +34,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { val timeout = 10.seconds test("nextBatchTime") { - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(100)) assert(processingTimeExecutor.nextBatchTime(0) === 100) assert(processingTimeExecutor.nextBatchTime(1) === 100) assert(processingTimeExecutor.nextBatchTime(99) === 100) @@ -49,7 +48,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { val clock = new StreamManualClock() @volatile var continueExecuting = true @volatile var clockIncrementInTrigger = 0L - val executor = ProcessingTimeExecutor(ProcessingTime("1000 milliseconds"), clock) + val executor = ProcessingTimeExecutor(ProcessingTimeTrigger("1000 milliseconds"), clock) val executorThread = new Thread() { override def run(): Unit = { executor.execute(() => { @@ -97,7 +96,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { test("calling nextBatchTime with the result of a previous call should return the next interval") { val intervalMS = 100 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMS)) val ITERATION = 10 var nextBatchTime: Long = 0 @@ -111,7 +110,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { private def testBatchTermination(intervalMs: Long): Unit = { var batchCounts = 0 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMs)) processingTimeExecutor.execute(() => { batchCounts += 1 // If the batch termination works correctly, batchCounts should be 3 after `execute` @@ -130,7 +129,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @volatile var batchFallingBehindCalled = false val t = new Thread() { override def run(): Unit = { - val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTime(100), clock) { + val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTimeTrigger(100), clock) { override def notifyBatchFallingBehind(realElapsedTimeMs: Long): Unit = { batchFallingBehindCalled = true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala index 2a1e7d615e5e3..7bca225dfdd8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS import org.apache.spark.util.Utils class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { @@ -124,7 +125,7 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { import spark.implicits._ coordRef = spark.streams.stateStoreCoordinator implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SHUFFLE_PARTITIONS.key, "1") // Start a query and run a batch to load state stores val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index af4369de800ec..a84d107f2cbc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -569,7 +569,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] val spark = SparkSession.builder().master("local[2]").getOrCreate() SparkSession.setActiveSession(spark) implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") import spark.implicits._ val inputData = MemoryStream[Int] 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 e3e5ddff96378..8edbb87706716 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 @@ -647,7 +647,7 @@ class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { .setMaster("local") .setAppName("test") .set(config.TASK_MAX_FAILURES, 1) // Don't retry the tasks to run this test quickly - .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly + .set(UI_RETAINED_EXECUTIONS.key, "50") // Set it to 50 to run this test quickly .set(ASYNC_TRACKING_ENABLED, false) withSpark(new SparkContext(conf)) { sc => quietly { 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 5f27e75addcff..89eaac8e5927c 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 @@ -857,10 +857,7 @@ class JDBCSuite extends QueryTest Some(ArrayType(DecimalType.SYSTEM_DEFAULT))) assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") - val errMsg = intercept[IllegalArgumentException] { - Postgres.getJDBCType(ByteType) - } - assert(errMsg.getMessage contains "Unsupported type in postgresql: ByteType") + assert(Postgres.getJDBCType(ByteType).map(_.databaseTypeDefinition).get == "SMALLINT") } test("DerbyDialect jdbc type mapping") { @@ -895,6 +892,17 @@ class JDBCSuite extends QueryTest "BIT") assert(msSqlServerDialect.getJDBCType(BinaryType).map(_.databaseTypeDefinition).get == "VARBINARY(MAX)") + assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get == + "SMALLINT") + } + + test("SPARK-28152 MsSqlServerDialect catalyst type mapping") { + val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver") + val metadata = new MetadataBuilder().putLong("scale", 1) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1, + metadata).get == ShortType) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1, + metadata).get == FloatType) } test("table exists query by jdbc dialect") { @@ -1322,7 +1330,7 @@ class JDBCSuite extends QueryTest testJdbcParitionColumn("THEID", "THEID") testJdbcParitionColumn("\"THEID\"", "THEID") - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { testJdbcParitionColumn("ThEiD", "THEID") } testJdbcParitionColumn("THE ID", "THE ID") 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 fc61050dc7458..75f68dea96bf0 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 @@ -63,7 +63,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(SQLConf.BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { withTable("t") { 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 d46029e84433c..5f9856656ac3b 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 @@ -26,6 +26,7 @@ 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.util.Utils @@ -252,7 +253,7 @@ class CreateTableAsSelectSuite val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // Within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { 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 96345e22dbd5d..c90090aca3d4e 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 @@ -21,12 +21,15 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.analysis.{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.V2_SESSION_CATALOG import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{LongType, StringType, StructType} +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -37,7 +40,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) spark.conf.set("spark.sql.catalog.testcat2", classOf[TestInMemoryTableCatalog].getName) - spark.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set(V2_SESSION_CATALOG.key, classOf[TestInMemoryTableCatalog].getName) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -47,8 +50,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn after { spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() - spark.sql("DROP TABLE source") - spark.sql("DROP TABLE source2") + spark.catalog("session").asInstanceOf[TestInMemoryTableCatalog].clearTables() } test("CreateTable: use v2 plan because catalog is set") { @@ -66,13 +68,13 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } - test("CreateTable: use v2 plan because provider is v2") { + 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("testcat").asTableCatalog + val testCatalog = spark.catalog("session").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(table.name == "testcat.table_name") + assert(table.name == "session.table_name") assert(table.partitioning.isEmpty) assert(table.properties == Map("provider" -> orc2).asJava) assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) @@ -137,22 +139,23 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd2, table.schema), Seq.empty) } - test("CreateTable: fail analysis when default catalog is needed but missing") { - val originalDefaultCatalog = conf.getConfString("spark.sql.default.catalog") - try { - conf.unsetConf("spark.sql.default.catalog") + 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") - val exc = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") - } + val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(exc.getMessage.contains("No catalog specified for table")) - assert(exc.getMessage.contains("table_name")) - assert(exc.getMessage.contains("no default catalog is set")) + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) - } finally { - conf.setConfString("spark.sql.default.catalog", originalDefaultCatalog) - } + // check that the table is empty + val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } test("CreateTableAsSelect: use v2 plan because catalog is set") { @@ -172,13 +175,13 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) } - test("CreateTableAsSelect: use v2 plan because provider is v2") { + 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("testcat").asTableCatalog + val testCatalog = spark.catalog("session").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) - assert(table.name == "testcat.table_name") + assert(table.name == "session.table_name") assert(table.partitioning.isEmpty) assert(table.properties == Map("provider" -> orc2).asJava) assert(table.schema == new StructType() @@ -251,22 +254,43 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd2, table.schema), spark.table("source")) } - test("CreateTableAsSelect: fail analysis when default catalog is needed but missing") { - val originalDefaultCatalog = conf.getConfString("spark.sql.default.catalog") - try { - conf.unsetConf("spark.sql.default.catalog") + 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") - val exc = intercept[AnalysisException] { - spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") - } + val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") - assert(exc.getMessage.contains("No catalog specified for table")) - assert(exc.getMessage.contains("table_name")) - assert(exc.getMessage.contains("no default catalog is set")) + // 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") - } finally { - conf.setConfString("spark.sql.default.catalog", originalDefaultCatalog) - } + val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .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")) + } + + test("CreateTableAsSelect: v2 session catalog can load v1 source table") { + val sparkSession = spark.newSession() + sparkSession.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + + val df = sparkSession.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") + + // use the catalog name to force loading with the v2 catalog + checkAnswer(sparkSession.sql(s"TABLE session.table_name"), sparkSession.table("source")) } test("DropTable: basic") { @@ -344,4 +368,834 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn df_joined) } } + + test("AlterTable: table does not exist") { + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id") + } + + assert(exc.getMessage.contains("testcat.ns1.table_name")) + assert(exc.getMessage.contains("Table or view not found")) + } + + test("AlterTable: change rejected by implementation") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[SparkException] { + sql(s"ALTER TABLE $t DROP COLUMN id") + } + + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: add top-level column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType).add("data", StringType)) + } + } + + test("AlterTable: add column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc")))) + } + } + + test("AlterTable: add multiple columns") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc"), + StructField("ts", TimestampType)))) + } + } + + test("AlterTable: add nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))))) + } + } + + test("AlterTable: add nested column to map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))), LongType))) + } + } + + test("AlterTable: add nested column to map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add nested column to array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add complex column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: add nested column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: add nested column parent must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD COLUMN point.z double") + } + + assert(exc.getMessage.contains("point")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type int -> long") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", LongType)) + } + } + + test("AlterTable: update nested type float -> double") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with struct type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point TYPE struct") + } + + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with array type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN points TYPE array") + } + + assert(exc.getMessage.contains("update the element by updating points.element")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(IntegerType))) + } + } + + test("AlterTable: update column array element type") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(LongType))) + } + } + + test("AlterTable: update column with map type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, m map) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN m TYPE map") + } + + assert(exc.getMessage.contains("update a map by updating m.key or m.value")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, IntegerType))) + } + } + + test("AlterTable: update column map value type") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, LongType))) + } + } + + test("AlterTable: update nested type in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))), LongType))) + } + } + + test("AlterTable: update nested type in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update nested type in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data TYPE string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type must be compatible") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE boolean") + } + + assert(exc.getMessage.contains("id")) + assert(exc.getMessage.contains("int cannot be cast to boolean")) + } + } + + test("AlterTable: update column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", IntegerType).withComment("doc")))) + } + } + + test("AlterTable: update column type and comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", LongType).withComment("doc")))) + } + } + + test("AlterTable: update nested column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))))) + } + } + + test("AlterTable: update nested column comment in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))), LongType))) + } + } + + test("AlterTable: update nested column comment in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: update nested column comment in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data COMMENT 'doc'") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x COMMENT 'doc'") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: rename column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("user_id", IntegerType)) + } + } + + test("AlterTable: rename nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))))) + } + } + + test("AlterTable: rename nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))), LongType))) + } + } + + test("AlterTable: rename nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN data TO some_string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN point.x TO z") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: drop column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: drop nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: drop nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType))), LongType))) + } + } + + test("AlterTable: drop nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN data") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN point.x") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: set location") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "location" -> "s3://bucket/path").asJava) + } + } + + test("AlterTable: set table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + 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 table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + } + } + + test("AlterTable: remove table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo TBLPROPERTIES('test' = '34')") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('test')") + + val updated = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(updated.name == "testcat.ns1.table_name") + assert(updated.properties == Map("provider" -> "foo").asJava) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 4e9f961016de3..380df7a365967 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -88,6 +88,12 @@ class TestInMemoryTableCatalog extends TableCatalog { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) + + // fail if the last column in the schema was dropped + if (schema.fields.isEmpty) { + throw new IllegalArgumentException(s"Cannot drop all fields") + } + val newTable = new InMemoryTable(table.name, schema, properties, table.data) tables.put(ident, newTable) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 4bf49ff4d5c61..92ec2a0c172ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -305,7 +305,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("update mode") { val inputData = MemoryStream[Int] - spark.conf.set("spark.sql.shuffle.partitions", "10") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) 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 2b8d77386925f..72f893845172d 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 @@ -1310,7 +1310,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val start = startId.map(new FileStreamSourceOffset(_)) val end = FileStreamSourceOffset(endId) - withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") { assert(fileSource.getBatch(start, end).as[String].collect().toSeq === expected) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index f2f5fad59eb2a..1ed2599444c5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -871,7 +871,7 @@ class StreamSuite extends StreamTest { testQuietly("specify custom state store provider") { val providerClassName = classOf[TestStateStoreProvider].getCanonicalName - withSQLConf("spark.sql.streaming.stateStore.providerClass" -> providerClassName) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> providerClassName) { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() val query = df.writeStream.outputMode("complete").format("memory").queryName("name").start() @@ -888,9 +888,9 @@ class StreamSuite extends StreamTest { testQuietly("custom state store provider read from offset log") { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() - val providerConf1 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf1 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider" - val providerConf2 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf2 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[TestStateStoreProvider].getCanonicalName def runQuery(queryName: String, checkpointLoc: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 88f510c726faa..da2f221aaf101 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS import org.apache.spark.sql.streaming.StreamingQueryListener._ @@ -29,7 +30,7 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { import testImplicits._ override protected def sparkConf: SparkConf = - super.sparkConf.set("spark.sql.streaming.streamingQueryListeners", + super.sparkConf.set(STREAMING_QUERY_LISTENERS.key, "org.apache.spark.sql.streaming.TestListener") test("test if the configured query lister is loaded") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index a5cb25c49b869..e6b56e5f46f89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -413,9 +413,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi sources.nonEmpty } // Disabled by default - assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + assert(spark.conf.get(SQLConf.STREAMING_METRICS_ENABLED.key).toBoolean === false) - withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "false") { testStream(inputData.toDF)( AssertOnQuery { q => !isMetricsRegistered(q) }, StopStream, @@ -424,7 +424,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } // Registered when enabled - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { testStream(inputData.toDF)( AssertOnQuery { q => isMetricsRegistered(q) }, StopStream, @@ -434,7 +434,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } test("SPARK-22975: MetricsReporter defaults when there was no progress reported") { - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { BlockingSource.latch = new CountDownLatch(1) withTempDir { tempDir => val sq = spark.readStream diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala index c5b95fa9b64a9..3ec4750c59fc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.streaming.continuous import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED import org.apache.spark.sql.streaming.OutputMode class ContinuousAggregationSuite extends ContinuousSuiteBase { @@ -36,7 +37,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("basic") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( @@ -112,7 +113,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("repeated restart") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index 9840c7f066780..c6921010a002f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf.CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE +import org.apache.spark.sql.internal.SQLConf.{CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE, MIN_BATCHES_TO_RETAIN} import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.TestSparkSession @@ -37,18 +37,43 @@ class ContinuousSuiteBase extends StreamTest { "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true"))) - protected def waitForRateSourceTriggers(query: StreamExecution, numTriggers: Int): Unit = { - query match { - case s: ContinuousExecution => - assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized") - val reader = s.lastExecution.executedPlan.collectFirst { - case ContinuousScanExec(_, _, r: RateStreamContinuousStream, _) => r - }.get - - val deltaMs = numTriggers * 1000 + 300 - while (System.currentTimeMillis < reader.creationTime + deltaMs) { - Thread.sleep(reader.creationTime + deltaMs - System.currentTimeMillis) + protected def waitForRateSourceTriggers(query: ContinuousExecution, numTriggers: Int): Unit = { + query.awaitEpoch(0) + + // This is called after waiting first epoch to be committed, so we can just treat + // it as partition readers for rate source are already initialized. + val firstCommittedTime = System.nanoTime() + val deltaNs = (numTriggers * 1000 + 300) * 1000000L + var toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + while (toWaitNs > 0) { + Thread.sleep(toWaitNs / 1000000) + toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + } + } + + protected def waitForRateSourceCommittedValue( + query: ContinuousExecution, + desiredValue: Long, + maxWaitTimeMs: Long): Unit = { + def readHighestCommittedValue(c: ContinuousExecution): Option[Long] = { + c.committedOffsets.lastOption.map { case (_, offset) => + offset match { + case o: RateStreamOffset => + o.partitionToValueAndRunTimeMs.map { + case (_, ValueRunTimeMsPair(value, _)) => value + }.max } + } + } + + val maxWait = System.currentTimeMillis() + maxWaitTimeMs + while (System.currentTimeMillis() < maxWait && + readHighestCommittedValue(query).getOrElse(Long.MinValue) < desiredValue) { + Thread.sleep(100) + } + if (System.currentTimeMillis() > maxWait) { + logWarning(s"Couldn't reach desired value in $maxWaitTimeMs milliseconds!" + + s"Current highest committed value is ${readHighestCommittedValue(query)}") } } @@ -216,14 +241,16 @@ class ContinuousSuite extends ContinuousSuiteBase { .queryName("noharness") .trigger(Trigger.Continuous(100)) .start() + + val expected = Set(0, 1, 2, 3) val continuousExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.asInstanceOf[ContinuousExecution] - continuousExecution.awaitEpoch(0) - waitForRateSourceTriggers(continuousExecution, 2) + waitForRateSourceCommittedValue(continuousExecution, expected.max, 20 * 1000) query.stop() val results = spark.read.table("noharness").collect() - assert(Set(0, 1, 2, 3).map(Row(_)).subsetOf(results.toSet)) + assert(expected.map(Row(_)).subsetOf(results.toSet), + s"Result set ${results.toSet} are not a superset of $expected!") } } @@ -241,7 +268,9 @@ class ContinuousStressSuite extends ContinuousSuiteBase { testStream(df)( StartStream(longContinuousTrigger), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 10)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_))) @@ -259,7 +288,9 @@ class ContinuousStressSuite extends ContinuousSuiteBase { testStream(df)( StartStream(Trigger.Continuous(2012)), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 10)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_)))) @@ -307,7 +338,7 @@ class ContinuousMetaSuite extends ContinuousSuiteBase { "local[10]", "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true") - .set("spark.sql.streaming.minBatchesToRetain", "2"))) + .set(MIN_BATCHES_TO_RETAIN.key, "2"))) test("SPARK-24351: check offsetLog/commitLog retained in the checkpoint directory") { withTempDir { checkpointDir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 7b2c1a56e8baa..4db605ee1b238 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -24,8 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} import org.apache.spark.sql.sources.v2._ @@ -242,7 +241,7 @@ class StreamingDataSourceV2Suite extends StreamTest { override def beforeAll(): Unit = { super.beforeAll() val fakeCheckpoint = Utils.createTempDir() - spark.conf.set("spark.sql.streaming.checkpointLocation", fakeCheckpoint.getCanonicalPath) + spark.conf.set(SQLConf.CHECKPOINT_LOCATION.key, fakeCheckpoint.getCanonicalPath) } override def afterEach(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8fb1400a9b5a7..c630f1497a17e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -203,7 +203,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .stop() assert(LastOptions.partitionColumns == Seq("a")) - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) 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 e9ab62800f84f..126e23e6e5926 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 @@ -409,7 +409,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be test("write path implements onTaskCommit API correctly") { withSQLConf( - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { withTempDir { dir => val path = dir.getCanonicalPath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index da0e5535df5eb..115536da8949e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -392,7 +392,7 @@ private[sql] trait SQLTestUtilsBase */ protected def stripSparkFilter(df: DataFrame): DataFrame = { val schema = df.schema - val withoutFilters = df.queryExecution.sparkPlan.transform { + val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index d1de9f0379924..b4d1d0d58aad6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -137,7 +137,7 @@ object HiveThriftServer2 extends Logging { } private[thriftserver] object ExecutionState extends Enumeration { - val STARTED, COMPILED, FAILED, FINISHED = Value + val STARTED, COMPILED, FAILED, FINISHED, CLOSED = Value type ExecutionState = Value } @@ -147,16 +147,17 @@ object HiveThriftServer2 extends Logging { val startTimestamp: Long, val userName: String) { var finishTimestamp: Long = 0L + var closeTimestamp: Long = 0L var executePlan: String = "" var detail: String = "" var state: ExecutionState.Value = ExecutionState.STARTED val jobId: ArrayBuffer[String] = ArrayBuffer[String]() var groupId: String = "" - def totalTime: Long = { - if (finishTimestamp == 0L) { + def totalTime(endTime: Long): Long = { + if (endTime == 0L) { System.currentTimeMillis - startTimestamp } else { - finishTimestamp - startTimestamp + endTime - startTimestamp } } } @@ -254,6 +255,11 @@ object HiveThriftServer2 extends Logging { trimExecutionIfNecessary() } + def onOperationClosed(id: String): Unit = synchronized { + executionList(id).closeTimestamp = System.currentTimeMillis + executionList(id).state = ExecutionState.CLOSED + } + private def trimExecutionIfNecessary() = { if (executionList.size > retainedStatements) { val toRemove = math.max(retainedStatements / 10, 1) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 820f76db6db3b..2f011c25fe2ce 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -70,11 +70,12 @@ private[hive] class SparkExecuteStatementOperation( } } - def close(): Unit = { + override def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. logDebug(s"CLOSING $statementId") cleanup(OperationState.CLOSED) sqlContext.sparkContext.clearJobGroup() + HiveThriftServer2.listener.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 99ba968e1ae83..89faff2f6f913 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -58,8 +58,15 @@ private[hive] class SparkGetColumnsOperation( val catalog: SessionCatalog = sqlContext.sessionState.catalog + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName, tablePattern : $tableName" val logMsg = s"Listing columns '$cmdStr, columnName : $columnName'" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 3ecbbd036c87f..87ef154bcc8ab 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -45,8 +45,15 @@ private[hive] class SparkGetSchemasOperation( schemaName: String) extends GetSchemasOperation(parentSession, catalogName, schemaName) with Logging { + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" val logMsg = s"Listing databases '$cmdStr'" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 878683692fb60..952de42083c42 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -55,8 +55,15 @@ private[hive] class SparkGetTablesOperation( extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes) with Logging{ + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" val tableTypesStr = if (tableTypes == null) "null" else tableTypes.asScala.mkString(",") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 27d2c997ca3e8..1747b5bafc934 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -70,8 +70,8 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { val numStatement = listener.getExecutionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -90,7 +90,9 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {info.groupId} {formatDate(info.startTimestamp)} {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index fdc9bee5ed056..a45c6e363cbf5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -79,8 +79,8 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = executionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -99,7 +99,9 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {info.groupId} {formatDate(info.startTimestamp)} {formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index b06856b054795..dd18add53fde8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -44,6 +44,7 @@ import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.HiveTestUtils +import org.apache.spark.sql.internal.StaticSQLConf.HIVE_THRIFT_SERVER_SINGLESESSION import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -536,9 +537,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } if (HiveUtils.isHive23) { - assert(conf.get("spark.sql.hive.version") === Some("2.3.5")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) } else { - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) } } } @@ -553,9 +554,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } if (HiveUtils.isHive23) { - assert(conf.get("spark.sql.hive.version") === Some("2.3.5")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) } else { - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) } } } @@ -659,7 +660,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary override protected def extraConf: Seq[String] = - "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil + s"--conf ${HIVE_THRIFT_SERVER_SINGLESESSION.key}=true" :: Nil test("share the temporary functions across JDBC connections") { withMultipleConnectionJdbcStatement()( diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index ad7a9a238f8a9..8fce9d9383438 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -344,6 +344,7 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) String ipAddress = getIpAddress(); TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol()); + res.setServerProtocolVersion(protocol); SessionHandle sessionHandle; if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && (userName != null)) { @@ -354,7 +355,6 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), ipAddress, req.getConfiguration()); } - res.setServerProtocolVersion(protocol); return sessionHandle; } diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index c3bcaf1e2d6c1..504e63dbc5e5e 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -86,7 +86,7 @@ public void run() { throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + " Not configured for SSL connection"); } - SslContextFactory sslContextFactory = new SslContextFactory(); + SslContextFactory sslContextFactory = new SslContextFactory.Server(); String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); sslContextFactory.addExcludeProtocols(excludedProtocols); diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 9552d9bd68cd5..d41c3b493bb47 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -345,6 +345,7 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) String ipAddress = getIpAddress(); TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol()); + res.setServerProtocolVersion(protocol); SessionHandle sessionHandle; if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && (userName != null)) { @@ -355,7 +356,6 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), ipAddress, req.getConfiguration()); } - res.setServerProtocolVersion(protocol); return sessionHandle; } diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 003898acf5986..08626e7eb146d 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -87,7 +87,7 @@ public void run() { throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + " Not configured for SSL connection"); } - SslContextFactory sslContextFactory = new SslContextFactory(); + SslContextFactory sslContextFactory = new SslContextFactory.Server(); String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); sslContextFactory.addExcludeProtocols(excludedProtocols); 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 b04b3f1031d73..2fa108825982f 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 @@ -74,7 +74,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: new FallBackFileSourceV2(session) +: - DataSourceResolution(conf, session.catalog(_)) +: + DataSourceResolution(conf, this) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 4351dc7036846..9bc0be87be5af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -27,9 +27,12 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.spark.{SecurityManager, SparkConf, TestUtils} +import org.apache.spark.internal.config.MASTER_REST_SERVER_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils @@ -184,11 +187,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { val args = Seq( "--name", "prepare testing tables", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", "spark.sql.hive.metastore.version=1.2.1", - "--conf", "spark.sql.hive.metastore.jars=maven", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--conf", s"spark.sql.test.version.index=$index", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", tempPyFile.getCanonicalPath) @@ -203,11 +206,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { "--class", PROCESS_TABLES.getClass.getName.stripSuffix("$"), "--name", "HiveExternalCatalog backward compatibility test", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", "spark.sql.hive.metastore.version=1.2.1", - "--conf", "spark.sql.hive.metastore.jars=maven", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", unusedJar.toString) runSparkSubmit(args) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 0ff22150658b9..e2ddec3427665 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -33,6 +33,8 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHiveContext} +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types.{DecimalType, StructType} import org.apache.spark.tags.ExtendedHiveTest import org.apache.spark.util.{ResetSystemProperties, Utils} @@ -338,10 +340,10 @@ object SetMetastoreURLTest extends Logging { val builder = SparkSession.builder() .config(sparkConf) .config(UI_ENABLED.key, "false") - .config("spark.sql.hive.metastore.version", "0.13.1") + .config(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") // The issue described in SPARK-16901 only appear when // spark.sql.hive.metastore.jars is not set to builtin. - .config("spark.sql.hive.metastore.jars", "maven") + .config(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .enableHiveSupport() val spark = builder.getOrCreate() @@ -392,16 +394,16 @@ object SetWarehouseLocationTest extends Logging { // We are expecting that the value of spark.sql.warehouse.dir will override the // value of hive.metastore.warehouse.dir. val session = new TestHiveContext(new SparkContext(sparkConf - .set("spark.sql.warehouse.dir", warehouseLocation.toString) + .set(WAREHOUSE_PATH.key, warehouseLocation.toString) .set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString))) .sparkSession (session, warehouseLocation.toString) } - if (sparkSession.conf.get("spark.sql.warehouse.dir") != expectedWarehouseLocation) { + if (sparkSession.conf.get(WAREHOUSE_PATH.key) != expectedWarehouseLocation) { throw new Exception( - "spark.sql.warehouse.dir is not set to the expected warehouse location " + + s"${WAREHOUSE_PATH.key} is not set to the expected warehouse location " + s"$expectedWarehouseLocation.") } @@ -564,7 +566,7 @@ object SparkSubmitClassLoaderTest extends Logging { val conf = new SparkConf() val hiveWarehouseLocation = Utils.createTempDir() conf.set(UI_ENABLED, false) - conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) + conf.set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j") @@ -642,14 +644,14 @@ object SparkSQLConfTest extends Logging { val conf = new SparkConf() { override def getAll: Array[(String, String)] = { def isMetastoreSetting(conf: String): Boolean = { - conf == "spark.sql.hive.metastore.version" || conf == "spark.sql.hive.metastore.jars" + conf == HiveUtils.HIVE_METASTORE_VERSION.key || conf == HiveUtils.HIVE_METASTORE_JARS.key } // If there is any metastore settings, remove them. val filteredSettings = super.getAll.filterNot(e => isMetastoreSetting(e._1)) // Always add these two metastore settings at the beginning. - ("spark.sql.hive.metastore.version" -> "0.12") +: - ("spark.sql.hive.metastore.jars" -> "maven") +: + (HiveUtils.HIVE_METASTORE_VERSION.key -> "0.12") +: + (HiveUtils.HIVE_METASTORE_JARS.key -> "maven") +: filteredSettings } @@ -676,10 +678,10 @@ object SPARK_9757 extends QueryTest { val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() - .set("spark.sql.hive.metastore.version", "0.13.1") - .set("spark.sql.hive.metastore.jars", "maven") + .set(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") + .set(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .set(UI_ENABLED, false) - .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) + .set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -725,7 +727,7 @@ object SPARK_11009 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() .set(UI_ENABLED, false) - .set("spark.sql.shuffle.partitions", "100")) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -756,7 +758,7 @@ object SPARK_14244 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() .set(UI_ENABLED, false) - .set("spark.sql.shuffle.partitions", "100")) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index ecd428780c671..d06cc1c0a88ac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -1028,7 +1028,7 @@ class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySu override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { Seq("true", "false").foreach { enableTwoLevelMaps => - withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enabled" -> + withSQLConf(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> enableTwoLevelMaps) { (1 to 3).foreach { fallbackStartsAt => withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index dc58c2d603570..fd505d1e4cc30 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1183,12 +1183,31 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Row(CalendarInterval.fromString("interval 100 milliseconds"))) checkAnswer(sql("select interval '10-9' year to month"), Row(CalendarInterval.fromString("interval 10 years 9 months"))) + checkAnswer(sql("select interval '20 15:40:32.99899999' day to hour"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours"))) + checkAnswer(sql("select interval '20 15:40:32.99899999' day to minute"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes"))) checkAnswer(sql("select interval '20 15:40:32.99899999' day to second"), Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes " + "32 seconds 998 milliseconds 999 microseconds"))) + checkAnswer(sql("select interval '15:40:32.99899999' hour to minute"), + Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) + checkAnswer(sql("select interval '15:40.99899999' hour to second"), + Row(CalendarInterval.fromString("interval 15 minutes 40 seconds 998 milliseconds " + + "999 microseconds"))) + checkAnswer(sql("select interval '15:40' hour to second"), + Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) checkAnswer(sql("select interval '15:40:32.99899999' hour to second"), Row(CalendarInterval.fromString("interval 15 hours 40 minutes 32 seconds 998 milliseconds " + "999 microseconds"))) + checkAnswer(sql("select interval '20 40:32.99899999' minute to second"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 40 minutes 32 seconds " + + "998 milliseconds 999 microseconds"))) + checkAnswer(sql("select interval '40:32.99899999' minute to second"), + Row(CalendarInterval.fromString("interval 40 minutes 32 seconds 998 milliseconds " + + "999 microseconds"))) + checkAnswer(sql("select interval '40:32' minute to second"), + Row(CalendarInterval.fromString("interval 40 minutes 32 seconds"))) checkAnswer(sql("select interval '30' year"), Row(CalendarInterval.fromString("interval 30 years"))) checkAnswer(sql("select interval '25' month"), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 8cdb8dd84fb2e..d68a47053f18c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf, WithTestConf} -import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.StaticSQLConf.{CATALOG_IMPLEMENTATION, WAREHOUSE_PATH} import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. @@ -57,9 +57,9 @@ object TestHive new SparkConf() .set("spark.sql.test", "") .set(SQLConf.CODEGEN_FALLBACK.key, "false") - .set("spark.sql.hive.metastore.barrierPrefixes", + .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, "org.apache.spark.sql.hive.execution.PairSerDe") - .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) + .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 .set(UI_ENABLED, false) .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) @@ -534,7 +534,7 @@ private[hive] class TestHiveSparkSession( } // Clean out the Hive warehouse between each suite - val warehouseDir = new File(new URI(sparkContext.conf.get("spark.sql.warehouse.dir")).getPath) + val warehouseDir = new File(new URI(sparkContext.conf.get(WAREHOUSE_PATH.key)).getPath) Utils.deleteRecursively(warehouseDir) warehouseDir.mkdir()