diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 7376d1ddc4818..e04524dde0a75 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -30,10 +30,10 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.*; -import io.netty.util.AbstractReferenceCounted; import org.apache.commons.crypto.stream.CryptoInputStream; import org.apache.commons.crypto.stream.CryptoOutputStream; +import org.apache.spark.network.util.AbstractFileRegion; import org.apache.spark.network.util.ByteArrayReadableChannel; import org.apache.spark.network.util.ByteArrayWritableChannel; @@ -161,7 +161,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } } - private static class EncryptedMessage extends AbstractReferenceCounted implements FileRegion { + private static class EncryptedMessage extends AbstractFileRegion { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; @@ -199,10 +199,45 @@ public long position() { } @Override - public long transfered() { + public long transferred() { return transferred; } + @Override + public EncryptedMessage touch(Object o) { + super.touch(o); + if (region != null) { + region.touch(o); + } + if (buf != null) { + buf.touch(o); + } + return this; + } + + @Override + public EncryptedMessage retain(int increment) { + super.retain(increment); + if (region != null) { + region.retain(increment); + } + if (buf != null) { + buf.retain(increment); + } + return this; + } + + @Override + public boolean release(int decrement) { + if (region != null) { + region.release(decrement); + } + if (buf != null) { + buf.release(decrement); + } + return super.release(decrement); + } + @Override public long transferTo(WritableByteChannel target, long position) throws IOException { Preconditions.checkArgument(position == transfered(), "Invalid position."); diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java index 4f8781b42a0e4..897d0f9e4fb89 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -25,17 +25,17 @@ import com.google.common.base.Preconditions; import io.netty.buffer.ByteBuf; import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCountUtil; import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.util.AbstractFileRegion; /** * A wrapper message that holds two separate pieces (a header and a body). * * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. */ -class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { +class MessageWithHeader extends AbstractFileRegion { @Nullable private final ManagedBuffer managedBuffer; private final ByteBuf header; @@ -91,7 +91,7 @@ public long position() { } @Override - public long transfered() { + public long transferred() { return totalBytesTransferred; } @@ -160,4 +160,37 @@ private int writeNioBuffer( return ret; } + + @Override + public MessageWithHeader touch(Object o) { + super.touch(o); + header.touch(o); + ReferenceCountUtil.touch(body, o); + return this; + } + + @Override + public MessageWithHeader retain(int increment) { + super.retain(increment); + header.retain(increment); + ReferenceCountUtil.retain(body, increment); + if (managedBuffer != null) { + for (int i = 0; i < increment; i++) { + managedBuffer.retain(); + } + } + return this; + } + + @Override + public boolean release(int decrement) { + header.release(decrement); + ReferenceCountUtil.release(body, decrement); + if (managedBuffer != null) { + for (int i = 0; i < decrement; i++) { + managedBuffer.release(); + } + } + return super.release(decrement); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java index 3d71ebaa7ea0c..16ab4efcd4f5f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java @@ -32,8 +32,8 @@ import io.netty.channel.ChannelPromise; import io.netty.channel.FileRegion; import io.netty.handler.codec.MessageToMessageDecoder; -import io.netty.util.AbstractReferenceCounted; +import org.apache.spark.network.util.AbstractFileRegion; import org.apache.spark.network.util.ByteArrayWritableChannel; import org.apache.spark.network.util.NettyUtils; @@ -129,7 +129,7 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf msg, List out) } @VisibleForTesting - static class EncryptedMessage extends AbstractReferenceCounted implements FileRegion { + static class EncryptedMessage extends AbstractFileRegion { private final SaslEncryptionBackend backend; private final boolean isByteBuf; @@ -183,10 +183,45 @@ public long position() { * Returns an approximation of the amount of data transferred. See {@link #count()}. */ @Override - public long transfered() { + public long transferred() { return transferred; } + @Override + public EncryptedMessage touch(Object o) { + super.touch(o); + if (buf != null) { + buf.touch(o); + } + if (region != null) { + region.touch(o); + } + return this; + } + + @Override + public EncryptedMessage retain(int increment) { + super.retain(increment); + if (buf != null) { + buf.retain(increment); + } + if (region != null) { + region.retain(increment); + } + return this; + } + + @Override + public boolean release(int decrement) { + if (region != null) { + region.release(decrement); + } + if (buf != null) { + buf.release(decrement); + } + return super.release(decrement); + } + /** * Transfers data from the original message to the channel, encrypting it in the process. * diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java b/common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java new file mode 100644 index 0000000000000..8651297d97ec2 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java @@ -0,0 +1,53 @@ +/* + * 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.network.util; + +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; + +public abstract class AbstractFileRegion extends AbstractReferenceCounted implements FileRegion { + + @Override + @SuppressWarnings("deprecation") + public final long transfered() { + return transferred(); + } + + @Override + public AbstractFileRegion retain() { + super.retain(); + return this; + } + + @Override + public AbstractFileRegion retain(int increment) { + super.retain(increment); + return this; + } + + @Override + public AbstractFileRegion touch() { + super.touch(); + return this; + } + + @Override + public AbstractFileRegion touch(Object o) { + return this; + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java index bb1c40c4b0e06..bc94f7ca63a96 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java @@ -56,7 +56,7 @@ private void testServerToClient(Message msg) { NettyUtils.createFrameDecoder(), MessageDecoder.INSTANCE); while (!serverChannel.outboundMessages().isEmpty()) { - clientChannel.writeInbound(serverChannel.readOutbound()); + clientChannel.writeOneInbound(serverChannel.readOutbound()); } assertEquals(1, clientChannel.inboundMessages().size()); @@ -72,7 +72,7 @@ private void testClientToServer(Message msg) { NettyUtils.createFrameDecoder(), MessageDecoder.INSTANCE); while (!clientChannel.outboundMessages().isEmpty()) { - serverChannel.writeInbound(clientChannel.readOutbound()); + serverChannel.writeOneInbound(clientChannel.readOutbound()); } assertEquals(1, serverChannel.inboundMessages().size()); diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java index b341c5681e00c..ecb66fcf2ff76 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -23,8 +23,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; +import org.apache.spark.network.util.AbstractFileRegion; import org.junit.Test; import org.mockito.Mockito; @@ -108,7 +107,7 @@ private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exc return Unpooled.wrappedBuffer(channel.getData()); } - private static class TestFileRegion extends AbstractReferenceCounted implements FileRegion { + private static class TestFileRegion extends AbstractFileRegion { private final int writeCount; private final int writesPerCall; @@ -130,7 +129,7 @@ public long position() { } @Override - public long transfered() { + public long transferred() { return 8 * written; } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 92e13ce1ba042..fcbeddd2a9ac3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -53,7 +53,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.local.LocalSchedulerBackend -import org.apache.spark.status.{AppStatusPlugin, AppStatusStore} +import org.apache.spark.status.AppStatusStore import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} @@ -416,7 +416,8 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. - _statusStore = AppStatusStore.createLiveStore(conf, l => listenerBus.addToStatusQueue(l)) + _statusStore = AppStatusStore.createLiveStore(conf) + listenerBus.addToStatusQueue(_statusStore.listener.get) // Create the Spark execution environment (cache, map output tracker, etc) _env = createSparkEnv(_conf, isLocal, listenerBus) @@ -445,14 +446,9 @@ class SparkContext(config: SparkConf) extends Logging { // For tests, do not enable the UI None } - _ui.foreach { ui => - // Load any plugins that might want to modify the UI. - AppStatusPlugin.loadPlugins().foreach(_.setupUI(ui)) - - // Bind the UI before starting the task scheduler to communicate - // the bound port to the cluster manager properly - ui.bind() - } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + _ui.foreach(_.bind()) _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ab834bb682041..cbe1f2c3e08a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1271,7 +1271,7 @@ private[spark] object SparkSubmitUtils { // retrieve all resolved dependencies ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId, packagesDirectory.getAbsolutePath + File.separator + - "[organization]_[artifact]-[revision].[ext]", + "[organization]_[artifact]-[revision](-[classifier]).[ext]", retrieveOptions.setConfs(Array(ivyConfName))) resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory) } finally { 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 fa2c5194aa41b..a299b79850613 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -44,7 +44,6 @@ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.status._ import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} -import org.apache.spark.status.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -322,15 +321,18 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) (new InMemoryStore(), true) } + val plugins = ServiceLoader.load( + classOf[AppHistoryServerPlugin], Utils.getContextOrSparkClassLoader).asScala val trackingStore = new ElementTrackingStore(kvstore, conf) if (needReplay) { val replayBus = new ReplayListenerBus() val listener = new AppStatusListener(trackingStore, conf, false, lastUpdateTime = Some(attempt.info.lastUpdated.getTime())) replayBus.addListener(listener) - AppStatusPlugin.loadPlugins().foreach { plugin => - plugin.setupListeners(conf, trackingStore, l => replayBus.addListener(l), false) - } + for { + plugin <- plugins + listener <- plugin.createListeners(conf, trackingStore) + } replayBus.addListener(listener) try { val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) @@ -353,9 +355,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - AppStatusPlugin.loadPlugins().foreach { plugin => - plugin.setupUI(ui) - } + plugins.foreach(_.setupUI(ui)) val loadedUI = LoadedAppUI(ui) diff --git a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala new file mode 100644 index 0000000000000..d144a0e998fa1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.ui.SparkUI + +/** + * An interface for creating history listeners(to replay event logs) defined in other modules like + * SQL, and setup the UI of the plugin to rebuild the history UI. + */ +private[spark] trait AppHistoryServerPlugin { + /** + * Creates listeners to replay the event logs. + */ + def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] + + /** + * Sets up UI of this plugin to rebuild the history UI. + */ + def setupUI(ui: SparkUI): Unit +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 1fb7b76d43d04..4db797e1d24c6 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -48,7 +48,7 @@ private[spark] class AppStatusListener( import config._ - private var sparkVersion = SPARK_VERSION + private val sparkVersion = SPARK_VERSION private var appInfo: v1.ApplicationInfo = null private var appSummary = new AppSummary(0, 0) private var coresPerTask: Int = 1 @@ -329,10 +329,6 @@ private[spark] class AppStatusListener( .toSeq stage.jobIds = stage.jobs.map(_.jobId).toSet - stage.schedulingPool = Option(event.properties).flatMap { p => - Option(p.getProperty("spark.scheduler.pool")) - }.getOrElse(SparkUI.DEFAULT_POOL_NAME) - stage.description = Option(event.properties).flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala deleted file mode 100644 index 4cada5c7b0de4..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala +++ /dev/null @@ -1,71 +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.status - -import java.util.ServiceLoader - -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkConf -import org.apache.spark.scheduler.SparkListener -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils -import org.apache.spark.util.kvstore.KVStore - -/** - * An interface that defines plugins for collecting and storing application state. - * - * The plugin implementations are invoked for both live and replayed applications. For live - * applications, it's recommended that plugins defer creation of UI tabs until there's actual - * data to be shown. - */ -private[spark] trait AppStatusPlugin { - - /** - * Install listeners to collect data about the running application and populate the given - * store. - * - * @param conf The Spark configuration. - * @param store The KVStore where to keep application data. - * @param addListenerFn Function to register listeners with a bus. - * @param live Whether this is a live application (or an application being replayed by the - * HistoryServer). - */ - def setupListeners( - conf: SparkConf, - store: ElementTrackingStore, - addListenerFn: SparkListener => Unit, - live: Boolean): Unit - - /** - * Install any needed extensions (tabs, pages, etc) to a Spark UI. The plugin can detect whether - * the app is live or replayed by looking at the UI's SparkContext field `sc`. - * - * @param ui The Spark UI instance for the application. - */ - def setupUI(ui: SparkUI): Unit - -} - -private[spark] object AppStatusPlugin { - - def loadPlugins(): Iterable[AppStatusPlugin] = { - ServiceLoader.load(classOf[AppStatusPlugin], Utils.getContextOrSparkClassLoader).asScala - } - -} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 9987419b170f6..5a942f5284018 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -17,16 +17,14 @@ package org.apache.spark.status -import java.io.File -import java.util.{Arrays, List => JList} +import java.util.{List => JList} import scala.collection.JavaConverters._ import org.apache.spark.{JobExecutionStatus, SparkConf} -import org.apache.spark.scheduler.SparkListener import org.apache.spark.status.api.v1 import org.apache.spark.ui.scope._ -import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.util.Distribution import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} /** @@ -34,7 +32,7 @@ import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} */ private[spark] class AppStatusStore( val store: KVStore, - listener: Option[AppStatusListener] = None) { + val listener: Option[AppStatusListener] = None) { def applicationInfo(): v1.ApplicationInfo = { store.view(classOf[ApplicationInfoWrapper]).max(1).iterator().next().info @@ -346,17 +344,10 @@ private[spark] object AppStatusStore { /** * Create an in-memory store for a live application. - * - * @param conf Configuration. - * @param addListenerFn Function to register a listener with a bus. */ - def createLiveStore(conf: SparkConf, addListenerFn: SparkListener => Unit): AppStatusStore = { + def createLiveStore(conf: SparkConf): AppStatusStore = { val store = new ElementTrackingStore(new InMemoryStore(), conf) val listener = new AppStatusListener(store, conf, true) - addListenerFn(listener) - AppStatusPlugin.loadPlugins().foreach { p => - p.setupListeners(conf, store, addListenerFn, true) - } new AppStatusStore(store, listener = Some(listener)) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 97abd92d4b70f..39249d411b582 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -26,12 +26,11 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ListBuffer import com.google.common.io.Closeables -import io.netty.channel.{DefaultFileRegion, FileRegion} -import io.netty.util.AbstractReferenceCounted +import io.netty.channel.DefaultFileRegion import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils +import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils} import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.util.Utils import org.apache.spark.util.io.ChunkedByteBuffer @@ -266,7 +265,7 @@ private class EncryptedBlockData( } private class ReadableChannelFileRegion(source: ReadableByteChannel, blockSize: Long) - extends AbstractReferenceCounted with FileRegion { + extends AbstractFileRegion { private var _transferred = 0L @@ -277,7 +276,7 @@ private class ReadableChannelFileRegion(source: ReadableByteChannel, blockSize: override def position(): Long = 0 - override def transfered(): Long = _transferred + override def transferred(): Long = _transferred override def transferTo(target: WritableByteChannel, pos: Long): Long = { assert(pos == transfered(), "Invalid position.") diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 46932a02f1a1b..661d0d48d2f37 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -22,7 +22,6 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.mockito.Matchers.anyString import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ @@ -30,7 +29,6 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.ui.jobs.{StagePage, StagesTab} -import org.apache.spark.util.Utils class StagePageSuite extends SparkFunSuite with LocalSparkContext { @@ -55,12 +53,12 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { * This also runs a dummy stage to populate the page with useful content. */ private def renderStagePage(conf: SparkConf): Seq[Node] = { - val bus = new ReplayListenerBus() - val store = AppStatusStore.createLiveStore(conf, l => bus.addListener(l)) + val statusStore = AppStatusStore.createLiveStore(conf) + val listener = statusStore.listener.get try { val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) - when(tab.store).thenReturn(store) + when(tab.store).thenReturn(statusStore) val request = mock(classOf[HttpServletRequest]) when(tab.conf).thenReturn(conf) @@ -68,7 +66,7 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { when(tab.headerTabs).thenReturn(Seq.empty) when(request.getParameter("id")).thenReturn("0") when(request.getParameter("attempt")).thenReturn("0") - val page = new StagePage(tab, store) + val page = new StagePage(tab, statusStore) // Simulate a stage in job progress listener val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") @@ -77,17 +75,17 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - bus.postToAll(SparkListenerStageSubmitted(stageInfo)) - bus.postToAll(SparkListenerTaskStart(0, 0, taskInfo)) + listener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) + listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) val taskMetrics = TaskMetrics.empty taskMetrics.incPeakExecutionMemory(peakExecutionMemory) - bus.postToAll(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) } - bus.postToAll(SparkListenerStageCompleted(stageInfo)) + listener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) page.render(request) } finally { - store.close() + statusStore.close() } } diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 1831f3378e852..fea642737cc0c 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -14,9 +14,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -arrow-format-0.4.0.jar -arrow-memory-0.4.0.jar -arrow-vector-0.4.0.jar +arrow-format-0.8.0.jar +arrow-memory-0.8.0.jar +arrow-vector-0.8.0.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -82,7 +82,7 @@ hadoop-yarn-server-web-proxy-2.6.5.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -hppc-0.7.1.jar +hppc-0.7.2.jar htrace-core-3.0.4.jar httpclient-4.5.2.jar httpcore-4.4.4.jar @@ -144,7 +144,7 @@ metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar netty-3.9.9.Final.jar -netty-all-4.0.47.Final.jar +netty-all-4.1.17.Final.jar objenesis-2.1.jar opencsv-2.3.jar orc-core-1.4.1-nohive.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index fe14c05987327..6dd44333f21ca 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -14,9 +14,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -arrow-format-0.4.0.jar -arrow-memory-0.4.0.jar -arrow-vector-0.4.0.jar +arrow-format-0.8.0.jar +arrow-memory-0.8.0.jar +arrow-vector-0.8.0.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar @@ -82,7 +82,7 @@ hadoop-yarn-server-web-proxy-2.7.3.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar -hppc-0.7.1.jar +hppc-0.7.2.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.2.jar httpcore-4.4.4.jar @@ -145,7 +145,7 @@ metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar netty-3.9.9.Final.jar -netty-all-4.0.47.Final.jar +netty-all-4.1.17.Final.jar objenesis-2.1.jar opencsv-2.3.jar orc-core-1.4.1-nohive.jar diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 25718f904cc49..3311de12dbd97 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -42,7 +42,7 @@ object BroadcastTest { val arr1 = (0 until num).toArray for (i <- 0 until 3) { - println("Iteration " + i) + println(s"Iteration $i") println("===========") val startTime = System.nanoTime val barr1 = sc.broadcast(arr1) diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 3bff7ce736d08..1a779716ec4c0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -49,12 +49,10 @@ object DFSReadWriteTest { } private def printUsage(): Unit = { - val usage: String = "DFS Read-Write Test\n" + - "\n" + - "Usage: localFile dfsDir\n" + - "\n" + - "localFile - (string) local file to use in test\n" + - "dfsDir - (string) DFS directory for read/write tests\n" + val usage = """DFS Read-Write Test + |Usage: localFile dfsDir + |localFile - (string) local file to use in test + |dfsDir - (string) DFS directory for read/write tests""".stripMargin println(usage) } @@ -69,13 +67,13 @@ object DFSReadWriteTest { localFilePath = new File(args(i)) if (!localFilePath.exists) { - System.err.println("Given path (" + args(i) + ") does not exist.\n") + System.err.println(s"Given path (${args(i)}) does not exist") printUsage() System.exit(1) } if (!localFilePath.isFile) { - System.err.println("Given path (" + args(i) + ") is not a file.\n") + System.err.println(s"Given path (${args(i)}) is not a file") printUsage() System.exit(1) } @@ -108,7 +106,7 @@ object DFSReadWriteTest { .getOrCreate() println("Writing local file to DFS") - val dfsFilename = dfsDirPath + "/dfs_read_write_test" + val dfsFilename = s"$dfsDirPath/dfs_read_write_test" val fileRDD = spark.sparkContext.parallelize(fileContents) fileRDD.saveAsTextFile(dfsFilename) @@ -127,11 +125,11 @@ object DFSReadWriteTest { spark.stop() if (localWordCount == dfsWordCount) { - println(s"Success! Local Word Count ($localWordCount) " + - s"and DFS Word Count ($dfsWordCount) agree.") + println(s"Success! Local Word Count $localWordCount and " + + s"DFS Word Count $dfsWordCount agree.") } else { - println(s"Failure! Local Word Count ($localWordCount) " + - s"and DFS Word Count ($dfsWordCount) disagree.") + println(s"Failure! Local Word Count $localWordCount " + + s"and DFS Word Count $dfsWordCount disagree.") } } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index aa8de69839e28..e1f985ece8c06 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -39,7 +39,7 @@ object HdfsTest { val start = System.currentTimeMillis() for (x <- mapped) { x + 2 } val end = System.currentTimeMillis() - println("Iteration " + iter + " took " + (end-start) + " ms") + println(s"Iteration $iter took ${end-start} ms") } spark.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 97aefac025e55..3f9cea35d6503 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -129,8 +129,7 @@ object LocalALS { println(s"Iteration $iter:") ms = (0 until M).map(i => updateMovie(i, ms(i), us, R)).toArray us = (0 until U).map(j => updateUser(j, us(j), ms, R)).toArray - println("RMSE = " + rmse(R, ms, us)) - println() + println(s"RMSE = ${rmse(R, ms, us)}") } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 8dbb7ee4e5307..5512e33e41ac3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -58,10 +58,10 @@ object LocalFileLR { // Initialize w to a random value val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} - println("Initial w: " + w) + println(s"Initial w: $w") for (i <- 1 to ITERATIONS) { - println("On iteration " + i) + println(s"On iteration $i") val gradient = DenseVector.zeros[Double](D) for (p <- points) { val scale = (1 / (1 + math.exp(-p.y * (w.dot(p.x)))) - 1) * p.y @@ -71,7 +71,7 @@ object LocalFileLR { } fileSrc.close() - println("Final w: " + w) + println(s"Final w: $w") } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index 963c9a56d6cac..f5162a59522f0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -88,7 +88,7 @@ object LocalKMeans { kPoints.put(i, iter.next()) } - println("Initial centers: " + kPoints) + println(s"Initial centers: $kPoints") while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) @@ -114,7 +114,7 @@ object LocalKMeans { } } - println("Final centers: " + kPoints) + println(s"Final centers: $kPoints") } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index eb5221f085937..bde8ccd305960 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -61,10 +61,10 @@ object LocalLR { val data = generateData // Initialize w to a random value val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} - println("Initial w: " + w) + println(s"Initial w: $w") for (i <- 1 to ITERATIONS) { - println("On iteration " + i) + println(s"On iteration $i") val gradient = DenseVector.zeros[Double](D) for (p <- data) { val scale = (1 / (1 + math.exp(-p.y * (w.dot(p.x)))) - 1) * p.y @@ -73,7 +73,7 @@ object LocalLR { w -= gradient } - println("Final w: " + w) + println(s"Final w: $w") } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index 121b768e4198e..a93c15c85cfc1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -28,7 +28,7 @@ object LocalPi { val y = random * 2 - 1 if (x*x + y*y <= 1) count += 1 } - println("Pi is roughly " + 4 * count / 100000.0) + println(s"Pi is roughly ${4 * count / 100000.0}") } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 8e1a574c92221..e64dcbd182d94 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -59,7 +59,7 @@ object SimpleSkewedGroupByTest { // Enforce that everything has been calculated and in cache pairs1.count - println("RESULT: " + pairs1.groupByKey(numReducers).count) + println(s"RESULT: ${pairs1.groupByKey(numReducers).count}") // Print how many keys each reducer got (for debugging) // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index a99ddd9fd37db..d3e7b7a967de7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -135,10 +135,8 @@ object SparkALS { .map(i => update(i, usb.value(i), msb.value, Rc.value.transpose())) .collect() usb = sc.broadcast(us) // Re-broadcast us because it was updated - println("RMSE = " + rmse(R, ms, us)) - println() + println(s"RMSE = ${rmse(R, ms, us)}") } - spark.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 9d675bbc18f38..23eaa879114a9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -79,17 +79,17 @@ object SparkHdfsLR { // Initialize w to a random value val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} - println("Initial w: " + w) + println(s"Initial w: $w") for (i <- 1 to ITERATIONS) { - println("On iteration " + i) + println(s"On iteration $i") val gradient = points.map { p => p.x * (1 / (1 + exp(-p.y * (w.dot(p.x)))) - 1) * p.y }.reduce(_ + _) w -= gradient } - println("Final w: " + w) + println(s"Final w: $w") spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index fec3160e9f37b..b005cb6971c16 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -95,7 +95,7 @@ object SparkKMeans { for (newP <- newPoints) { kPoints(newP._1) = newP._2 } - println("Finished iteration (delta = " + tempDist + ")") + println(s"Finished iteration (delta = $tempDist)") } println("Final centers:") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index c18e3d31f149e..4b1497345af82 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -73,17 +73,17 @@ object SparkLR { // Initialize w to a random value val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} - println("Initial w: " + w) + println(s"Initial w: $w") for (i <- 1 to ITERATIONS) { - println("On iteration " + i) + println(s"On iteration $i") val gradient = points.map { p => p.x * (1 / (1 + exp(-p.y * (w.dot(p.x)))) - 1) * p.y }.reduce(_ + _) w -= gradient } - println("Final w: " + w) + println(s"Final w: $w") spark.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 5d8831265e4ad..9299bad5d3290 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -77,7 +77,7 @@ object SparkPageRank { } val output = ranks.collect() - output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) + output.foreach(tup => println(s"${tup._1} has rank: ${tup._2} .")) spark.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index a5cacf17a5cca..828d98b5001d7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -36,7 +36,7 @@ object SparkPi { val y = random * 2 - 1 if (x*x + y*y <= 1) 1 else 0 }.reduce(_ + _) - println("Pi is roughly " + 4.0 * count / (n - 1)) + println(s"Pi is roughly ${4.0 * count / (n - 1)}") spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 558295ab928af..f5d42141f5dd2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -68,7 +68,7 @@ object SparkTC { nextCount = tc.count() } while (nextCount != oldCount) - println("TC has " + tc.count() + " edges.") + println(s"TC has ${tc.count()} edges.") spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 619e585b6ca17..92936bd30dbc0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -27,6 +27,7 @@ import org.apache.spark.graphx.lib._ import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel + /** * Driver program for running graph algorithms. */ @@ -34,12 +35,12 @@ object Analytics extends Logging { def main(args: Array[String]): Unit = { if (args.length < 2) { - System.err.println( - "Usage: Analytics --numEPart= [other options]") - System.err.println("Supported 'taskType' as follows:") - System.err.println(" pagerank Compute PageRank") - System.err.println(" cc Compute the connected components of vertices") - System.err.println(" triangles Count the number of triangles") + val usage = """Usage: Analytics --numEPart= + |[other options] Supported 'taskType' as follows: + |pagerank Compute PageRank + |cc Compute the connected components of vertices + |triangles Count the number of triangles""".stripMargin + System.err.println(usage) System.exit(1) } @@ -48,7 +49,7 @@ object Analytics extends Logging { val optionsList = args.drop(2).map { arg => arg.dropWhile(_ == '-').split('=') match { case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + case _ => throw new IllegalArgumentException(s"Invalid argument: $arg") } } val options = mutable.Map(optionsList: _*) @@ -74,14 +75,14 @@ object Analytics extends Logging { val numIterOpt = options.remove("numIter").map(_.toInt) options.foreach { - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + case (opt, _) => throw new IllegalArgumentException(s"Invalid option: $opt") } println("======================================") println("| PageRank |") println("======================================") - val sc = new SparkContext(conf.setAppName("PageRank(" + fname + ")")) + val sc = new SparkContext(conf.setAppName(s"PageRank($fname)")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, numEdgePartitions = numEPart, @@ -89,18 +90,18 @@ object Analytics extends Logging { vertexStorageLevel = vertexStorageLevel).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - println("GRAPHX: Number of vertices " + graph.vertices.count) - println("GRAPHX: Number of edges " + graph.edges.count) + println(s"GRAPHX: Number of vertices ${graph.vertices.count}") + println(s"GRAPHX: Number of edges ${graph.edges.count}") val pr = (numIterOpt match { case Some(numIter) => PageRank.run(graph, numIter) case None => PageRank.runUntilConvergence(graph, tol) }).vertices.cache() - println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_ + _)) + println(s"GRAPHX: Total rank: ${pr.map(_._2).reduce(_ + _)}") if (!outFname.isEmpty) { - logWarning("Saving pageranks of pages to " + outFname) + logWarning(s"Saving pageranks of pages to $outFname") pr.map { case (id, r) => id + "\t" + r }.saveAsTextFile(outFname) } @@ -108,14 +109,14 @@ object Analytics extends Logging { case "cc" => options.foreach { - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + case (opt, _) => throw new IllegalArgumentException(s"Invalid option: $opt") } println("======================================") println("| Connected Components |") println("======================================") - val sc = new SparkContext(conf.setAppName("ConnectedComponents(" + fname + ")")) + val sc = new SparkContext(conf.setAppName(s"ConnectedComponents($fname)")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, numEdgePartitions = numEPart, edgeStorageLevel = edgeStorageLevel, @@ -123,19 +124,19 @@ object Analytics extends Logging { val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map { case (vid, data) => data }.distinct()) + println(s"Components: ${cc.vertices.map { case (vid, data) => data }.distinct()}") sc.stop() case "triangles" => options.foreach { - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + case (opt, _) => throw new IllegalArgumentException(s"Invalid option: $opt") } println("======================================") println("| Triangle Count |") println("======================================") - val sc = new SparkContext(conf.setAppName("TriangleCount(" + fname + ")")) + val sc = new SparkContext(conf.setAppName(s"TriangleCount($fname)")) val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, numEdgePartitions = numEPart, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 87f31fcc20ae6..e9cff04ba5f2e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 6e24423df4abc..b5da415b3097e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.execution.streaming.SerializedOffset -import org.apache.spark.sql.sources.v2.reader.Offset +import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 9cac0e5ae7117..2034b9be07f24 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala index cae41edb7aca8..9bed74a9f2c05 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes, UnsafeArrayData} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, TypedImperativeAggregate} import org.apache.spark.sql.functions.lit import org.apache.spark.sql.types._ @@ -41,7 +41,7 @@ sealed abstract class SummaryBuilder { /** * Returns an aggregate object that contains the summary of the column with the requested metrics. * @param featuresCol a column that contains features Vector object. - * @param weightCol a column that contains weight value. + * @param weightCol a column that contains weight value. Default weight is 1.0. * @return an aggregate column that contains the statistics. The exact content of this * structure is determined during the creation of the builder. */ @@ -50,6 +50,7 @@ sealed abstract class SummaryBuilder { @Since("2.3.0") def summary(featuresCol: Column): Column = summary(featuresCol, lit(1.0)) + } /** @@ -60,15 +61,18 @@ sealed abstract class SummaryBuilder { * This class lets users pick the statistics they would like to extract for a given column. Here is * an example in Scala: * {{{ - * val dataframe = ... // Some dataframe containing a feature column - * val allStats = dataframe.select(Summarizer.metrics("min", "max").summary($"features")) - * val Row(Row(min_, max_)) = allStats.first() + * import org.apache.spark.ml.linalg._ + * import org.apache.spark.sql.Row + * val dataframe = ... // Some dataframe containing a feature column and a weight column + * val multiStatsDF = dataframe.select( + * Summarizer.metrics("min", "max", "count").summary($"features", $"weight") + * val Row(Row(minVec, maxVec, count)) = multiStatsDF.first() * }}} * * If one wants to get a single metric, shortcuts are also available: * {{{ * val meanDF = dataframe.select(Summarizer.mean($"features")) - * val Row(mean_) = meanDF.first() + * val Row(meanVec) = meanDF.first() * }}} * * Note: Currently, the performance of this interface is about 2x~3x slower then using the RDD @@ -94,8 +98,7 @@ object Summarizer extends Logging { * - min: the minimum for each coefficient. * - normL2: the Euclidian norm for each coefficient. * - normL1: the L1 norm of each coefficient (sum of the absolute values). - * @param firstMetric the metric being provided - * @param metrics additional metrics that can be provided. + * @param metrics metrics that can be provided. * @return a builder. * @throws IllegalArgumentException if one of the metric names is not understood. * @@ -103,37 +106,79 @@ object Summarizer extends Logging { * interface. */ @Since("2.3.0") - def metrics(firstMetric: String, metrics: String*): SummaryBuilder = { - val (typedMetrics, computeMetrics) = getRelevantMetrics(Seq(firstMetric) ++ metrics) + @scala.annotation.varargs + def metrics(metrics: String*): SummaryBuilder = { + require(metrics.size >= 1, "Should include at least one metric") + val (typedMetrics, computeMetrics) = getRelevantMetrics(metrics) new SummaryBuilderImpl(typedMetrics, computeMetrics) } @Since("2.3.0") - def mean(col: Column): Column = getSingleMetric(col, "mean") + def mean(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "mean") + } + + @Since("2.3.0") + def mean(col: Column): Column = mean(col, lit(1.0)) + + @Since("2.3.0") + def variance(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "variance") + } + + @Since("2.3.0") + def variance(col: Column): Column = variance(col, lit(1.0)) + + @Since("2.3.0") + def count(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "count") + } + + @Since("2.3.0") + def count(col: Column): Column = count(col, lit(1.0)) @Since("2.3.0") - def variance(col: Column): Column = getSingleMetric(col, "variance") + def numNonZeros(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "numNonZeros") + } + + @Since("2.3.0") + def numNonZeros(col: Column): Column = numNonZeros(col, lit(1.0)) + + @Since("2.3.0") + def max(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "max") + } + + @Since("2.3.0") + def max(col: Column): Column = max(col, lit(1.0)) @Since("2.3.0") - def count(col: Column): Column = getSingleMetric(col, "count") + def min(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "min") + } @Since("2.3.0") - def numNonZeros(col: Column): Column = getSingleMetric(col, "numNonZeros") + def min(col: Column): Column = min(col, lit(1.0)) @Since("2.3.0") - def max(col: Column): Column = getSingleMetric(col, "max") + def normL1(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "normL1") + } @Since("2.3.0") - def min(col: Column): Column = getSingleMetric(col, "min") + def normL1(col: Column): Column = normL1(col, lit(1.0)) @Since("2.3.0") - def normL1(col: Column): Column = getSingleMetric(col, "normL1") + def normL2(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "normL2") + } @Since("2.3.0") - def normL2(col: Column): Column = getSingleMetric(col, "normL2") + def normL2(col: Column): Column = normL2(col, lit(1.0)) - private def getSingleMetric(col: Column, metric: String): Column = { - val c1 = metrics(metric).summary(col) + private def getSingleMetric(col: Column, weightCol: Column, metric: String): Column = { + val c1 = metrics(metric).summary(col, weightCol) c1.getField(metric).as(s"$metric($col)") } } @@ -187,8 +232,7 @@ private[ml] object SummaryBuilderImpl extends Logging { StructType(fields) } - private val arrayDType = ArrayType(DoubleType, containsNull = false) - private val arrayLType = ArrayType(LongType, containsNull = false) + private val vectorUDT = new VectorUDT /** * All the metrics that can be currently computed by Spark for vectors. @@ -197,14 +241,14 @@ private[ml] object SummaryBuilderImpl extends Logging { * metrics that need to de computed internally to get the final result. */ private val allMetrics: Seq[(String, Metric, DataType, Seq[ComputeMetric])] = Seq( - ("mean", Mean, arrayDType, Seq(ComputeMean, ComputeWeightSum)), - ("variance", Variance, arrayDType, Seq(ComputeWeightSum, ComputeMean, ComputeM2n)), + ("mean", Mean, vectorUDT, Seq(ComputeMean, ComputeWeightSum)), + ("variance", Variance, vectorUDT, Seq(ComputeWeightSum, ComputeMean, ComputeM2n)), ("count", Count, LongType, Seq()), - ("numNonZeros", NumNonZeros, arrayLType, Seq(ComputeNNZ)), - ("max", Max, arrayDType, Seq(ComputeMax, ComputeNNZ)), - ("min", Min, arrayDType, Seq(ComputeMin, ComputeNNZ)), - ("normL2", NormL2, arrayDType, Seq(ComputeM2)), - ("normL1", NormL1, arrayDType, Seq(ComputeL1)) + ("numNonZeros", NumNonZeros, vectorUDT, Seq(ComputeNNZ)), + ("max", Max, vectorUDT, Seq(ComputeMax, ComputeNNZ)), + ("min", Min, vectorUDT, Seq(ComputeMin, ComputeNNZ)), + ("normL2", NormL2, vectorUDT, Seq(ComputeM2)), + ("normL1", NormL1, vectorUDT, Seq(ComputeL1)) ) /** @@ -527,27 +571,28 @@ private[ml] object SummaryBuilderImpl extends Logging { weightExpr: Expression, mutableAggBufferOffset: Int, inputAggBufferOffset: Int) - extends TypedImperativeAggregate[SummarizerBuffer] { + extends TypedImperativeAggregate[SummarizerBuffer] with ImplicitCastInputTypes { - override def eval(state: SummarizerBuffer): InternalRow = { + override def eval(state: SummarizerBuffer): Any = { val metrics = requestedMetrics.map { - case Mean => UnsafeArrayData.fromPrimitiveArray(state.mean.toArray) - case Variance => UnsafeArrayData.fromPrimitiveArray(state.variance.toArray) + case Mean => vectorUDT.serialize(state.mean) + case Variance => vectorUDT.serialize(state.variance) case Count => state.count - case NumNonZeros => UnsafeArrayData.fromPrimitiveArray( - state.numNonzeros.toArray.map(_.toLong)) - case Max => UnsafeArrayData.fromPrimitiveArray(state.max.toArray) - case Min => UnsafeArrayData.fromPrimitiveArray(state.min.toArray) - case NormL2 => UnsafeArrayData.fromPrimitiveArray(state.normL2.toArray) - case NormL1 => UnsafeArrayData.fromPrimitiveArray(state.normL1.toArray) + case NumNonZeros => vectorUDT.serialize(state.numNonzeros) + case Max => vectorUDT.serialize(state.max) + case Min => vectorUDT.serialize(state.min) + case NormL2 => vectorUDT.serialize(state.normL2) + case NormL1 => vectorUDT.serialize(state.normL1) } InternalRow.apply(metrics: _*) } + override def inputTypes: Seq[DataType] = vectorUDT :: DoubleType :: Nil + override def children: Seq[Expression] = featuresExpr :: weightExpr :: Nil override def update(state: SummarizerBuffer, row: InternalRow): SummarizerBuffer = { - val features = udt.deserialize(featuresExpr.eval(row)) + val features = vectorUDT.deserialize(featuresExpr.eval(row)) val weight = weightExpr.eval(row).asInstanceOf[Double] state.add(features, weight) state @@ -591,7 +636,4 @@ private[ml] object SummaryBuilderImpl extends Logging { override def prettyName: String = "aggregate_metrics" } - - private[this] val udt = new VectorUDT - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index ae98e24a75681..9b9c70cfe5109 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -197,7 +197,9 @@ class BisectingKMeans private ( newClusters = summarize(d, newAssignments) newClusterCenters = newClusters.mapValues(_.center).map(identity) } - if (preIndices != null) preIndices.unpersist() + if (preIndices != null) { + preIndices.unpersist(false) + } preIndices = indices indices = updateAssignments(assignments, divisibleIndices, newClusterCenters).keys .persist(StorageLevel.MEMORY_AND_DISK) @@ -212,7 +214,13 @@ class BisectingKMeans private ( } level += 1 } - if(indices != null) indices.unpersist() + if (preIndices != null) { + preIndices.unpersist(false) + } + if (indices != null) { + indices.unpersist(false) + } + norms.unpersist(false) val clusters = activeClusters ++ inactiveClusters val root = buildTree(clusters) new BisectingKMeansModel(root) diff --git a/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java new file mode 100644 index 0000000000000..38ab39aa0f492 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java @@ -0,0 +1,64 @@ +/* + * 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.ml.stat; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Dataset; +import static org.apache.spark.sql.functions.col; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; + +public class JavaSummarizerSuite extends SharedSparkSession { + + private transient Dataset dataset; + + @Override + public void setUp() throws IOException { + super.setUp(); + List points = new ArrayList(); + points.add(new LabeledPoint(0.0, Vectors.dense(1.0, 2.0))); + points.add(new LabeledPoint(0.0, Vectors.dense(3.0, 4.0))); + + dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); + } + + @Test + public void testSummarizer() { + dataset.select(col("features")); + Row result = dataset + .select(Summarizer.metrics("mean", "max", "count").summary(col("features"))) + .first().getStruct(0); + Vector meanVec = result.getAs("mean"); + Vector maxVec = result.getAs("max"); + long count = result.getAs("count"); + + assertEquals(2L, count); + assertArrayEquals(new double[]{2.0, 3.0}, meanVec.toArray(), 0.0); + assertArrayEquals(new double[]{3.0, 4.0}, maxVec.toArray(), 0.0); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala index 1ea851ef2d676..5e4f402989697 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml.stat -import org.scalatest.exceptions.TestFailedException - import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, Statistics} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.Row class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -35,237 +32,262 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { import SummaryBuilderImpl._ private case class ExpectedMetrics( - mean: Seq[Double], - variance: Seq[Double], + mean: Vector, + variance: Vector, count: Long, - numNonZeros: Seq[Long], - max: Seq[Double], - min: Seq[Double], - normL2: Seq[Double], - normL1: Seq[Double]) + numNonZeros: Vector, + max: Vector, + min: Vector, + normL2: Vector, + normL1: Vector) /** - * The input is expected to be either a sparse vector, a dense vector or an array of doubles - * (which will be converted to a dense vector) - * The expected is the list of all the known metrics. + * The input is expected to be either a sparse vector, a dense vector. * - * The tests take an list of input vectors and a list of all the summary values that - * are expected for this input. They currently test against some fixed subset of the - * metrics, but should be made fuzzy in the future. + * The tests take an list of input vectors, and compare results with + * `mllib.stat.MultivariateOnlineSummarizer`. They currently test against some fixed subset + * of the metrics, but should be made fuzzy in the future. */ - private def testExample(name: String, input: Seq[Any], exp: ExpectedMetrics): Unit = { + private def testExample(name: String, inputVec: Seq[(Vector, Double)], + exp: ExpectedMetrics, expWithoutWeight: ExpectedMetrics): Unit = { - def inputVec: Seq[Vector] = input.map { - case x: Array[Double @unchecked] => Vectors.dense(x) - case x: Seq[Double @unchecked] => Vectors.dense(x.toArray) - case x: Vector => x - case x => throw new Exception(x.toString) + val summarizer = { + val _summarizer = new MultivariateOnlineSummarizer + inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v._1), v._2)) + _summarizer } - val summarizer = { + val summarizerWithoutWeight = { val _summarizer = new MultivariateOnlineSummarizer - inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v))) + inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v._1))) _summarizer } // Because the Spark context is reset between tests, we cannot hold a reference onto it. def wrappedInit() = { - val df = inputVec.map(Tuple1.apply).toDF("features") - val col = df.col("features") - (df, col) + val df = inputVec.toDF("features", "weight") + val featuresCol = df.col("features") + val weightCol = df.col("weight") + (df, featuresCol, weightCol) } registerTest(s"$name - mean only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("mean").summary(c), mean(c)), Seq(Row(exp.mean), summarizer.mean)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("mean").summary(c, w), mean(c, w)).first(), + Row(Row(summarizer.mean), exp.mean)) } - registerTest(s"$name - mean only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(mean(c)), Seq(exp.mean)) + registerTest(s"$name - mean only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("mean").summary(c), mean(c)).first(), + Row(Row(summarizerWithoutWeight.mean), expWithoutWeight.mean)) } registerTest(s"$name - variance only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("variance").summary(c), variance(c)), - Seq(Row(exp.variance), summarizer.variance)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("variance").summary(c, w), variance(c, w)).first(), + Row(Row(summarizer.variance), exp.variance)) } - registerTest(s"$name - variance only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(variance(c)), Seq(summarizer.variance)) + registerTest(s"$name - variance only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("variance").summary(c), variance(c)).first(), + Row(Row(summarizerWithoutWeight.variance), expWithoutWeight.variance)) } registerTest(s"$name - count only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("count").summary(c), count(c)), - Seq(Row(exp.count), exp.count)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("count").summary(c, w), count(c, w)).first(), + Row(Row(summarizer.count), exp.count)) } - registerTest(s"$name - count only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(count(c)), - Seq(exp.count)) + registerTest(s"$name - count only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("count").summary(c), count(c)).first(), + Row(Row(summarizerWithoutWeight.count), expWithoutWeight.count)) } registerTest(s"$name - numNonZeros only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("numNonZeros").summary(c), numNonZeros(c)), - Seq(Row(exp.numNonZeros), exp.numNonZeros)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("numNonZeros").summary(c, w), numNonZeros(c, w)).first(), + Row(Row(summarizer.numNonzeros), exp.numNonZeros)) } - registerTest(s"$name - numNonZeros only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(numNonZeros(c)), - Seq(exp.numNonZeros)) + registerTest(s"$name - numNonZeros only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("numNonZeros").summary(c), numNonZeros(c)).first(), + Row(Row(summarizerWithoutWeight.numNonzeros), expWithoutWeight.numNonZeros)) } registerTest(s"$name - min only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("min").summary(c), min(c)), - Seq(Row(exp.min), exp.min)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("min").summary(c, w), min(c, w)).first(), + Row(Row(summarizer.min), exp.min)) + } + + registerTest(s"$name - min only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("min").summary(c), min(c)).first(), + Row(Row(summarizerWithoutWeight.min), expWithoutWeight.min)) } registerTest(s"$name - max only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("max").summary(c), max(c)), - Seq(Row(exp.max), exp.max)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("max").summary(c, w), max(c, w)).first(), + Row(Row(summarizer.max), exp.max)) } - registerTest(s"$name - normL1 only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("normL1").summary(c), normL1(c)), - Seq(Row(exp.normL1), exp.normL1)) + registerTest(s"$name - max only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("max").summary(c), max(c)).first(), + Row(Row(summarizerWithoutWeight.max), expWithoutWeight.max)) } - registerTest(s"$name - normL2 only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("normL2").summary(c), normL2(c)), - Seq(Row(exp.normL2), exp.normL2)) + registerTest(s"$name - normL1 only") { + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("normL1").summary(c, w), normL1(c, w)).first(), + Row(Row(summarizer.normL1), exp.normL1)) } - registerTest(s"$name - all metrics at once") { - val (df, c) = wrappedInit() - compare(df.select( - metrics("mean", "variance", "count", "numNonZeros").summary(c), - mean(c), variance(c), count(c), numNonZeros(c)), - Seq(Row(exp.mean, exp.variance, exp.count, exp.numNonZeros), - exp.mean, exp.variance, exp.count, exp.numNonZeros)) + registerTest(s"$name - normL1 only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("normL1").summary(c), normL1(c)).first(), + Row(Row(summarizerWithoutWeight.normL1), expWithoutWeight.normL1)) } - } - private def denseData(input: Seq[Seq[Double]]): DataFrame = { - input.map(_.toArray).map(Vectors.dense).map(Tuple1.apply).toDF("features") - } + registerTest(s"$name - normL2 only") { + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("normL2").summary(c, w), normL2(c, w)).first(), + Row(Row(summarizer.normL2), exp.normL2)) + } - private def compare(df: DataFrame, exp: Seq[Any]): Unit = { - val coll = df.collect().toSeq - val Seq(row) = coll - val res = row.toSeq - val names = df.schema.fieldNames.zipWithIndex.map { case (n, idx) => s"$n ($idx)" } - assert(res.size === exp.size, (res.size, exp.size)) - for (((x1, x2), name) <- res.zip(exp).zip(names)) { - compareStructures(x1, x2, name) + registerTest(s"$name - normL2 only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("normL2").summary(c), normL2(c)).first(), + Row(Row(summarizerWithoutWeight.normL2), expWithoutWeight.normL2)) } - } - // Compares structured content. - private def compareStructures(x1: Any, x2: Any, name: String): Unit = (x1, x2) match { - case (y1: Seq[Double @unchecked], v1: OldVector) => - compareStructures(y1, v1.toArray.toSeq, name) - case (d1: Double, d2: Double) => - assert2(Vectors.dense(d1) ~== Vectors.dense(d2) absTol 1e-4, name) - case (r1: GenericRowWithSchema, r2: Row) => - assert(r1.size === r2.size, (r1, r2)) - for (((fname, x1), x2) <- r1.schema.fieldNames.zip(r1.toSeq).zip(r2.toSeq)) { - compareStructures(x1, x2, s"$name.$fname") - } - case (r1: Row, r2: Row) => - assert(r1.size === r2.size, (r1, r2)) - for ((x1, x2) <- r1.toSeq.zip(r2.toSeq)) { compareStructures(x1, x2, name) } - case (v1: Vector, v2: Vector) => - assert2(v1 ~== v2 absTol 1e-4, name) - case (l1: Long, l2: Long) => assert(l1 === l2) - case (s1: Seq[_], s2: Seq[_]) => - assert(s1.size === s2.size, s"$name ${(s1, s2)}") - for (((x1, idx), x2) <- s1.zipWithIndex.zip(s2)) { - compareStructures(x1, x2, s"$name.$idx") - } - case (arr1: Array[_], arr2: Array[_]) => - assert(arr1.toSeq === arr2.toSeq) - case _ => throw new Exception(s"$name: ${x1.getClass} ${x2.getClass} $x1 $x2") - } + registerTest(s"$name - multiple metrics at once") { + val (df, c, w) = wrappedInit() + compareRow(df.select( + metrics("mean", "variance", "count", "numNonZeros").summary(c, w)).first(), + Row(Row(exp.mean, exp.variance, exp.count, exp.numNonZeros)) + ) + } - private def assert2(x: => Boolean, hint: String): Unit = { - try { - assert(x, hint) - } catch { - case tfe: TestFailedException => - throw new TestFailedException(Some(s"Failure with hint $hint"), Some(tfe), 1) + registerTest(s"$name - multiple metrics at once w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select( + metrics("mean", "variance", "count", "numNonZeros").summary(c)).first(), + Row(Row(expWithoutWeight.mean, expWithoutWeight.variance, + expWithoutWeight.count, expWithoutWeight.numNonZeros)) + ) } } - test("debugging test") { - val df = denseData(Nil) - val c = df.col("features") - val c1 = metrics("mean").summary(c) - val res = df.select(c1) - intercept[SparkException] { - compare(res, Seq.empty) + private def compareRow(r1: Row, r2: Row): Unit = { + assert(r1.size === r2.size, (r1, r2)) + r1.toSeq.zip(r2.toSeq).foreach { + case (v1: Vector, v2: Vector) => + assert(v1 ~== v2 absTol 1e-4) + case (v1: Vector, v2: OldVector) => + assert(v1 ~== v2.asML absTol 1e-4) + case (l1: Long, l2: Long) => + assert(l1 === l2) + case (r1: Row, r2: Row) => + compareRow(r1, r2) + case (x1: Any, x2: Any) => + throw new Exception(s"type mismatch: ${x1.getClass} ${x2.getClass} $x1 $x2") } } - test("basic error handling") { - val df = denseData(Nil) + test("no element") { + val df = Seq[Tuple1[Vector]]().toDF("features") val c = df.col("features") - val res = df.select(metrics("mean").summary(c), mean(c)) intercept[SparkException] { - compare(res, Seq.empty) + df.select(metrics("mean").summary(c), mean(c)).first() } + compareRow(df.select(metrics("count").summary(c), count(c)).first(), + Row(Row(0L), 0L)) } - test("no element, working metrics") { - val df = denseData(Nil) - val c = df.col("features") - val res = df.select(metrics("count").summary(c), count(c)) - compare(res, Seq(Row(0L), 0L)) - } + val singleElem = Vectors.dense(0.0, 1.0, 2.0) + testExample("single element", Seq((singleElem, 2.0)), + ExpectedMetrics( + mean = singleElem, + variance = Vectors.dense(0.0, 0.0, 0.0), + count = 1L, + numNonZeros = Vectors.dense(0.0, 1.0, 1.0), + max = singleElem, + min = singleElem, + normL1 = Vectors.dense(0.0, 2.0, 4.0), + normL2 = Vectors.dense(0.0, 1.414213, 2.828427) + ), + ExpectedMetrics( + mean = singleElem, + variance = Vectors.dense(0.0, 0.0, 0.0), + count = 1L, + numNonZeros = Vectors.dense(0.0, 1.0, 1.0), + max = singleElem, + min = singleElem, + normL1 = singleElem, + normL2 = singleElem + ) + ) + + testExample("multiple elements (dense)", + Seq( + (Vectors.dense(-1.0, 0.0, 6.0), 0.5), + (Vectors.dense(3.0, -3.0, 0.0), 2.8), + (Vectors.dense(1.0, -3.0, 0.0), 0.0) + ), + ExpectedMetrics( + mean = Vectors.dense(2.393939, -2.545454, 0.909090), + variance = Vectors.dense(8.0, 4.5, 18.0), + count = 2L, + numNonZeros = Vectors.dense(2.0, 1.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(8.9, 8.4, 3.0), + normL2 = Vectors.dense(5.069516, 5.019960, 4.242640) + ), + ExpectedMetrics( + mean = Vectors.dense(1.0, -2.0, 2.0), + variance = Vectors.dense(4.0, 3.0, 12.0), + count = 3L, + numNonZeros = Vectors.dense(3.0, 2.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(5.0, 6.0, 6.0), + normL2 = Vectors.dense(3.316624, 4.242640, 6.0) + ) + ) - val singleElem = Seq(0.0, 1.0, 2.0) - testExample("single element", Seq(singleElem), ExpectedMetrics( - mean = singleElem, - variance = Seq(0.0, 0.0, 0.0), - count = 1, - numNonZeros = Seq(0, 1, 1), - max = singleElem, - min = singleElem, - normL1 = singleElem, - normL2 = singleElem - )) - - testExample("two elements", Seq(Seq(0.0, 1.0, 2.0), Seq(0.0, -1.0, -2.0)), ExpectedMetrics( - mean = Seq(0.0, 0.0, 0.0), - // TODO: I have a doubt about these values, they are not normalized. - variance = Seq(0.0, 2.0, 8.0), - count = 2, - numNonZeros = Seq(0, 2, 2), - max = Seq(0.0, 1.0, 2.0), - min = Seq(0.0, -1.0, -2.0), - normL1 = Seq(0.0, 2.0, 4.0), - normL2 = Seq(0.0, math.sqrt(2.0), math.sqrt(2.0) * 2.0) - )) - - testExample("dense vector input", - Seq(Seq(-1.0, 0.0, 6.0), Seq(3.0, -3.0, 0.0)), + testExample("multiple elements (sparse)", + Seq( + (Vectors.dense(-1.0, 0.0, 6.0).toSparse, 0.5), + (Vectors.dense(3.0, -3.0, 0.0).toSparse, 2.8), + (Vectors.dense(1.0, -3.0, 0.0).toSparse, 0.0) + ), + ExpectedMetrics( + mean = Vectors.dense(2.393939, -2.545454, 0.909090), + variance = Vectors.dense(8.0, 4.5, 18.0), + count = 2L, + numNonZeros = Vectors.dense(2.0, 1.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(8.9, 8.4, 3.0), + normL2 = Vectors.dense(5.069516, 5.019960, 4.242640) + ), ExpectedMetrics( - mean = Seq(1.0, -1.5, 3.0), - variance = Seq(8.0, 4.5, 18.0), - count = 2, - numNonZeros = Seq(2, 1, 1), - max = Seq(3.0, 0.0, 6.0), - min = Seq(-1.0, -3, 0.0), - normL1 = Seq(4.0, 3.0, 6.0), - normL2 = Seq(math.sqrt(10), 3, 6.0) + mean = Vectors.dense(1.0, -2.0, 2.0), + variance = Vectors.dense(4.0, 3.0, 12.0), + count = 3L, + numNonZeros = Vectors.dense(3.0, 2.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(5.0, 6.0, 6.0), + normL2 = Vectors.dense(3.316624, 4.242640, 6.0) ) ) diff --git a/pom.xml b/pom.xml index 52db79eaf036b..92f897095f087 100644 --- a/pom.xml +++ b/pom.xml @@ -185,7 +185,7 @@ 2.8 1.8 1.0.0 - 0.4.0 + 0.8.0 ${java.home} @@ -580,7 +580,7 @@ io.netty netty-all - 4.0.47.Final + 4.1.17.Final io.netty @@ -1972,6 +1972,14 @@ com.fasterxml.jackson.core jackson-databind + + io.netty + netty-buffer + + + io.netty + netty-common + io.netty netty-handler diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 75703380cdb4a..7469f11df0294 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -239,14 +239,14 @@ object SparkBuild extends PomBuild { javacOptions in Compile ++= Seq( "-encoding", "UTF-8", - "-source", javacJVMVersion.value, - "-Xlint:unchecked" + "-source", javacJVMVersion.value ), - // This -target option cannot be set in the Compile configuration scope since `javadoc` doesn't - // play nicely with it; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 for - // additional discussion and explanation. + // This -target and Xlint:unchecked options cannot be set in the Compile configuration scope since + // `javadoc` doesn't play nicely with them; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 + // for additional discussion and explanation. javacOptions in (Compile, compile) ++= Seq( - "-target", javacJVMVersion.value + "-target", javacJVMVersion.value, + "-Xlint:unchecked" ), scalacOptions in Compile ++= Seq( @@ -254,6 +254,21 @@ object SparkBuild extends PomBuild { "-sourcepath", (baseDirectory in ThisBuild).value.getAbsolutePath // Required for relative source links in scaladoc ), + // Remove certain packages from Scaladoc + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" + ) ++ { + // Do not attempt to scaladoc javadoc comments under 2.12 since it can't handle inner classes + if (scalaBinaryVersion.value == "2.12") Seq("-no-java-comments") else Seq.empty + }, + // Implements -Xfatal-warnings, ignoring deprecation warnings. // Code snippet taken from https://issues.scala-lang.org/browse/SI-8410. compile in Compile := { @@ -828,18 +843,7 @@ object TestSettings { } Seq.empty[File] }).value, - concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - // Remove certain packages from Scaladoc - scalacOptions in (Compile, doc) := Seq( - "-groups", - "-skip-packages", Seq( - "org.apache.spark.api.python", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":"), - "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" - ) + concurrentRestrictions in Global += Tags.limit(Tags.Test, 1) ) } diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 130d1a0bae7f0..d55d209d09398 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -154,7 +154,8 @@ def get$Name(self): ("aggregationDepth", "suggested depth for treeAggregate (>= 2).", "2", "TypeConverters.toInt"), ("parallelism", "the number of threads to use when running parallel algorithms (>= 1).", - "1", "TypeConverters.toInt")] + "1", "TypeConverters.toInt"), + ("loss", "the loss function to be optimized.", None, "TypeConverters.toString")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 4041d9c43b236..e5c5ddfba6c1f 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -632,6 +632,29 @@ def getParallelism(self): return self.getOrDefault(self.parallelism) +class HasLoss(Params): + """ + Mixin for param loss: the loss function to be optimized. + """ + + loss = Param(Params._dummy(), "loss", "the loss function to be optimized.", typeConverter=TypeConverters.toString) + + def __init__(self): + super(HasLoss, self).__init__() + + def setLoss(self, value): + """ + Sets the value of :py:attr:`loss`. + """ + return self._set(loss=value) + + def getLoss(self): + """ + Gets the value of loss or its default value. + """ + return self.getOrDefault(self.loss) + + class DecisionTreeParams(Params): """ Mixin for Decision Tree parameters. diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 9d5b768091cf4..f0812bd1d4a39 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -39,23 +39,26 @@ @inherit_doc class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, - HasStandardization, HasSolver, HasWeightCol, HasAggregationDepth, + HasStandardization, HasSolver, HasWeightCol, HasAggregationDepth, HasLoss, JavaMLWritable, JavaMLReadable): """ Linear regression. - The learning objective is to minimize the squared error, with regularization. - The specific squared error loss function used is: L = 1/2n ||A coefficients - y||^2^ + The learning objective is to minimize the specified loss function, with regularization. + This supports two kinds of loss: - This supports multiple types of regularization: - - * none (a.k.a. ordinary least squares) + * squaredError (a.k.a squared loss) + * huber (a hybrid of squared error for relatively small errors and absolute error for \ + relatively large ones, and we estimate the scale parameter from training data) - * L2 (ridge regression) + This supports multiple types of regularization: - * L1 (Lasso) + * none (a.k.a. ordinary least squares) + * L2 (ridge regression) + * L1 (Lasso) + * L2 + L1 (elastic net) - * L2 + L1 (elastic net) + Note: Fitting with huber loss only supports none and L2 regularization. >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([ @@ -98,19 +101,28 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: auto, normal, l-bfgs.", typeConverter=TypeConverters.toString) + loss = Param(Params._dummy(), "loss", "The loss function to be optimized. Supported " + + "options: squaredError, huber.", typeConverter=TypeConverters.toString) + + epsilon = Param(Params._dummy(), "epsilon", "The shape parameter to control the amount of " + + "robustness. Must be > 1.0. Only valid when loss is huber", + typeConverter=TypeConverters.toFloat) + @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, solver="auto", weightCol=None, aggregationDepth=2): + standardization=True, solver="auto", weightCol=None, aggregationDepth=2, + loss="squaredError", epsilon=1.35): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ - standardization=True, solver="auto", weightCol=None, aggregationDepth=2) + standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ + loss="squaredError", epsilon=1.35) """ super(LinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.LinearRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -118,11 +130,13 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred @since("1.4.0") def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, solver="auto", weightCol=None, aggregationDepth=2): + standardization=True, solver="auto", weightCol=None, aggregationDepth=2, + loss="squaredError", epsilon=1.35): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ - standardization=True, solver="auto", weightCol=None, aggregationDepth=2) + standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ + loss="squaredError", epsilon=1.35) Sets params for linear regression. """ kwargs = self._input_kwargs @@ -131,6 +145,20 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return LinearRegressionModel(java_model) + @since("2.3.0") + def setEpsilon(self, value): + """ + Sets the value of :py:attr:`epsilon`. + """ + return self._set(epsilon=value) + + @since("2.3.0") + def getEpsilon(self): + """ + Gets the value of epsilon or its default value. + """ + return self.getOrDefault(self.epsilon) + class LinearRegressionModel(JavaModel, JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ @@ -155,6 +183,14 @@ def intercept(self): """ return self._call_java("intercept") + @property + @since("2.3.0") + def scale(self): + """ + The value by which \|y - X'w\| is scaled down when loss is "huber", otherwise 1.0. + """ + return self._call_java("scale") + @property @since("2.0.0") def summary(self): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index be1521154f042..afcb0881c4dcb 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -1726,6 +1726,27 @@ def test_offset(self): self.assertTrue(np.isclose(model.intercept, -1.561613, atol=1E-4)) +class LinearRegressionTest(SparkSessionTestCase): + + def test_linear_regression_with_huber_loss(self): + + data_path = "data/mllib/sample_linear_regression_data.txt" + df = self.spark.read.format("libsvm").load(data_path) + + lir = LinearRegression(loss="huber", epsilon=2.0) + model = lir.fit(df) + + expectedCoefficients = [0.136, 0.7648, -0.7761, 2.4236, 0.537, + 1.2612, -0.333, -0.5694, -0.6311, 0.6053] + expectedIntercept = 0.1607 + expectedScale = 9.758 + + self.assertTrue( + np.allclose(model.coefficients.toArray(), expectedCoefficients, atol=1E-3)) + self.assertTrue(np.isclose(model.intercept, expectedIntercept, atol=1E-3)) + self.assertTrue(np.isclose(model.scale, expectedScale, atol=1E-3)) + + class LogisticRegressionTest(SparkSessionTestCase): def test_binomial_logistic_regression_with_bound(self): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 37e7cf3fa662e..88d6a191babca 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -223,27 +223,14 @@ def _create_batch(series, timezone): series = [series] series = ((s, None) if not isinstance(s, (list, tuple)) else s for s in series) - # If a nullable integer series has been promoted to floating point with NaNs, need to cast - # NOTE: this is not necessary with Arrow >= 0.7 - def cast_series(s, t): - if type(t) == pa.TimestampType: - # NOTE: convert to 'us' with astype here, unit ignored in `from_pandas` see ARROW-1680 - return _check_series_convert_timestamps_internal(s.fillna(0), timezone)\ - .values.astype('datetime64[us]', copy=False) - # NOTE: can not compare None with pyarrow.DataType(), fixed with Arrow >= 0.7.1 - elif t is not None and t == pa.date32(): - # TODO: this converts the series to Python objects, possibly avoid with Arrow >= 0.8 - return s.dt.date - elif t is None or s.dtype == t.to_pandas_dtype(): - return s - else: - return s.fillna(0).astype(t.to_pandas_dtype(), copy=False) - - # Some object types don't support masks in Arrow, see ARROW-1721 def create_array(s, t): - casted = cast_series(s, t) - mask = None if casted.dtype == 'object' else s.isnull() - return pa.Array.from_pandas(casted, mask=mask, type=t) + mask = s.isnull() + # Ensure timestamp series are in expected form for Spark internal representation + if t is not None and pa.types.is_timestamp(t): + s = _check_series_convert_timestamps_internal(s.fillna(0), timezone) + # TODO: need cast after Arrow conversion, ns values cause error with pandas 0.19.2 + return pa.Array.from_pandas(s, mask=mask).cast(t, safe=False) + return pa.Array.from_pandas(s, mask=mask, type=t) arrs = [create_array(s, t) for s, t in series] return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in xrange(len(arrs))]) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 75395a754a831..440684d3edfa6 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1906,7 +1906,9 @@ def toPandas(self): if self.sql_ctx.getConf("spark.sql.execution.arrow.enabled", "false").lower() == "true": try: from pyspark.sql.types import _check_dataframe_localize_timestamps + from pyspark.sql.utils import _require_minimum_pyarrow_version import pyarrow + _require_minimum_pyarrow_version() tables = self._collectAsArrow() if tables: table = pyarrow.concat_tables(tables) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 54530055dfa85..ddd8df3b15bf6 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2159,16 +2159,17 @@ def pandas_udf(f=None, returnType=None, functionType=None): >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> from pyspark.sql.types import IntegerType, StringType - >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) - >>> @pandas_udf(StringType()) + >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) # doctest: +SKIP + >>> @pandas_udf(StringType()) # doctest: +SKIP ... def to_upper(s): ... return s.str.upper() ... - >>> @pandas_udf("integer", PandasUDFType.SCALAR) + >>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP ... def add_one(x): ... return x + 1 ... - >>> df = spark.createDataFrame([(1, "John Doe", 21)], ("id", "name", "age")) + >>> df = spark.createDataFrame([(1, "John Doe", 21)], + ... ("id", "name", "age")) # doctest: +SKIP >>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")) \\ ... .show() # doctest: +SKIP +----------+--------------+------------+ @@ -2189,8 +2190,8 @@ def pandas_udf(f=None, returnType=None, functionType=None): >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) - >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) + ... ("id", "v")) # doctest: +SKIP + >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP ... def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 4d47dd6a3e878..09fae46adf014 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -218,7 +218,7 @@ def apply(self, udf): >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ... ("id", "v")) - >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) + >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP ... def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index e2435e09af23d..86db16eca7889 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -495,11 +495,14 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): from pyspark.serializers import ArrowSerializer, _create_batch from pyspark.sql.types import from_arrow_schema, to_arrow_type, \ _old_pandas_exception_message, TimestampType + from pyspark.sql.utils import _require_minimum_pyarrow_version try: from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype except ImportError as e: raise ImportError(_old_pandas_exception_message(e)) + _require_minimum_pyarrow_version() + # Determine arrow types to coerce data when creating batches if isinstance(schema, StructType): arrow_types = [to_arrow_type(f.dataType) for f in schema.fields] diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b4d32d8de8a22..6fdfda1cc831b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -3339,10 +3339,11 @@ def test_createDataFrame_with_single_data_type(self): self.spark.createDataFrame(pd.DataFrame({"a": [1]}), schema="int") def test_createDataFrame_does_not_modify_input(self): + import pandas as pd # Some series get converted for Spark to consume, this makes sure input is unchanged pdf = self.create_pandas_data_frame() # Use a nanosecond value to make sure it is not truncated - pdf.ix[0, '7_timestamp_t'] = 1 + pdf.ix[0, '7_timestamp_t'] = pd.Timestamp(1) # Integers with nulls will get NaNs filled with 0 and will be casted pdf.ix[1, '2_int_t'] = None pdf_copy = pdf.copy(deep=True) @@ -3356,6 +3357,7 @@ def test_schema_conversion_roundtrip(self): self.assertEquals(self.schema, schema_rt) +@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed") class PandasUDFTests(ReusedSQLTestCase): def test_pandas_udf_basic(self): from pyspark.rdd import PythonEvalType @@ -3671,9 +3673,9 @@ def test_vectorized_udf_chained(self): def test_vectorized_udf_wrong_return_type(self): from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) - f = pandas_udf(lambda x: x * 1.0, StringType()) + f = pandas_udf(lambda x: x * 1.0, ArrayType(LongType())) with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, 'Invalid.*type'): + with self.assertRaisesRegexp(Exception, 'Unsupported.*type.*conversion'): df.select(f(col('id'))).collect() def test_vectorized_udf_return_scalar(self): @@ -3974,12 +3976,12 @@ def test_wrong_return_type(self): foo = pandas_udf( lambda pdf: pdf, - 'id long, v string', + 'id long, v array', PandasUDFType.GROUP_MAP ) with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, 'Invalid.*type'): + with self.assertRaisesRegexp(Exception, 'Unsupported.*type.*conversion'): df.groupby('id').apply(foo).sort('id').toPandas() def test_wrong_args(self): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 78abc32a35a1c..46d9a417414b5 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1642,29 +1642,28 @@ def to_arrow_schema(schema): def from_arrow_type(at): """ Convert pyarrow type to Spark data type. """ - # TODO: newer pyarrow has is_boolean(at) functions that would be better to check type - import pyarrow as pa - if at == pa.bool_(): + import pyarrow.types as types + if types.is_boolean(at): spark_type = BooleanType() - elif at == pa.int8(): + elif types.is_int8(at): spark_type = ByteType() - elif at == pa.int16(): + elif types.is_int16(at): spark_type = ShortType() - elif at == pa.int32(): + elif types.is_int32(at): spark_type = IntegerType() - elif at == pa.int64(): + elif types.is_int64(at): spark_type = LongType() - elif at == pa.float32(): + elif types.is_float32(at): spark_type = FloatType() - elif at == pa.float64(): + elif types.is_float64(at): spark_type = DoubleType() - elif type(at) == pa.DecimalType: + elif types.is_decimal(at): spark_type = DecimalType(precision=at.precision, scale=at.scale) - elif at == pa.string(): + elif types.is_string(at): spark_type = StringType() - elif at == pa.date32(): + elif types.is_date32(at): spark_type = DateType() - elif type(at) == pa.TimestampType: + elif types.is_timestamp(at): spark_type = TimestampType() else: raise TypeError("Unsupported type in conversion from Arrow: " + str(at)) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index c3301a41ccd5a..50c87ba1ac882 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -33,19 +33,23 @@ def _wrap_function(sc, func, returnType): def _create_udf(f, returnType, evalType): - if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF: + + if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF or \ + evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF: import inspect + from pyspark.sql.utils import _require_minimum_pyarrow_version + + _require_minimum_pyarrow_version() argspec = inspect.getargspec(f) - if len(argspec.args) == 0 and argspec.varargs is None: + + if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF and len(argspec.args) == 0 and \ + argspec.varargs is None: raise ValueError( "Invalid function: 0-arg pandas_udfs are not supported. " "Instead, create a 1-arg pandas_udf and ignore the arg in your function." ) - elif evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF: - import inspect - argspec = inspect.getargspec(f) - if len(argspec.args) != 1: + if evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF and len(argspec.args) != 1: raise ValueError( "Invalid function: pandas_udfs with function type GROUP_MAP " "must take a single arg that is a pandas DataFrame." diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 7bc6a59ad3b26..cc7dabb64b3ec 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -110,3 +110,12 @@ def toJArray(gateway, jtype, arr): for i in range(0, len(arr)): jarr[i] = arr[i] return jarr + + +def _require_minimum_pyarrow_version(): + """ Raise ImportError if minimum version of pyarrow is not installed + """ + from distutils.version import LooseVersion + import pyarrow + if LooseVersion(pyarrow.__version__) < LooseVersion('0.8.0'): + raise ImportError("pyarrow >= 0.8.0 must be installed on calling Python process") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 04aadb4b06af4..45f527959cbe1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -102,10 +102,10 @@ private[spark] object Config extends Logging { val KUBERNETES_ALLOCATION_BATCH_DELAY = ConfigBuilder("spark.kubernetes.allocation.batch.delay") - .doc("Number of seconds to wait between each round of executor allocation.") - .longConf - .checkValue(value => value > 0, "Allocation batch delay should be a positive integer") - .createWithDefault(1) + .doc("Time to wait between each round of executor allocation.") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(value => value > 0, "Allocation batch delay must be a positive time value.") + .createWithDefaultString("1s") val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS = ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index e79c987852db2..9de4b16c30d3c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -217,7 +217,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .watch(new ExecutorPodsWatcher())) allocatorExecutor.scheduleWithFixedDelay( - allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.MILLISECONDS) if (!Utils.isDynamicAllocationEnabled(conf)) { doRequestTotalExecutors(initialExecutors) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 13c09033a50ee..b2f26f205a329 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -46,7 +46,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private val NAMESPACE = "test-namespace" private val SPARK_DRIVER_HOST = "localhost" private val SPARK_DRIVER_PORT = 7077 - private val POD_ALLOCATION_INTERVAL = 60L + private val POD_ALLOCATION_INTERVAL = "1m" private val DRIVER_URL = RpcEndpointAddress( SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val FIRST_EXECUTOR_POD = new PodBuilder() @@ -144,7 +144,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .set(KUBERNETES_NAMESPACE, NAMESPACE) .set("spark.driver.host", SPARK_DRIVER_HOST) .set("spark.driver.port", SPARK_DRIVER_PORT.toString) - .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY.key, POD_ALLOCATION_INTERVAL) executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) @@ -162,8 +162,8 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(allocatorExecutor.scheduleWithFixedDelay( allocatorRunnable.capture(), mockitoEq(0L), - mockitoEq(POD_ALLOCATION_INTERVAL), - mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + mockitoEq(TimeUnit.MINUTES.toMillis(1)), + mockitoEq(TimeUnit.MILLISECONDS))).thenReturn(null) // Creating Futures in Scala backed by a Java executor service resolves to running // ExecutorService#execute (as opposed to submit) doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index d3a8cb5804717..8bb14598a6d7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -602,13 +602,13 @@ case class Least(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val evalChildren = children.map(_.genCode(ctx)) - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "leastTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) val evals = evalChildren.map(eval => s""" |${eval.code} - |if (!${eval.isNull} && ($tmpIsNull || + |if (!${eval.isNull} && (${ev.isNull} || | ${ctx.genGreater(dataType, ev.value, eval.value)})) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; |} """.stripMargin @@ -628,10 +628,9 @@ case class Least(children: Seq[Expression]) extends Expression { foldFunctions = _.map(funcCall => s"${ev.value} = $funcCall;").mkString("\n")) ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; |$codes - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } @@ -682,13 +681,13 @@ case class Greatest(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val evalChildren = children.map(_.genCode(ctx)) - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "greatestTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) val evals = evalChildren.map(eval => s""" |${eval.code} - |if (!${eval.isNull} && ($tmpIsNull || + |if (!${eval.isNull} && (${ev.isNull} || | ${ctx.genGreater(dataType, eval.value, ev.value)})) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; |} """.stripMargin @@ -708,10 +707,9 @@ case class Greatest(children: Seq[Expression]) extends Expression { foldFunctions = _.map(funcCall => s"${ev.value} = $funcCall;").mkString("\n")) ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; |$codes - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 41a920ba3d677..9adf632ddcde8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -128,7 +128,7 @@ class CodegenContext { * `currentVars` to null, or set `currentVars(i)` to null for certain columns, before calling * `Expression.genCode`. */ - final var INPUT_ROW = "i" + var INPUT_ROW = "i" /** * Holding a list of generated columns as input of current operator, will be used by @@ -146,22 +146,30 @@ class CodegenContext { * as a member variable * * They will be kept as member variables in generated classes like `SpecificProjection`. + * + * Exposed for tests only. */ - val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] = + private[catalyst] val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] = mutable.ArrayBuffer.empty[(String, String)] /** * The mapping between mutable state types and corrseponding compacted arrays. * The keys are java type string. The values are [[MutableStateArrays]] which encapsulates * the compacted arrays for the mutable states with the same java type. + * + * Exposed for tests only. */ - val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] = + private[catalyst] val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] = mutable.Map.empty[String, MutableStateArrays] // An array holds the code that will initialize each state - val mutableStateInitCode: mutable.ArrayBuffer[String] = + // Exposed for tests only. + private[catalyst] val mutableStateInitCode: mutable.ArrayBuffer[String] = mutable.ArrayBuffer.empty[String] + // Tracks the names of all the mutable states. + private val mutableStateNames: mutable.HashSet[String] = mutable.HashSet.empty + /** * This class holds a set of names of mutableStateArrays that is used for compacting mutable * states for a certain type, and holds the next available slot of the current compacted array. @@ -172,7 +180,11 @@ class CodegenContext { private[this] var currentIndex = 0 - private def createNewArray() = arrayNames.append(freshName("mutableStateArray")) + private def createNewArray() = { + val newArrayName = freshName("mutableStateArray") + mutableStateNames += newArrayName + arrayNames.append(newArrayName) + } def getCurrentIndex: Int = currentIndex @@ -241,6 +253,7 @@ class CodegenContext { val initCode = initFunc(varName) inlinedMutableStates += ((javaType, varName)) mutableStateInitCode += initCode + mutableStateNames += varName varName } else { val arrays = arrayCompactedMutableStates.getOrElseUpdate(javaType, new MutableStateArrays) @@ -930,6 +943,15 @@ class CodegenContext { // inline execution if only one block blocks.head } else { + if (Utils.isTesting) { + // Passing global variables to the split method is dangerous, as any mutating to it is + // ignored and may lead to unexpected behavior. + arguments.foreach { case (_, name) => + assert(!mutableStateNames.contains(name), + s"split function argument $name cannot be a global variable.") + } + } + val func = freshName(funcName) val argString = arguments.map { case (t, name) => s"$t $name" }.mkString(", ") val functions = blocks.zipWithIndex.map { case (body, i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 1a9b68222a7f4..142dfb02be0a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -190,7 +190,7 @@ case class CaseWhen( // It is initialized to `NOT_MATCHED`, and if it's set to `HAS_NULL` or `HAS_NONNULL`, // We won't go on anymore on the computation. val resultState = ctx.freshName("caseWhenResultState") - val tmpResult = ctx.addMutableState(ctx.javaType(dataType), "caseWhenTmpResult") + ev.value = ctx.addMutableState(ctx.javaType(dataType), ev.value) // these blocks are meant to be inside a // do { @@ -205,7 +205,7 @@ case class CaseWhen( |if (!${cond.isNull} && ${cond.value}) { | ${res.code} | $resultState = (byte)(${res.isNull} ? $HAS_NULL : $HAS_NONNULL); - | $tmpResult = ${res.value}; + | ${ev.value} = ${res.value}; | continue; |} """.stripMargin @@ -216,7 +216,7 @@ case class CaseWhen( s""" |${res.code} |$resultState = (byte)(${res.isNull} ? $HAS_NULL : $HAS_NONNULL); - |$tmpResult = ${res.value}; + |${ev.value} = ${res.value}; """.stripMargin } @@ -264,13 +264,11 @@ case class CaseWhen( ev.copy(code = s""" |${ctx.JAVA_BYTE} $resultState = $NOT_MATCHED; - |$tmpResult = ${ctx.defaultValue(dataType)}; |do { | $codes |} while (false); |// TRUE if any condition is met and the result is null, or no any condition is met. |final boolean ${ev.isNull} = ($resultState != $HAS_NONNULL); - |final ${ctx.javaType(dataType)} ${ev.value} = $tmpResult; """.stripMargin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 1cd73a92a8635..69af7a250a5ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -214,11 +214,11 @@ case class Stack(children: Seq[Expression]) extends Generator { // Create the collection. val wrapperClass = classOf[mutable.WrappedArray[_]].getName - ctx.addMutableState( - s"$wrapperClass", - ev.value, - v => s"$v = $wrapperClass$$.MODULE$$.make($rowData);", useFreshName = false) - ev.copy(code = code, isNull = "false") + ev.copy(code = + s""" + |$code + |$wrapperClass ${ev.value} = $wrapperClass$$.MODULE$$.make($rowData); + """.stripMargin, isNull = "false") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index b4f895fffda38..470d5da041ea5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -72,7 +72,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "coalesceTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) // all the evals are meant to be in a do { ... } while (false); loop val evals = children.map { e => @@ -80,7 +80,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { s""" |${eval.code} |if (!${eval.isNull}) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; | continue; |} @@ -103,7 +103,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { foldFunctions = _.map { funcCall => s""" |${ev.value} = $funcCall; - |if (!$tmpIsNull) { + |if (!${ev.isNull}) { | continue; |} """.stripMargin @@ -112,12 +112,11 @@ case class Coalesce(children: Seq[Expression]) extends Expression { ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |$resultType ${ev.value} = ${ctx.defaultValue(dataType)}; |do { | $codes |} while (false); - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } 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 ac9f56f78eb2e..f4ee3d10f3f43 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 @@ -285,7 +285,7 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { |${valueGen.code} |byte $tmpResult = $HAS_NULL; |if (!${valueGen.isNull}) { - | $tmpResult = 0; + | $tmpResult = $NOT_MATCHED; | $javaDataType $valueArg = ${valueGen.value}; | do { | $codes diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index e11e3a105f597..220cc4f885d7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -251,8 +251,8 @@ case class SpecifiedWindowFrame( TypeCheckFailure(s"Window frame $location bound '$e' is not a literal.") case e: Expression if !frameType.inputType.acceptsType(e.dataType) => TypeCheckFailure( - s"The data type of the $location bound '${e.dataType} does not match " + - s"the expected data type '${frameType.inputType}'.") + s"The data type of the $location bound '${e.dataType}' does not match " + + s"the expected data type '${frameType.inputType.simpleString}'.") case _ => TypeCheckSuccess } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java index d53e1fcab0c5a..528f66f342dc9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java @@ -275,30 +275,30 @@ public byte[] getBinary(int rowId) { public ArrowColumnVector(ValueVector vector) { super(ArrowUtils.fromArrowField(vector.getField())); - if (vector instanceof NullableBitVector) { - accessor = new BooleanAccessor((NullableBitVector) vector); - } else if (vector instanceof NullableTinyIntVector) { - accessor = new ByteAccessor((NullableTinyIntVector) vector); - } else if (vector instanceof NullableSmallIntVector) { - accessor = new ShortAccessor((NullableSmallIntVector) vector); - } else if (vector instanceof NullableIntVector) { - accessor = new IntAccessor((NullableIntVector) vector); - } else if (vector instanceof NullableBigIntVector) { - accessor = new LongAccessor((NullableBigIntVector) vector); - } else if (vector instanceof NullableFloat4Vector) { - accessor = new FloatAccessor((NullableFloat4Vector) vector); - } else if (vector instanceof NullableFloat8Vector) { - accessor = new DoubleAccessor((NullableFloat8Vector) vector); - } else if (vector instanceof NullableDecimalVector) { - accessor = new DecimalAccessor((NullableDecimalVector) vector); - } else if (vector instanceof NullableVarCharVector) { - accessor = new StringAccessor((NullableVarCharVector) vector); - } else if (vector instanceof NullableVarBinaryVector) { - accessor = new BinaryAccessor((NullableVarBinaryVector) vector); - } else if (vector instanceof NullableDateDayVector) { - accessor = new DateAccessor((NullableDateDayVector) vector); - } else if (vector instanceof NullableTimeStampMicroTZVector) { - accessor = new TimestampAccessor((NullableTimeStampMicroTZVector) vector); + if (vector instanceof BitVector) { + accessor = new BooleanAccessor((BitVector) vector); + } else if (vector instanceof TinyIntVector) { + accessor = new ByteAccessor((TinyIntVector) vector); + } else if (vector instanceof SmallIntVector) { + accessor = new ShortAccessor((SmallIntVector) vector); + } else if (vector instanceof IntVector) { + accessor = new IntAccessor((IntVector) vector); + } else if (vector instanceof BigIntVector) { + accessor = new LongAccessor((BigIntVector) vector); + } else if (vector instanceof Float4Vector) { + accessor = new FloatAccessor((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + accessor = new DoubleAccessor((Float8Vector) vector); + } else if (vector instanceof DecimalVector) { + accessor = new DecimalAccessor((DecimalVector) vector); + } else if (vector instanceof VarCharVector) { + accessor = new StringAccessor((VarCharVector) vector); + } else if (vector instanceof VarBinaryVector) { + accessor = new BinaryAccessor((VarBinaryVector) vector); + } else if (vector instanceof DateDayVector) { + accessor = new DateAccessor((DateDayVector) vector); + } else if (vector instanceof TimeStampMicroTZVector) { + accessor = new TimestampAccessor((TimeStampMicroTZVector) vector); } else if (vector instanceof ListVector) { ListVector listVector = (ListVector) vector; accessor = new ArrayAccessor(listVector); @@ -321,23 +321,21 @@ public ArrowColumnVector(ValueVector vector) { private abstract static class ArrowVectorAccessor { private final ValueVector vector; - private final ValueVector.Accessor nulls; ArrowVectorAccessor(ValueVector vector) { this.vector = vector; - this.nulls = vector.getAccessor(); } final boolean isNullAt(int rowId) { - return nulls.isNull(rowId); + return vector.isNull(rowId); } final int getValueCount() { - return nulls.getValueCount(); + return vector.getValueCount(); } final int getNullCount() { - return nulls.getNullCount(); + return vector.getNullCount(); } final void close() { @@ -395,11 +393,11 @@ int getArrayOffset(int rowId) { private static class BooleanAccessor extends ArrowVectorAccessor { - private final NullableBitVector.Accessor accessor; + private final BitVector accessor; - BooleanAccessor(NullableBitVector vector) { + BooleanAccessor(BitVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -410,11 +408,11 @@ final boolean getBoolean(int rowId) { private static class ByteAccessor extends ArrowVectorAccessor { - private final NullableTinyIntVector.Accessor accessor; + private final TinyIntVector accessor; - ByteAccessor(NullableTinyIntVector vector) { + ByteAccessor(TinyIntVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -425,11 +423,11 @@ final byte getByte(int rowId) { private static class ShortAccessor extends ArrowVectorAccessor { - private final NullableSmallIntVector.Accessor accessor; + private final SmallIntVector accessor; - ShortAccessor(NullableSmallIntVector vector) { + ShortAccessor(SmallIntVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -440,11 +438,11 @@ final short getShort(int rowId) { private static class IntAccessor extends ArrowVectorAccessor { - private final NullableIntVector.Accessor accessor; + private final IntVector accessor; - IntAccessor(NullableIntVector vector) { + IntAccessor(IntVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -455,11 +453,11 @@ final int getInt(int rowId) { private static class LongAccessor extends ArrowVectorAccessor { - private final NullableBigIntVector.Accessor accessor; + private final BigIntVector accessor; - LongAccessor(NullableBigIntVector vector) { + LongAccessor(BigIntVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -470,11 +468,11 @@ final long getLong(int rowId) { private static class FloatAccessor extends ArrowVectorAccessor { - private final NullableFloat4Vector.Accessor accessor; + private final Float4Vector accessor; - FloatAccessor(NullableFloat4Vector vector) { + FloatAccessor(Float4Vector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -485,11 +483,11 @@ final float getFloat(int rowId) { private static class DoubleAccessor extends ArrowVectorAccessor { - private final NullableFloat8Vector.Accessor accessor; + private final Float8Vector accessor; - DoubleAccessor(NullableFloat8Vector vector) { + DoubleAccessor(Float8Vector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -500,11 +498,11 @@ final double getDouble(int rowId) { private static class DecimalAccessor extends ArrowVectorAccessor { - private final NullableDecimalVector.Accessor accessor; + private final DecimalVector accessor; - DecimalAccessor(NullableDecimalVector vector) { + DecimalAccessor(DecimalVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -516,12 +514,12 @@ final Decimal getDecimal(int rowId, int precision, int scale) { private static class StringAccessor extends ArrowVectorAccessor { - private final NullableVarCharVector.Accessor accessor; + private final VarCharVector accessor; private final NullableVarCharHolder stringResult = new NullableVarCharHolder(); - StringAccessor(NullableVarCharVector vector) { + StringAccessor(VarCharVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -539,11 +537,11 @@ final UTF8String getUTF8String(int rowId) { private static class BinaryAccessor extends ArrowVectorAccessor { - private final NullableVarBinaryVector.Accessor accessor; + private final VarBinaryVector accessor; - BinaryAccessor(NullableVarBinaryVector vector) { + BinaryAccessor(VarBinaryVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -554,11 +552,11 @@ final byte[] getBinary(int rowId) { private static class DateAccessor extends ArrowVectorAccessor { - private final NullableDateDayVector.Accessor accessor; + private final DateDayVector accessor; - DateAccessor(NullableDateDayVector vector) { + DateAccessor(DateDayVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -569,11 +567,11 @@ final int getInt(int rowId) { private static class TimestampAccessor extends ArrowVectorAccessor { - private final NullableTimeStampMicroTZVector.Accessor accessor; + private final TimeStampMicroTZVector accessor; - TimestampAccessor(NullableTimeStampMicroTZVector vector) { + TimestampAccessor(TimeStampMicroTZVector vector) { super(vector); - this.accessor = vector.getAccessor(); + this.accessor = vector; } @Override @@ -584,21 +582,21 @@ final long getLong(int rowId) { private static class ArrayAccessor extends ArrowVectorAccessor { - private final UInt4Vector.Accessor accessor; + private final ListVector accessor; ArrayAccessor(ListVector vector) { super(vector); - this.accessor = vector.getOffsetVector().getAccessor(); + this.accessor = vector; } @Override final int getArrayLength(int rowId) { - return accessor.get(rowId + 1) - accessor.get(rowId); + return accessor.getInnerValueCountAt(rowId); } @Override final int getArrayOffset(int rowId) { - return accessor.get(rowId); + return accessor.getOffsetBuffer().getInt(rowId * accessor.OFFSET_WIDTH); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java new file mode 100644 index 0000000000000..0b5b6ac675f2c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.annotation.InterfaceStability; + +import java.util.List; +import java.util.Map; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * propagate session configs with the specified key-prefix to all data source operations in this + * session. + */ +@InterfaceStability.Evolving +public interface SessionConfigSupport { + + /** + * Key prefix of the session configs to propagate. Spark will extract all session configs that + * starts with `spark.datasource.$keyPrefix`, turn `spark.datasource.$keyPrefix.xxx -> yyy` + * into `xxx -> yyy`, and propagate them to all data source operations in this session. + */ + String keyPrefix(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java index 1ebd35356f1a3..ce1c489742054 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Offset.java @@ -23,7 +23,7 @@ * restart checkpoints. Sources should provide an Offset implementation which they can use to * reconstruct the stream position where the offset was taken. */ -public abstract class Offset { +public abstract class Offset extends org.apache.spark.sql.execution.streaming.Offset { /** * A JSON-serialized representation of an Offset that is * used for saving offsets to the offset log. @@ -41,8 +41,8 @@ public abstract class Offset { */ @Override public boolean equals(Object obj) { - if (obj instanceof Offset) { - return this.json().equals(((Offset) obj).json()); + if (obj instanceof org.apache.spark.sql.execution.streaming.Offset) { + return this.json().equals(((org.apache.spark.sql.execution.streaming.Offset) obj).json()); } else { return false; } diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin new file mode 100644 index 0000000000000..0bba2f88b92a5 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -0,0 +1 @@ +org.apache.spark.sql.execution.ui.SQLHistoryServerPlugin diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppStatusPlugin b/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppStatusPlugin deleted file mode 100644 index ac6d7f6962f85..0000000000000 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppStatusPlugin +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.sql.execution.ui.SQLAppStatusPlugin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 39fec8f983b65..c43ee91294a27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -184,9 +185,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val options = new DataSourceV2Options(extraOptions.asJava) + val ds = cls.newInstance() + val options = new DataSourceV2Options((extraOptions ++ + DataSourceV2Utils.extractSessionConfigs( + ds = ds.asInstanceOf[DataSourceV2], + conf = sparkSession.sessionState.conf)).asJava) - val reader = (cls.newInstance(), userSpecifiedSchema) match { + val reader = (ds, userSpecifiedSchema) match { case (ds: ReadSupportWithSchema, Some(schema)) => ds.createReader(schema, options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 59a01e61124f7..7ccda0ad36d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -30,9 +30,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, WriteSupport} +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.StructType /** @@ -236,14 +237,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - cls.newInstance() match { - case ds: WriteSupport => - val options = new DataSourceV2Options(extraOptions.asJava) + val ds = cls.newInstance() + ds match { + case ws: WriteSupport => + val options = new DataSourceV2Options((extraOptions ++ + DataSourceV2Utils.extractSessionConfigs( + ds = ds.asInstanceOf[DataSourceV2], + conf = df.sparkSession.sessionState.conf)).asJava) // Using a timestamp and a random UUID to distinguish different writing jobs. This is good // enough as there won't be tons of writing jobs created at the same second. val jobId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) .format(new Date()) + "-" + UUID.randomUUID() - val writer = ds.createWriter(jobId, df.logicalPlan.schema, mode, options) + val writer = ws.createWriter(jobId, df.logicalPlan.schema, mode, options) if (writer.isPresent) { runCommand(df.sparkSession, "save") { WriteToDataSourceV2(writer.get(), df.logicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 3cafb344ef553..bcfc412430263 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -24,8 +24,8 @@ import scala.collection.JavaConverters._ import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector._ -import org.apache.arrow.vector.file._ -import org.apache.arrow.vector.schema.ArrowRecordBatch +import org.apache.arrow.vector.ipc.{ArrowFileReader, ArrowFileWriter} +import org.apache.arrow.vector.ipc.message.ArrowRecordBatch import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel import org.apache.spark.TaskContext @@ -86,13 +86,9 @@ private[sql] object ArrowConverters { val root = VectorSchemaRoot.create(arrowSchema, allocator) val arrowWriter = ArrowWriter.create(root) - var closed = false - context.addTaskCompletionListener { _ => - if (!closed) { - root.close() - allocator.close() - } + root.close() + allocator.close() } new Iterator[ArrowPayload] { @@ -100,7 +96,6 @@ private[sql] object ArrowConverters { override def hasNext: Boolean = rowIter.hasNext || { root.close() allocator.close() - closed = true false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index e4af4f65da127..0258056d9de49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -46,17 +46,17 @@ object ArrowWriter { private def createFieldWriter(vector: ValueVector): ArrowFieldWriter = { val field = vector.getField() (ArrowUtils.fromArrowField(field), vector) match { - case (BooleanType, vector: NullableBitVector) => new BooleanWriter(vector) - case (ByteType, vector: NullableTinyIntVector) => new ByteWriter(vector) - case (ShortType, vector: NullableSmallIntVector) => new ShortWriter(vector) - case (IntegerType, vector: NullableIntVector) => new IntegerWriter(vector) - case (LongType, vector: NullableBigIntVector) => new LongWriter(vector) - case (FloatType, vector: NullableFloat4Vector) => new FloatWriter(vector) - case (DoubleType, vector: NullableFloat8Vector) => new DoubleWriter(vector) - case (StringType, vector: NullableVarCharVector) => new StringWriter(vector) - case (BinaryType, vector: NullableVarBinaryVector) => new BinaryWriter(vector) - case (DateType, vector: NullableDateDayVector) => new DateWriter(vector) - case (TimestampType, vector: NullableTimeStampMicroTZVector) => new TimestampWriter(vector) + case (BooleanType, vector: BitVector) => new BooleanWriter(vector) + case (ByteType, vector: TinyIntVector) => new ByteWriter(vector) + case (ShortType, vector: SmallIntVector) => new ShortWriter(vector) + case (IntegerType, vector: IntVector) => new IntegerWriter(vector) + case (LongType, vector: BigIntVector) => new LongWriter(vector) + case (FloatType, vector: Float4Vector) => new FloatWriter(vector) + case (DoubleType, vector: Float8Vector) => new DoubleWriter(vector) + case (StringType, vector: VarCharVector) => new StringWriter(vector) + case (BinaryType, vector: VarBinaryVector) => new BinaryWriter(vector) + case (DateType, vector: DateDayVector) => new DateWriter(vector) + case (TimestampType, vector: TimeStampMicroTZVector) => new TimestampWriter(vector) case (ArrayType(_, _), vector: ListVector) => val elementVector = createFieldWriter(vector.getDataVector()) new ArrayWriter(vector, elementVector) @@ -103,7 +103,6 @@ class ArrowWriter(val root: VectorSchemaRoot, fields: Array[ArrowFieldWriter]) { private[arrow] abstract class ArrowFieldWriter { def valueVector: ValueVector - def valueMutator: ValueVector.Mutator def name: String = valueVector.getField().getName() def dataType: DataType = ArrowUtils.fromArrowField(valueVector.getField()) @@ -124,161 +123,144 @@ private[arrow] abstract class ArrowFieldWriter { } def finish(): Unit = { - valueMutator.setValueCount(count) + valueVector.setValueCount(count) } def reset(): Unit = { - valueMutator.reset() + // TODO: reset() should be in a common interface + valueVector match { + case fixedWidthVector: BaseFixedWidthVector => fixedWidthVector.reset() + case variableWidthVector: BaseVariableWidthVector => variableWidthVector.reset() + case _ => + } count = 0 } } -private[arrow] class BooleanWriter(val valueVector: NullableBitVector) extends ArrowFieldWriter { - - override def valueMutator: NullableBitVector#Mutator = valueVector.getMutator() +private[arrow] class BooleanWriter(val valueVector: BitVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, if (input.getBoolean(ordinal)) 1 else 0) + valueVector.setSafe(count, if (input.getBoolean(ordinal)) 1 else 0) } } -private[arrow] class ByteWriter(val valueVector: NullableTinyIntVector) extends ArrowFieldWriter { - - override def valueMutator: NullableTinyIntVector#Mutator = valueVector.getMutator() +private[arrow] class ByteWriter(val valueVector: TinyIntVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getByte(ordinal)) + valueVector.setSafe(count, input.getByte(ordinal)) } } -private[arrow] class ShortWriter(val valueVector: NullableSmallIntVector) extends ArrowFieldWriter { - - override def valueMutator: NullableSmallIntVector#Mutator = valueVector.getMutator() +private[arrow] class ShortWriter(val valueVector: SmallIntVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getShort(ordinal)) + valueVector.setSafe(count, input.getShort(ordinal)) } } -private[arrow] class IntegerWriter(val valueVector: NullableIntVector) extends ArrowFieldWriter { - - override def valueMutator: NullableIntVector#Mutator = valueVector.getMutator() +private[arrow] class IntegerWriter(val valueVector: IntVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getInt(ordinal)) + valueVector.setSafe(count, input.getInt(ordinal)) } } -private[arrow] class LongWriter(val valueVector: NullableBigIntVector) extends ArrowFieldWriter { - - override def valueMutator: NullableBigIntVector#Mutator = valueVector.getMutator() +private[arrow] class LongWriter(val valueVector: BigIntVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getLong(ordinal)) + valueVector.setSafe(count, input.getLong(ordinal)) } } -private[arrow] class FloatWriter(val valueVector: NullableFloat4Vector) extends ArrowFieldWriter { - - override def valueMutator: NullableFloat4Vector#Mutator = valueVector.getMutator() +private[arrow] class FloatWriter(val valueVector: Float4Vector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getFloat(ordinal)) + valueVector.setSafe(count, input.getFloat(ordinal)) } } -private[arrow] class DoubleWriter(val valueVector: NullableFloat8Vector) extends ArrowFieldWriter { - - override def valueMutator: NullableFloat8Vector#Mutator = valueVector.getMutator() +private[arrow] class DoubleWriter(val valueVector: Float8Vector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getDouble(ordinal)) + valueVector.setSafe(count, input.getDouble(ordinal)) } } -private[arrow] class StringWriter(val valueVector: NullableVarCharVector) extends ArrowFieldWriter { - - override def valueMutator: NullableVarCharVector#Mutator = valueVector.getMutator() +private[arrow] class StringWriter(val valueVector: VarCharVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { val utf8 = input.getUTF8String(ordinal) val utf8ByteBuffer = utf8.getByteBuffer // todo: for off-heap UTF8String, how to pass in to arrow without copy? - valueMutator.setSafe(count, utf8ByteBuffer, utf8ByteBuffer.position(), utf8.numBytes()) + valueVector.setSafe(count, utf8ByteBuffer, utf8ByteBuffer.position(), utf8.numBytes()) } } private[arrow] class BinaryWriter( - val valueVector: NullableVarBinaryVector) extends ArrowFieldWriter { - - override def valueMutator: NullableVarBinaryVector#Mutator = valueVector.getMutator() + val valueVector: VarBinaryVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { val bytes = input.getBinary(ordinal) - valueMutator.setSafe(count, bytes, 0, bytes.length) + valueVector.setSafe(count, bytes, 0, bytes.length) } } -private[arrow] class DateWriter(val valueVector: NullableDateDayVector) extends ArrowFieldWriter { - - override def valueMutator: NullableDateDayVector#Mutator = valueVector.getMutator() +private[arrow] class DateWriter(val valueVector: DateDayVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getInt(ordinal)) + valueVector.setSafe(count, input.getInt(ordinal)) } } private[arrow] class TimestampWriter( - val valueVector: NullableTimeStampMicroTZVector) extends ArrowFieldWriter { - - override def valueMutator: NullableTimeStampMicroTZVector#Mutator = valueVector.getMutator() + val valueVector: TimeStampMicroTZVector) extends ArrowFieldWriter { override def setNull(): Unit = { - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { - valueMutator.setSafe(count, input.getLong(ordinal)) + valueVector.setSafe(count, input.getLong(ordinal)) } } @@ -286,20 +268,18 @@ private[arrow] class ArrayWriter( val valueVector: ListVector, val elementWriter: ArrowFieldWriter) extends ArrowFieldWriter { - override def valueMutator: ListVector#Mutator = valueVector.getMutator() - override def setNull(): Unit = { } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { val array = input.getArray(ordinal) var i = 0 - valueMutator.startNewValue(count) + valueVector.startNewValue(count) while (i < array.numElements()) { elementWriter.write(array, i) i += 1 } - valueMutator.endValue(count, array.numElements()) + valueVector.endValue(count, array.numElements()) } override def finish(): Unit = { @@ -317,8 +297,6 @@ private[arrow] class StructWriter( val valueVector: NullableMapVector, children: Array[ArrowFieldWriter]) extends ArrowFieldWriter { - override def valueMutator: NullableMapVector#Mutator = valueVector.getMutator() - override def setNull(): Unit = { var i = 0 while (i < children.length) { @@ -326,7 +304,7 @@ private[arrow] class StructWriter( children(i).count += 1 i += 1 } - valueMutator.setNull(count) + valueVector.setNull(count) } override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { @@ -336,7 +314,7 @@ private[arrow] class StructWriter( children(i).write(struct, i) i += 1 } - valueMutator.setIndexDefined(count) + valueVector.setIndexDefined(count) } override def finish(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala new file mode 100644 index 0000000000000..5267f5f1580c3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -0,0 +1,58 @@ +/* + * 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.regex.Pattern + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.{DataSourceV2, SessionConfigSupport} + +private[sql] object DataSourceV2Utils extends Logging { + + /** + * Helper method that extracts and transforms session configs into k/v pairs, the k/v pairs will + * be used to create data source options. + * Only extract when `ds` implements [[SessionConfigSupport]], in this case we may fetch the + * specified key-prefix from `ds`, and extract session configs with config keys that start with + * `spark.datasource.$keyPrefix`. A session config `spark.datasource.$keyPrefix.xxx -> yyy` will + * be transformed into `xxx -> yyy`. + * + * @param ds a [[DataSourceV2]] object + * @param conf the session conf + * @return an immutable map that contains all the extracted and transformed k/v pairs. + */ + def extractSessionConfigs(ds: DataSourceV2, conf: SQLConf): Map[String, String] = ds match { + case cs: SessionConfigSupport => + val keyPrefix = cs.keyPrefix() + require(keyPrefix != null, "The data source config key prefix can't be null.") + + val pattern = Pattern.compile(s"^spark\\.datasource\\.$keyPrefix\\.(.+)") + + conf.getAllConfs.flatMap { case (key, value) => + val m = pattern.matcher(key) + if (m.matches() && m.groupCount() > 0) { + Seq((m.group(1), value)) + } else { + Seq.empty + } + } + + case _ => Map.empty + } +} 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 82f0b9f5cd060..c8e236be28b42 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 @@ -252,54 +252,56 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { operator.withNewChildren(children) } - /** - * When the physical operators are created for JOIN, the ordering of join keys is based on order - * in which the join keys appear in the user query. That might not match with the output - * partitioning of the join node's children (thus leading to extra sort / shuffle being - * introduced). This rule will change the ordering of the join keys to match with the - * partitioning of the join nodes' children. - */ - def reorderJoinPredicates(plan: SparkPlan): SparkPlan = { - def reorderJoinKeys( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - leftPartitioning: Partitioning, - rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = { - - def reorder(expectedOrderOfKeys: Seq[Expression], - currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - val leftKeysBuffer = ArrayBuffer[Expression]() - val rightKeysBuffer = ArrayBuffer[Expression]() + private def reorder( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + expectedOrderOfKeys: Seq[Expression], + currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + val leftKeysBuffer = ArrayBuffer[Expression]() + val rightKeysBuffer = ArrayBuffer[Expression]() - expectedOrderOfKeys.foreach(expression => { - val index = currentOrderOfKeys.indexWhere(e => e.semanticEquals(expression)) - leftKeysBuffer.append(leftKeys(index)) - rightKeysBuffer.append(rightKeys(index)) - }) - (leftKeysBuffer, rightKeysBuffer) - } + expectedOrderOfKeys.foreach(expression => { + val index = currentOrderOfKeys.indexWhere(e => e.semanticEquals(expression)) + leftKeysBuffer.append(leftKeys(index)) + rightKeysBuffer.append(rightKeys(index)) + }) + (leftKeysBuffer, rightKeysBuffer) + } - 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(leftExpressions, leftKeys) + private def reorderJoinKeys( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + 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(rightExpressions, rightKeys) + 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) - } + case _ => (leftKeys, rightKeys) } - } else { - (leftKeys, rightKeys) } + } else { + (leftKeys, rightKeys) } + } + /** + * When the physical operators are created for JOIN, the ordering of join keys is based on order + * in which the join keys appear in the user query. That might not match with the output + * partitioning of the join node's children (thus leading to extra sort / shuffle being + * introduced). This rule will change the ordering of the join keys to match with the + * partitioning of the join nodes' children. + */ + private def reorderJoinPredicates(plan: SparkPlan): SparkPlan = { plan.transformUp { case BroadcastHashJoinExec(leftKeys, rightKeys, joinType, buildSide, condition, left, right) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 9a94d771a01b0..5cc8ed3535654 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.stream.{ArrowStreamReader, ArrowStreamWriter} +import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} import org.apache.spark._ import org.apache.spark.api.python._ @@ -74,13 +74,9 @@ class ArrowPythonRunner( val root = VectorSchemaRoot.create(arrowSchema, allocator) val arrowWriter = ArrowWriter.create(root) - var closed = false - context.addTaskCompletionListener { _ => - if (!closed) { - root.close() - allocator.close() - } + root.close() + allocator.close() } val writer = new ArrowStreamWriter(root, null, dataOut) @@ -102,7 +98,6 @@ class ArrowPythonRunner( writer.end() root.close() allocator.close() - closed = true } } } @@ -126,18 +121,11 @@ class ArrowPythonRunner( private var schema: StructType = _ private var vectors: Array[ColumnVector] = _ - private var closed = false - context.addTaskCompletionListener { _ => - // todo: we need something like `reader.end()`, which release all the resources, but leave - // the input stream open. `reader.close()` will close the socket and we can't reuse worker. - // So here we simply not close the reader, which is problematic. - if (!closed) { - if (root != null) { - root.close() - } - allocator.close() + if (reader != null) { + reader.close(false) } + allocator.close() } private var batchLoaded = true @@ -154,9 +142,8 @@ class ArrowPythonRunner( batch.setNumRows(root.getRowCount) batch } else { - root.close() + reader.close(false) allocator.close() - closed = true // Reach end of stream. Call `read()` again to read control data. read() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index a33b785126765..0debd7db84757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -27,7 +27,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala index 431e5b99e3e98..a2b49d944a688 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala @@ -22,8 +22,6 @@ import scala.util.control.Exception._ import org.json4s.NoTypeHints import org.json4s.jackson.Serialization -import org.apache.spark.sql.sources.v2.reader.Offset - /** * Offset for the [[FileStreamSource]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala index 7ea31462ca7b0..5f0b195fcfcb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.sources.v2.reader.Offset - /** * A simple offset for sources that produce a single linear stream of data. */ 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 a67dda99dc01b..4a3de8bae4bc9 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 @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} import org.apache.spark.util.{Clock, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java new file mode 100644 index 0000000000000..80aa5505db991 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java @@ -0,0 +1,61 @@ +/* + * 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; + +/** + * This is an internal, deprecated interface. New source implementations should use the + * org.apache.spark.sql.sources.v2.reader.Offset class, which is the one that will be supported + * in the long term. + * + * This class will be removed in a future release. + */ +public abstract class Offset { + /** + * A JSON-serialized representation of an Offset that is + * used for saving offsets to the offset log. + * Note: We assume that equivalent/equal offsets serialize to + * identical JSON strings. + * + * @return JSON string encoding + */ + public abstract String json(); + + /** + * Equality based on JSON string representation. We leverage the + * JSON representation for normalization between the Offset's + * in memory and on disk representations. + */ + @Override + public boolean equals(Object obj) { + if (obj instanceof Offset) { + return this.json().equals(((Offset) obj).json()); + } else { + return false; + } + } + + @Override + public int hashCode() { + return this.json().hashCode(); + } + + @Override + public String toString() { + return this.json(); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index dcc5935890c8d..4e0a468b962a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -23,7 +23,6 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.internal.SQLConf.{SHUFFLE_PARTITIONS, STATE_STORE_PROVIDER_CLASS} -import org.apache.spark.sql.sources.v2.reader.Offset /** * An ordered collection of offsets, used to track the progress of processing data from one or more diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala index bfdbc65296165..e3f4abcf9f1dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala @@ -24,7 +24,6 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.Offset /** * This class is used to log offsets to persistent files in HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala index 50671a46599e6..41761324cf6ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateSourceProvider.scala @@ -30,9 +30,10 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.execution.streaming.continuous.ContinuousRateStreamReader +import org.apache.spark.sql.execution.streaming.sources.RateStreamV2Reader import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, MicroBatchReader, Offset} +import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, MicroBatchReader} import org.apache.spark.sql.types._ import org.apache.spark.util.{ManualClock, SystemClock} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala index 13679dfbe446b..726d8574af52b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.execution.streaming import org.json4s.DefaultFormats import org.json4s.jackson.Serialization -import org.apache.spark.sql.sources.v2.reader.Offset +import org.apache.spark.sql.sources.v2 case class RateStreamOffset(partitionToValueAndRunTimeMs: Map[Int, (Long, Long)]) - extends Offset { + extends v2.reader.Offset { implicit val defaultFormats: DefaultFormats = DefaultFormats override val json = Serialization.write(partitionToValueAndRunTimeMs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala index 73f0c6221c5c1..129cfed860eb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SerializedOffset.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.sources.v2.reader.Offset - - /** * Used when loading a JSON serialized offset from external storage. * We are currently not responsible for converting JSON serialized diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index dbb408ffc98d8..311942f6dbd84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.StructType /** 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 7946889e85e37..129995dcf3607 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 @@ -36,7 +36,6 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 770db401c9fd7..a3f3662e6f4c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.streaming import scala.collection.{immutable, GenTraversableOnce} -import org.apache.spark.sql.sources.v2.reader.Offset - /** * A helper class that looks like a Map[Source, Offset]. */ 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 77fc26730e52c..4c3a1ee201ac1 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 @@ -25,7 +25,8 @@ import org.json4s.jackson.Serialization import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.{RateSourceProvider, RateStreamOffset} +import org.apache.spark.sql.execution.streaming.sources.RateStreamSourceV2 import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2, DataSourceV2Options} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index db0717510a2cb..3041d4d703cb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, Statistics} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index 440cae016a173..0b22cbc46e6bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -31,7 +31,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider} -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamSourceV2.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamSourceV2.scala index 102551c238bfb..45dc7d75cbc8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamSourceV2.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.sources import java.util.Optional @@ -27,6 +27,7 @@ import org.json4s.jackson.Serialization import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.streaming.RateStreamOffset import org.apache.spark.sql.sources.v2.DataSourceV2Options import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} @@ -59,7 +60,9 @@ class RateStreamV2Reader(options: DataSourceV2Options) private var start: RateStreamOffset = _ private var end: RateStreamOffset = _ - override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = { + override def setOffsetRange( + start: Optional[Offset], + end: Optional[Offset]): Unit = { this.start = start.orElse( RateStreamSourceV2.createInitialOffset(numPartitions, creationTimeMs)) .asInstanceOf[RateStreamOffset] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala index 437040cc12472..94c5dd63089b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memoryV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming +package org.apache.spark.sql.execution.streaming.sources import javax.annotation.concurrent.GuardedBy @@ -26,6 +26,7 @@ import scala.util.control.NonFatal import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update} +import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.sources.v2.{ContinuousWriteSupport, DataSourceV2, DataSourceV2Options, MicroBatchWriteSupport} import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.streaming.OutputMode diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index cf4c2a2fb9969..2295b8dd5fe36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -30,14 +30,11 @@ import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} import org.apache.spark.status.config._ -import org.apache.spark.ui.SparkUI -private[sql] class SQLAppStatusListener( +class SQLAppStatusListener( conf: SparkConf, kvstore: ElementTrackingStore, - live: Boolean, - ui: Option[SparkUI] = None) - extends SparkListener with Logging { + live: Boolean) extends SparkListener with Logging { // How often to flush intermediate state of a live execution to the store. When replaying logs, // never flush (only do the very last write). @@ -49,7 +46,10 @@ private[sql] class SQLAppStatusListener( private val liveExecutions = new ConcurrentHashMap[Long, LiveExecutionData]() private val stageMetrics = new ConcurrentHashMap[Int, LiveStageMetrics]() - private var uiInitialized = false + // Returns true if this listener has no live data. Exposed for tests only. + private[sql] def noLiveData(): Boolean = { + liveExecutions.isEmpty && stageMetrics.isEmpty + } kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get(UI_RETAINED_EXECUTIONS)) { count => cleanupExecutions(count) @@ -229,14 +229,6 @@ private[sql] class SQLAppStatusListener( } private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { - // Install the SQL tab in a live app if it hasn't been initialized yet. - if (!uiInitialized) { - ui.foreach { _ui => - new SQLTab(new SQLAppStatusStore(kvstore, Some(this)), _ui) - } - uiInitialized = true - } - val SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) = event @@ -388,7 +380,7 @@ private class LiveStageMetrics( val accumulatorIds: Array[Long], val taskMetrics: ConcurrentHashMap[Long, LiveTaskMetrics]) -private[sql] class LiveTaskMetrics( +private class LiveTaskMetrics( val ids: Array[Long], val values: Array[Long], val succeeded: Boolean) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala index 7fd5f7395cdf3..910f2e52fdbb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala @@ -25,21 +25,17 @@ import scala.collection.mutable.ArrayBuffer import com.fasterxml.jackson.databind.annotation.JsonDeserialize -import org.apache.spark.{JobExecutionStatus, SparkConf} -import org.apache.spark.scheduler.SparkListener -import org.apache.spark.status.{AppStatusPlugin, ElementTrackingStore} +import org.apache.spark.JobExecutionStatus import org.apache.spark.status.KVUtils.KVIndexParam -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils import org.apache.spark.util.kvstore.KVStore /** * Provides a view of a KVStore with methods that make it easy to query SQL-specific state. There's * no state kept in this class, so it's ok to have multiple instances of it in an application. */ -private[sql] class SQLAppStatusStore( +class SQLAppStatusStore( store: KVStore, - listener: Option[SQLAppStatusListener] = None) { + val listener: Option[SQLAppStatusListener] = None) { def executionsList(): Seq[SQLExecutionUIData] = { store.view(classOf[SQLExecutionUIData]).asScala.toSeq @@ -74,48 +70,9 @@ private[sql] class SQLAppStatusStore( def planGraph(executionId: Long): SparkPlanGraph = { store.read(classOf[SparkPlanGraphWrapper], executionId).toSparkPlanGraph() } - -} - -/** - * An AppStatusPlugin for handling the SQL UI and listeners. - */ -private[sql] class SQLAppStatusPlugin extends AppStatusPlugin { - - override def setupListeners( - conf: SparkConf, - store: ElementTrackingStore, - addListenerFn: SparkListener => Unit, - live: Boolean): Unit = { - // For live applications, the listener is installed in [[setupUI]]. This also avoids adding - // the listener when the UI is disabled. Force installation during testing, though. - if (!live || Utils.isTesting) { - val listener = new SQLAppStatusListener(conf, store, live, None) - addListenerFn(listener) - } - } - - override def setupUI(ui: SparkUI): Unit = { - ui.sc match { - case Some(sc) => - // If this is a live application, then install a listener that will enable the SQL - // tab as soon as there's a SQL event posted to the bus. - val listener = new SQLAppStatusListener(sc.conf, - ui.store.store.asInstanceOf[ElementTrackingStore], true, Some(ui)) - sc.listenerBus.addToStatusQueue(listener) - - case _ => - // For a replayed application, only add the tab if the store already contains SQL data. - val sqlStore = new SQLAppStatusStore(ui.store.store) - if (sqlStore.executionsCount() > 0) { - new SQLTab(sqlStore, ui) - } - } - } - } -private[sql] class SQLExecutionUIData( +class SQLExecutionUIData( @KVIndexParam val executionId: Long, val description: String, val details: String, @@ -133,10 +90,9 @@ private[sql] class SQLExecutionUIData( * from the SQL listener instance. */ @JsonDeserialize(keyAs = classOf[JLong]) - val metricValues: Map[Long, String] - ) + val metricValues: Map[Long, String]) -private[sql] class SparkPlanGraphWrapper( +class SparkPlanGraphWrapper( @KVIndexParam val executionId: Long, val nodes: Seq[SparkPlanGraphNodeWrapper], val edges: Seq[SparkPlanGraphEdge]) { @@ -147,7 +103,7 @@ private[sql] class SparkPlanGraphWrapper( } -private[sql] class SparkPlanGraphClusterWrapper( +class SparkPlanGraphClusterWrapper( val id: Long, val name: String, val desc: String, @@ -163,7 +119,7 @@ private[sql] class SparkPlanGraphClusterWrapper( } /** Only one of the values should be set. */ -private[sql] class SparkPlanGraphNodeWrapper( +class SparkPlanGraphNodeWrapper( val node: SparkPlanGraphNode, val cluster: SparkPlanGraphClusterWrapper) { @@ -174,7 +130,7 @@ private[sql] class SparkPlanGraphNodeWrapper( } -private[sql] case class SQLPlanMetric( +case class SQLPlanMetric( name: String, accumulatorId: Long, metricType: String) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala new file mode 100644 index 0000000000000..522d0cf79bffa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.ui + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} +import org.apache.spark.ui.SparkUI + +class SQLHistoryServerPlugin extends AppHistoryServerPlugin { + override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { + Seq(new SQLAppStatusListener(conf, store, live = false)) + } + + override def setupUI(ui: SparkUI): Unit = { + val sqlStatusStore = new SQLAppStatusStore(ui.store.store) + if (sqlStatusStore.executionsCount() > 0) { + new SQLTab(sqlStatusStore, ui) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 3e479faed72ac..baea4ceebf8e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -28,11 +28,12 @@ import org.apache.hadoop.fs.FsUrlStreamHandlerFactory import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager +import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.{MutableURLClassLoader, Utils} @@ -82,6 +83,19 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { */ val cacheManager: CacheManager = new CacheManager + /** + * A status store to query SQL status/metrics of this Spark application, based on SQL-specific + * [[org.apache.spark.scheduler.SparkListenerEvent]]s. + */ + val statusStore: SQLAppStatusStore = { + val kvStore = sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + val listener = new SQLAppStatusListener(sparkContext.conf, kvStore, live = true) + sparkContext.listenerBus.addToStatusQueue(listener) + val statusStore = new SQLAppStatusStore(kvStore, Some(listener)) + sparkContext.ui.foreach(new SQLTab(statusStore, _)) + statusStore + } + /** * A catalog that interacts with external systems. */ diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql new file mode 100644 index 0000000000000..8b04864b18ce3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql @@ -0,0 +1,1448 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql new file mode 100644 index 0000000000000..5cd3538757499 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql @@ -0,0 +1,44 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out new file mode 100644 index 0000000000000..ebc8201ed5a1d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -0,0 +1,9514 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1145 + + +-- !query 0 +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t +-- !query 1 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) + CAST(1 AS DECIMAL(3,0))):decimal(4,0)> +-- !query 1 output +2 + + +-- !query 2 +SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t +-- !query 2 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 2 output +2 + + +-- !query 3 +SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t +-- !query 3 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 3 output +2 + + +-- !query 4 +SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t +-- !query 4 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 4 output +2 + + +-- !query 5 +SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t +-- !query 5 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 5 output +2 + + +-- !query 6 +SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t +-- !query 6 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) + CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query 6 output +2 + + +-- !query 7 +SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t +-- !query 7 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 7 output +2 + + +-- !query 8 +SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t +-- !query 8 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 8 output +2 + + +-- !query 9 +SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t +-- !query 9 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 9 output +2 + + +-- !query 10 +SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t +-- !query 10 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 10 output +2 + + +-- !query 11 +SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t +-- !query 11 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 11 output +2 + + +-- !query 12 +SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t +-- !query 12 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 12 output +2 + + +-- !query 13 +SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t +-- !query 13 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 13 output +2 + + +-- !query 14 +SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t +-- !query 14 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 14 output +2 + + +-- !query 15 +SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t +-- !query 15 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 15 output +2 + + +-- !query 16 +SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t +-- !query 16 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query 16 output +2 + + +-- !query 17 +SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t +-- !query 17 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 17 output +2.0 + + +-- !query 18 +SELECT cast(1 as float) + cast(1 as decimal(5, 0)) FROM t +-- !query 18 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 18 output +2.0 + + +-- !query 19 +SELECT cast(1 as float) + cast(1 as decimal(10, 0)) FROM t +-- !query 19 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 19 output +2.0 + + +-- !query 20 +SELECT cast(1 as float) + cast(1 as decimal(20, 0)) FROM t +-- !query 20 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 20 output +2.0 + + +-- !query 21 +SELECT cast(1 as double) + cast(1 as decimal(3, 0)) FROM t +-- !query 21 schema +struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 21 output +2.0 + + +-- !query 22 +SELECT cast(1 as double) + cast(1 as decimal(5, 0)) FROM t +-- !query 22 schema +struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 22 output +2.0 + + +-- !query 23 +SELECT cast(1 as double) + cast(1 as decimal(10, 0)) FROM t +-- !query 23 schema +struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 23 output +2.0 + + +-- !query 24 +SELECT cast(1 as double) + cast(1 as decimal(20, 0)) FROM t +-- !query 24 schema +struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 24 output +2.0 + + +-- !query 25 +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t +-- !query 25 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 25 output +2 + + +-- !query 26 +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t +-- !query 26 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 26 output +2 + + +-- !query 27 +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query 27 schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 27 output +2 + + +-- !query 28 +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t +-- !query 28 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 28 output +2 + + +-- !query 29 +SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 30 +SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t +-- !query 30 schema +struct<> +-- !query 30 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 31 +SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 32 +SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 33 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 34 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 35 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t +-- !query 35 schema +struct<> +-- !query 35 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 36 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 37 +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 38 +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 39 +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t +-- !query 39 schema +struct<> +-- !query 39 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 40 +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 41 +SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t +-- !query 41 schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(4,0)> +-- !query 41 output +2 + + +-- !query 42 +SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t +-- !query 42 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 42 output +2 + + +-- !query 43 +SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t +-- !query 43 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 43 output +2 + + +-- !query 44 +SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t +-- !query 44 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 44 output +2 + + +-- !query 45 +SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t +-- !query 45 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 45 output +2 + + +-- !query 46 +SELECT cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t +-- !query 46 schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(6,0)> +-- !query 46 output +2 + + +-- !query 47 +SELECT cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t +-- !query 47 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 47 output +2 + + +-- !query 48 +SELECT cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t +-- !query 48 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 48 output +2 + + +-- !query 49 +SELECT cast(1 as decimal(3, 0)) + cast(1 as int) FROM t +-- !query 49 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 49 output +2 + + +-- !query 50 +SELECT cast(1 as decimal(5, 0)) + cast(1 as int) FROM t +-- !query 50 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 50 output +2 + + +-- !query 51 +SELECT cast(1 as decimal(10, 0)) + cast(1 as int) FROM t +-- !query 51 schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(11,0)> +-- !query 51 output +2 + + +-- !query 52 +SELECT cast(1 as decimal(20, 0)) + cast(1 as int) FROM t +-- !query 52 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 52 output +2 + + +-- !query 53 +SELECT cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t +-- !query 53 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 53 output +2 + + +-- !query 54 +SELECT cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t +-- !query 54 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 54 output +2 + + +-- !query 55 +SELECT cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t +-- !query 55 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 55 output +2 + + +-- !query 56 +SELECT cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t +-- !query 56 schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(21,0)> +-- !query 56 output +2 + + +-- !query 57 +SELECT cast(1 as decimal(3, 0)) + cast(1 as float) FROM t +-- !query 57 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 57 output +2.0 + + +-- !query 58 +SELECT cast(1 as decimal(5, 0)) + cast(1 as float) FROM t +-- !query 58 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 58 output +2.0 + + +-- !query 59 +SELECT cast(1 as decimal(10, 0)) + cast(1 as float) FROM t +-- !query 59 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 59 output +2.0 + + +-- !query 60 +SELECT cast(1 as decimal(20, 0)) + cast(1 as float) FROM t +-- !query 60 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 60 output +2.0 + + +-- !query 61 +SELECT cast(1 as decimal(3, 0)) + cast(1 as double) FROM t +-- !query 61 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> +-- !query 61 output +2.0 + + +-- !query 62 +SELECT cast(1 as decimal(5, 0)) + cast(1 as double) FROM t +-- !query 62 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> +-- !query 62 output +2.0 + + +-- !query 63 +SELECT cast(1 as decimal(10, 0)) + cast(1 as double) FROM t +-- !query 63 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> +-- !query 63 output +2.0 + + +-- !query 64 +SELECT cast(1 as decimal(20, 0)) + cast(1 as double) FROM t +-- !query 64 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> +-- !query 64 output +2.0 + + +-- !query 65 +SELECT cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query 65 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 65 output +2 + + +-- !query 66 +SELECT cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query 66 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 66 output +2 + + +-- !query 67 +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query 67 schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 67 output +2 + + +-- !query 68 +SELECT cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query 68 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 68 output +2 + + +-- !query 69 +SELECT cast(1 as decimal(3, 0)) + cast(1 as string) FROM t +-- !query 69 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 69 output +2.0 + + +-- !query 70 +SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t +-- !query 70 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 70 output +2.0 + + +-- !query 71 +SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t +-- !query 71 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 71 output +2.0 + + +-- !query 72 +SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t +-- !query 72 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 72 output +2.0 + + +-- !query 73 +SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t +-- !query 73 schema +struct<> +-- !query 73 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 74 +SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t +-- !query 74 schema +struct<> +-- !query 74 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 75 +SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t +-- !query 75 schema +struct<> +-- !query 75 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 76 +SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t +-- !query 76 schema +struct<> +-- !query 76 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 77 +SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t +-- !query 77 schema +struct<> +-- !query 77 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 78 +SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t +-- !query 78 schema +struct<> +-- !query 78 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 79 +SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t +-- !query 79 schema +struct<> +-- !query 79 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 80 +SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t +-- !query 80 schema +struct<> +-- !query 80 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 81 +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 81 schema +struct<> +-- !query 81 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 82 +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 82 schema +struct<> +-- !query 82 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 83 +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 83 schema +struct<> +-- !query 83 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 84 +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 84 schema +struct<> +-- !query 84 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 85 +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query 85 schema +struct<> +-- !query 85 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 86 +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query 86 schema +struct<> +-- !query 86 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 87 +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query 87 schema +struct<> +-- !query 87 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 88 +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query 88 schema +struct<> +-- !query 88 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 89 +SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t +-- !query 89 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) - CAST(1 AS DECIMAL(3,0))):decimal(4,0)> +-- !query 89 output +0 + + +-- !query 90 +SELECT cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t +-- !query 90 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 90 output +0 + + +-- !query 91 +SELECT cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t +-- !query 91 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 91 output +0 + + +-- !query 92 +SELECT cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t +-- !query 92 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 92 output +0 + + +-- !query 93 +SELECT cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t +-- !query 93 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 93 output +0 + + +-- !query 94 +SELECT cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t +-- !query 94 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) - CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query 94 output +0 + + +-- !query 95 +SELECT cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t +-- !query 95 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 95 output +0 + + +-- !query 96 +SELECT cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t +-- !query 96 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 96 output +0 + + +-- !query 97 +SELECT cast(1 as int) - cast(1 as decimal(3, 0)) FROM t +-- !query 97 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 97 output +0 + + +-- !query 98 +SELECT cast(1 as int) - cast(1 as decimal(5, 0)) FROM t +-- !query 98 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 98 output +0 + + +-- !query 99 +SELECT cast(1 as int) - cast(1 as decimal(10, 0)) FROM t +-- !query 99 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 99 output +0 + + +-- !query 100 +SELECT cast(1 as int) - cast(1 as decimal(20, 0)) FROM t +-- !query 100 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 100 output +0 + + +-- !query 101 +SELECT cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t +-- !query 101 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 101 output +0 + + +-- !query 102 +SELECT cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t +-- !query 102 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 102 output +0 + + +-- !query 103 +SELECT cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t +-- !query 103 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 103 output +0 + + +-- !query 104 +SELECT cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t +-- !query 104 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query 104 output +0 + + +-- !query 105 +SELECT cast(1 as float) - cast(1 as decimal(3, 0)) FROM t +-- !query 105 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 105 output +0.0 + + +-- !query 106 +SELECT cast(1 as float) - cast(1 as decimal(5, 0)) FROM t +-- !query 106 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 106 output +0.0 + + +-- !query 107 +SELECT cast(1 as float) - cast(1 as decimal(10, 0)) FROM t +-- !query 107 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 107 output +0.0 + + +-- !query 108 +SELECT cast(1 as float) - cast(1 as decimal(20, 0)) FROM t +-- !query 108 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 108 output +0.0 + + +-- !query 109 +SELECT cast(1 as double) - cast(1 as decimal(3, 0)) FROM t +-- !query 109 schema +struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 109 output +0.0 + + +-- !query 110 +SELECT cast(1 as double) - cast(1 as decimal(5, 0)) FROM t +-- !query 110 schema +struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 110 output +0.0 + + +-- !query 111 +SELECT cast(1 as double) - cast(1 as decimal(10, 0)) FROM t +-- !query 111 schema +struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 111 output +0.0 + + +-- !query 112 +SELECT cast(1 as double) - cast(1 as decimal(20, 0)) FROM t +-- !query 112 schema +struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 112 output +0.0 + + +-- !query 113 +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t +-- !query 113 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 113 output +0 + + +-- !query 114 +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t +-- !query 114 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 114 output +0 + + +-- !query 115 +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query 115 schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 115 output +0 + + +-- !query 116 +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t +-- !query 116 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 116 output +0 + + +-- !query 117 +SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t +-- !query 117 schema +struct<> +-- !query 117 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 118 +SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t +-- !query 118 schema +struct<> +-- !query 118 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 119 +SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t +-- !query 119 schema +struct<> +-- !query 119 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 120 +SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t +-- !query 120 schema +struct<> +-- !query 120 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 121 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t +-- !query 121 schema +struct<> +-- !query 121 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 122 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t +-- !query 122 schema +struct<> +-- !query 122 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 123 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t +-- !query 123 schema +struct<> +-- !query 123 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 124 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t +-- !query 124 schema +struct<> +-- !query 124 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 125 +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t +-- !query 125 schema +struct<> +-- !query 125 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 126 +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t +-- !query 126 schema +struct<> +-- !query 126 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 127 +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t +-- !query 127 schema +struct<> +-- !query 127 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 128 +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t +-- !query 128 schema +struct<> +-- !query 128 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 129 +SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t +-- !query 129 schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(4,0)> +-- !query 129 output +0 + + +-- !query 130 +SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t +-- !query 130 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 130 output +0 + + +-- !query 131 +SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t +-- !query 131 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 131 output +0 + + +-- !query 132 +SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t +-- !query 132 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 132 output +0 + + +-- !query 133 +SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t +-- !query 133 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> +-- !query 133 output +0 + + +-- !query 134 +SELECT cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t +-- !query 134 schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(6,0)> +-- !query 134 output +0 + + +-- !query 135 +SELECT cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t +-- !query 135 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 135 output +0 + + +-- !query 136 +SELECT cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t +-- !query 136 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 136 output +0 + + +-- !query 137 +SELECT cast(1 as decimal(3, 0)) - cast(1 as int) FROM t +-- !query 137 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 137 output +0 + + +-- !query 138 +SELECT cast(1 as decimal(5, 0)) - cast(1 as int) FROM t +-- !query 138 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 138 output +0 + + +-- !query 139 +SELECT cast(1 as decimal(10, 0)) - cast(1 as int) FROM t +-- !query 139 schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(11,0)> +-- !query 139 output +0 + + +-- !query 140 +SELECT cast(1 as decimal(20, 0)) - cast(1 as int) FROM t +-- !query 140 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 140 output +0 + + +-- !query 141 +SELECT cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t +-- !query 141 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 141 output +0 + + +-- !query 142 +SELECT cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t +-- !query 142 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 142 output +0 + + +-- !query 143 +SELECT cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t +-- !query 143 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 143 output +0 + + +-- !query 144 +SELECT cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t +-- !query 144 schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(21,0)> +-- !query 144 output +0 + + +-- !query 145 +SELECT cast(1 as decimal(3, 0)) - cast(1 as float) FROM t +-- !query 145 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 145 output +0.0 + + +-- !query 146 +SELECT cast(1 as decimal(5, 0)) - cast(1 as float) FROM t +-- !query 146 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 146 output +0.0 + + +-- !query 147 +SELECT cast(1 as decimal(10, 0)) - cast(1 as float) FROM t +-- !query 147 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 147 output +0.0 + + +-- !query 148 +SELECT cast(1 as decimal(20, 0)) - cast(1 as float) FROM t +-- !query 148 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 148 output +0.0 + + +-- !query 149 +SELECT cast(1 as decimal(3, 0)) - cast(1 as double) FROM t +-- !query 149 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> +-- !query 149 output +0.0 + + +-- !query 150 +SELECT cast(1 as decimal(5, 0)) - cast(1 as double) FROM t +-- !query 150 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> +-- !query 150 output +0.0 + + +-- !query 151 +SELECT cast(1 as decimal(10, 0)) - cast(1 as double) FROM t +-- !query 151 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> +-- !query 151 output +0.0 + + +-- !query 152 +SELECT cast(1 as decimal(20, 0)) - cast(1 as double) FROM t +-- !query 152 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> +-- !query 152 output +0.0 + + +-- !query 153 +SELECT cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query 153 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 153 output +0 + + +-- !query 154 +SELECT cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query 154 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> +-- !query 154 output +0 + + +-- !query 155 +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query 155 schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query 155 output +0 + + +-- !query 156 +SELECT cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query 156 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> +-- !query 156 output +0 + + +-- !query 157 +SELECT cast(1 as decimal(3, 0)) - cast(1 as string) FROM t +-- !query 157 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 157 output +0.0 + + +-- !query 158 +SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t +-- !query 158 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 158 output +0.0 + + +-- !query 159 +SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t +-- !query 159 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 159 output +0.0 + + +-- !query 160 +SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t +-- !query 160 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 160 output +0.0 + + +-- !query 161 +SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t +-- !query 161 schema +struct<> +-- !query 161 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 162 +SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t +-- !query 162 schema +struct<> +-- !query 162 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 163 +SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t +-- !query 163 schema +struct<> +-- !query 163 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 164 +SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t +-- !query 164 schema +struct<> +-- !query 164 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 165 +SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t +-- !query 165 schema +struct<> +-- !query 165 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 166 +SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t +-- !query 166 schema +struct<> +-- !query 166 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 167 +SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t +-- !query 167 schema +struct<> +-- !query 167 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 168 +SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t +-- !query 168 schema +struct<> +-- !query 168 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 169 +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 169 schema +struct<> +-- !query 169 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 170 +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 170 schema +struct<> +-- !query 170 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 171 +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 171 schema +struct<> +-- !query 171 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 172 +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 172 schema +struct<> +-- !query 172 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 173 +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query 173 schema +struct<> +-- !query 173 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 174 +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query 174 schema +struct<> +-- !query 174 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 175 +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query 175 schema +struct<> +-- !query 175 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 176 +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query 176 schema +struct<> +-- !query 176 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 177 +SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t +-- !query 177 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) * CAST(1 AS DECIMAL(3,0))):decimal(7,0)> +-- !query 177 output +1 + + +-- !query 178 +SELECT cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t +-- !query 178 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,0)> +-- !query 178 output +1 + + +-- !query 179 +SELECT cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t +-- !query 179 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 179 output +1 + + +-- !query 180 +SELECT cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t +-- !query 180 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,0)> +-- !query 180 output +1 + + +-- !query 181 +SELECT cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t +-- !query 181 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(9,0)> +-- !query 181 output +1 + + +-- !query 182 +SELECT cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t +-- !query 182 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) * CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query 182 output +1 + + +-- !query 183 +SELECT cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t +-- !query 183 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 183 output +1 + + +-- !query 184 +SELECT cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t +-- !query 184 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,0)> +-- !query 184 output +1 + + +-- !query 185 +SELECT cast(1 as int) * cast(1 as decimal(3, 0)) FROM t +-- !query 185 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 185 output +1 + + +-- !query 186 +SELECT cast(1 as int) * cast(1 as decimal(5, 0)) FROM t +-- !query 186 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 186 output +1 + + +-- !query 187 +SELECT cast(1 as int) * cast(1 as decimal(10, 0)) FROM t +-- !query 187 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query 187 output +1 + + +-- !query 188 +SELECT cast(1 as int) * cast(1 as decimal(20, 0)) FROM t +-- !query 188 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 188 output +1 + + +-- !query 189 +SELECT cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t +-- !query 189 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(24,0)> +-- !query 189 output +1 + + +-- !query 190 +SELECT cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t +-- !query 190 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,0)> +-- !query 190 output +1 + + +-- !query 191 +SELECT cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t +-- !query 191 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 191 output +1 + + +-- !query 192 +SELECT cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t +-- !query 192 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) * CAST(1 AS DECIMAL(20,0))):decimal(38,0)> +-- !query 192 output +1 + + +-- !query 193 +SELECT cast(1 as float) * cast(1 as decimal(3, 0)) FROM t +-- !query 193 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 193 output +1.0 + + +-- !query 194 +SELECT cast(1 as float) * cast(1 as decimal(5, 0)) FROM t +-- !query 194 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 194 output +1.0 + + +-- !query 195 +SELECT cast(1 as float) * cast(1 as decimal(10, 0)) FROM t +-- !query 195 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 195 output +1.0 + + +-- !query 196 +SELECT cast(1 as float) * cast(1 as decimal(20, 0)) FROM t +-- !query 196 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 196 output +1.0 + + +-- !query 197 +SELECT cast(1 as double) * cast(1 as decimal(3, 0)) FROM t +-- !query 197 schema +struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 197 output +1.0 + + +-- !query 198 +SELECT cast(1 as double) * cast(1 as decimal(5, 0)) FROM t +-- !query 198 schema +struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 198 output +1.0 + + +-- !query 199 +SELECT cast(1 as double) * cast(1 as decimal(10, 0)) FROM t +-- !query 199 schema +struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 199 output +1.0 + + +-- !query 200 +SELECT cast(1 as double) * cast(1 as decimal(20, 0)) FROM t +-- !query 200 schema +struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 200 output +1.0 + + +-- !query 201 +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t +-- !query 201 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 201 output +1 + + +-- !query 202 +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t +-- !query 202 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 202 output +1 + + +-- !query 203 +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query 203 schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query 203 output +1 + + +-- !query 204 +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t +-- !query 204 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 204 output +1 + + +-- !query 205 +SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t +-- !query 205 schema +struct<> +-- !query 205 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 206 +SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t +-- !query 206 schema +struct<> +-- !query 206 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 207 +SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t +-- !query 207 schema +struct<> +-- !query 207 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 208 +SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t +-- !query 208 schema +struct<> +-- !query 208 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 209 +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t +-- !query 209 schema +struct<> +-- !query 209 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 210 +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t +-- !query 210 schema +struct<> +-- !query 210 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 211 +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t +-- !query 211 schema +struct<> +-- !query 211 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 212 +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t +-- !query 212 schema +struct<> +-- !query 212 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 213 +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t +-- !query 213 schema +struct<> +-- !query 213 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 214 +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t +-- !query 214 schema +struct<> +-- !query 214 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 215 +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t +-- !query 215 schema +struct<> +-- !query 215 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 216 +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t +-- !query 216 schema +struct<> +-- !query 216 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 217 +SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t +-- !query 217 schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(7,0)> +-- !query 217 output +1 + + +-- !query 218 +SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t +-- !query 218 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(9,0)> +-- !query 218 output +1 + + +-- !query 219 +SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t +-- !query 219 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 219 output +1 + + +-- !query 220 +SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t +-- !query 220 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(24,0)> +-- !query 220 output +1 + + +-- !query 221 +SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t +-- !query 221 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,0)> +-- !query 221 output +1 + + +-- !query 222 +SELECT cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t +-- !query 222 schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,0)> +-- !query 222 output +1 + + +-- !query 223 +SELECT cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t +-- !query 223 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 223 output +1 + + +-- !query 224 +SELECT cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t +-- !query 224 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,0)> +-- !query 224 output +1 + + +-- !query 225 +SELECT cast(1 as decimal(3, 0)) * cast(1 as int) FROM t +-- !query 225 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 225 output +1 + + +-- !query 226 +SELECT cast(1 as decimal(5, 0)) * cast(1 as int) FROM t +-- !query 226 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 226 output +1 + + +-- !query 227 +SELECT cast(1 as decimal(10, 0)) * cast(1 as int) FROM t +-- !query 227 schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,0)> +-- !query 227 output +1 + + +-- !query 228 +SELECT cast(1 as decimal(20, 0)) * cast(1 as int) FROM t +-- !query 228 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 228 output +1 + + +-- !query 229 +SELECT cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t +-- !query 229 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,0)> +-- !query 229 output +1 + + +-- !query 230 +SELECT cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t +-- !query 230 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,0)> +-- !query 230 output +1 + + +-- !query 231 +SELECT cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t +-- !query 231 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 231 output +1 + + +-- !query 232 +SELECT cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t +-- !query 232 schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,0)> +-- !query 232 output +1 + + +-- !query 233 +SELECT cast(1 as decimal(3, 0)) * cast(1 as float) FROM t +-- !query 233 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 233 output +1.0 + + +-- !query 234 +SELECT cast(1 as decimal(5, 0)) * cast(1 as float) FROM t +-- !query 234 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 234 output +1.0 + + +-- !query 235 +SELECT cast(1 as decimal(10, 0)) * cast(1 as float) FROM t +-- !query 235 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 235 output +1.0 + + +-- !query 236 +SELECT cast(1 as decimal(20, 0)) * cast(1 as float) FROM t +-- !query 236 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 236 output +1.0 + + +-- !query 237 +SELECT cast(1 as decimal(3, 0)) * cast(1 as double) FROM t +-- !query 237 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> +-- !query 237 output +1.0 + + +-- !query 238 +SELECT cast(1 as decimal(5, 0)) * cast(1 as double) FROM t +-- !query 238 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> +-- !query 238 output +1.0 + + +-- !query 239 +SELECT cast(1 as decimal(10, 0)) * cast(1 as double) FROM t +-- !query 239 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> +-- !query 239 output +1.0 + + +-- !query 240 +SELECT cast(1 as decimal(20, 0)) * cast(1 as double) FROM t +-- !query 240 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> +-- !query 240 output +1.0 + + +-- !query 241 +SELECT cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query 241 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> +-- !query 241 output +1 + + +-- !query 242 +SELECT cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query 242 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> +-- !query 242 output +1 + + +-- !query 243 +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query 243 schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query 243 output +1 + + +-- !query 244 +SELECT cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query 244 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> +-- !query 244 output +1 + + +-- !query 245 +SELECT cast(1 as decimal(3, 0)) * cast(1 as string) FROM t +-- !query 245 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 245 output +1.0 + + +-- !query 246 +SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t +-- !query 246 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 246 output +1.0 + + +-- !query 247 +SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t +-- !query 247 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 247 output +1.0 + + +-- !query 248 +SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t +-- !query 248 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 248 output +1.0 + + +-- !query 249 +SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t +-- !query 249 schema +struct<> +-- !query 249 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 250 +SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t +-- !query 250 schema +struct<> +-- !query 250 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 251 +SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t +-- !query 251 schema +struct<> +-- !query 251 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 252 +SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t +-- !query 252 schema +struct<> +-- !query 252 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 253 +SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t +-- !query 253 schema +struct<> +-- !query 253 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 254 +SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t +-- !query 254 schema +struct<> +-- !query 254 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 255 +SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t +-- !query 255 schema +struct<> +-- !query 255 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 256 +SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t +-- !query 256 schema +struct<> +-- !query 256 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 257 +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query 257 schema +struct<> +-- !query 257 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 258 +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query 258 schema +struct<> +-- !query 258 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 259 +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query 259 schema +struct<> +-- !query 259 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 260 +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query 260 schema +struct<> +-- !query 260 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 261 +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query 261 schema +struct<> +-- !query 261 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 262 +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query 262 schema +struct<> +-- !query 262 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 263 +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query 263 schema +struct<> +-- !query 263 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 264 +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query 264 schema +struct<> +-- !query 264 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 265 +SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t +-- !query 265 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> +-- !query 265 output +1 + + +-- !query 266 +SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t +-- !query 266 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> +-- !query 266 output +1 + + +-- !query 267 +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t +-- !query 267 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> +-- !query 267 output +1 + + +-- !query 268 +SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t +-- !query 268 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> +-- !query 268 output +1 + + +-- !query 269 +SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t +-- !query 269 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> +-- !query 269 output +1 + + +-- !query 270 +SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t +-- !query 270 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> +-- !query 270 output +1 + + +-- !query 271 +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t +-- !query 271 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> +-- !query 271 output +1 + + +-- !query 272 +SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t +-- !query 272 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> +-- !query 272 output +1 + + +-- !query 273 +SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t +-- !query 273 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 273 output +1 + + +-- !query 274 +SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t +-- !query 274 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 274 output +1 + + +-- !query 275 +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t +-- !query 275 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query 275 output +1 + + +-- !query 276 +SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t +-- !query 276 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> +-- !query 276 output +1 + + +-- !query 277 +SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t +-- !query 277 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> +-- !query 277 output +1 + + +-- !query 278 +SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t +-- !query 278 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> +-- !query 278 output +1 + + +-- !query 279 +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t +-- !query 279 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> +-- !query 279 output +1 + + +-- !query 280 +SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t +-- !query 280 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(20,0))):decimal(38,19)> +-- !query 280 output +1 + + +-- !query 281 +SELECT cast(1 as float) / cast(1 as decimal(3, 0)) FROM t +-- !query 281 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) AS DOUBLE)):double> +-- !query 281 output +1.0 + + +-- !query 282 +SELECT cast(1 as float) / cast(1 as decimal(5, 0)) FROM t +-- !query 282 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) AS DOUBLE)):double> +-- !query 282 output +1.0 + + +-- !query 283 +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t +-- !query 283 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) AS DOUBLE)):double> +-- !query 283 output +1.0 + + +-- !query 284 +SELECT cast(1 as float) / cast(1 as decimal(20, 0)) FROM t +-- !query 284 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) AS DOUBLE)):double> +-- !query 284 output +1.0 + + +-- !query 285 +SELECT cast(1 as double) / cast(1 as decimal(3, 0)) FROM t +-- !query 285 schema +struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 285 output +1.0 + + +-- !query 286 +SELECT cast(1 as double) / cast(1 as decimal(5, 0)) FROM t +-- !query 286 schema +struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 286 output +1.0 + + +-- !query 287 +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t +-- !query 287 schema +struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 287 output +1.0 + + +-- !query 288 +SELECT cast(1 as double) / cast(1 as decimal(20, 0)) FROM t +-- !query 288 schema +struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 288 output +1.0 + + +-- !query 289 +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t +-- !query 289 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 289 output +1 + + +-- !query 290 +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t +-- !query 290 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 290 output +1 + + +-- !query 291 +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query 291 schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query 291 output +1 + + +-- !query 292 +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t +-- !query 292 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> +-- !query 292 output +1 + + +-- !query 293 +SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t +-- !query 293 schema +struct<> +-- !query 293 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 294 +SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t +-- !query 294 schema +struct<> +-- !query 294 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 295 +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t +-- !query 295 schema +struct<> +-- !query 295 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 296 +SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t +-- !query 296 schema +struct<> +-- !query 296 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 297 +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t +-- !query 297 schema +struct<> +-- !query 297 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 298 +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t +-- !query 298 schema +struct<> +-- !query 298 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 299 +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t +-- !query 299 schema +struct<> +-- !query 299 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 300 +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t +-- !query 300 schema +struct<> +-- !query 300 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 301 +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t +-- !query 301 schema +struct<> +-- !query 301 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 302 +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t +-- !query 302 schema +struct<> +-- !query 302 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 303 +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t +-- !query 303 schema +struct<> +-- !query 303 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 304 +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t +-- !query 304 schema +struct<> +-- !query 304 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 305 +SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t +-- !query 305 schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(9,6)> +-- !query 305 output +1 + + +-- !query 306 +SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t +-- !query 306 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> +-- !query 306 output +1 + + +-- !query 307 +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t +-- !query 307 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 307 output +1 + + +-- !query 308 +SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t +-- !query 308 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> +-- !query 308 output +1 + + +-- !query 309 +SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t +-- !query 309 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> +-- !query 309 output +1 + + +-- !query 310 +SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t +-- !query 310 schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,6)> +-- !query 310 output +1 + + +-- !query 311 +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t +-- !query 311 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> +-- !query 311 output +1 + + +-- !query 312 +SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t +-- !query 312 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> +-- !query 312 output +1 + + +-- !query 313 +SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t +-- !query 313 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> +-- !query 313 output +1 + + +-- !query 314 +SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t +-- !query 314 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> +-- !query 314 output +1 + + +-- !query 315 +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t +-- !query 315 schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> +-- !query 315 output +1 + + +-- !query 316 +SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t +-- !query 316 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> +-- !query 316 output +1 + + +-- !query 317 +SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t +-- !query 317 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> +-- !query 317 output +1 + + +-- !query 318 +SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t +-- !query 318 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> +-- !query 318 output +1 + + +-- !query 319 +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t +-- !query 319 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> +-- !query 319 output +1 + + +-- !query 320 +SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t +-- !query 320 schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,19)> +-- !query 320 output +1 + + +-- !query 321 +SELECT cast(1 as decimal(3, 0)) / cast(1 as float) FROM t +-- !query 321 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 321 output +1.0 + + +-- !query 322 +SELECT cast(1 as decimal(5, 0)) / cast(1 as float) FROM t +-- !query 322 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 322 output +1.0 + + +-- !query 323 +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t +-- !query 323 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 323 output +1.0 + + +-- !query 324 +SELECT cast(1 as decimal(20, 0)) / cast(1 as float) FROM t +-- !query 324 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 324 output +1.0 + + +-- !query 325 +SELECT cast(1 as decimal(3, 0)) / cast(1 as double) FROM t +-- !query 325 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> +-- !query 325 output +1.0 + + +-- !query 326 +SELECT cast(1 as decimal(5, 0)) / cast(1 as double) FROM t +-- !query 326 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> +-- !query 326 output +1.0 + + +-- !query 327 +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t +-- !query 327 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> +-- !query 327 output +1.0 + + +-- !query 328 +SELECT cast(1 as decimal(20, 0)) / cast(1 as double) FROM t +-- !query 328 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> +-- !query 328 output +1.0 + + +-- !query 329 +SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query 329 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> +-- !query 329 output +1 + + +-- !query 330 +SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query 330 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> +-- !query 330 output +1 + + +-- !query 331 +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query 331 schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query 331 output +1 + + +-- !query 332 +SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query 332 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> +-- !query 332 output +1 + + +-- !query 333 +SELECT cast(1 as decimal(3, 0)) / cast(1 as string) FROM t +-- !query 333 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 333 output +1.0 + + +-- !query 334 +SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t +-- !query 334 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 334 output +1.0 + + +-- !query 335 +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t +-- !query 335 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 335 output +1.0 + + +-- !query 336 +SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t +-- !query 336 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 336 output +1.0 + + +-- !query 337 +SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t +-- !query 337 schema +struct<> +-- !query 337 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 338 +SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t +-- !query 338 schema +struct<> +-- !query 338 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 339 +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t +-- !query 339 schema +struct<> +-- !query 339 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 340 +SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t +-- !query 340 schema +struct<> +-- !query 340 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 341 +SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t +-- !query 341 schema +struct<> +-- !query 341 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 342 +SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t +-- !query 342 schema +struct<> +-- !query 342 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 343 +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t +-- !query 343 schema +struct<> +-- !query 343 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 344 +SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t +-- !query 344 schema +struct<> +-- !query 344 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 345 +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query 345 schema +struct<> +-- !query 345 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 346 +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query 346 schema +struct<> +-- !query 346 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 347 +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query 347 schema +struct<> +-- !query 347 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 348 +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query 348 schema +struct<> +-- !query 348 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 349 +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query 349 schema +struct<> +-- !query 349 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 350 +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query 350 schema +struct<> +-- !query 350 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 351 +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query 351 schema +struct<> +-- !query 351 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 352 +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query 352 schema +struct<> +-- !query 352 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 353 +SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t +-- !query 353 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query 353 output +0 + + +-- !query 354 +SELECT cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t +-- !query 354 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(3,0)> +-- !query 354 output +0 + + +-- !query 355 +SELECT cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t +-- !query 355 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 355 output +0 + + +-- !query 356 +SELECT cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t +-- !query 356 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 356 output +0 + + +-- !query 357 +SELECT cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t +-- !query 357 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(3,0)> +-- !query 357 output +0 + + +-- !query 358 +SELECT cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t +-- !query 358 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> +-- !query 358 output +0 + + +-- !query 359 +SELECT cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t +-- !query 359 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 359 output +0 + + +-- !query 360 +SELECT cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t +-- !query 360 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(5,0)> +-- !query 360 output +0 + + +-- !query 361 +SELECT cast(1 as int) % cast(1 as decimal(3, 0)) FROM t +-- !query 361 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 361 output +0 + + +-- !query 362 +SELECT cast(1 as int) % cast(1 as decimal(5, 0)) FROM t +-- !query 362 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 362 output +0 + + +-- !query 363 +SELECT cast(1 as int) % cast(1 as decimal(10, 0)) FROM t +-- !query 363 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query 363 output +0 + + +-- !query 364 +SELECT cast(1 as int) % cast(1 as decimal(20, 0)) FROM t +-- !query 364 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 364 output +0 + + +-- !query 365 +SELECT cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t +-- !query 365 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 365 output +0 + + +-- !query 366 +SELECT cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t +-- !query 366 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(5,0)> +-- !query 366 output +0 + + +-- !query 367 +SELECT cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t +-- !query 367 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 367 output +0 + + +-- !query 368 +SELECT cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t +-- !query 368 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) % CAST(1 AS DECIMAL(20,0))):decimal(20,0)> +-- !query 368 output +0 + + +-- !query 369 +SELECT cast(1 as float) % cast(1 as decimal(3, 0)) FROM t +-- !query 369 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 369 output +0.0 + + +-- !query 370 +SELECT cast(1 as float) % cast(1 as decimal(5, 0)) FROM t +-- !query 370 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 370 output +0.0 + + +-- !query 371 +SELECT cast(1 as float) % cast(1 as decimal(10, 0)) FROM t +-- !query 371 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 371 output +0.0 + + +-- !query 372 +SELECT cast(1 as float) % cast(1 as decimal(20, 0)) FROM t +-- !query 372 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 372 output +0.0 + + +-- !query 373 +SELECT cast(1 as double) % cast(1 as decimal(3, 0)) FROM t +-- !query 373 schema +struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> +-- !query 373 output +0.0 + + +-- !query 374 +SELECT cast(1 as double) % cast(1 as decimal(5, 0)) FROM t +-- !query 374 schema +struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> +-- !query 374 output +0.0 + + +-- !query 375 +SELECT cast(1 as double) % cast(1 as decimal(10, 0)) FROM t +-- !query 375 schema +struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> +-- !query 375 output +0.0 + + +-- !query 376 +SELECT cast(1 as double) % cast(1 as decimal(20, 0)) FROM t +-- !query 376 schema +struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> +-- !query 376 output +0.0 + + +-- !query 377 +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t +-- !query 377 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 377 output +0 + + +-- !query 378 +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t +-- !query 378 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 378 output +0 + + +-- !query 379 +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query 379 schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query 379 output +0 + + +-- !query 380 +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t +-- !query 380 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 380 output +0 + + +-- !query 381 +SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t +-- !query 381 schema +struct<> +-- !query 381 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 382 +SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t +-- !query 382 schema +struct<> +-- !query 382 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 383 +SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t +-- !query 383 schema +struct<> +-- !query 383 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 384 +SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t +-- !query 384 schema +struct<> +-- !query 384 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 385 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t +-- !query 385 schema +struct<> +-- !query 385 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 386 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t +-- !query 386 schema +struct<> +-- !query 386 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 387 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t +-- !query 387 schema +struct<> +-- !query 387 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 388 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t +-- !query 388 schema +struct<> +-- !query 388 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 389 +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t +-- !query 389 schema +struct<> +-- !query 389 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 390 +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t +-- !query 390 schema +struct<> +-- !query 390 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 391 +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t +-- !query 391 schema +struct<> +-- !query 391 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 392 +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t +-- !query 392 schema +struct<> +-- !query 392 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 393 +SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t +-- !query 393 schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(3,0)> +-- !query 393 output +0 + + +-- !query 394 +SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t +-- !query 394 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(3,0)> +-- !query 394 output +0 + + +-- !query 395 +SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t +-- !query 395 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 395 output +0 + + +-- !query 396 +SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t +-- !query 396 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 396 output +0 + + +-- !query 397 +SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t +-- !query 397 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(3,0)> +-- !query 397 output +0 + + +-- !query 398 +SELECT cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t +-- !query 398 schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(5,0)> +-- !query 398 output +0 + + +-- !query 399 +SELECT cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t +-- !query 399 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 399 output +0 + + +-- !query 400 +SELECT cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t +-- !query 400 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(5,0)> +-- !query 400 output +0 + + +-- !query 401 +SELECT cast(1 as decimal(3, 0)) % cast(1 as int) FROM t +-- !query 401 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 401 output +0 + + +-- !query 402 +SELECT cast(1 as decimal(5, 0)) % cast(1 as int) FROM t +-- !query 402 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 402 output +0 + + +-- !query 403 +SELECT cast(1 as decimal(10, 0)) % cast(1 as int) FROM t +-- !query 403 schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(10,0)> +-- !query 403 output +0 + + +-- !query 404 +SELECT cast(1 as decimal(20, 0)) % cast(1 as int) FROM t +-- !query 404 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 404 output +0 + + +-- !query 405 +SELECT cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t +-- !query 405 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 405 output +0 + + +-- !query 406 +SELECT cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t +-- !query 406 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(5,0)> +-- !query 406 output +0 + + +-- !query 407 +SELECT cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t +-- !query 407 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 407 output +0 + + +-- !query 408 +SELECT cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t +-- !query 408 schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(20,0)> +-- !query 408 output +0 + + +-- !query 409 +SELECT cast(1 as decimal(3, 0)) % cast(1 as float) FROM t +-- !query 409 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 409 output +0.0 + + +-- !query 410 +SELECT cast(1 as decimal(5, 0)) % cast(1 as float) FROM t +-- !query 410 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 410 output +0.0 + + +-- !query 411 +SELECT cast(1 as decimal(10, 0)) % cast(1 as float) FROM t +-- !query 411 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 411 output +0.0 + + +-- !query 412 +SELECT cast(1 as decimal(20, 0)) % cast(1 as float) FROM t +-- !query 412 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> +-- !query 412 output +0.0 + + +-- !query 413 +SELECT cast(1 as decimal(3, 0)) % cast(1 as double) FROM t +-- !query 413 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> +-- !query 413 output +0.0 + + +-- !query 414 +SELECT cast(1 as decimal(5, 0)) % cast(1 as double) FROM t +-- !query 414 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> +-- !query 414 output +0.0 + + +-- !query 415 +SELECT cast(1 as decimal(10, 0)) % cast(1 as double) FROM t +-- !query 415 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> +-- !query 415 output +0.0 + + +-- !query 416 +SELECT cast(1 as decimal(20, 0)) % cast(1 as double) FROM t +-- !query 416 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> +-- !query 416 output +0.0 + + +-- !query 417 +SELECT cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query 417 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> +-- !query 417 output +0 + + +-- !query 418 +SELECT cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query 418 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> +-- !query 418 output +0 + + +-- !query 419 +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query 419 schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query 419 output +0 + + +-- !query 420 +SELECT cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query 420 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> +-- !query 420 output +0 + + +-- !query 421 +SELECT cast(1 as decimal(3, 0)) % cast(1 as string) FROM t +-- !query 421 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 421 output +0.0 + + +-- !query 422 +SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t +-- !query 422 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 422 output +0.0 + + +-- !query 423 +SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t +-- !query 423 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 423 output +0.0 + + +-- !query 424 +SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t +-- !query 424 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> +-- !query 424 output +0.0 + + +-- !query 425 +SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t +-- !query 425 schema +struct<> +-- !query 425 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 426 +SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t +-- !query 426 schema +struct<> +-- !query 426 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 427 +SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t +-- !query 427 schema +struct<> +-- !query 427 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 428 +SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t +-- !query 428 schema +struct<> +-- !query 428 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 429 +SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t +-- !query 429 schema +struct<> +-- !query 429 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 430 +SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t +-- !query 430 schema +struct<> +-- !query 430 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 431 +SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t +-- !query 431 schema +struct<> +-- !query 431 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 432 +SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t +-- !query 432 schema +struct<> +-- !query 432 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 433 +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 433 schema +struct<> +-- !query 433 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 434 +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 434 schema +struct<> +-- !query 434 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 435 +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 435 schema +struct<> +-- !query 435 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 436 +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 436 schema +struct<> +-- !query 436 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 437 +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query 437 schema +struct<> +-- !query 437 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 438 +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query 438 schema +struct<> +-- !query 438 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 439 +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query 439 schema +struct<> +-- !query 439 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 440 +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query 440 schema +struct<> +-- !query 440 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 441 +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t +-- !query 441 schema +struct +-- !query 441 output +0 + + +-- !query 442 +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t +-- !query 442 schema +struct +-- !query 442 output +0 + + +-- !query 443 +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t +-- !query 443 schema +struct +-- !query 443 output +0 + + +-- !query 444 +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t +-- !query 444 schema +struct +-- !query 444 output +0 + + +-- !query 445 +SELECT pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t +-- !query 445 schema +struct +-- !query 445 output +0 + + +-- !query 446 +SELECT pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t +-- !query 446 schema +struct +-- !query 446 output +0 + + +-- !query 447 +SELECT pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t +-- !query 447 schema +struct +-- !query 447 output +0 + + +-- !query 448 +SELECT pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t +-- !query 448 schema +struct +-- !query 448 output +0 + + +-- !query 449 +SELECT pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t +-- !query 449 schema +struct +-- !query 449 output +0 + + +-- !query 450 +SELECT pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t +-- !query 450 schema +struct +-- !query 450 output +0 + + +-- !query 451 +SELECT pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t +-- !query 451 schema +struct +-- !query 451 output +0 + + +-- !query 452 +SELECT pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t +-- !query 452 schema +struct +-- !query 452 output +0 + + +-- !query 453 +SELECT pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t +-- !query 453 schema +struct +-- !query 453 output +0 + + +-- !query 454 +SELECT pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t +-- !query 454 schema +struct +-- !query 454 output +0 + + +-- !query 455 +SELECT pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t +-- !query 455 schema +struct +-- !query 455 output +0 + + +-- !query 456 +SELECT pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t +-- !query 456 schema +struct +-- !query 456 output +0 + + +-- !query 457 +SELECT pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t +-- !query 457 schema +struct +-- !query 457 output +0.0 + + +-- !query 458 +SELECT pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t +-- !query 458 schema +struct +-- !query 458 output +0.0 + + +-- !query 459 +SELECT pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t +-- !query 459 schema +struct +-- !query 459 output +0.0 + + +-- !query 460 +SELECT pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t +-- !query 460 schema +struct +-- !query 460 output +0.0 + + +-- !query 461 +SELECT pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t +-- !query 461 schema +struct +-- !query 461 output +0.0 + + +-- !query 462 +SELECT pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t +-- !query 462 schema +struct +-- !query 462 output +0.0 + + +-- !query 463 +SELECT pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t +-- !query 463 schema +struct +-- !query 463 output +0.0 + + +-- !query 464 +SELECT pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t +-- !query 464 schema +struct +-- !query 464 output +0.0 + + +-- !query 465 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t +-- !query 465 schema +struct +-- !query 465 output +0 + + +-- !query 466 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t +-- !query 466 schema +struct +-- !query 466 output +0 + + +-- !query 467 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query 467 schema +struct +-- !query 467 output +0 + + +-- !query 468 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t +-- !query 468 schema +struct +-- !query 468 output +0 + + +-- !query 469 +SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t +-- !query 469 schema +struct<> +-- !query 469 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 470 +SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t +-- !query 470 schema +struct<> +-- !query 470 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 471 +SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t +-- !query 471 schema +struct<> +-- !query 471 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 472 +SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t +-- !query 472 schema +struct<> +-- !query 472 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 473 +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t +-- !query 473 schema +struct<> +-- !query 473 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 474 +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t +-- !query 474 schema +struct<> +-- !query 474 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 475 +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t +-- !query 475 schema +struct<> +-- !query 475 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 476 +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t +-- !query 476 schema +struct<> +-- !query 476 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 477 +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t +-- !query 477 schema +struct<> +-- !query 477 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 478 +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t +-- !query 478 schema +struct<> +-- !query 478 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 479 +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t +-- !query 479 schema +struct<> +-- !query 479 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 480 +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t +-- !query 480 schema +struct<> +-- !query 480 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 481 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t +-- !query 481 schema +struct +-- !query 481 output +0 + + +-- !query 482 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t +-- !query 482 schema +struct +-- !query 482 output +0 + + +-- !query 483 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t +-- !query 483 schema +struct +-- !query 483 output +0 + + +-- !query 484 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t +-- !query 484 schema +struct +-- !query 484 output +0 + + +-- !query 485 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t +-- !query 485 schema +struct +-- !query 485 output +0 + + +-- !query 486 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t +-- !query 486 schema +struct +-- !query 486 output +0 + + +-- !query 487 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t +-- !query 487 schema +struct +-- !query 487 output +0 + + +-- !query 488 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t +-- !query 488 schema +struct +-- !query 488 output +0 + + +-- !query 489 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t +-- !query 489 schema +struct +-- !query 489 output +0 + + +-- !query 490 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t +-- !query 490 schema +struct +-- !query 490 output +0 + + +-- !query 491 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t +-- !query 491 schema +struct +-- !query 491 output +0 + + +-- !query 492 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t +-- !query 492 schema +struct +-- !query 492 output +0 + + +-- !query 493 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t +-- !query 493 schema +struct +-- !query 493 output +0 + + +-- !query 494 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t +-- !query 494 schema +struct +-- !query 494 output +0 + + +-- !query 495 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t +-- !query 495 schema +struct +-- !query 495 output +0 + + +-- !query 496 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t +-- !query 496 schema +struct +-- !query 496 output +0 + + +-- !query 497 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t +-- !query 497 schema +struct +-- !query 497 output +0.0 + + +-- !query 498 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t +-- !query 498 schema +struct +-- !query 498 output +0.0 + + +-- !query 499 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t +-- !query 499 schema +struct +-- !query 499 output +0.0 + + +-- !query 500 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t +-- !query 500 schema +struct +-- !query 500 output +0.0 + + +-- !query 501 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t +-- !query 501 schema +struct +-- !query 501 output +0.0 + + +-- !query 502 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t +-- !query 502 schema +struct +-- !query 502 output +0.0 + + +-- !query 503 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t +-- !query 503 schema +struct +-- !query 503 output +0.0 + + +-- !query 504 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t +-- !query 504 schema +struct +-- !query 504 output +0.0 + + +-- !query 505 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query 505 schema +struct +-- !query 505 output +0 + + +-- !query 506 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query 506 schema +struct +-- !query 506 output +0 + + +-- !query 507 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query 507 schema +struct +-- !query 507 output +0 + + +-- !query 508 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query 508 schema +struct +-- !query 508 output +0 + + +-- !query 509 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t +-- !query 509 schema +struct +-- !query 509 output +0.0 + + +-- !query 510 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t +-- !query 510 schema +struct +-- !query 510 output +0.0 + + +-- !query 511 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t +-- !query 511 schema +struct +-- !query 511 output +0.0 + + +-- !query 512 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t +-- !query 512 schema +struct +-- !query 512 output +0.0 + + +-- !query 513 +SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t +-- !query 513 schema +struct<> +-- !query 513 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 514 +SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t +-- !query 514 schema +struct<> +-- !query 514 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 515 +SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t +-- !query 515 schema +struct<> +-- !query 515 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 516 +SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t +-- !query 516 schema +struct<> +-- !query 516 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 517 +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t +-- !query 517 schema +struct<> +-- !query 517 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 518 +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t +-- !query 518 schema +struct<> +-- !query 518 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 519 +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t +-- !query 519 schema +struct<> +-- !query 519 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 520 +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t +-- !query 520 schema +struct<> +-- !query 520 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 521 +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query 521 schema +struct<> +-- !query 521 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 522 +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query 522 schema +struct<> +-- !query 522 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 523 +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query 523 schema +struct<> +-- !query 523 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 524 +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query 524 schema +struct<> +-- !query 524 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 525 +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query 525 schema +struct<> +-- !query 525 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 526 +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query 526 schema +struct<> +-- !query 526 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 527 +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query 527 schema +struct<> +-- !query 527 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 528 +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query 528 schema +struct<> +-- !query 528 output +org.apache.spark.sql.AnalysisException +cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 529 +SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t +-- !query 529 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 529 output +true + + +-- !query 530 +SELECT cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t +-- !query 530 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 530 output +true + + +-- !query 531 +SELECT cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t +-- !query 531 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 531 output +true + + +-- !query 532 +SELECT cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t +-- !query 532 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 532 output +true + + +-- !query 533 +SELECT cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t +-- !query 533 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 533 output +true + + +-- !query 534 +SELECT cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t +-- !query 534 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 534 output +true + + +-- !query 535 +SELECT cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t +-- !query 535 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 535 output +true + + +-- !query 536 +SELECT cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t +-- !query 536 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 536 output +true + + +-- !query 537 +SELECT cast(1 as int) = cast(1 as decimal(3, 0)) FROM t +-- !query 537 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 537 output +true + + +-- !query 538 +SELECT cast(1 as int) = cast(1 as decimal(5, 0)) FROM t +-- !query 538 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 538 output +true + + +-- !query 539 +SELECT cast(1 as int) = cast(1 as decimal(10, 0)) FROM t +-- !query 539 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 539 output +true + + +-- !query 540 +SELECT cast(1 as int) = cast(1 as decimal(20, 0)) FROM t +-- !query 540 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 540 output +true + + +-- !query 541 +SELECT cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t +-- !query 541 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 541 output +true + + +-- !query 542 +SELECT cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t +-- !query 542 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 542 output +true + + +-- !query 543 +SELECT cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t +-- !query 543 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 543 output +true + + +-- !query 544 +SELECT cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t +-- !query 544 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 544 output +true + + +-- !query 545 +SELECT cast(1 as float) = cast(1 as decimal(3, 0)) FROM t +-- !query 545 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 545 output +true + + +-- !query 546 +SELECT cast(1 as float) = cast(1 as decimal(5, 0)) FROM t +-- !query 546 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 546 output +true + + +-- !query 547 +SELECT cast(1 as float) = cast(1 as decimal(10, 0)) FROM t +-- !query 547 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 547 output +true + + +-- !query 548 +SELECT cast(1 as float) = cast(1 as decimal(20, 0)) FROM t +-- !query 548 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 548 output +true + + +-- !query 549 +SELECT cast(1 as double) = cast(1 as decimal(3, 0)) FROM t +-- !query 549 schema +struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 549 output +true + + +-- !query 550 +SELECT cast(1 as double) = cast(1 as decimal(5, 0)) FROM t +-- !query 550 schema +struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 550 output +true + + +-- !query 551 +SELECT cast(1 as double) = cast(1 as decimal(10, 0)) FROM t +-- !query 551 schema +struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 551 output +true + + +-- !query 552 +SELECT cast(1 as double) = cast(1 as decimal(20, 0)) FROM t +-- !query 552 schema +struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 552 output +true + + +-- !query 553 +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t +-- !query 553 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 553 output +true + + +-- !query 554 +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t +-- !query 554 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 554 output +true + + +-- !query 555 +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query 555 schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 555 output +true + + +-- !query 556 +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t +-- !query 556 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 556 output +true + + +-- !query 557 +SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t +-- !query 557 schema +struct<> +-- !query 557 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 558 +SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t +-- !query 558 schema +struct<> +-- !query 558 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 559 +SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t +-- !query 559 schema +struct<> +-- !query 559 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 560 +SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t +-- !query 560 schema +struct<> +-- !query 560 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 561 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t +-- !query 561 schema +struct<> +-- !query 561 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 562 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t +-- !query 562 schema +struct<> +-- !query 562 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 563 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t +-- !query 563 schema +struct<> +-- !query 563 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 564 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t +-- !query 564 schema +struct<> +-- !query 564 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 565 +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t +-- !query 565 schema +struct<> +-- !query 565 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 566 +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t +-- !query 566 schema +struct<> +-- !query 566 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 567 +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t +-- !query 567 schema +struct<> +-- !query 567 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 568 +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t +-- !query 568 schema +struct<> +-- !query 568 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 569 +SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t +-- !query 569 schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 569 output +true + + +-- !query 570 +SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t +-- !query 570 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 570 output +true + + +-- !query 571 +SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t +-- !query 571 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 571 output +true + + +-- !query 572 +SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t +-- !query 572 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 572 output +true + + +-- !query 573 +SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t +-- !query 573 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 573 output +true + + +-- !query 574 +SELECT cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t +-- !query 574 schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 574 output +true + + +-- !query 575 +SELECT cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t +-- !query 575 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 575 output +true + + +-- !query 576 +SELECT cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t +-- !query 576 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 576 output +true + + +-- !query 577 +SELECT cast(1 as decimal(3, 0)) = cast(1 as int) FROM t +-- !query 577 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 577 output +true + + +-- !query 578 +SELECT cast(1 as decimal(5, 0)) = cast(1 as int) FROM t +-- !query 578 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 578 output +true + + +-- !query 579 +SELECT cast(1 as decimal(10, 0)) = cast(1 as int) FROM t +-- !query 579 schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 579 output +true + + +-- !query 580 +SELECT cast(1 as decimal(20, 0)) = cast(1 as int) FROM t +-- !query 580 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 580 output +true + + +-- !query 581 +SELECT cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t +-- !query 581 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 581 output +true + + +-- !query 582 +SELECT cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t +-- !query 582 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 582 output +true + + +-- !query 583 +SELECT cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t +-- !query 583 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 583 output +true + + +-- !query 584 +SELECT cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t +-- !query 584 schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 584 output +true + + +-- !query 585 +SELECT cast(1 as decimal(3, 0)) = cast(1 as float) FROM t +-- !query 585 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 585 output +true + + +-- !query 586 +SELECT cast(1 as decimal(5, 0)) = cast(1 as float) FROM t +-- !query 586 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 586 output +true + + +-- !query 587 +SELECT cast(1 as decimal(10, 0)) = cast(1 as float) FROM t +-- !query 587 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 587 output +true + + +-- !query 588 +SELECT cast(1 as decimal(20, 0)) = cast(1 as float) FROM t +-- !query 588 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 588 output +true + + +-- !query 589 +SELECT cast(1 as decimal(3, 0)) = cast(1 as double) FROM t +-- !query 589 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> +-- !query 589 output +true + + +-- !query 590 +SELECT cast(1 as decimal(5, 0)) = cast(1 as double) FROM t +-- !query 590 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> +-- !query 590 output +true + + +-- !query 591 +SELECT cast(1 as decimal(10, 0)) = cast(1 as double) FROM t +-- !query 591 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> +-- !query 591 output +true + + +-- !query 592 +SELECT cast(1 as decimal(20, 0)) = cast(1 as double) FROM t +-- !query 592 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> +-- !query 592 output +true + + +-- !query 593 +SELECT cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query 593 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 593 output +true + + +-- !query 594 +SELECT cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query 594 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 594 output +true + + +-- !query 595 +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query 595 schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 595 output +true + + +-- !query 596 +SELECT cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query 596 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 596 output +true + + +-- !query 597 +SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t +-- !query 597 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 597 output +true + + +-- !query 598 +SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t +-- !query 598 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 598 output +true + + +-- !query 599 +SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t +-- !query 599 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 599 output +true + + +-- !query 600 +SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t +-- !query 600 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 600 output +true + + +-- !query 601 +SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t +-- !query 601 schema +struct<> +-- !query 601 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 602 +SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t +-- !query 602 schema +struct<> +-- !query 602 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 603 +SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t +-- !query 603 schema +struct<> +-- !query 603 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 604 +SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t +-- !query 604 schema +struct<> +-- !query 604 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 605 +SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t +-- !query 605 schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0))):boolean> +-- !query 605 output +true + + +-- !query 606 +SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t +-- !query 606 schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0))):boolean> +-- !query 606 output +true + + +-- !query 607 +SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t +-- !query 607 schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0))):boolean> +-- !query 607 output +true + + +-- !query 608 +SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t +-- !query 608 schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0))):boolean> +-- !query 608 output +true + + +-- !query 609 +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 609 schema +struct<> +-- !query 609 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 610 +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 610 schema +struct<> +-- !query 610 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 611 +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 611 schema +struct<> +-- !query 611 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 612 +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 612 schema +struct<> +-- !query 612 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 613 +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query 613 schema +struct<> +-- !query 613 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 614 +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query 614 schema +struct<> +-- !query 614 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 615 +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query 615 schema +struct<> +-- !query 615 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 616 +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query 616 schema +struct<> +-- !query 616 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 617 +SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 617 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 617 output +true + + +-- !query 618 +SELECT cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 618 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 618 output +true + + +-- !query 619 +SELECT cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 619 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 619 output +true + + +-- !query 620 +SELECT cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 620 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 620 output +true + + +-- !query 621 +SELECT cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 621 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 621 output +true + + +-- !query 622 +SELECT cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 622 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 622 output +true + + +-- !query 623 +SELECT cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 623 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 623 output +true + + +-- !query 624 +SELECT cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 624 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 624 output +true + + +-- !query 625 +SELECT cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 625 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 625 output +true + + +-- !query 626 +SELECT cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 626 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 626 output +true + + +-- !query 627 +SELECT cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 627 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 627 output +true + + +-- !query 628 +SELECT cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 628 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 628 output +true + + +-- !query 629 +SELECT cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 629 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 629 output +true + + +-- !query 630 +SELECT cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 630 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 630 output +true + + +-- !query 631 +SELECT cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 631 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 631 output +true + + +-- !query 632 +SELECT cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 632 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 632 output +true + + +-- !query 633 +SELECT cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 633 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 633 output +true + + +-- !query 634 +SELECT cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 634 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 634 output +true + + +-- !query 635 +SELECT cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 635 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 635 output +true + + +-- !query 636 +SELECT cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 636 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 636 output +true + + +-- !query 637 +SELECT cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 637 schema +struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 637 output +true + + +-- !query 638 +SELECT cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 638 schema +struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 638 output +true + + +-- !query 639 +SELECT cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 639 schema +struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 639 output +true + + +-- !query 640 +SELECT cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 640 schema +struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 640 output +true + + +-- !query 641 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 641 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 641 output +true + + +-- !query 642 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 642 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 642 output +true + + +-- !query 643 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 643 schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 643 output +true + + +-- !query 644 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 644 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 644 output +true + + +-- !query 645 +SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 645 schema +struct<> +-- !query 645 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 646 +SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 646 schema +struct<> +-- !query 646 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 647 +SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 647 schema +struct<> +-- !query 647 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 648 +SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 648 schema +struct<> +-- !query 648 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 649 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 649 schema +struct<> +-- !query 649 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 650 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 650 schema +struct<> +-- !query 650 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 651 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 651 schema +struct<> +-- !query 651 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 652 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 652 schema +struct<> +-- !query 652 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 653 +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t +-- !query 653 schema +struct<> +-- !query 653 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 654 +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t +-- !query 654 schema +struct<> +-- !query 654 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 655 +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 655 schema +struct<> +-- !query 655 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 656 +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t +-- !query 656 schema +struct<> +-- !query 656 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 657 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t +-- !query 657 schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 657 output +true + + +-- !query 658 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t +-- !query 658 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 658 output +true + + +-- !query 659 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t +-- !query 659 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 659 output +true + + +-- !query 660 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t +-- !query 660 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 660 output +true + + +-- !query 661 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t +-- !query 661 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 661 output +true + + +-- !query 662 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t +-- !query 662 schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 662 output +true + + +-- !query 663 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t +-- !query 663 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 663 output +true + + +-- !query 664 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t +-- !query 664 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 664 output +true + + +-- !query 665 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t +-- !query 665 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 665 output +true + + +-- !query 666 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t +-- !query 666 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 666 output +true + + +-- !query 667 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t +-- !query 667 schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 667 output +true + + +-- !query 668 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t +-- !query 668 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 668 output +true + + +-- !query 669 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t +-- !query 669 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 669 output +true + + +-- !query 670 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t +-- !query 670 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 670 output +true + + +-- !query 671 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t +-- !query 671 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 671 output +true + + +-- !query 672 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t +-- !query 672 schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 672 output +true + + +-- !query 673 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t +-- !query 673 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 673 output +true + + +-- !query 674 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t +-- !query 674 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 674 output +true + + +-- !query 675 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t +-- !query 675 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 675 output +true + + +-- !query 676 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t +-- !query 676 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 676 output +true + + +-- !query 677 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t +-- !query 677 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> +-- !query 677 output +true + + +-- !query 678 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t +-- !query 678 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> +-- !query 678 output +true + + +-- !query 679 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t +-- !query 679 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> +-- !query 679 output +true + + +-- !query 680 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t +-- !query 680 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> +-- !query 680 output +true + + +-- !query 681 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 681 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 681 output +true + + +-- !query 682 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 682 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 682 output +true + + +-- !query 683 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 683 schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 683 output +true + + +-- !query 684 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query 684 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 684 output +true + + +-- !query 685 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t +-- !query 685 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 685 output +true + + +-- !query 686 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t +-- !query 686 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 686 output +true + + +-- !query 687 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t +-- !query 687 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 687 output +true + + +-- !query 688 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t +-- !query 688 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 688 output +true + + +-- !query 689 +SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t +-- !query 689 schema +struct<> +-- !query 689 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 690 +SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t +-- !query 690 schema +struct<> +-- !query 690 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 691 +SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t +-- !query 691 schema +struct<> +-- !query 691 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 692 +SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t +-- !query 692 schema +struct<> +-- !query 692 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 693 +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t +-- !query 693 schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0))):boolean> +-- !query 693 output +true + + +-- !query 694 +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t +-- !query 694 schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0))):boolean> +-- !query 694 output +true + + +-- !query 695 +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t +-- !query 695 schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0))):boolean> +-- !query 695 output +true + + +-- !query 696 +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t +-- !query 696 schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0))):boolean> +-- !query 696 output +true + + +-- !query 697 +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 697 schema +struct<> +-- !query 697 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 698 +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 698 schema +struct<> +-- !query 698 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 699 +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 699 schema +struct<> +-- !query 699 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 700 +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 700 schema +struct<> +-- !query 700 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 701 +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 701 schema +struct<> +-- !query 701 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 702 +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 702 schema +struct<> +-- !query 702 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 703 +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 703 schema +struct<> +-- !query 703 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 704 +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 704 schema +struct<> +-- !query 704 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 705 +SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t +-- !query 705 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 705 output +false + + +-- !query 706 +SELECT cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t +-- !query 706 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 706 output +false + + +-- !query 707 +SELECT cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t +-- !query 707 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 707 output +false + + +-- !query 708 +SELECT cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t +-- !query 708 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 708 output +false + + +-- !query 709 +SELECT cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t +-- !query 709 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 709 output +false + + +-- !query 710 +SELECT cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t +-- !query 710 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 710 output +false + + +-- !query 711 +SELECT cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t +-- !query 711 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 711 output +false + + +-- !query 712 +SELECT cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t +-- !query 712 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 712 output +false + + +-- !query 713 +SELECT cast(1 as int) < cast(1 as decimal(3, 0)) FROM t +-- !query 713 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 713 output +false + + +-- !query 714 +SELECT cast(1 as int) < cast(1 as decimal(5, 0)) FROM t +-- !query 714 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 714 output +false + + +-- !query 715 +SELECT cast(1 as int) < cast(1 as decimal(10, 0)) FROM t +-- !query 715 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 715 output +false + + +-- !query 716 +SELECT cast(1 as int) < cast(1 as decimal(20, 0)) FROM t +-- !query 716 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 716 output +false + + +-- !query 717 +SELECT cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t +-- !query 717 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 717 output +false + + +-- !query 718 +SELECT cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t +-- !query 718 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 718 output +false + + +-- !query 719 +SELECT cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t +-- !query 719 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 719 output +false + + +-- !query 720 +SELECT cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t +-- !query 720 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 720 output +false + + +-- !query 721 +SELECT cast(1 as float) < cast(1 as decimal(3, 0)) FROM t +-- !query 721 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 721 output +false + + +-- !query 722 +SELECT cast(1 as float) < cast(1 as decimal(5, 0)) FROM t +-- !query 722 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 722 output +false + + +-- !query 723 +SELECT cast(1 as float) < cast(1 as decimal(10, 0)) FROM t +-- !query 723 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 723 output +false + + +-- !query 724 +SELECT cast(1 as float) < cast(1 as decimal(20, 0)) FROM t +-- !query 724 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 724 output +false + + +-- !query 725 +SELECT cast(1 as double) < cast(1 as decimal(3, 0)) FROM t +-- !query 725 schema +struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 725 output +false + + +-- !query 726 +SELECT cast(1 as double) < cast(1 as decimal(5, 0)) FROM t +-- !query 726 schema +struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 726 output +false + + +-- !query 727 +SELECT cast(1 as double) < cast(1 as decimal(10, 0)) FROM t +-- !query 727 schema +struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 727 output +false + + +-- !query 728 +SELECT cast(1 as double) < cast(1 as decimal(20, 0)) FROM t +-- !query 728 schema +struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 728 output +false + + +-- !query 729 +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t +-- !query 729 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 729 output +false + + +-- !query 730 +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t +-- !query 730 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 730 output +false + + +-- !query 731 +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query 731 schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 731 output +false + + +-- !query 732 +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t +-- !query 732 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 732 output +false + + +-- !query 733 +SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t +-- !query 733 schema +struct<> +-- !query 733 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 734 +SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t +-- !query 734 schema +struct<> +-- !query 734 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 735 +SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t +-- !query 735 schema +struct<> +-- !query 735 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 736 +SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t +-- !query 736 schema +struct<> +-- !query 736 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 737 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t +-- !query 737 schema +struct<> +-- !query 737 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 738 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t +-- !query 738 schema +struct<> +-- !query 738 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 739 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t +-- !query 739 schema +struct<> +-- !query 739 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 740 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t +-- !query 740 schema +struct<> +-- !query 740 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 741 +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t +-- !query 741 schema +struct<> +-- !query 741 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 742 +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t +-- !query 742 schema +struct<> +-- !query 742 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 743 +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t +-- !query 743 schema +struct<> +-- !query 743 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 744 +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t +-- !query 744 schema +struct<> +-- !query 744 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 745 +SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t +-- !query 745 schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 745 output +false + + +-- !query 746 +SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t +-- !query 746 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 746 output +false + + +-- !query 747 +SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t +-- !query 747 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 747 output +false + + +-- !query 748 +SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t +-- !query 748 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 748 output +false + + +-- !query 749 +SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t +-- !query 749 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 749 output +false + + +-- !query 750 +SELECT cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t +-- !query 750 schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 750 output +false + + +-- !query 751 +SELECT cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t +-- !query 751 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 751 output +false + + +-- !query 752 +SELECT cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t +-- !query 752 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 752 output +false + + +-- !query 753 +SELECT cast(1 as decimal(3, 0)) < cast(1 as int) FROM t +-- !query 753 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 753 output +false + + +-- !query 754 +SELECT cast(1 as decimal(5, 0)) < cast(1 as int) FROM t +-- !query 754 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 754 output +false + + +-- !query 755 +SELECT cast(1 as decimal(10, 0)) < cast(1 as int) FROM t +-- !query 755 schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 755 output +false + + +-- !query 756 +SELECT cast(1 as decimal(20, 0)) < cast(1 as int) FROM t +-- !query 756 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 756 output +false + + +-- !query 757 +SELECT cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t +-- !query 757 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 757 output +false + + +-- !query 758 +SELECT cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t +-- !query 758 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 758 output +false + + +-- !query 759 +SELECT cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t +-- !query 759 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 759 output +false + + +-- !query 760 +SELECT cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t +-- !query 760 schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 760 output +false + + +-- !query 761 +SELECT cast(1 as decimal(3, 0)) < cast(1 as float) FROM t +-- !query 761 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 761 output +false + + +-- !query 762 +SELECT cast(1 as decimal(5, 0)) < cast(1 as float) FROM t +-- !query 762 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 762 output +false + + +-- !query 763 +SELECT cast(1 as decimal(10, 0)) < cast(1 as float) FROM t +-- !query 763 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 763 output +false + + +-- !query 764 +SELECT cast(1 as decimal(20, 0)) < cast(1 as float) FROM t +-- !query 764 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 764 output +false + + +-- !query 765 +SELECT cast(1 as decimal(3, 0)) < cast(1 as double) FROM t +-- !query 765 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> +-- !query 765 output +false + + +-- !query 766 +SELECT cast(1 as decimal(5, 0)) < cast(1 as double) FROM t +-- !query 766 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> +-- !query 766 output +false + + +-- !query 767 +SELECT cast(1 as decimal(10, 0)) < cast(1 as double) FROM t +-- !query 767 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> +-- !query 767 output +false + + +-- !query 768 +SELECT cast(1 as decimal(20, 0)) < cast(1 as double) FROM t +-- !query 768 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> +-- !query 768 output +false + + +-- !query 769 +SELECT cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query 769 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 769 output +false + + +-- !query 770 +SELECT cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query 770 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 770 output +false + + +-- !query 771 +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query 771 schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 771 output +false + + +-- !query 772 +SELECT cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query 772 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 772 output +false + + +-- !query 773 +SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t +-- !query 773 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 773 output +false + + +-- !query 774 +SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t +-- !query 774 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 774 output +false + + +-- !query 775 +SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t +-- !query 775 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 775 output +false + + +-- !query 776 +SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t +-- !query 776 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 776 output +false + + +-- !query 777 +SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t +-- !query 777 schema +struct<> +-- !query 777 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 778 +SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t +-- !query 778 schema +struct<> +-- !query 778 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 779 +SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t +-- !query 779 schema +struct<> +-- !query 779 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 780 +SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t +-- !query 780 schema +struct<> +-- !query 780 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 781 +SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t +-- !query 781 schema +struct<> +-- !query 781 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 782 +SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t +-- !query 782 schema +struct<> +-- !query 782 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 783 +SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t +-- !query 783 schema +struct<> +-- !query 783 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 784 +SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t +-- !query 784 schema +struct<> +-- !query 784 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 785 +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 785 schema +struct<> +-- !query 785 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 786 +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 786 schema +struct<> +-- !query 786 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 787 +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 787 schema +struct<> +-- !query 787 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 788 +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 788 schema +struct<> +-- !query 788 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 789 +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query 789 schema +struct<> +-- !query 789 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 790 +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query 790 schema +struct<> +-- !query 790 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 791 +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query 791 schema +struct<> +-- !query 791 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 792 +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query 792 schema +struct<> +-- !query 792 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 793 +SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t +-- !query 793 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 793 output +true + + +-- !query 794 +SELECT cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t +-- !query 794 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 794 output +true + + +-- !query 795 +SELECT cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t +-- !query 795 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 795 output +true + + +-- !query 796 +SELECT cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t +-- !query 796 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 796 output +true + + +-- !query 797 +SELECT cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t +-- !query 797 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 797 output +true + + +-- !query 798 +SELECT cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t +-- !query 798 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 798 output +true + + +-- !query 799 +SELECT cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t +-- !query 799 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 799 output +true + + +-- !query 800 +SELECT cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t +-- !query 800 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 800 output +true + + +-- !query 801 +SELECT cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t +-- !query 801 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 801 output +true + + +-- !query 802 +SELECT cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t +-- !query 802 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 802 output +true + + +-- !query 803 +SELECT cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t +-- !query 803 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 803 output +true + + +-- !query 804 +SELECT cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t +-- !query 804 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 804 output +true + + +-- !query 805 +SELECT cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t +-- !query 805 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 805 output +true + + +-- !query 806 +SELECT cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t +-- !query 806 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 806 output +true + + +-- !query 807 +SELECT cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t +-- !query 807 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 807 output +true + + +-- !query 808 +SELECT cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t +-- !query 808 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 808 output +true + + +-- !query 809 +SELECT cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t +-- !query 809 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 809 output +true + + +-- !query 810 +SELECT cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t +-- !query 810 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 810 output +true + + +-- !query 811 +SELECT cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t +-- !query 811 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 811 output +true + + +-- !query 812 +SELECT cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t +-- !query 812 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 812 output +true + + +-- !query 813 +SELECT cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t +-- !query 813 schema +struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 813 output +true + + +-- !query 814 +SELECT cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t +-- !query 814 schema +struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 814 output +true + + +-- !query 815 +SELECT cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t +-- !query 815 schema +struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 815 output +true + + +-- !query 816 +SELECT cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t +-- !query 816 schema +struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 816 output +true + + +-- !query 817 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t +-- !query 817 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 817 output +true + + +-- !query 818 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t +-- !query 818 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 818 output +true + + +-- !query 819 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query 819 schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 819 output +true + + +-- !query 820 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t +-- !query 820 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 820 output +true + + +-- !query 821 +SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t +-- !query 821 schema +struct<> +-- !query 821 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 822 +SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t +-- !query 822 schema +struct<> +-- !query 822 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 823 +SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t +-- !query 823 schema +struct<> +-- !query 823 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 824 +SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t +-- !query 824 schema +struct<> +-- !query 824 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 825 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t +-- !query 825 schema +struct<> +-- !query 825 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 826 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t +-- !query 826 schema +struct<> +-- !query 826 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 827 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t +-- !query 827 schema +struct<> +-- !query 827 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 828 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t +-- !query 828 schema +struct<> +-- !query 828 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 829 +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t +-- !query 829 schema +struct<> +-- !query 829 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 830 +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t +-- !query 830 schema +struct<> +-- !query 830 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 831 +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t +-- !query 831 schema +struct<> +-- !query 831 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 832 +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t +-- !query 832 schema +struct<> +-- !query 832 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 833 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t +-- !query 833 schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 833 output +true + + +-- !query 834 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t +-- !query 834 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 834 output +true + + +-- !query 835 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t +-- !query 835 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 835 output +true + + +-- !query 836 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t +-- !query 836 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 836 output +true + + +-- !query 837 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t +-- !query 837 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 837 output +true + + +-- !query 838 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t +-- !query 838 schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 838 output +true + + +-- !query 839 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t +-- !query 839 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 839 output +true + + +-- !query 840 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t +-- !query 840 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 840 output +true + + +-- !query 841 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t +-- !query 841 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 841 output +true + + +-- !query 842 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t +-- !query 842 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 842 output +true + + +-- !query 843 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t +-- !query 843 schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 843 output +true + + +-- !query 844 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t +-- !query 844 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 844 output +true + + +-- !query 845 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t +-- !query 845 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 845 output +true + + +-- !query 846 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t +-- !query 846 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 846 output +true + + +-- !query 847 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t +-- !query 847 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 847 output +true + + +-- !query 848 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t +-- !query 848 schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 848 output +true + + +-- !query 849 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t +-- !query 849 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 849 output +true + + +-- !query 850 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t +-- !query 850 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 850 output +true + + +-- !query 851 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t +-- !query 851 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 851 output +true + + +-- !query 852 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t +-- !query 852 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 852 output +true + + +-- !query 853 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t +-- !query 853 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> +-- !query 853 output +true + + +-- !query 854 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t +-- !query 854 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> +-- !query 854 output +true + + +-- !query 855 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t +-- !query 855 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> +-- !query 855 output +true + + +-- !query 856 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t +-- !query 856 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> +-- !query 856 output +true + + +-- !query 857 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query 857 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 857 output +true + + +-- !query 858 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query 858 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 858 output +true + + +-- !query 859 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query 859 schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 859 output +true + + +-- !query 860 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query 860 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 860 output +true + + +-- !query 861 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t +-- !query 861 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 861 output +true + + +-- !query 862 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t +-- !query 862 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 862 output +true + + +-- !query 863 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t +-- !query 863 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 863 output +true + + +-- !query 864 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t +-- !query 864 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 864 output +true + + +-- !query 865 +SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t +-- !query 865 schema +struct<> +-- !query 865 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 866 +SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t +-- !query 866 schema +struct<> +-- !query 866 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 867 +SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t +-- !query 867 schema +struct<> +-- !query 867 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 868 +SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t +-- !query 868 schema +struct<> +-- !query 868 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 869 +SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t +-- !query 869 schema +struct<> +-- !query 869 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 870 +SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t +-- !query 870 schema +struct<> +-- !query 870 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 871 +SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t +-- !query 871 schema +struct<> +-- !query 871 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 872 +SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t +-- !query 872 schema +struct<> +-- !query 872 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 873 +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 873 schema +struct<> +-- !query 873 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 874 +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 874 schema +struct<> +-- !query 874 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 875 +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 875 schema +struct<> +-- !query 875 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 876 +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 876 schema +struct<> +-- !query 876 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 877 +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 877 schema +struct<> +-- !query 877 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 878 +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 878 schema +struct<> +-- !query 878 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 879 +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 879 schema +struct<> +-- !query 879 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 880 +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 880 schema +struct<> +-- !query 880 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 881 +SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t +-- !query 881 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 881 output +false + + +-- !query 882 +SELECT cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t +-- !query 882 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 882 output +false + + +-- !query 883 +SELECT cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t +-- !query 883 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 883 output +false + + +-- !query 884 +SELECT cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t +-- !query 884 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 884 output +false + + +-- !query 885 +SELECT cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t +-- !query 885 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 885 output +false + + +-- !query 886 +SELECT cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t +-- !query 886 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 886 output +false + + +-- !query 887 +SELECT cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t +-- !query 887 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 887 output +false + + +-- !query 888 +SELECT cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t +-- !query 888 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 888 output +false + + +-- !query 889 +SELECT cast(1 as int) > cast(1 as decimal(3, 0)) FROM t +-- !query 889 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 889 output +false + + +-- !query 890 +SELECT cast(1 as int) > cast(1 as decimal(5, 0)) FROM t +-- !query 890 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 890 output +false + + +-- !query 891 +SELECT cast(1 as int) > cast(1 as decimal(10, 0)) FROM t +-- !query 891 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 891 output +false + + +-- !query 892 +SELECT cast(1 as int) > cast(1 as decimal(20, 0)) FROM t +-- !query 892 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 892 output +false + + +-- !query 893 +SELECT cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t +-- !query 893 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 893 output +false + + +-- !query 894 +SELECT cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t +-- !query 894 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 894 output +false + + +-- !query 895 +SELECT cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t +-- !query 895 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 895 output +false + + +-- !query 896 +SELECT cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t +-- !query 896 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 896 output +false + + +-- !query 897 +SELECT cast(1 as float) > cast(1 as decimal(3, 0)) FROM t +-- !query 897 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 897 output +false + + +-- !query 898 +SELECT cast(1 as float) > cast(1 as decimal(5, 0)) FROM t +-- !query 898 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 898 output +false + + +-- !query 899 +SELECT cast(1 as float) > cast(1 as decimal(10, 0)) FROM t +-- !query 899 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 899 output +false + + +-- !query 900 +SELECT cast(1 as float) > cast(1 as decimal(20, 0)) FROM t +-- !query 900 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 900 output +false + + +-- !query 901 +SELECT cast(1 as double) > cast(1 as decimal(3, 0)) FROM t +-- !query 901 schema +struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 901 output +false + + +-- !query 902 +SELECT cast(1 as double) > cast(1 as decimal(5, 0)) FROM t +-- !query 902 schema +struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 902 output +false + + +-- !query 903 +SELECT cast(1 as double) > cast(1 as decimal(10, 0)) FROM t +-- !query 903 schema +struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 903 output +false + + +-- !query 904 +SELECT cast(1 as double) > cast(1 as decimal(20, 0)) FROM t +-- !query 904 schema +struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 904 output +false + + +-- !query 905 +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t +-- !query 905 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 905 output +false + + +-- !query 906 +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t +-- !query 906 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 906 output +false + + +-- !query 907 +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query 907 schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 907 output +false + + +-- !query 908 +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t +-- !query 908 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 908 output +false + + +-- !query 909 +SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t +-- !query 909 schema +struct<> +-- !query 909 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 910 +SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t +-- !query 910 schema +struct<> +-- !query 910 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 911 +SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t +-- !query 911 schema +struct<> +-- !query 911 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 912 +SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t +-- !query 912 schema +struct<> +-- !query 912 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 913 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t +-- !query 913 schema +struct<> +-- !query 913 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 914 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t +-- !query 914 schema +struct<> +-- !query 914 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 915 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t +-- !query 915 schema +struct<> +-- !query 915 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 916 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t +-- !query 916 schema +struct<> +-- !query 916 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 917 +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t +-- !query 917 schema +struct<> +-- !query 917 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 918 +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t +-- !query 918 schema +struct<> +-- !query 918 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 919 +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t +-- !query 919 schema +struct<> +-- !query 919 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 920 +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t +-- !query 920 schema +struct<> +-- !query 920 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 921 +SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t +-- !query 921 schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 921 output +false + + +-- !query 922 +SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t +-- !query 922 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 922 output +false + + +-- !query 923 +SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t +-- !query 923 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 923 output +false + + +-- !query 924 +SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t +-- !query 924 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 924 output +false + + +-- !query 925 +SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t +-- !query 925 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 925 output +false + + +-- !query 926 +SELECT cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t +-- !query 926 schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 926 output +false + + +-- !query 927 +SELECT cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t +-- !query 927 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 927 output +false + + +-- !query 928 +SELECT cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t +-- !query 928 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 928 output +false + + +-- !query 929 +SELECT cast(1 as decimal(3, 0)) > cast(1 as int) FROM t +-- !query 929 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 929 output +false + + +-- !query 930 +SELECT cast(1 as decimal(5, 0)) > cast(1 as int) FROM t +-- !query 930 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 930 output +false + + +-- !query 931 +SELECT cast(1 as decimal(10, 0)) > cast(1 as int) FROM t +-- !query 931 schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 931 output +false + + +-- !query 932 +SELECT cast(1 as decimal(20, 0)) > cast(1 as int) FROM t +-- !query 932 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 932 output +false + + +-- !query 933 +SELECT cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t +-- !query 933 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 933 output +false + + +-- !query 934 +SELECT cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t +-- !query 934 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 934 output +false + + +-- !query 935 +SELECT cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t +-- !query 935 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 935 output +false + + +-- !query 936 +SELECT cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t +-- !query 936 schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 936 output +false + + +-- !query 937 +SELECT cast(1 as decimal(3, 0)) > cast(1 as float) FROM t +-- !query 937 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 937 output +false + + +-- !query 938 +SELECT cast(1 as decimal(5, 0)) > cast(1 as float) FROM t +-- !query 938 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 938 output +false + + +-- !query 939 +SELECT cast(1 as decimal(10, 0)) > cast(1 as float) FROM t +-- !query 939 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 939 output +false + + +-- !query 940 +SELECT cast(1 as decimal(20, 0)) > cast(1 as float) FROM t +-- !query 940 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 940 output +false + + +-- !query 941 +SELECT cast(1 as decimal(3, 0)) > cast(1 as double) FROM t +-- !query 941 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> +-- !query 941 output +false + + +-- !query 942 +SELECT cast(1 as decimal(5, 0)) > cast(1 as double) FROM t +-- !query 942 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> +-- !query 942 output +false + + +-- !query 943 +SELECT cast(1 as decimal(10, 0)) > cast(1 as double) FROM t +-- !query 943 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> +-- !query 943 output +false + + +-- !query 944 +SELECT cast(1 as decimal(20, 0)) > cast(1 as double) FROM t +-- !query 944 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> +-- !query 944 output +false + + +-- !query 945 +SELECT cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query 945 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 945 output +false + + +-- !query 946 +SELECT cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query 946 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 946 output +false + + +-- !query 947 +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query 947 schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 947 output +false + + +-- !query 948 +SELECT cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query 948 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 948 output +false + + +-- !query 949 +SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t +-- !query 949 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 949 output +false + + +-- !query 950 +SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t +-- !query 950 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 950 output +false + + +-- !query 951 +SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t +-- !query 951 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 951 output +false + + +-- !query 952 +SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t +-- !query 952 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 952 output +false + + +-- !query 953 +SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t +-- !query 953 schema +struct<> +-- !query 953 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 954 +SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t +-- !query 954 schema +struct<> +-- !query 954 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 955 +SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t +-- !query 955 schema +struct<> +-- !query 955 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 956 +SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t +-- !query 956 schema +struct<> +-- !query 956 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 957 +SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t +-- !query 957 schema +struct<> +-- !query 957 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 958 +SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t +-- !query 958 schema +struct<> +-- !query 958 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 959 +SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t +-- !query 959 schema +struct<> +-- !query 959 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 960 +SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t +-- !query 960 schema +struct<> +-- !query 960 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 961 +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 961 schema +struct<> +-- !query 961 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 962 +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 962 schema +struct<> +-- !query 962 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 963 +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 963 schema +struct<> +-- !query 963 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 964 +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 964 schema +struct<> +-- !query 964 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 965 +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query 965 schema +struct<> +-- !query 965 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 966 +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query 966 schema +struct<> +-- !query 966 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 967 +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query 967 schema +struct<> +-- !query 967 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 968 +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query 968 schema +struct<> +-- !query 968 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 969 +SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t +-- !query 969 schema +struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query 969 output +true + + +-- !query 970 +SELECT cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t +-- !query 970 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 970 output +true + + +-- !query 971 +SELECT cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t +-- !query 971 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 971 output +true + + +-- !query 972 +SELECT cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t +-- !query 972 schema +struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 972 output +true + + +-- !query 973 +SELECT cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t +-- !query 973 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 973 output +true + + +-- !query 974 +SELECT cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t +-- !query 974 schema +struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query 974 output +true + + +-- !query 975 +SELECT cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t +-- !query 975 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 975 output +true + + +-- !query 976 +SELECT cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t +-- !query 976 schema +struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 976 output +true + + +-- !query 977 +SELECT cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t +-- !query 977 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 977 output +true + + +-- !query 978 +SELECT cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t +-- !query 978 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 978 output +true + + +-- !query 979 +SELECT cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t +-- !query 979 schema +struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 979 output +true + + +-- !query 980 +SELECT cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t +-- !query 980 schema +struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 980 output +true + + +-- !query 981 +SELECT cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t +-- !query 981 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 981 output +true + + +-- !query 982 +SELECT cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t +-- !query 982 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 982 output +true + + +-- !query 983 +SELECT cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t +-- !query 983 schema +struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 983 output +true + + +-- !query 984 +SELECT cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t +-- !query 984 schema +struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query 984 output +true + + +-- !query 985 +SELECT cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t +-- !query 985 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 985 output +true + + +-- !query 986 +SELECT cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t +-- !query 986 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 986 output +true + + +-- !query 987 +SELECT cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t +-- !query 987 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 987 output +true + + +-- !query 988 +SELECT cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t +-- !query 988 schema +struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 988 output +true + + +-- !query 989 +SELECT cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t +-- !query 989 schema +struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> +-- !query 989 output +true + + +-- !query 990 +SELECT cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t +-- !query 990 schema +struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> +-- !query 990 output +true + + +-- !query 991 +SELECT cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t +-- !query 991 schema +struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> +-- !query 991 output +true + + +-- !query 992 +SELECT cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t +-- !query 992 schema +struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> +-- !query 992 output +true + + +-- !query 993 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t +-- !query 993 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 993 output +true + + +-- !query 994 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t +-- !query 994 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 994 output +true + + +-- !query 995 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query 995 schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 995 output +true + + +-- !query 996 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t +-- !query 996 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 996 output +true + + +-- !query 997 +SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t +-- !query 997 schema +struct<> +-- !query 997 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 998 +SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t +-- !query 998 schema +struct<> +-- !query 998 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 999 +SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t +-- !query 999 schema +struct<> +-- !query 999 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 1000 +SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t +-- !query 1000 schema +struct<> +-- !query 1000 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 1001 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t +-- !query 1001 schema +struct<> +-- !query 1001 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 1002 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t +-- !query 1002 schema +struct<> +-- !query 1002 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 1003 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1003 schema +struct<> +-- !query 1003 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 1004 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t +-- !query 1004 schema +struct<> +-- !query 1004 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 1005 +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t +-- !query 1005 schema +struct<> +-- !query 1005 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 1006 +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t +-- !query 1006 schema +struct<> +-- !query 1006 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 1007 +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1007 schema +struct<> +-- !query 1007 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 1008 +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t +-- !query 1008 schema +struct<> +-- !query 1008 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 1009 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t +-- !query 1009 schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> +-- !query 1009 output +true + + +-- !query 1010 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t +-- !query 1010 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> +-- !query 1010 output +true + + +-- !query 1011 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t +-- !query 1011 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> +-- !query 1011 output +true + + +-- !query 1012 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t +-- !query 1012 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> +-- !query 1012 output +true + + +-- !query 1013 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t +-- !query 1013 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> +-- !query 1013 output +true + + +-- !query 1014 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t +-- !query 1014 schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> +-- !query 1014 output +true + + +-- !query 1015 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t +-- !query 1015 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> +-- !query 1015 output +true + + +-- !query 1016 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t +-- !query 1016 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> +-- !query 1016 output +true + + +-- !query 1017 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t +-- !query 1017 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 1017 output +true + + +-- !query 1018 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t +-- !query 1018 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 1018 output +true + + +-- !query 1019 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t +-- !query 1019 schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> +-- !query 1019 output +true + + +-- !query 1020 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t +-- !query 1020 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 1020 output +true + + +-- !query 1021 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t +-- !query 1021 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 1021 output +true + + +-- !query 1022 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t +-- !query 1022 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 1022 output +true + + +-- !query 1023 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t +-- !query 1023 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> +-- !query 1023 output +true + + +-- !query 1024 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t +-- !query 1024 schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> +-- !query 1024 output +true + + +-- !query 1025 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t +-- !query 1025 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 1025 output +true + + +-- !query 1026 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t +-- !query 1026 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 1026 output +true + + +-- !query 1027 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t +-- !query 1027 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 1027 output +true + + +-- !query 1028 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t +-- !query 1028 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> +-- !query 1028 output +true + + +-- !query 1029 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t +-- !query 1029 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> +-- !query 1029 output +true + + +-- !query 1030 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t +-- !query 1030 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> +-- !query 1030 output +true + + +-- !query 1031 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t +-- !query 1031 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> +-- !query 1031 output +true + + +-- !query 1032 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t +-- !query 1032 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> +-- !query 1032 output +true + + +-- !query 1033 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1033 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 1033 output +true + + +-- !query 1034 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1034 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> +-- !query 1034 output +true + + +-- !query 1035 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1035 schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query 1035 output +true + + +-- !query 1036 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query 1036 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> +-- !query 1036 output +true + + +-- !query 1037 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t +-- !query 1037 schema +struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 1037 output +true + + +-- !query 1038 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t +-- !query 1038 schema +struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 1038 output +true + + +-- !query 1039 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t +-- !query 1039 schema +struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 1039 output +true + + +-- !query 1040 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t +-- !query 1040 schema +struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> +-- !query 1040 output +true + + +-- !query 1041 +SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t +-- !query 1041 schema +struct<> +-- !query 1041 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 1042 +SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t +-- !query 1042 schema +struct<> +-- !query 1042 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 1043 +SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t +-- !query 1043 schema +struct<> +-- !query 1043 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 1044 +SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t +-- !query 1044 schema +struct<> +-- !query 1044 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 1045 +SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t +-- !query 1045 schema +struct<> +-- !query 1045 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 + + +-- !query 1046 +SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t +-- !query 1046 schema +struct<> +-- !query 1046 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 + + +-- !query 1047 +SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t +-- !query 1047 schema +struct<> +-- !query 1047 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 + + +-- !query 1048 +SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t +-- !query 1048 schema +struct<> +-- !query 1048 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 + + +-- !query 1049 +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1049 schema +struct<> +-- !query 1049 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 1050 +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1050 schema +struct<> +-- !query 1050 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 1051 +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1051 schema +struct<> +-- !query 1051 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 1052 +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1052 schema +struct<> +-- !query 1052 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 1053 +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1053 schema +struct<> +-- !query 1053 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 1054 +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1054 schema +struct<> +-- !query 1054 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 1055 +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1055 schema +struct<> +-- !query 1055 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 1056 +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1056 schema +struct<> +-- !query 1056 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 + + +-- !query 1057 +SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1057 schema +struct<(NOT (CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query 1057 output +false + + +-- !query 1058 +SELECT cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1058 schema +struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)))):boolean> +-- !query 1058 output +false + + +-- !query 1059 +SELECT cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1059 schema +struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1059 output +false + + +-- !query 1060 +SELECT cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1060 schema +struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1060 output +false + + +-- !query 1061 +SELECT cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1061 schema +struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)))):boolean> +-- !query 1061 output +false + + +-- !query 1062 +SELECT cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1062 schema +struct<(NOT (CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query 1062 output +false + + +-- !query 1063 +SELECT cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1063 schema +struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1063 output +false + + +-- !query 1064 +SELECT cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1064 schema +struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1064 output +false + + +-- !query 1065 +SELECT cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1065 schema +struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1065 output +false + + +-- !query 1066 +SELECT cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1066 schema +struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1066 output +false + + +-- !query 1067 +SELECT cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1067 schema +struct<(NOT (CAST(CAST(1 AS INT) AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query 1067 output +false + + +-- !query 1068 +SELECT cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1068 schema +struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1068 output +false + + +-- !query 1069 +SELECT cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1069 schema +struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1069 output +false + + +-- !query 1070 +SELECT cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1070 schema +struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1070 output +false + + +-- !query 1071 +SELECT cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1071 schema +struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1071 output +false + + +-- !query 1072 +SELECT cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1072 schema +struct<(NOT (CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query 1072 output +false + + +-- !query 1073 +SELECT cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1073 schema +struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE))):boolean> +-- !query 1073 output +false + + +-- !query 1074 +SELECT cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1074 schema +struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE))):boolean> +-- !query 1074 output +false + + +-- !query 1075 +SELECT cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1075 schema +struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> +-- !query 1075 output +false + + +-- !query 1076 +SELECT cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1076 schema +struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE))):boolean> +-- !query 1076 output +false + + +-- !query 1077 +SELECT cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1077 schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE))):boolean> +-- !query 1077 output +false + + +-- !query 1078 +SELECT cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1078 schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE))):boolean> +-- !query 1078 output +false + + +-- !query 1079 +SELECT cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1079 schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> +-- !query 1079 output +false + + +-- !query 1080 +SELECT cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1080 schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE))):boolean> +-- !query 1080 output +false + + +-- !query 1081 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1081 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1081 output +false + + +-- !query 1082 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1082 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1082 output +false + + +-- !query 1083 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1083 schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query 1083 output +false + + +-- !query 1084 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1084 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1084 output +false + + +-- !query 1085 +SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1085 schema +struct<> +-- !query 1085 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 + + +-- !query 1086 +SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1086 schema +struct<> +-- !query 1086 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 + + +-- !query 1087 +SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1087 schema +struct<> +-- !query 1087 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 + + +-- !query 1088 +SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1088 schema +struct<> +-- !query 1088 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 + + +-- !query 1089 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1089 schema +struct<> +-- !query 1089 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 + + +-- !query 1090 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1090 schema +struct<> +-- !query 1090 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 + + +-- !query 1091 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1091 schema +struct<> +-- !query 1091 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 + + +-- !query 1092 +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1092 schema +struct<> +-- !query 1092 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 + + +-- !query 1093 +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t +-- !query 1093 schema +struct<> +-- !query 1093 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 + + +-- !query 1094 +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t +-- !query 1094 schema +struct<> +-- !query 1094 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 + + +-- !query 1095 +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1095 schema +struct<> +-- !query 1095 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 + + +-- !query 1096 +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t +-- !query 1096 schema +struct<> +-- !query 1096 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 + + +-- !query 1097 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t +-- !query 1097 schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)))):boolean> +-- !query 1097 output +false + + +-- !query 1098 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t +-- !query 1098 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)))):boolean> +-- !query 1098 output +false + + +-- !query 1099 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t +-- !query 1099 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1099 output +false + + +-- !query 1100 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t +-- !query 1100 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1100 output +false + + +-- !query 1101 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t +-- !query 1101 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)))):boolean> +-- !query 1101 output +false + + +-- !query 1102 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t +-- !query 1102 schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)))):boolean> +-- !query 1102 output +false + + +-- !query 1103 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t +-- !query 1103 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1103 output +false + + +-- !query 1104 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t +-- !query 1104 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1104 output +false + + +-- !query 1105 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t +-- !query 1105 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1105 output +false + + +-- !query 1106 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t +-- !query 1106 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1106 output +false + + +-- !query 1107 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t +-- !query 1107 schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS INT) AS DECIMAL(10,0)))):boolean> +-- !query 1107 output +false + + +-- !query 1108 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t +-- !query 1108 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1108 output +false + + +-- !query 1109 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t +-- !query 1109 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1109 output +false + + +-- !query 1110 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t +-- !query 1110 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1110 output +false + + +-- !query 1111 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t +-- !query 1111 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1111 output +false + + +-- !query 1112 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t +-- !query 1112 schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)))):boolean> +-- !query 1112 output +false + + +-- !query 1113 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t +-- !query 1113 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> +-- !query 1113 output +false + + +-- !query 1114 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t +-- !query 1114 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> +-- !query 1114 output +false + + +-- !query 1115 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t +-- !query 1115 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> +-- !query 1115 output +false + + +-- !query 1116 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t +-- !query 1116 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> +-- !query 1116 output +false + + +-- !query 1117 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t +-- !query 1117 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> +-- !query 1117 output +false + + +-- !query 1118 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t +-- !query 1118 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> +-- !query 1118 output +false + + +-- !query 1119 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t +-- !query 1119 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> +-- !query 1119 output +false + + +-- !query 1120 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t +-- !query 1120 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> +-- !query 1120 output +false + + +-- !query 1121 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1121 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1121 output +false + + +-- !query 1122 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1122 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> +-- !query 1122 output +false + + +-- !query 1123 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1123 schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query 1123 output +false + + +-- !query 1124 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query 1124 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> +-- !query 1124 output +false + + +-- !query 1125 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t +-- !query 1125 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> +-- !query 1125 output +false + + +-- !query 1126 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t +-- !query 1126 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> +-- !query 1126 output +false + + +-- !query 1127 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t +-- !query 1127 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> +-- !query 1127 output +false + + +-- !query 1128 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t +-- !query 1128 schema +struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> +-- !query 1128 output +false + + +-- !query 1129 +SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t +-- !query 1129 schema +struct<> +-- !query 1129 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 + + +-- !query 1130 +SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t +-- !query 1130 schema +struct<> +-- !query 1130 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 + + +-- !query 1131 +SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t +-- !query 1131 schema +struct<> +-- !query 1131 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 + + +-- !query 1132 +SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t +-- !query 1132 schema +struct<> +-- !query 1132 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 + + +-- !query 1133 +SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t +-- !query 1133 schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0)))):boolean> +-- !query 1133 output +false + + +-- !query 1134 +SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t +-- !query 1134 schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0)))):boolean> +-- !query 1134 output +false + + +-- !query 1135 +SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t +-- !query 1135 schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0)))):boolean> +-- !query 1135 output +false + + +-- !query 1136 +SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t +-- !query 1136 schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0)))):boolean> +-- !query 1136 output +false + + +-- !query 1137 +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1137 schema +struct<> +-- !query 1137 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 + + +-- !query 1138 +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1138 schema +struct<> +-- !query 1138 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 + + +-- !query 1139 +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1139 schema +struct<> +-- !query 1139 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 + + +-- !query 1140 +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query 1140 schema +struct<> +-- !query 1140 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 + + +-- !query 1141 +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1141 schema +struct<> +-- !query 1141 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 + + +-- !query 1142 +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1142 schema +struct<> +-- !query 1142 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 + + +-- !query 1143 +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1143 schema +struct<> +-- !query 1143 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 + + +-- !query 1144 +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query 1144 schema +struct<> +-- !query 1144 output +org.apache.spark.sql.AnalysisException +cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out new file mode 100644 index 0000000000000..5dd257ba6a0bb --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -0,0 +1,206 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 25 + + +-- !query 0 +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t +-- !query 1 schema +struct +-- !query 1 output +1 + + +-- !query 2 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t +-- !query 2 schema +struct +-- !query 2 output +1 + + +-- !query 3 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t +-- !query 5 schema +struct +-- !query 5 output +1 + + +-- !query 6 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t +-- !query 6 schema +struct +-- !query 6 output +1 + + +-- !query 7 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t +-- !query 8 schema +struct +-- !query 8 output +1 + + +-- !query 9 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query 11 schema +struct +-- !query 11 output +1 + + +-- !query 12 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 15 schema +struct +-- !query 15 output +1 + + +-- !query 16 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 16 schema +struct +-- !query 16 output +1 + + +-- !query 17 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 17 schema +struct +-- !query 17 output +1 + + +-- !query 18 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 18 schema +struct +-- !query 18 output +1 + + +-- !query 19 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS STRING) FOLLOWING' due to data type mismatch: The data type of the upper bound 'StringType' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 + + +-- !query 21 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY 1 ORDER BY CAST('1' AS BINARY) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'BinaryType' used in the order specification does not match the data type 'IntegerType' which is used in the range frame.; line 1 pos 21 + + +-- !query 22 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BOOLEAN) FOLLOWING' due to data type mismatch: The data type of the upper bound 'BooleanType' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 + + +-- !query 23 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY 1 ORDER BY CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'TimestampType' used in the order specification does not match the data type 'IntegerType' which is used in the range frame.; line 1 pos 21 + + +-- !query 24 +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query 24 schema +struct +-- !query 24 output +1 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 73ad27e5bf8ce..a52e198eb9a8f 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -61,7 +61,7 @@ ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'LongType does not match the expected data type 'IntegerType'.; line 1 pos 41 +cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'LongType' does not match the expected data type 'int'.; line 1 pos 41 -- !query 4 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 57958f7239224..fd5a3df6abc68 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 @@ -25,7 +25,7 @@ import java.util.Locale import com.google.common.io.Files import org.apache.arrow.memory.RootAllocator import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot} -import org.apache.arrow.vector.file.json.JsonFileReader +import org.apache.arrow.vector.ipc.JsonFileReader import org.apache.arrow.vector.util.Validator import org.scalatest.BeforeAndAfterAll @@ -76,16 +76,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 16 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 16 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_s", | "type" : { @@ -94,16 +85,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 16 | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 16 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -143,16 +125,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_i", | "type" : { @@ -161,16 +134,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -210,16 +174,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 64 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_l", | "type" : { @@ -228,16 +183,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 64 | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -276,16 +222,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "SINGLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_f", | "type" : { @@ -293,16 +230,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "SINGLE" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -341,16 +269,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "DOUBLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_d", | "type" : { @@ -358,16 +277,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "DOUBLE" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -408,16 +318,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -449,16 +350,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 16 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 16 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b", | "type" : { @@ -466,16 +358,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "SINGLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "c", | "type" : { @@ -484,16 +367,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "d", | "type" : { @@ -501,16 +375,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "DOUBLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | }, { | "name" : "e", | "type" : { @@ -519,16 +384,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 64 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -583,57 +439,21 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "name" : "utf8" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 8 - | } ] - | } + | "children" : [ ] | }, { | "name" : "lower_case", | "type" : { | "name" : "utf8" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 8 - | } ] - | } + | "children" : [ ] | }, { | "name" : "null_str", | "type" : { | "name" : "utf8" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 8 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -681,16 +501,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "name" : "bool" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 1 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -721,16 +532,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 8 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 8 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -760,19 +562,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "name" : "binary" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 8 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -807,16 +597,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "unit" : "DAY" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -855,16 +636,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "timezone" : "America/Los_Angeles" | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -904,16 +676,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "SINGLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "NaN_d", | "type" : { @@ -921,16 +684,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "precision" : "DOUBLE" | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 64 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -939,12 +693,12 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "name" : "NaN_f", | "count" : 2, | "VALIDITY" : [ 1, 1 ], - | "DATA" : [ 1.2000000476837158, "NaN" ] + | "DATA" : [ 1.2000000476837158, NaN ] | }, { | "name" : "NaN_d", | "count" : 2, | "VALIDITY" : [ 1, 1 ], - | "DATA" : [ "NaN", 1.2 ] + | "DATA" : [ NaN, 1.2 ] | } ] | } ] |} @@ -976,26 +730,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "b_arr", | "nullable" : true, @@ -1010,26 +746,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "c_arr", | "nullable" : true, @@ -1044,26 +762,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "d_arr", | "nullable" : true, @@ -1084,36 +784,9 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "OFFSET", - | "typeBitWidth" : 32 + | "children" : [ ] | } ] - | } + | } ] | } ] | }, | "batches" : [ { @@ -1204,23 +877,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "b_struct", | "nullable" : true, @@ -1235,23 +893,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "c_struct", | "nullable" : false, @@ -1266,23 +909,8 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | } ] - | } + | "children" : [ ] + | } ] | }, { | "name" : "d_struct", | "nullable" : true, @@ -1303,30 +931,9 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32, | "isSigned" : true | }, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | } ] - | } - | } ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 + | "children" : [ ] | } ] - | } + | } ] | } ] | }, | "batches" : [ { @@ -1413,16 +1020,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b", | "type" : { @@ -1431,16 +1029,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -1471,16 +1060,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b", | "type" : { @@ -1489,16 +1069,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -1600,16 +1171,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "b_i", | "type" : { @@ -1618,16 +1180,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { @@ -1658,16 +1211,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : true, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | }, { | "name" : "a_i", | "type" : { @@ -1676,16 +1220,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { | "bitWidth" : 32 | }, | "nullable" : false, - | "children" : [ ], - | "typeLayout" : { - | "vectors" : [ { - | "type" : "VALIDITY", - | "typeBitWidth" : 1 - | }, { - | "type" : "DATA", - | "typeBitWidth" : 32 - | } ] - | } + | "children" : [ ] | } ] | }, | "batches" : [ { 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 d588af3e19dde..fc3483379c817 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 @@ -33,14 +33,6 @@ import org.apache.spark.util.{AccumulatorContext, JsonProtocol} class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with SharedSQLContext { import testImplicits._ - private def statusStore: SQLAppStatusStore = { - new SQLAppStatusStore(sparkContext.statusStore.store) - } - - private def currentExecutionIds(): Set[Long] = { - statusStore.executionsList.map(_.executionId).toSet - } - /** * Generates a `DataFrame` by filling randomly generated bytes for hash collision. */ 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 d89c4b14619fa..122d28798136f 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 @@ -31,17 +31,14 @@ import org.apache.spark.util.Utils trait SQLMetricsTestUtils extends SQLTestUtils { - import testImplicits._ - private def statusStore: SQLAppStatusStore = { - new SQLAppStatusStore(sparkContext.statusStore.store) - } - - private def currentExecutionIds(): Set[Long] = { + protected def currentExecutionIds(): Set[Long] = { statusStore.executionsList.map(_.executionId).toSet } + protected def statusStore: SQLAppStatusStore = spark.sharedState.statusStore + /** * Get execution metrics for the SQL execution and verify metrics values. * @@ -57,7 +54,6 @@ trait SQLMetricsTestUtils extends SQLTestUtils { assert(executionIds.size == 1) val executionId = executionIds.head - val executionData = statusStore.execution(executionId).get val executedNode = statusStore.planGraph(executionId).nodes.head val metricsNames = Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala index be4b490754986..00d4f0b8503d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.streaming import org.scalatest.BeforeAndAfter import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.streaming.{OutputMode, StreamTest} class MemorySinkV2Suite extends StreamTest with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala index ef801ceb1310c..6514c5f0fdfeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceV2Suite.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.execution.streaming.sources.{RateStreamBatchTask, RateStreamSourceV2, RateStreamV2Reader} import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2Options, MicroBatchReadSupport} import org.apache.spark.sql.streaming.StreamTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala similarity index 73% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 8c16d46bffaa2..78809354efd5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -41,7 +41,8 @@ import org.apache.spark.status.config._ import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator} import org.apache.spark.util.kvstore.InMemoryStore -class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils { + +class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils { import testImplicits._ override protected def sparkConf = { @@ -61,21 +62,21 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest properties } - private def createStageInfo(stageId: Int, attemptId: Int): StageInfo = new StageInfo( - stageId = stageId, - attemptId = attemptId, - // The following fields are not used in tests - name = "", - numTasks = 0, - rddInfos = Nil, - parentIds = Nil, - details = "" - ) + private def createStageInfo(stageId: Int, attemptId: Int): StageInfo = { + new StageInfo(stageId = stageId, + attemptId = attemptId, + // The following fields are not used in tests + name = "", + numTasks = 0, + rddInfos = Nil, + parentIds = Nil, + details = "") + } private def createTaskInfo( taskId: Int, attemptNumber: Int, - accums: Map[Long, Long] = Map()): TaskInfo = { + accums: Map[Long, Long] = Map.empty): TaskInfo = { val info = new TaskInfo( taskId = taskId, attemptNumber = attemptNumber, @@ -99,29 +100,37 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest }.toSeq } - /** Return the shared SQL store from the active SparkSession. */ - private def statusStore: SQLAppStatusStore = - new SQLAppStatusStore(spark.sparkContext.statusStore.store) - - /** - * Runs a test with a temporary SQLAppStatusStore tied to a listener bus. Events can be sent to - * the listener bus to update the store, and all data will be cleaned up at the end of the test. - */ - private def sqlStoreTest(name: String) - (fn: (SQLAppStatusStore, SparkListenerBus) => Unit): Unit = { - test(name) { - val conf = sparkConf - val store = new ElementTrackingStore(new InMemoryStore(), conf) - val bus = new ReplayListenerBus() - val listener = new SQLAppStatusListener(conf, store, true) - bus.addListener(listener) - store.close(false) - val sqlStore = new SQLAppStatusStore(store, Some(listener)) - fn(sqlStore, bus) + private def assertJobs( + exec: Option[SQLExecutionUIData], + running: Seq[Int] = Nil, + completed: Seq[Int] = Nil, + failed: Seq[Int] = Nil): Unit = { + val actualRunning = new ListBuffer[Int]() + val actualCompleted = new ListBuffer[Int]() + val actualFailed = new ListBuffer[Int]() + + exec.get.jobs.foreach { case (jobId, jobStatus) => + jobStatus match { + case JobExecutionStatus.RUNNING => actualRunning += jobId + case JobExecutionStatus.SUCCEEDED => actualCompleted += jobId + case JobExecutionStatus.FAILED => actualFailed += jobId + case _ => fail(s"Unexpected status $jobStatus") + } } + + assert(actualRunning.sorted === running) + assert(actualCompleted.sorted === completed) + assert(actualFailed.sorted === failed) + } + + private def createStatusStore(): SQLAppStatusStore = { + val conf = sparkContext.conf + val store = new ElementTrackingStore(new InMemoryStore, conf) + val listener = new SQLAppStatusListener(conf, store, live = true) + new SQLAppStatusStore(store, Some(listener)) } - sqlStoreTest("basic") { (store, bus) => + test("basic") { def checkAnswer(actual: Map[Long, String], expected: Map[Long, Long]): Unit = { assert(actual.size == expected.size) expected.foreach { case (id, value) => @@ -135,6 +144,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest } } + val statusStore = createStatusStore() + val listener = statusStore.listener.get + val executionId = 0 val df = createTestDataFrame val accumulatorIds = @@ -147,7 +159,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest (id, accumulatorValue) }.toMap - bus.postToAll(SparkListenerSQLExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", @@ -155,7 +167,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), System.currentTimeMillis())) - bus.postToAll(SparkListenerJobStart( + listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Seq( @@ -163,45 +175,45 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest createStageInfo(1, 0) ), createProperties(executionId))) - bus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 0))) + listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 0))) - assert(store.executionMetrics(0).isEmpty) + assert(statusStore.executionMetrics(executionId).isEmpty) - bus.postToAll(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) (0L, 0, 0, createAccumulatorInfos(accumulatorUpdates)), (1L, 0, 0, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) // Driver accumulator updates don't belong to this execution should be filtered and no // exception will be thrown. - bus.postToAll(SparkListenerDriverAccumUpdates(0, Seq((999L, 2L)))) + listener.onOtherEvent(SparkListenerDriverAccumUpdates(0, Seq((999L, 2L)))) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) - bus.postToAll(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) (0L, 0, 0, createAccumulatorInfos(accumulatorUpdates)), (1L, 0, 0, createAccumulatorInfos(accumulatorUpdates.mapValues(_ * 2))) ))) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 3)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 3)) // Retrying a stage should reset the metrics - bus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1))) + listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 1))) - bus.postToAll(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) (0L, 0, 1, createAccumulatorInfos(accumulatorUpdates)), (1L, 0, 1, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) // Ignore the task end for the first attempt - bus.postToAll(SparkListenerTaskEnd( + listener.onTaskEnd(SparkListenerTaskEnd( stageId = 0, stageAttemptId = 0, taskType = "", @@ -209,17 +221,17 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 100)), null)) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) // Finish two tasks - bus.postToAll(SparkListenerTaskEnd( + listener.onTaskEnd(SparkListenerTaskEnd( stageId = 0, stageAttemptId = 1, taskType = "", reason = null, createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 2)), null)) - bus.postToAll(SparkListenerTaskEnd( + listener.onTaskEnd(SparkListenerTaskEnd( stageId = 0, stageAttemptId = 1, taskType = "", @@ -227,28 +239,28 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), null)) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 5)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 5)) // Summit a new stage - bus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 0))) + listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) - bus.postToAll(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) (0L, 1, 0, createAccumulatorInfos(accumulatorUpdates)), (1L, 1, 0, createAccumulatorInfos(accumulatorUpdates)) ))) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 7)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 7)) // Finish two tasks - bus.postToAll(SparkListenerTaskEnd( + listener.onTaskEnd(SparkListenerTaskEnd( stageId = 1, stageAttemptId = 0, taskType = "", reason = null, createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 3)), null)) - bus.postToAll(SparkListenerTaskEnd( + listener.onTaskEnd(SparkListenerTaskEnd( stageId = 1, stageAttemptId = 0, taskType = "", @@ -256,109 +268,119 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), null)) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 11)) + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) - assertJobs(store.execution(0), running = Seq(0)) + assertJobs(statusStore.execution(executionId), running = Seq(0)) - bus.postToAll(SparkListenerJobEnd( + listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), JobSucceeded )) - bus.postToAll(SparkListenerSQLExecutionEnd( + listener.onOtherEvent(SparkListenerSQLExecutionEnd( executionId, System.currentTimeMillis())) - assertJobs(store.execution(0), completed = Seq(0)) - checkAnswer(store.executionMetrics(0), accumulatorUpdates.mapValues(_ * 11)) + assertJobs(statusStore.execution(executionId), completed = Seq(0)) + + checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) } - sqlStoreTest("onExecutionEnd happens before onJobEnd(JobSucceeded)") { (store, bus) => + test("onExecutionEnd happens before onJobEnd(JobSucceeded)") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + val executionId = 0 val df = createTestDataFrame - bus.postToAll(SparkListenerSQLExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), System.currentTimeMillis())) - bus.postToAll(SparkListenerJobStart( + listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - bus.postToAll(SparkListenerSQLExecutionEnd( + listener.onOtherEvent(SparkListenerSQLExecutionEnd( executionId, System.currentTimeMillis())) - bus.postToAll(SparkListenerJobEnd( + listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), JobSucceeded )) - assertJobs(store.execution(0), completed = Seq(0)) + assertJobs(statusStore.execution(executionId), completed = Seq(0)) } - sqlStoreTest("onExecutionEnd happens before multiple onJobEnd(JobSucceeded)s") { (store, bus) => + test("onExecutionEnd happens before multiple onJobEnd(JobSucceeded)s") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + val executionId = 0 val df = createTestDataFrame - bus.postToAll(SparkListenerSQLExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), System.currentTimeMillis())) - bus.postToAll(SparkListenerJobStart( + listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - bus.postToAll(SparkListenerJobEnd( + listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), JobSucceeded )) - bus.postToAll(SparkListenerJobStart( + listener.onJobStart(SparkListenerJobStart( jobId = 1, time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - bus.postToAll(SparkListenerSQLExecutionEnd( + listener.onOtherEvent(SparkListenerSQLExecutionEnd( executionId, System.currentTimeMillis())) - bus.postToAll(SparkListenerJobEnd( + listener.onJobEnd(SparkListenerJobEnd( jobId = 1, time = System.currentTimeMillis(), JobSucceeded )) - assertJobs(store.execution(0), completed = Seq(0, 1)) + assertJobs(statusStore.execution(executionId), completed = Seq(0, 1)) } - sqlStoreTest("onExecutionEnd happens before onJobEnd(JobFailed)") { (store, bus) => + test("onExecutionEnd happens before onJobEnd(JobFailed)") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + val executionId = 0 val df = createTestDataFrame - bus.postToAll(SparkListenerSQLExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), System.currentTimeMillis())) - bus.postToAll(SparkListenerJobStart( + listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Seq.empty, createProperties(executionId))) - bus.postToAll(SparkListenerSQLExecutionEnd( + listener.onOtherEvent(SparkListenerSQLExecutionEnd( executionId, System.currentTimeMillis())) - bus.postToAll(SparkListenerJobEnd( + listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), JobFailed(new RuntimeException("Oops")) )) - assertJobs(store.execution(0), failed = Seq(0)) + assertJobs(statusStore.execution(executionId), failed = Seq(0)) } sqlStoreTest("handle one execution with multiple jobs") { (store, bus) => @@ -402,21 +424,21 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest } test("SPARK-11126: no memory leak when running non SQL jobs") { - val previousStageNumber = statusStore.executionsList().size + val listener = spark.sharedState.statusStore.listener.get + // At the beginning of this test case, there should be no live data in the listener. + assert(listener.noLiveData()) spark.sparkContext.parallelize(1 to 10).foreach(i => ()) spark.sparkContext.listenerBus.waitUntilEmpty(10000) - // listener should ignore the non SQL stage - assert(statusStore.executionsList().size == previousStageNumber) - - spark.sparkContext.parallelize(1 to 10).toDF().foreach(i => ()) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) - // listener should save the SQL stage - assert(statusStore.executionsList().size == previousStageNumber + 1) + // Listener should ignore the non-SQL stages, as the stage data are only removed when SQL + // execution ends, which will not be triggered for non-SQL jobs. + assert(listener.noLiveData()) } test("driver side SQL metrics") { + val statusStore = spark.sharedState.statusStore val oldCount = statusStore.executionsList().size - val expectedAccumValue = 12345L + + val expectedAccumValue = 12345 val physicalPlan = MyPlan(sqlContext.sparkContext, expectedAccumValue) val dummyQueryExecution = new QueryExecution(spark, LocalRelation()) { override lazy val sparkPlan = physicalPlan @@ -427,7 +449,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest physicalPlan.execute().collect() } - while (statusStore.executionsList().size < oldCount) { + // Wait until the new execution is started and being tracked. + while (statusStore.executionsCount() < oldCount) { Thread.sleep(100) } @@ -445,30 +468,6 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTest assert(metrics(driverMetric.id) === expectedValue) } - private def assertJobs( - exec: Option[SQLExecutionUIData], - running: Seq[Int] = Nil, - completed: Seq[Int] = Nil, - failed: Seq[Int] = Nil): Unit = { - - val actualRunning = new ListBuffer[Int]() - val actualCompleted = new ListBuffer[Int]() - val actualFailed = new ListBuffer[Int]() - - exec.get.jobs.foreach { case (jobId, jobStatus) => - jobStatus match { - case JobExecutionStatus.RUNNING => actualRunning += jobId - case JobExecutionStatus.SUCCEEDED => actualCompleted += jobId - case JobExecutionStatus.FAILED => actualFailed += jobId - case _ => fail(s"Unexpected status $jobStatus") - } - } - - assert(actualRunning.toSeq.sorted === running) - assert(actualCompleted.toSeq.sorted === completed) - assert(actualFailed.toSeq.sorted === failed) - } - test("roundtripping SparkListenerDriverAccumUpdates through JsonProtocol (SPARK-18462)") { val event = SparkListenerDriverAccumUpdates(1L, Seq((2L, 3L))) val json = JsonProtocol.sparkEventToJson(event) @@ -534,7 +533,7 @@ private case class MyPlan(sc: SparkContext, expectedValue: Long) extends LeafExe } -class SQLListenerMemoryLeakSuite extends SparkFunSuite { +class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { test("no memory leak") { val conf = new SparkConf() @@ -562,9 +561,10 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { } } sc.listenerBus.waitUntilEmpty(10000) - - val statusStore = new SQLAppStatusStore(sc.statusStore.store) - assert(statusStore.executionsList().size <= 50) + val statusStore = spark.sharedState.statusStore + assert(statusStore.executionsCount() <= 50) + // No live data should be left behind after all executions end. + assert(statusStore.listener.get.noLiveData()) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala index e460d0721e7bf..03490ad15a655 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala @@ -30,15 +30,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("boolean") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("boolean", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("boolean", BooleanType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableBitVector] + .createVector(allocator).asInstanceOf[BitVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, if (i % 2 == 0) 1 else 0) + vector.setSafe(i, if (i % 2 == 0) 1 else 0) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === BooleanType) @@ -58,15 +57,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("byte") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("byte", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("byte", ByteType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableTinyIntVector] + .createVector(allocator).asInstanceOf[TinyIntVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i.toByte) + vector.setSafe(i, i.toByte) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === ByteType) @@ -86,15 +84,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("short") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("short", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("short", ShortType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableSmallIntVector] + .createVector(allocator).asInstanceOf[SmallIntVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i.toShort) + vector.setSafe(i, i.toShort) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === ShortType) @@ -114,15 +111,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("int") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("int", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("int", IntegerType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableIntVector] + .createVector(allocator).asInstanceOf[IntVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i) + vector.setSafe(i, i) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === IntegerType) @@ -142,15 +138,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("long") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("long", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("long", LongType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableBigIntVector] + .createVector(allocator).asInstanceOf[BigIntVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i.toLong) + vector.setSafe(i, i.toLong) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === LongType) @@ -170,15 +165,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("float") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("float", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("float", FloatType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableFloat4Vector] + .createVector(allocator).asInstanceOf[Float4Vector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i.toFloat) + vector.setSafe(i, i.toFloat) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === FloatType) @@ -198,15 +192,14 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("double") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("double", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("double", DoubleType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableFloat8Vector] + .createVector(allocator).asInstanceOf[Float8Vector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => - mutator.setSafe(i, i.toDouble) + vector.setSafe(i, i.toDouble) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === DoubleType) @@ -226,16 +219,15 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("string") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("string", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("string", StringType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableVarCharVector] + .createVector(allocator).asInstanceOf[VarCharVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => val utf8 = s"str$i".getBytes("utf8") - mutator.setSafe(i, utf8, 0, utf8.length) + vector.setSafe(i, utf8, 0, utf8.length) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === StringType) @@ -253,16 +245,15 @@ class ArrowColumnVectorSuite extends SparkFunSuite { test("binary") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("binary", 0, Long.MaxValue) val vector = ArrowUtils.toArrowField("binary", BinaryType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableVarBinaryVector] + .createVector(allocator).asInstanceOf[VarBinaryVector] vector.allocateNew() - val mutator = vector.getMutator() (0 until 10).foreach { i => val utf8 = s"str$i".getBytes("utf8") - mutator.setSafe(i, utf8, 0, utf8.length) + vector.setSafe(i, utf8, 0, utf8.length) } - mutator.setNull(10) - mutator.setValueCount(11) + vector.setNull(10) + vector.setValueCount(11) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === BinaryType) @@ -282,31 +273,29 @@ class ArrowColumnVectorSuite extends SparkFunSuite { val vector = ArrowUtils.toArrowField("array", ArrayType(IntegerType), nullable = true, null) .createVector(allocator).asInstanceOf[ListVector] vector.allocateNew() - val mutator = vector.getMutator() - val elementVector = vector.getDataVector().asInstanceOf[NullableIntVector] - val elementMutator = elementVector.getMutator() + val elementVector = vector.getDataVector().asInstanceOf[IntVector] // [1, 2] - mutator.startNewValue(0) - elementMutator.setSafe(0, 1) - elementMutator.setSafe(1, 2) - mutator.endValue(0, 2) + vector.startNewValue(0) + elementVector.setSafe(0, 1) + elementVector.setSafe(1, 2) + vector.endValue(0, 2) // [3, null, 5] - mutator.startNewValue(1) - elementMutator.setSafe(2, 3) - elementMutator.setNull(3) - elementMutator.setSafe(4, 5) - mutator.endValue(1, 3) + vector.startNewValue(1) + elementVector.setSafe(2, 3) + elementVector.setNull(3) + elementVector.setSafe(4, 5) + vector.endValue(1, 3) // null // [] - mutator.startNewValue(3) - mutator.endValue(3, 0) + vector.startNewValue(3) + vector.endValue(3, 0) - elementMutator.setValueCount(5) - mutator.setValueCount(4) + elementVector.setValueCount(5) + vector.setValueCount(4) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === ArrayType(IntegerType)) @@ -338,38 +327,35 @@ class ArrowColumnVectorSuite extends SparkFunSuite { val vector = ArrowUtils.toArrowField("struct", schema, nullable = true, null) .createVector(allocator).asInstanceOf[NullableMapVector] vector.allocateNew() - val mutator = vector.getMutator() - val intVector = vector.getChildByOrdinal(0).asInstanceOf[NullableIntVector] - val intMutator = intVector.getMutator() - val longVector = vector.getChildByOrdinal(1).asInstanceOf[NullableBigIntVector] - val longMutator = longVector.getMutator() + val intVector = vector.getChildByOrdinal(0).asInstanceOf[IntVector] + val longVector = vector.getChildByOrdinal(1).asInstanceOf[BigIntVector] // (1, 1L) - mutator.setIndexDefined(0) - intMutator.setSafe(0, 1) - longMutator.setSafe(0, 1L) + vector.setIndexDefined(0) + intVector.setSafe(0, 1) + longVector.setSafe(0, 1L) // (2, null) - mutator.setIndexDefined(1) - intMutator.setSafe(1, 2) - longMutator.setNull(1) + vector.setIndexDefined(1) + intVector.setSafe(1, 2) + longVector.setNull(1) // (null, 3L) - mutator.setIndexDefined(2) - intMutator.setNull(2) - longMutator.setSafe(2, 3L) + vector.setIndexDefined(2) + intVector.setNull(2) + longVector.setSafe(2, 3L) // null - mutator.setNull(3) + vector.setNull(3) // (5, 5L) - mutator.setIndexDefined(4) - intMutator.setSafe(4, 5) - longMutator.setSafe(4, 5L) + vector.setIndexDefined(4) + intVector.setSafe(4, 5) + longVector.setSafe(4, 5L) - intMutator.setValueCount(5) - longMutator.setValueCount(5) - mutator.setValueCount(5) + intVector.setValueCount(5) + longVector.setValueCount(5) + vector.setValueCount(5) val columnVector = new ArrowColumnVector(vector) assert(columnVector.dataType === schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index d3ed8276b8f10..7848ebdcab6d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Random -import org.apache.arrow.vector.NullableIntVector +import org.apache.arrow.vector.IntVector import org.apache.spark.SparkFunSuite import org.apache.spark.memory.MemoryMode @@ -1137,22 +1137,20 @@ class ColumnarBatchSuite extends SparkFunSuite { test("create columnar batch from Arrow column vectors") { val allocator = ArrowUtils.rootAllocator.newChildAllocator("int", 0, Long.MaxValue) val vector1 = ArrowUtils.toArrowField("int1", IntegerType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableIntVector] + .createVector(allocator).asInstanceOf[IntVector] vector1.allocateNew() - val mutator1 = vector1.getMutator() val vector2 = ArrowUtils.toArrowField("int2", IntegerType, nullable = true, null) - .createVector(allocator).asInstanceOf[NullableIntVector] + .createVector(allocator).asInstanceOf[IntVector] vector2.allocateNew() - val mutator2 = vector2.getMutator() (0 until 10).foreach { i => - mutator1.setSafe(i, i) - mutator2.setSafe(i + 1, i) + vector1.setSafe(i, i) + vector2.setSafe(i + 1, i) } - mutator1.setNull(10) - mutator1.setValueCount(11) - mutator2.setNull(0) - mutator2.setValueCount(11) + vector1.setNull(10) + vector1.setValueCount(11) + vector2.setNull(0) + vector2.setValueCount(11) val columnVectors = Seq(new ArrowColumnVector(vector1), new ArrowColumnVector(vector2)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 9025859e91066..fb61fa716b946 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -620,7 +620,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { |) ab |JOIN table2 c |ON ab.i = c.i - |""".stripMargin), + """.stripMargin), sql(""" |SELECT a.i, a.j, a.k, c.i, c.j, c.k |FROM bucketed_table a @@ -628,7 +628,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { |ON a.i = b.i |JOIN table2 c |ON a.i = c.i - |""".stripMargin)) + """.stripMargin)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala new file mode 100644 index 0000000000000..4911e3225552d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2 + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.internal.SQLConf + +class DataSourceV2UtilsSuite extends SparkFunSuite { + + private val keyPrefix = new DataSourceV2WithSessionConfig().keyPrefix + + test("method withSessionConfig() should propagate session configs correctly") { + // Only match configs with keys start with "spark.datasource.${keyPrefix}". + val conf = new SQLConf + conf.setConfString(s"spark.datasource.$keyPrefix.foo.bar", "false") + conf.setConfString(s"spark.datasource.$keyPrefix.whateverConfigName", "123") + conf.setConfString(s"spark.sql.$keyPrefix.config.name", "false") + conf.setConfString("spark.datasource.another.config.name", "123") + conf.setConfString(s"spark.datasource.$keyPrefix.", "123") + val cs = classOf[DataSourceV2WithSessionConfig].newInstance() + val confs = DataSourceV2Utils.extractSessionConfigs(cs.asInstanceOf[DataSourceV2], conf) + assert(confs.size == 2) + assert(confs.keySet.filter(_.startsWith("spark.datasource")).size == 0) + assert(confs.keySet.filter(_.startsWith("not.exist.prefix")).size == 0) + assert(confs.keySet.contains("foo.bar")) + assert(confs.keySet.contains("whateverConfigName")) + } +} + +class DataSourceV2WithSessionConfig extends SimpleDataSourceV2 with SessionConfigSupport { + + override def keyPrefix: String = "userDefinedDataSource" +} 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 f4fa7fa7954d6..39bb572740617 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 @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala index 429748261f1ea..f208f9bd9b6e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.streaming.{LongOffset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.Offset +import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, SerializedOffset} trait OffsetSuite extends SparkFunSuite { /** Creates test to check all the comparisons of offsets given a `one` that is less than `two`. */ 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 fa4b2dd6a6c9b..755490308b5b9 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 @@ -39,7 +39,6 @@ import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreCon import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index fb88c5d327043..71a474ef63e84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 38aa5171314f2..97e065193fd05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode._ import org.apache.spark.sql.streaming.util.{MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index fc9ac2a56c4e5..9ff02dee288fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -33,7 +33,6 @@ import org.apache.spark.scheduler._ import org.apache.spark.sql.{Encoder, SparkSession} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.util.JsonProtocol 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 ad4d3abd01aa5..2fa4595dab376 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 @@ -33,12 +33,10 @@ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType import org.apache.spark.util.ManualClock - class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging with MockitoSugar { import AwaitTerminationTester._ 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 952908f21ca60..aa163d2211c38 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 @@ -32,7 +32,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.{ProcessingTime => DeprecatedProcessingTime, _} import org.apache.spark.sql.streaming.Trigger._ import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala index 9a35f097e6e40..19ab2ff13e14e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql.streaming.util import java.util.concurrent.CountDownLatch import org.apache.spark.sql.{SQLContext, _} -import org.apache.spark.sql.execution.streaming.{LongOffset, Sink, Source} +import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Sink, Source} import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} -import org.apache.spark.sql.sources.v2.reader.Offset import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{IntegerType, StructField, StructType} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 6a3b3200dccdb..a6997359d64d2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -29,7 +29,9 @@ private[streaming] object HdfsUtils { // If the file exists and we have append support, append instead of creating a new file val stream: FSDataOutputStream = { if (dfs.isFile(dfsPath)) { - if (conf.getBoolean("hdfs.append.support", false) || dfs.isInstanceOf[RawLocalFileSystem]) { + if (conf.getBoolean("dfs.support.append", true) || + conf.getBoolean("hdfs.append.support", false) || + dfs.isInstanceOf[RawLocalFileSystem]) { dfs.append(dfsPath) } else { throw new IllegalStateException("File exists and there is no append support!")