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/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 2259d1a2d555d..538ae05e4eea1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -181,20 +181,25 @@ class KryoSerializer(conf: SparkConf) // We can't load those class directly in order to avoid unnecessary jar dependencies. // We load them safely, ignore it if the class not found. - Seq("org.apache.spark.mllib.linalg.Vector", - "org.apache.spark.mllib.linalg.DenseVector", - "org.apache.spark.mllib.linalg.SparseVector", - "org.apache.spark.mllib.linalg.Matrix", - "org.apache.spark.mllib.linalg.DenseMatrix", - "org.apache.spark.mllib.linalg.SparseMatrix", - "org.apache.spark.ml.linalg.Vector", + Seq( + "org.apache.spark.ml.feature.Instance", + "org.apache.spark.ml.feature.LabeledPoint", + "org.apache.spark.ml.feature.OffsetInstance", + "org.apache.spark.ml.linalg.DenseMatrix", "org.apache.spark.ml.linalg.DenseVector", - "org.apache.spark.ml.linalg.SparseVector", "org.apache.spark.ml.linalg.Matrix", - "org.apache.spark.ml.linalg.DenseMatrix", "org.apache.spark.ml.linalg.SparseMatrix", - "org.apache.spark.ml.feature.Instance", - "org.apache.spark.ml.feature.OffsetInstance" + "org.apache.spark.ml.linalg.SparseVector", + "org.apache.spark.ml.linalg.Vector", + "org.apache.spark.ml.tree.impl.TreePoint", + "org.apache.spark.mllib.clustering.VectorWithNorm", + "org.apache.spark.mllib.linalg.DenseMatrix", + "org.apache.spark.mllib.linalg.DenseVector", + "org.apache.spark.mllib.linalg.Matrix", + "org.apache.spark.mllib.linalg.SparseMatrix", + "org.apache.spark.mllib.linalg.SparseVector", + "org.apache.spark.mllib.linalg.Vector", + "org.apache.spark.mllib.regression.LabeledPoint" ).foreach { name => try { val clazz = Utils.classForName(name) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala similarity index 54% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala rename to core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala index 01717479fddd9..d144a0e998fa1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -15,27 +15,24 @@ * limitations under the License. */ -package org.apache.spark.deploy.k8s +package org.apache.spark.status import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.ui.SparkUI -private[spark] object ConfigurationUtils { - +/** + * 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 { /** - * Extract and parse Spark configuration properties with a given name prefix and - * return the result as a Map. Keys must not have more than one value. - * - * @param sparkConf Spark configuration - * @param prefix the given property name prefix - * @return a Map storing the configuration property keys and values + * Creates listeners to replay the event logs. */ - def parsePrefixedKeyValuePairs( - sparkConf: SparkConf, - prefix: String): Map[String, String] = { - sparkConf.getAllWithPrefix(prefix).toMap - } + def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { - opt1.foreach { _ => require(opt2.isEmpty, errMessage) } - } + /** + * 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..5253297137323 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -90,6 +90,11 @@ private[spark] class AppStatusListener( } } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case SparkListenerLogStart(version) => sparkVersion = version + case _ => + } + override def onApplicationStart(event: SparkListenerApplicationStart): Unit = { assert(event.appId.isDefined, "Application without IDs are not supported.") @@ -329,10 +334,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/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 9cf4f7efb24a8..c0b3a79fe981e 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -103,6 +103,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("scheduler events") { val listener = new AppStatusListener(store, conf, true) + listener.onOtherEvent(SparkListenerLogStart("TestSparkVersion")) + // Start the application. time += 1 listener.onApplicationStart(SparkListenerApplicationStart( @@ -125,6 +127,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(attempt.endTime.getTime() === -1L) assert(attempt.sparkUser === "user") assert(!attempt.completed) + assert(attempt.appSparkVersion === "TestSparkVersion") } // Start a couple of executors. 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/docs/_layouts/global.html b/docs/_layouts/global.html index 67b05ecf7a858..e5af5ae4561c7 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -99,6 +99,7 @@
  • Spark Standalone
  • Mesos
  • YARN
  • +
  • Kubernetes
  • diff --git a/docs/building-spark.md b/docs/building-spark.md index 98f7df155456f..c391255a91596 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -49,7 +49,7 @@ To create a Spark distribution like those distributed by the to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured with Maven profile settings and so on like the direct Maven build. Example: - ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn + ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` @@ -90,6 +90,10 @@ like ZooKeeper and Hadoop itself. ## Building with Mesos support ./build/mvn -Pmesos -DskipTests clean package + +## Building with Kubernetes support + + ./build/mvn -Pkubernetes -DskipTests clean package ## Building with Kafka 0.8 support diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index c42bb4bb8377e..658e67f99dd71 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -52,11 +52,8 @@ The system currently supports three cluster managers: * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce and service applications. * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2. -* [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- In addition to the above, -there is experimental support for Kubernetes. Kubernetes is an open-source platform -for providing container-centric infrastructure. Kubernetes support is being actively -developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization. -For documentation, refer to that project's README. +* [Kubernetes](running-on-kubernetes.html) -- [Kubernetes](https://kubernetes.io/docs/concepts/overview/what-is-kubernetes/) +is an open-source platform that provides container-centric infrastructure. A third-party project (not supported by the Spark project) exists to add support for [Nomad](https://github.com/hashicorp/nomad-spark) as a cluster manager. diff --git a/docs/configuration.md b/docs/configuration.md index d70bac134808f..1189aea2aa71f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2376,6 +2376,8 @@ can be found on the pages for each mode: #### [Mesos](running-on-mesos.html#configuration) +#### [Kubernetes](running-on-kubernetes.html#configuration) + #### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables diff --git a/docs/img/k8s-cluster-mode.png b/docs/img/k8s-cluster-mode.png new file mode 100644 index 0000000000000..12a6288ed5823 Binary files /dev/null and b/docs/img/k8s-cluster-mode.png differ diff --git a/docs/index.md b/docs/index.md index b867c972b4b48..2f009417fafb0 100644 --- a/docs/index.md +++ b/docs/index.md @@ -81,6 +81,7 @@ options for deployment: * [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster * [Apache Mesos](running-on-mesos.html) * [Hadoop YARN](running-on-yarn.html) +* [Kubernetes](running-on-kubernetes.html) # Where to Go from Here @@ -112,7 +113,7 @@ options for deployment: * [Mesos](running-on-mesos.html): deploy a private cluster using [Apache Mesos](http://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) - * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark): deploy Spark on top of Kubernetes + * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes **Other Documents:** diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md new file mode 100644 index 0000000000000..0048bd90b48ae --- /dev/null +++ b/docs/running-on-kubernetes.md @@ -0,0 +1,578 @@ +--- +layout: global +title: Running Spark on Kubernetes +--- +* This will become a table of contents (this text will be scraped). +{:toc} + +Spark can run on clusters managed by [Kubernetes](https://kubernetes.io). This feature makes use of native +Kubernetes scheduler that has been added to Spark. + +# Prerequisites + +* A runnable distribution of Spark 2.3 or above. +* A running Kubernetes cluster at version >= 1.6 with access configured to it using +[kubectl](https://kubernetes.io/docs/user-guide/prereqs/). If you do not already have a working Kubernetes cluster, +you may setup a test cluster on your local machine using +[minikube](https://kubernetes.io/docs/getting-started-guides/minikube/). + * We recommend using the latest release of minikube with the DNS addon enabled. +* You must have appropriate permissions to list, create, edit and delete +[pods](https://kubernetes.io/docs/user-guide/pods/) in your cluster. You can verify that you can list these resources +by running `kubectl auth can-i pods`. + * The service account credentials used by the driver pods must be allowed to create pods, services and configmaps. +* You must have [Kubernetes DNS](https://kubernetes.io/docs/concepts/services-networking/dns-pod-service/) configured in your cluster. + +# How it works + +

    + Spark cluster components +

    + +spark-submit can be directly used to submit a Spark application to a Kubernetes cluster. +The submission mechanism works as follows: + +* Spark creates a Spark driver running within a [Kubernetes pod](https://kubernetes.io/docs/concepts/workloads/pods/pod/). +* The driver creates executors which are also running within Kubernetes pods and connects to them, and executes application code. +* When the application completes, the executor pods terminate and are cleaned up, but the driver pod persists +logs and remains in "completed" state in the Kubernetes API until it's eventually garbage collected or manually cleaned up. + +Note that in the completed state, the driver pod does *not* use any computational or memory resources. + +The driver and executor pod scheduling is handled by Kubernetes. It will be possible to affect Kubernetes scheduling +decisions for driver and executor pods using advanced primitives like +[node selectors](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector) +and [node/pod affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity) +in a future release. + +# Submitting Applications to Kubernetes + +## Docker Images + +Kubernetes requires users to supply images that can be deployed into containers within pods. The images are built to +be run in a container runtime environment that Kubernetes supports. Docker is a container runtime environment that is +frequently used with Kubernetes. With Spark 2.3, there are Dockerfiles provided in the runnable distribution that can be customized +and built for your usage. + +You may build these docker images from sources. +There is a script, `sbin/build-push-docker-images.sh` that you can use to build and push +customized Spark distribution images consisting of all the above components. + +Example usage is: + + ./sbin/build-push-docker-images.sh -r -t my-tag build + ./sbin/build-push-docker-images.sh -r -t my-tag push + +Docker files are under the `kubernetes/dockerfiles/` directory and can be customized further before +building using the supplied script, or manually. + +## Cluster Mode + +To launch Spark Pi in cluster mode, + +{% highlight bash %} +$ bin/spark-submit \ + --master k8s://https://: \ + --deploy-mode cluster \ + --name spark-pi \ + --class org.apache.spark.examples.SparkPi \ + --conf spark.executor.instances=5 \ + --conf spark.kubernetes.driver.docker.image= \ + --conf spark.kubernetes.executor.docker.image= \ + local:///path/to/examples.jar +{% endhighlight %} + +The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting +`spark.master` in the application's configuration, must be a URL with the format `k8s://`. Prefixing the +master string with `k8s://` will cause the Spark application to launch on the Kubernetes cluster, with the API server +being contacted at `api_server_url`. If no HTTP protocol is specified in the URL, it defaults to `https`. For example, +setting the master to `k8s://example.com:443` is equivalent to setting it to `k8s://https://example.com:443`, but to +connect without TLS on a different port, the master would be set to `k8s://http://example.com:8080`. + +In Kubernetes mode, the Spark application name that is specified by `spark.app.name` or the `--name` argument to +`spark-submit` is used by default to name the Kubernetes resources created like drivers and executors. So, application names +must consist of lower case alphanumeric characters, `-`, and `.` and must start and end with an alphanumeric character. + +If you have a Kubernetes cluster setup, one way to discover the apiserver URL is by executing `kubectl cluster-info`. + +```bash +kubectl cluster-info +Kubernetes master is running at http://127.0.0.1:6443 +``` + +In the above example, the specific Kubernetes cluster can be used with spark-submit by specifying +`--master k8s://http://127.0.0.1:6443` as an argument to spark-submit. Additionally, it is also possible to use the +authenticating proxy, `kubectl proxy` to communicate to the Kubernetes API. + +The local proxy can be started by: + +```bash +kubectl proxy +``` + +If the local proxy is running at localhost:8001, `--master k8s://http://127.0.0.1:8001` can be used as the argument to +spark-submit. Finally, notice that in the above example we specify a jar with a specific URI with a scheme of `local://`. +This URI is the location of the example jar that is already in the Docker image. + +## Dependency Management + +If your application's dependencies are all hosted in remote locations like HDFS or HTTP servers, they may be referred to +by their appropriate remote URIs. Also, application dependencies can be pre-mounted into custom-built Docker images. +Those dependencies can be added to the classpath by referencing them with `local://` URIs and/or setting the +`SPARK_EXTRA_CLASSPATH` environment variable in your Dockerfiles. + +## Introspection and Debugging + +These are the different ways in which you can investigate a running/completed Spark application, monitor progress, and +take actions. + +### Accessing Logs + +Logs can be accessed using the Kubernetes API and the `kubectl` CLI. When a Spark application is running, it's possible +to stream logs from the application using: + +```bash +kubectl -n= logs -f +``` + +The same logs can also be accessed through the +[Kubernetes dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) if installed on +the cluster. + +### Accessing Driver UI + +The UI associated with any application can be accessed locally using +[`kubectl port-forward`](https://kubernetes.io/docs/tasks/access-application-cluster/port-forward-access-application-cluster/#forward-a-local-port-to-a-port-on-the-pod). + +```bash +kubectl port-forward 4040:4040 +``` + +Then, the Spark driver UI can be accessed on `http://localhost:4040`. + +### Debugging + +There may be several kinds of failures. If the Kubernetes API server rejects the request made from spark-submit, or the +connection is refused for a different reason, the submission logic should indicate the error encountered. However, if there +are errors during the running of the application, often, the best way to investigate may be through the Kubernetes CLI. + +To get some basic information about the scheduling decisions made around the driver pod, you can run: + +```bash +kubectl describe pod +``` + +If the pod has encountered a runtime error, the status can be probed further using: + +```bash +kubectl logs +``` + +Status and logs of failed executor pods can be checked in similar ways. Finally, deleting the driver pod will clean up the entire spark +application, includling all executors, associated service, etc. The driver pod can be thought of as the Kubernetes representation of +the Spark application. + +## Kubernetes Features + +### Namespaces + +Kubernetes has the concept of [namespaces](https://kubernetes.io/docs/concepts/overview/working-with-objects/namespaces/). +Namespaces are ways to divide cluster resources between multiple users (via resource quota). Spark on Kubernetes can +use namespaces to launch Spark applications. This can be made use of through the `spark.kubernetes.namespace` configuration. + +Kubernetes allows using [ResourceQuota](https://kubernetes.io/docs/concepts/policy/resource-quotas/) to set limits on +resources, number of objects, etc on individual namespaces. Namespaces and ResourceQuota can be used in combination by +administrator to control sharing and resource allocation in a Kubernetes cluster running Spark applications. + +### RBAC + +In Kubernetes clusters with [RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) enabled, users can configure +Kubernetes RBAC roles and service accounts used by the various Spark on Kubernetes components to access the Kubernetes +API server. + +The Spark driver pod uses a Kubernetes service account to access the Kubernetes API server to create and watch executor +pods. The service account used by the driver pod must have the appropriate permission for the driver to be able to do +its work. Specifically, at minimum, the service account must be granted a +[`Role` or `ClusterRole`](https://kubernetes.io/docs/admin/authorization/rbac/#role-and-clusterrole) that allows driver +pods to create pods and services. By default, the driver pod is automatically assigned the `default` service account in +the namespace specified by `spark.kubernetes.namespace`, if no service account is specified when the pod gets created. + +Depending on the version and setup of Kubernetes deployed, this `default` service account may or may not have the role +that allows driver pods to create pods and services under the default Kubernetes +[RBAC](https://kubernetes.io/docs/admin/authorization/rbac/) policies. Sometimes users may need to specify a custom +service account that has the right role granted. Spark on Kubernetes supports specifying a custom service account to +be used by the driver pod through the configuration property +`spark.kubernetes.authenticate.driver.serviceAccountName=`. For example to make the driver pod +use the `spark` service account, a user simply adds the following option to the `spark-submit` command: + +``` +--conf spark.kubernetes.authenticate.driver.serviceAccountName=spark +``` + +To create a custom service account, a user can use the `kubectl create serviceaccount` command. For example, the +following command creates a service account named `spark`: + +```bash +kubectl create serviceaccount spark +``` + +To grant a service account a `Role` or `ClusterRole`, a `RoleBinding` or `ClusterRoleBinding` is needed. To create +a `RoleBinding` or `ClusterRoleBinding`, a user can use the `kubectl create rolebinding` (or `clusterrolebinding` +for `ClusterRoleBinding`) command. For example, the following command creates an `edit` `ClusterRole` in the `default` +namespace and grants it to the `spark` service account created above: + +```bash +kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=default:spark --namespace=default +``` + +Note that a `Role` can only be used to grant access to resources (like pods) within a single namespace, whereas a +`ClusterRole` can be used to grant access to cluster-scoped resources (like nodes) as well as namespaced resources +(like pods) across all namespaces. For Spark on Kubernetes, since the driver always creates executor pods in the +same namespace, a `Role` is sufficient, although users may use a `ClusterRole` instead. For more information on +RBAC authorization and how to configure Kubernetes service accounts for pods, please refer to +[Using RBAC Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and +[Configure Service Accounts for Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). + +## Client Mode + +Client mode is not currently supported. + +## Future Work + +There are several Spark on Kubernetes features that are currently being incubated in a fork - +[apache-spark-on-k8s/spark](https://github.com/apache-spark-on-k8s/spark), which are expected to eventually make it into +future versions of the spark-kubernetes integration. + +Some of these include: + +* PySpark +* R +* Dynamic Executor Scaling +* Local File Dependency Management +* Spark Application Management +* Job Queues and Resource Management + +You can refer to the [documentation](https://apache-spark-on-k8s.github.io/userdocs/) if you want to try these features +and provide feedback to the development team. + +# Configuration + +See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are +specific to Spark on Kubernetes. + +#### Spark Properties + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.kubernetes.namespacedefault + The namespace that will be used for running the driver and executor pods. +
    spark.kubernetes.driver.container.image(none) + Container image to use for the driver. + This is usually of the form `example.com/repo/spark-driver:v1.0.0`. + This configuration is required and must be provided by the user. +
    spark.kubernetes.executor.container.image(none) + Container image to use for the executors. + This is usually of the form `example.com/repo/spark-executor:v1.0.0`. + This configuration is required and must be provided by the user. +
    spark.kubernetes.container.image.pullPolicyIfNotPresent + Container image pull policy used when pulling images within Kubernetes. +
    spark.kubernetes.allocation.batch.size5 + Number of pods to launch at once in each round of executor pod allocation. +
    spark.kubernetes.allocation.batch.delay1s + Time to wait between each round of executor pod allocation. Specifying values less than 1 second may lead to + excessive CPU usage on the spark driver. +
    spark.kubernetes.authenticate.submission.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS when starting the driver. This file + must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide + a scheme). +
    spark.kubernetes.authenticate.submission.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server when starting the driver. This file + must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide + a scheme). +
    spark.kubernetes.authenticate.submission.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server when starting the driver. This + file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not + provide a scheme). +
    spark.kubernetes.authenticate.submission.oauthToken(none) + OAuth token to use when authenticating against the Kubernetes API server when starting the driver. Note + that unlike the other authentication options, this is expected to be the exact string value of the token to use for + the authentication. +
    spark.kubernetes.authenticate.submission.oauthTokenFile(none) + Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server when starting the driver. + This file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not + provide a scheme). +
    spark.kubernetes.authenticate.driver.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting + executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting + executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). If this is specified, it is highly + recommended to set up TLS for the driver submission server, as this value is sensitive information that would be + passed to the driver pod in plaintext otherwise. +
    spark.kubernetes.authenticate.driver.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when + requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the + driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.oauthToken(none) + OAuth token to use when authenticating against the Kubernetes API server from the driver pod when + requesting executors. Note that unlike the other authentication options, this must be the exact string value of + the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is + highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would + be passed to the driver pod in plaintext otherwise. +
    spark.kubernetes.authenticate.driver.oauthTokenFile(none) + Path to the OAuth token file containing the token to use when authenticating against the Kubernetes API server from the driver pod when + requesting executors. Note that unlike the other authentication options, this file must contain the exact string value of + the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is + highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would + be passed to the driver pod in plaintext otherwise. +
    spark.kubernetes.authenticate.driver.mounted.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting + executors. This path must be accessible from the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.mounted.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting + executors. This path must be accessible from the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.mounted.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when + requesting executors. This path must be accessible from the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.mounted.oauthTokenFile(none) + Path to the file containing the OAuth token to use when authenticating against the Kubernetes API server from the driver pod when + requesting executors. This path must be accessible from the driver pod. + Note that unlike the other authentication options, this file must contain the exact string value of the token to use for the authentication. +
    spark.kubernetes.authenticate.driver.serviceAccountNamedefault + Service account that is used when running the driver pod. The driver pod uses this service account when requesting + executor pods from the API server. Note that this cannot be specified alongside a CA cert file, client key file, + client cert file, and/or OAuth token. +
    spark.kubernetes.driver.label.[LabelName](none) + Add the label specified by LabelName to the driver pod. + For example, spark.kubernetes.driver.label.something=true. + Note that Spark also adds its own labels to the driver pod + for bookkeeping purposes. +
    spark.kubernetes.driver.annotation.[AnnotationName](none) + Add the annotation specified by AnnotationName to the driver pod. + For example, spark.kubernetes.driver.annotation.something=true. +
    spark.kubernetes.executor.label.[LabelName](none) + Add the label specified by LabelName to the executor pods. + For example, spark.kubernetes.executor.label.something=true. + Note that Spark also adds its own labels to the driver pod + for bookkeeping purposes. +
    spark.kubernetes.executor.annotation.[AnnotationName](none) + Add the annotation specified by AnnotationName to the executor pods. + For example, spark.kubernetes.executor.annotation.something=true. +
    spark.kubernetes.driver.pod.name(none) + Name of the driver pod. If not set, the driver pod name is set to "spark.app.name" suffixed by the current timestamp + to avoid name conflicts. +
    spark.kubernetes.executor.podNamePrefix(none) + Prefix for naming the executor pods. + If not set, the executor pod name is set to driver pod name suffixed by an integer. +
    spark.kubernetes.executor.lostCheck.maxAttempts10 + Number of times that the driver will try to ascertain the loss reason for a specific executor. + The loss reason is used to ascertain whether the executor failure is due to a framework or an application error + which in turn decides whether the executor is removed and replaced, or placed into a failed state for debugging. +
    spark.kubernetes.submission.waitAppCompletiontrue + In cluster mode, whether to wait for the application to finish before exiting the launcher process. When changed to + false, the launcher has a "fire-and-forget" behavior when launching the Spark job. +
    spark.kubernetes.report.interval1s + Interval between reports of the current Spark job status in cluster mode. +
    spark.kubernetes.driver.limit.cores(none) + Specify the hard CPU [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for the driver pod. +
    spark.kubernetes.executor.limit.cores(none) + Specify the hard CPU [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for each executor pod launched for the Spark Application. +
    spark.kubernetes.node.selector.[labelKey](none) + Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the + configuration's value. For example, setting spark.kubernetes.node.selector.identifier to myIdentifier + will result in the driver pod and executors having a node selector with key identifier and value + myIdentifier. Multiple node selector keys can be added by setting multiple configurations with this prefix. +
    spark.kubernetes.driverEnv.[EnvironmentVariableName](none) + Add the environment variable specified by EnvironmentVariableName to + the Driver process. The user can specify multiple of these to set multiple environment variables. +
    spark.kubernetes.mountDependencies.jarsDownloadDir/var/spark-data/spark-jars + Location to download jars to in the driver and executors. + This directory must be empty and will be mounted as an empty directory volume on the driver and executor pods. +
    spark.kubernetes.mountDependencies.filesDownloadDir/var/spark-data/spark-files + Location to download jars to in the driver and executors. + This directory must be empty and will be mounted as an empty directory volume on the driver and executor pods. +
    \ No newline at end of file diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 7e2386f33b583..e7edec5990363 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -18,7 +18,9 @@ Spark application's configuration (driver, executors, and the AM when running in There are two deploy modes that can be used to launch Spark applications on YARN. In `cluster` mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In `client` mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. -Unlike [Spark standalone](spark-standalone.html) and [Mesos](running-on-mesos.html) modes, in which the master's address is specified in the `--master` parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the `--master` parameter is `yarn`. +Unlike other cluster managers supported by Spark in which the master's address is specified in the `--master` +parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. +Thus, the `--master` parameter is `yarn`. To launch a Spark application in `cluster` mode: diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 866d6e527549c..0473ab73a5e6c 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -127,6 +127,16 @@ export HADOOP_CONF_DIR=XXX http://path/to/examples.jar \ 1000 +# Run on a Kubernetes cluster in cluster deploy mode +./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://xx.yy.zz.ww:443 \ + --deploy-mode cluster \ + --executor-memory 20G \ + --num-executors 50 \ + http://path/to/examples.jar \ + 1000 + {% endhighlight %} # Master URLs @@ -155,6 +165,12 @@ The master URL passed to Spark can be in one of the following formats: client or cluster mode depending on the value of --deploy-mode. The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. + k8s://HOST:PORT Connect to a Kubernetes cluster in + cluster mode. Client mode is currently unsupported and will be supported in future releases. + The HOST and PORT refer to the [Kubernetes API Server](https://kubernetes.io/docs/reference/generated/kube-apiserver/). + It connects using TLS by default. In order to force it to use an unsecured connection, you can use + k8s://http://HOST:PORT. + diff --git a/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/feature/VectorSizeHint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala new file mode 100644 index 0000000000000..1fe3cfc74c76d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSizeHint.scala @@ -0,0 +1,195 @@ +/* + * 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.feature + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} +import org.apache.spark.sql.{Column, DataFrame, Dataset} +import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.StructType + +/** + * :: Experimental :: + * A feature transformer that adds size information to the metadata of a vector column. + * VectorAssembler needs size information for its input columns and cannot be used on streaming + * dataframes without this metadata. + * + */ +@Experimental +@Since("2.3.0") +class VectorSizeHint @Since("2.3.0") (@Since("2.3.0") override val uid: String) + extends Transformer with HasInputCol with HasHandleInvalid with DefaultParamsWritable { + + @Since("2.3.0") + def this() = this(Identifiable.randomUID("vectSizeHint")) + + /** + * The size of Vectors in `inputCol`. + * @group param + */ + @Since("2.3.0") + val size: IntParam = new IntParam( + this, + "size", + "Size of vectors in column.", + {s: Int => s >= 0}) + + /** group getParam */ + @Since("2.3.0") + def getSize: Int = getOrDefault(size) + + /** @group setParam */ + @Since("2.3.0") + def setSize(value: Int): this.type = set(size, value) + + /** @group setParam */ + @Since("2.3.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** + * Param for how to handle invalid entries. Invalid vectors include nulls and vectors with the + * wrong size. The options are `skip` (filter out rows with invalid vectors), `error` (throw an + * error) and `optimistic` (do not check the vector size, and keep all rows). `error` by default. + * + * Note: Users should take care when setting this param to `optimistic`. The use of the + * `optimistic` option will prevent the transformer from validating the sizes of vectors in + * `inputCol`. A mismatch between the metadata of a column and its contents could result in + * unexpected behaviour or errors when using that column. + * + * @group param + */ + @Since("2.3.0") + override val handleInvalid: Param[String] = new Param[String]( + this, + "handleInvalid", + "How to handle invalid vectors in inputCol. Invalid vectors include nulls and vectors with " + + "the wrong size. The options are `skip` (filter out rows with invalid vectors), `error` " + + "(throw an error) and `optimistic` (do not check the vector size, and keep all rows). " + + "`error` by default.", + ParamValidators.inArray(VectorSizeHint.supportedHandleInvalids)) + + /** @group setParam */ + @Since("2.3.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + setDefault(handleInvalid, VectorSizeHint.ERROR_INVALID) + + @Since("2.3.0") + override def transform(dataset: Dataset[_]): DataFrame = { + val localInputCol = getInputCol + val localSize = getSize + val localHandleInvalid = getHandleInvalid + + val group = AttributeGroup.fromStructField(dataset.schema(localInputCol)) + val newGroup = validateSchemaAndSize(dataset.schema, group) + if (localHandleInvalid == VectorSizeHint.OPTIMISTIC_INVALID && group.size == localSize) { + dataset.toDF() + } else { + val newCol: Column = localHandleInvalid match { + case VectorSizeHint.OPTIMISTIC_INVALID => col(localInputCol) + case VectorSizeHint.ERROR_INVALID => + val checkVectorSizeUDF = udf { vector: Vector => + if (vector == null) { + throw new SparkException(s"Got null vector in VectorSizeHint, set `handleInvalid` " + + s"to 'skip' to filter invalid rows.") + } + if (vector.size != localSize) { + throw new SparkException(s"VectorSizeHint Expecting a vector of size $localSize but" + + s" got ${vector.size}") + } + vector + }.asNondeterministic() + checkVectorSizeUDF(col(localInputCol)) + case VectorSizeHint.SKIP_INVALID => + val checkVectorSizeUDF = udf { vector: Vector => + if (vector != null && vector.size == localSize) { + vector + } else { + null + } + } + checkVectorSizeUDF(col(localInputCol)) + } + + val res = dataset.withColumn(localInputCol, newCol.as(localInputCol, newGroup.toMetadata())) + if (localHandleInvalid == VectorSizeHint.SKIP_INVALID) { + res.na.drop(Array(localInputCol)) + } else { + res + } + } + } + + /** + * Checks that schema can be updated with new size and returns a new attribute group with + * updated size. + */ + private def validateSchemaAndSize(schema: StructType, group: AttributeGroup): AttributeGroup = { + // This will throw a NoSuchElementException if params are not set. + val localSize = getSize + val localInputCol = getInputCol + + val inputColType = schema(getInputCol).dataType + require( + inputColType.isInstanceOf[VectorUDT], + s"Input column, $getInputCol must be of Vector type, got $inputColType" + ) + group.size match { + case `localSize` => group + case -1 => new AttributeGroup(localInputCol, localSize) + case _ => + val msg = s"Trying to set size of vectors in `$localInputCol` to $localSize but size " + + s"already set to ${group.size}." + throw new IllegalArgumentException(msg) + } + } + + @Since("2.3.0") + override def transformSchema(schema: StructType): StructType = { + val fieldIndex = schema.fieldIndex(getInputCol) + val fields = schema.fields.clone() + val inputField = fields(fieldIndex) + val group = AttributeGroup.fromStructField(inputField) + val newGroup = validateSchemaAndSize(schema, group) + fields(fieldIndex) = inputField.copy(metadata = newGroup.toMetadata()) + StructType(fields) + } + + @Since("2.3.0") + override def copy(extra: ParamMap): this.type = defaultCopy(extra) +} + +/** :: Experimental :: */ +@Experimental +@Since("2.3.0") +object VectorSizeHint extends DefaultParamsReadable[VectorSizeHint] { + + private[feature] val OPTIMISTIC_INVALID = "optimistic" + private[feature] val ERROR_INVALID = "error" + private[feature] val SKIP_INVALID = "skip" + private[feature] val supportedHandleInvalids: Array[String] = + Array(OPTIMISTIC_INVALID, ERROR_INVALID, SKIP_INVALID) + + @Since("2.3.0") + override def load(path: String): VectorSizeHint = super.load(path) +} 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/feature/InstanceSuit.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala similarity index 79% rename from mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuit.scala rename to mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index 88c85a9425e78..cca7399b4b9c5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuit.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -17,31 +17,29 @@ package org.apache.spark.ml.feature -import scala.reflect.ClassTag - import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer -class InstanceSuit extends SparkFunSuite{ +class InstanceSuite extends SparkFunSuite{ test("Kryo class register") { val conf = new SparkConf(false) conf.set("spark.kryo.registrationRequired", "true") - val ser = new KryoSerializer(conf) - val serInstance = new KryoSerializer(conf).newInstance() - - def check[T: ClassTag](t: T) { - assert(serInstance.deserialize[T](serInstance.serialize(t)) === t) - } + val ser = new KryoSerializer(conf).newInstance() val instance1 = Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)) val instance2 = Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse) + Seq(instance1, instance2).foreach { i => + val i2 = ser.deserialize[Instance](ser.serialize(i)) + assert(i === i2) + } + val oInstance1 = OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)) val oInstance2 = OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse) - check(instance1) - check(instance2) - check(oInstance1) - check(oInstance2) + Seq(oInstance1, oInstance2).foreach { o => + val o2 = ser.deserialize[OffsetInstance](ser.serialize(o)) + assert(o === o2) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala new file mode 100644 index 0000000000000..05c7a58ee5ffd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala @@ -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.ml.feature + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.serializer.KryoSerializer + +class LabeledPointSuite extends SparkFunSuite { + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + val ser = new KryoSerializer(conf).newInstance() + + val labeled1 = LabeledPoint(1.0, Vectors.dense(Array(1.0, 2.0))) + val labeled2 = LabeledPoint(1.0, Vectors.sparse(10, Array(5, 7), Array(1.0, 2.0))) + + Seq(labeled1, labeled2).foreach { l => + val l2 = ser.deserialize[LabeledPoint](ser.serialize(l)) + assert(l === l2) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala new file mode 100644 index 0000000000000..f6c9a76599fae --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSizeHintSuite.scala @@ -0,0 +1,189 @@ +/* + * 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.feature + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.StreamTest + +class VectorSizeHintSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("Test Param Validators") { + intercept[IllegalArgumentException] (new VectorSizeHint().setHandleInvalid("invalidValue")) + intercept[IllegalArgumentException] (new VectorSizeHint().setSize(-3)) + } + + test("Required params must be set before transform.") { + val data = Seq((Vectors.dense(1, 2), 0)).toDF("vector", "intValue") + + val noSizeTransformer = new VectorSizeHint().setInputCol("vector") + intercept[NoSuchElementException] (noSizeTransformer.transform(data)) + intercept[NoSuchElementException] (noSizeTransformer.transformSchema(data.schema)) + + val noInputColTransformer = new VectorSizeHint().setSize(2) + intercept[NoSuchElementException] (noInputColTransformer.transform(data)) + intercept[NoSuchElementException] (noInputColTransformer.transformSchema(data.schema)) + } + + test("Adding size to column of vectors.") { + + val size = 3 + val vectorColName = "vector" + val denseVector = Vectors.dense(1, 2, 3) + val sparseVector = Vectors.sparse(size, Array(), Array()) + + val data = Seq(denseVector, denseVector, sparseVector).map(Tuple1.apply) + val dataFrame = data.toDF(vectorColName) + assert( + AttributeGroup.fromStructField(dataFrame.schema(vectorColName)).size == -1, + s"This test requires that column '$vectorColName' not have size metadata.") + + for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) { + val transformer = new VectorSizeHint() + .setInputCol(vectorColName) + .setSize(size) + .setHandleInvalid(handleInvalid) + val withSize = transformer.transform(dataFrame) + assert( + AttributeGroup.fromStructField(withSize.schema(vectorColName)).size == size, + "Transformer did not add expected size data.") + val numRows = withSize.collect().length + assert(numRows === data.length, s"Expecting ${data.length} rows, got $numRows.") + } + } + + test("Size hint preserves attributes.") { + + val size = 3 + val vectorColName = "vector" + val data = Seq((1, 2, 3), (2, 3, 3)) + val dataFrame = data.toDF("x", "y", "z") + + val assembler = new VectorAssembler() + .setInputCols(Array("x", "y", "z")) + .setOutputCol(vectorColName) + val dataFrameWithMetadata = assembler.transform(dataFrame) + val group = AttributeGroup.fromStructField(dataFrameWithMetadata.schema(vectorColName)) + + for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) { + val transformer = new VectorSizeHint() + .setInputCol(vectorColName) + .setSize(size) + .setHandleInvalid(handleInvalid) + val withSize = transformer.transform(dataFrameWithMetadata) + + val newGroup = AttributeGroup.fromStructField(withSize.schema(vectorColName)) + assert(newGroup.size === size, "Column has incorrect size metadata.") + assert( + newGroup.attributes.get === group.attributes.get, + "VectorSizeHint did not preserve attributes.") + withSize.collect + } + } + + test("Size mismatch between current and target size raises an error.") { + val size = 4 + val vectorColName = "vector" + val data = Seq((1, 2, 3), (2, 3, 3)) + val dataFrame = data.toDF("x", "y", "z") + + val assembler = new VectorAssembler() + .setInputCols(Array("x", "y", "z")) + .setOutputCol(vectorColName) + val dataFrameWithMetadata = assembler.transform(dataFrame) + + for (handleInvalid <- VectorSizeHint.supportedHandleInvalids) { + val transformer = new VectorSizeHint() + .setInputCol(vectorColName) + .setSize(size) + .setHandleInvalid(handleInvalid) + intercept[IllegalArgumentException](transformer.transform(dataFrameWithMetadata)) + } + } + + test("Handle invalid does the right thing.") { + + val vector = Vectors.dense(1, 2, 3) + val short = Vectors.dense(2) + val dataWithNull = Seq(vector, null).map(Tuple1.apply).toDF("vector") + val dataWithShort = Seq(vector, short).map(Tuple1.apply).toDF("vector") + + val sizeHint = new VectorSizeHint() + .setInputCol("vector") + .setHandleInvalid("error") + .setSize(3) + + intercept[SparkException](sizeHint.transform(dataWithNull).collect()) + intercept[SparkException](sizeHint.transform(dataWithShort).collect()) + + sizeHint.setHandleInvalid("skip") + assert(sizeHint.transform(dataWithNull).count() === 1) + assert(sizeHint.transform(dataWithShort).count() === 1) + + sizeHint.setHandleInvalid("optimistic") + assert(sizeHint.transform(dataWithNull).count() === 2) + assert(sizeHint.transform(dataWithShort).count() === 2) + } + + test("read/write") { + val sizeHint = new VectorSizeHint() + .setInputCol("myInputCol") + .setSize(11) + .setHandleInvalid("skip") + testDefaultReadWrite(sizeHint) + } +} + +class VectorSizeHintStreamingSuite extends StreamTest { + + import testImplicits._ + + test("Test assemble vectors with size hint in streaming.") { + val a = Vectors.dense(0, 1, 2) + val b = Vectors.sparse(4, Array(0, 3), Array(3, 6)) + + val stream = MemoryStream[(Vector, Vector)] + val streamingDF = stream.toDS.toDF("a", "b") + val sizeHintA = new VectorSizeHint() + .setSize(3) + .setInputCol("a") + val sizeHintB = new VectorSizeHint() + .setSize(4) + .setInputCol("b") + val vectorAssembler = new VectorAssembler() + .setInputCols(Array("a", "b")) + .setOutputCol("assembled") + val pipeline = new Pipeline().setStages(Array(sizeHintA, sizeHintB, vectorAssembler)) + val output = pipeline.fit(streamingDF).transform(streamingDF).select("assembled") + + val expected = Vectors.dense(0, 1, 2, 3, 0, 0, 6) + + testStream (output) ( + AddData(stream, (a, b), (a, b)), + CheckAnswer(Tuple1(expected), Tuple1(expected)) + ) + } +} 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/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala new file mode 100644 index 0000000000000..f41abe48f2c58 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala @@ -0,0 +1,35 @@ +/* + * 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.tree.impl + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.KryoSerializer + +class TreePointSuite extends SparkFunSuite { + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + val ser = new KryoSerializer(conf).newInstance() + + val point = new TreePoint(1.0, Array(1, 2, 3)) + val point2 = ser.deserialize[TreePoint](ser.serialize(point)) + assert(point.label === point2.label) + assert(point.binnedFeatures === point2.binnedFeatures) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 48bd41dc3e3bf..00d7e2f2d3864 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -311,6 +312,21 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { assert(returnModel.clusterCenters(1) === initialModel.clusterCenters(1)) } + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + val ser = new KryoSerializer(conf).newInstance() + + val vec1 = new VectorWithNorm(Vectors.dense(Array(1.0, 2.0))) + val vec2 = new VectorWithNorm(Vectors.sparse(10, Array(5, 8), Array(1.0, 2.0))) + + Seq(vec1, vec2).foreach { v => + val v2 = ser.deserialize[VectorWithNorm](ser.serialize(v)) + assert(v2.norm === v.norm) + assert(v2.vector === v.vector) + } + } } object KMeansSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index 252a068dcd72f..c1449ece740d4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.regression -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.serializer.KryoSerializer class LabeledPointSuite extends SparkFunSuite { @@ -53,4 +54,19 @@ class LabeledPointSuite extends SparkFunSuite { assert(p1 === LabeledPoint.fromML(p2)) } } + + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + + val ser = new KryoSerializer(conf).newInstance() + + val labeled1 = LabeledPoint(1.0, Vectors.dense(Array(1.0, 2.0))) + val labeled2 = LabeledPoint(1.0, Vectors.sparse(10, Array(5, 7), Array(1.0, 2.0))) + + Seq(labeled1, labeled2).foreach { l => + val l2 = ser.deserialize[LabeledPoint](ser.serialize(l)) + assert(l === l2) + } + } } 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..95eca76fa9888 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..6e5eec48e8aca 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -493,12 +493,14 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): data types will be used to coerce the data in Pandas to Arrow conversion. """ from pyspark.serializers import ArrowSerializer, _create_batch - from pyspark.sql.types import from_arrow_schema, to_arrow_type, \ - _old_pandas_exception_message, TimestampType - try: - from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype - except ImportError as e: - raise ImportError(_old_pandas_exception_message(e)) + from pyspark.sql.types import from_arrow_schema, to_arrow_type, TimestampType + from pyspark.sql.utils import require_minimum_pandas_version, \ + require_minimum_pyarrow_version + + require_minimum_pandas_version() + require_minimum_pyarrow_version() + + from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype # Determine arrow types to coerce data when creating batches if isinstance(schema, StructType): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b4d32d8de8a22..b977160af566d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -53,7 +53,8 @@ try: import pandas try: - import pandas.api + from pyspark.sql.utils import require_minimum_pandas_version + require_minimum_pandas_version() _have_pandas = True except: _have_old_pandas = True @@ -2600,7 +2601,7 @@ def test_to_pandas(self): @unittest.skipIf(not _have_old_pandas, "Old Pandas not installed") def test_to_pandas_old(self): with QuietTest(self.sc): - with self.assertRaisesRegexp(ImportError, 'Pandas \(.*\) must be installed'): + with self.assertRaisesRegexp(ImportError, 'Pandas >= .* must be installed'): self._to_pandas() @unittest.skipIf(not _have_pandas, "Pandas not installed") @@ -2643,7 +2644,7 @@ def test_create_dataframe_from_old_pandas(self): pdf = pd.DataFrame({"ts": [datetime(2017, 10, 31, 1, 1, 1)], "d": [pd.Timestamp.now().date()]}) with QuietTest(self.sc): - with self.assertRaisesRegexp(ImportError, 'Pandas \(.*\) must be installed'): + with self.assertRaisesRegexp(ImportError, 'Pandas >= .* must be installed'): self.spark.createDataFrame(pdf) @@ -3339,10 +3340,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 +3358,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 +3674,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 +3977,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..063264a89379c 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)) @@ -1679,13 +1678,6 @@ def from_arrow_schema(arrow_schema): for field in arrow_schema]) -def _old_pandas_exception_message(e): - """ Create an error message for importing old Pandas. - """ - msg = "note: Pandas (>=0.19.2) must be installed and available on calling Python process" - return "%s\n%s" % (_exception_message(e), msg) - - def _check_dataframe_localize_timestamps(pdf, timezone): """ Convert timezone aware timestamps to timezone-naive in the specified timezone or local timezone @@ -1694,10 +1686,10 @@ def _check_dataframe_localize_timestamps(pdf, timezone): :param timezone: the timezone to convert. if None then use local timezone :return pandas.DataFrame where any timezone aware columns have been converted to tz-naive """ - try: - from pandas.api.types import is_datetime64tz_dtype - except ImportError as e: - raise ImportError(_old_pandas_exception_message(e)) + from pyspark.sql.utils import require_minimum_pandas_version + require_minimum_pandas_version() + + from pandas.api.types import is_datetime64tz_dtype tz = timezone or 'tzlocal()' for column, series in pdf.iteritems(): # TODO: handle nested timestamps, such as ArrayType(TimestampType())? @@ -1715,10 +1707,10 @@ def _check_series_convert_timestamps_internal(s, timezone): :param timezone: the timezone to convert. if None then use local timezone :return pandas.Series where if it is a timestamp, has been UTC normalized without a time zone """ - try: - from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype - except ImportError as e: - raise ImportError(_old_pandas_exception_message(e)) + from pyspark.sql.utils import require_minimum_pandas_version + require_minimum_pandas_version() + + from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype # TODO: handle nested timestamps, such as ArrayType(TimestampType())? if is_datetime64_dtype(s.dtype): tz = timezone or 'tzlocal()' @@ -1738,11 +1730,11 @@ def _check_series_convert_timestamps_localize(s, from_timezone, to_timezone): :param to_timezone: the timezone to convert to. if None then use local timezone :return pandas.Series where if it is a timestamp, has been converted to tz-naive """ - try: - import pandas as pd - from pandas.api.types import is_datetime64tz_dtype, is_datetime64_dtype - except ImportError as e: - raise ImportError(_old_pandas_exception_message(e)) + from pyspark.sql.utils import require_minimum_pandas_version + require_minimum_pandas_version() + + import pandas as pd + from pandas.api.types import is_datetime64tz_dtype, is_datetime64_dtype from_tz = from_timezone or 'tzlocal()' to_tz = to_timezone or 'tzlocal()' # TODO: handle nested timestamps, such as ArrayType(TimestampType())? diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index c3301a41ccd5a..123138117fdc3 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..fb7d42a35d8f4 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -110,3 +110,21 @@ def toJArray(gateway, jtype, arr): for i in range(0, len(arr)): jarr[i] = arr[i] return jarr + + +def require_minimum_pandas_version(): + """ Raise ImportError if minimum version of Pandas is not installed + """ + from distutils.version import LooseVersion + import pandas + if LooseVersion(pandas.__version__) < LooseVersion('0.19.2'): + raise ImportError("Pandas >= 0.19.2 must be installed on calling Python process") + + +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..e5d79d9a9d9da 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 @@ -20,7 +20,6 @@ import java.util.concurrent.TimeUnit import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder -import org.apache.spark.network.util.ByteUnit private[spark] object Config extends Logging { @@ -102,10 +101,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") @@ -132,20 +131,72 @@ private[spark] object Config extends Logging { val JARS_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir") - .doc("Location to download jars to in the driver and executors. When using" + - " spark-submit, this directory must be empty and will be mounted as an empty directory" + - " volume on the driver and executor pod.") + .doc("Location to download jars to in the driver and executors. When using " + + "spark-submit, this directory must be empty and will be mounted as an empty directory " + + "volume on the driver and executor pod.") .stringConf .createWithDefault("/var/spark-data/spark-jars") val FILES_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir") - .doc("Location to download files to in the driver and executors. When using" + - " spark-submit, this directory must be empty and will be mounted as an empty directory" + - " volume on the driver and executor pods.") + .doc("Location to download files to in the driver and executors. When using " + + "spark-submit, this directory must be empty and will be mounted as an empty directory " + + "volume on the driver and executor pods.") .stringConf .createWithDefault("/var/spark-data/spark-files") + val INIT_CONTAINER_IMAGE = + ConfigBuilder("spark.kubernetes.initContainer.image") + .doc("Image for the driver and executor's init-container for downloading dependencies.") + .stringConf + .createOptional + + val INIT_CONTAINER_MOUNT_TIMEOUT = + ConfigBuilder("spark.kubernetes.mountDependencies.timeout") + .doc("Timeout before aborting the attempt to download and unpack dependencies from remote " + + "locations into the driver and executor pods.") + .timeConf(TimeUnit.SECONDS) + .createWithDefault(300) + + val INIT_CONTAINER_MAX_THREAD_POOL_SIZE = + ConfigBuilder("spark.kubernetes.mountDependencies.maxSimultaneousDownloads") + .doc("Maximum number of remote dependencies to download simultaneously in a driver or " + + "executor pod.") + .intConf + .createWithDefault(5) + + val INIT_CONTAINER_REMOTE_JARS = + ConfigBuilder("spark.kubernetes.initContainer.remoteJars") + .doc("Comma-separated list of jar URIs to download in the init-container. This is " + + "calculated from spark.jars.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_REMOTE_FILES = + ConfigBuilder("spark.kubernetes.initContainer.remoteFiles") + .doc("Comma-separated list of file URIs to download in the init-container. This is " + + "calculated from spark.files.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_CONFIG_MAP_NAME = + ConfigBuilder("spark.kubernetes.initContainer.configMapName") + .doc("Name of the config map to use in the init-container that retrieves submitted files " + + "for the executor.") + .internal() + .stringConf + .createOptional + + val INIT_CONTAINER_CONFIG_MAP_KEY_CONF = + ConfigBuilder("spark.kubernetes.initContainer.configMapKey") + .doc("Key for the entry in the init container config map for submitted files that " + + "corresponds to the properties for this init-container.") + .internal() + .stringConf + .createOptional + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" @@ -153,9 +204,11 @@ private[spark] object Config extends Logging { val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." + val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 0b91145405d3a..111cb2a3b75e5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -69,6 +69,17 @@ private[spark] object Constants { val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS" val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" + val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" + + // Bootstrapping dependencies with the init-container + val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume" + val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files-volume" + val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" + val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" + val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" + val INIT_CONTAINER_PROPERTIES_FILE_PATH = + s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" + val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala new file mode 100644 index 0000000000000..dfeccf9e2bd1c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerBootstrap.scala @@ -0,0 +1,119 @@ +/* + * 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.deploy.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Bootstraps an init-container for downloading remote dependencies. This is separated out from + * the init-container steps API because this component can be used to bootstrap init-containers + * for both the driver and executors. + */ +private[spark] class InitContainerBootstrap( + initContainerImage: String, + imagePullPolicy: String, + jarsDownloadPath: String, + filesDownloadPath: String, + configMapName: String, + configMapKey: String, + sparkRole: String, + sparkConf: SparkConf) { + + /** + * Bootstraps an init-container that downloads dependencies to be used by a main container. + */ + def bootstrapInitContainer( + original: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { + val sharedVolumeMounts = Seq[VolumeMount]( + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withMountPath(jarsDownloadPath) + .build(), + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withMountPath(filesDownloadPath) + .build()) + + val customEnvVarKeyPrefix = sparkRole match { + case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY + case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv." + case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role") + } + val customEnvVars = sparkConf.getAllWithPrefix(customEnvVarKeyPrefix).toSeq.map { + case (key, value) => + new EnvVarBuilder() + .withName(key) + .withValue(value) + .build() + } + + val initContainer = new ContainerBuilder(original.initContainer) + .withName("spark-init") + .withImage(initContainerImage) + .withImagePullPolicy(imagePullPolicy) + .addAllToEnv(customEnvVars.asJava) + .addNewVolumeMount() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR) + .endVolumeMount() + .addToVolumeMounts(sharedVolumeMounts: _*) + .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) + .build() + + val podWithBasicVolumes = new PodBuilder(original.pod) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withNewConfigMap() + .withName(configMapName) + .addNewItem() + .withKey(configMapKey) + .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .endSpec() + .build() + + val mainContainer = new ContainerBuilder(original.mainContainer) + .addToVolumeMounts(sharedVolumeMounts: _*) + .addNewEnv() + .withName(ENV_MOUNTED_FILES_DIR) + .withValue(filesDownloadPath) + .endEnv() + .build() + + PodWithDetachedInitContainer( + podWithBasicVolumes, + initContainer, + mainContainer) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala similarity index 57% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index a38cf55fc3d58..37331d8bbf9b7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -14,13 +14,49 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.submit +package org.apache.spark.deploy.k8s import java.io.File +import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} + +import org.apache.spark.SparkConf import org.apache.spark.util.Utils -private[spark] object KubernetesFileUtils { +private[spark] object KubernetesUtils { + + /** + * Extract and parse Spark configuration properties with a given name prefix and + * return the result as a Map. Keys must not have more than one value. + * + * @param sparkConf Spark configuration + * @param prefix the given property name prefix + * @return a Map storing the configuration property keys and values + */ + def parsePrefixedKeyValuePairs( + sparkConf: SparkConf, + prefix: String): Map[String, String] = { + sparkConf.getAllWithPrefix(prefix).toMap + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } + + /** + * Append the given init-container to a pod's list of init-containers. + * + * @param originalPodSpec original specification of the pod + * @param initContainer the init-container to add to the pod + * @return the pod with the init-container added to the list of InitContainers + */ + def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { + new PodBuilder(originalPodSpec) + .editOrNewSpec() + .addToInitContainers(initContainer) + .endSpec() + .build() + } /** * For the given collection of file URIs, resolves them as follows: @@ -47,6 +83,16 @@ private[spark] object KubernetesFileUtils { } } + /** + * Get from a given collection of file URIs the ones that represent remote files. + */ + def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { + uris.filter { uri => + val scheme = Utils.resolveURI(uri).getScheme + scheme != "file" && scheme != "local" + } + } + private def resolveFileUri( uri: String, fileDownloadPath: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala new file mode 100644 index 0000000000000..8286546ce0641 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/MountSecretsBootstrap.scala @@ -0,0 +1,62 @@ +/* + * 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.deploy.k8s + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} + +/** + * Bootstraps a driver or executor container or an init-container with needed secrets mounted. + */ +private[spark] class MountSecretsBootstrap(secretNamesToMountPaths: Map[String, String]) { + + /** + * Mounts Kubernetes secrets as secret volumes into the given container in the given pod. + * + * @param pod the pod into which the secret volumes are being added. + * @param container the container into which the secret volumes are being mounted. + * @return the updated pod and container with the secrets mounted. + */ + def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { + var podBuilder = new PodBuilder(pod) + secretNamesToMountPaths.keys.foreach { name => + podBuilder = podBuilder + .editOrNewSpec() + .addNewVolume() + .withName(secretVolumeName(name)) + .withNewSecret() + .withSecretName(name) + .endSecret() + .endVolume() + .endSpec() + } + + var containerBuilder = new ContainerBuilder(container) + secretNamesToMountPaths.foreach { case (name, path) => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(name)) + .withMountPath(path) + .endVolumeMount() + } + + (podBuilder.build(), containerBuilder.build()) + } + + private def secretVolumeName(secretName: String): String = { + secretName + "-volume" + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala new file mode 100644 index 0000000000000..0b79f8b12e806 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala @@ -0,0 +1,31 @@ +/* + * 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.deploy.k8s + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +/** + * Represents a pod with a detached init-container (not yet added to the pod). + * + * @param pod the pod + * @param initContainer the init-container in the pod + * @param mainContainer the main container in the pod + */ +private[spark] case class PodWithDetachedInitContainer( + pod: Pod, + initContainer: Container, + mainContainer: Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 1e3f055e05766..c47e78cbf19e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -48,7 +48,7 @@ private[spark] object SparkKubernetesClientFactory { .map(new File(_)) .orElse(defaultServiceAccountToken) val oauthTokenValue = sparkConf.getOption(oauthTokenConf) - ConfigurationUtils.requireNandDefined( + KubernetesUtils.requireNandDefined( oauthTokenFile, oauthTokenValue, s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a " + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala similarity index 53% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala index 1411e6f40b468..00c9c4ee49177 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestrator.scala @@ -21,25 +21,31 @@ import java.util.UUID import com.google.common.primitives.Longs import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.{KubernetesUtils, MountSecretsBootstrap} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.steps._ +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.InitContainerConfigOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.SystemClock +import org.apache.spark.util.Utils /** - * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. + * Figures out and returns the complete ordered list of needed DriverConfigurationSteps to + * configure the Spark driver pod. The returned steps will be applied one by one in the given + * order to produce a final KubernetesDriverSpec that is used in KubernetesClientApplication + * to construct and create the driver pod. It uses the InitContainerConfigOrchestrator to + * configure the driver init-container if one is needed, i.e., when there are remote dependencies + * to localize. */ -private[spark] class DriverConfigurationStepsOrchestrator( - namespace: String, +private[spark] class DriverConfigOrchestrator( kubernetesAppId: String, launchTime: Long, mainAppResource: Option[MainAppResource], appName: String, mainClass: String, appArgs: Array[String], - submissionSparkConf: SparkConf) { + sparkConf: SparkConf) { // The resource name prefix is derived from the Spark application name, making it easy to connect // the names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the @@ -49,13 +55,14 @@ private[spark] class DriverConfigurationStepsOrchestrator( s"$appName-$uuid".toLowerCase.replaceAll("\\.", "-") } - private val imagePullPolicy = submissionSparkConf.get(CONTAINER_IMAGE_PULL_POLICY) - private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION) - private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION) + private val imagePullPolicy = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) + private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + private val jarsDownloadPath = sparkConf.get(JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = sparkConf.get(FILES_DOWNLOAD_LOCATION) - def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { - val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, + def getAllConfigurationSteps: Seq[DriverConfigurationStep] = { + val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + s"$SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping " + @@ -64,11 +71,15 @@ private[spark] class DriverConfigurationStepsOrchestrator( s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + "operations.") + val secretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX) + val allDriverLabels = driverCustomLabels ++ Map( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val initialSubmissionStep = new BaseDriverConfigurationStep( + val initialSubmissionStep = new BasicDriverConfigurationStep( kubernetesAppId, kubernetesResourceNamePrefix, allDriverLabels, @@ -76,16 +87,16 @@ private[spark] class DriverConfigurationStepsOrchestrator( appName, mainClass, appArgs, - submissionSparkConf) + sparkConf) - val driverAddressStep = new DriverServiceBootstrapStep( + val serviceBootstrapStep = new DriverServiceBootstrapStep( kubernetesResourceNamePrefix, allDriverLabels, - submissionSparkConf, + sparkConf, new SystemClock) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( - submissionSparkConf, kubernetesResourceNamePrefix) + sparkConf, kubernetesResourceNamePrefix) val additionalMainAppJar = if (mainAppResource.nonEmpty) { val mayBeResource = mainAppResource.get match { @@ -98,28 +109,62 @@ private[spark] class DriverConfigurationStepsOrchestrator( None } - val sparkJars = submissionSparkConf.getOption("spark.jars") + val sparkJars = sparkConf.getOption("spark.jars") .map(_.split(",")) .getOrElse(Array.empty[String]) ++ additionalMainAppJar.toSeq - val sparkFiles = submissionSparkConf.getOption("spark.files") + val sparkFiles = sparkConf.getOption("spark.files") .map(_.split(",")) .getOrElse(Array.empty[String]) - val maybeDependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) { - Some(new DependencyResolutionStep( + val dependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) { + Seq(new DependencyResolutionStep( sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath)) } else { - None + Nil + } + + val initContainerBootstrapStep = if (existNonContainerLocalFiles(sparkJars ++ sparkFiles)) { + val orchestrator = new InitContainerConfigOrchestrator( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + imagePullPolicy, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME, + sparkConf) + val bootstrapStep = new DriverInitContainerBootstrapStep( + orchestrator.getAllConfigurationSteps, + initContainerConfigMapName, + INIT_CONTAINER_PROPERTIES_FILE_NAME) + + Seq(bootstrapStep) + } else { + Nil + } + + val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + Seq(new DriverMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) + } else { + Nil } Seq( initialSubmissionStep, - driverAddressStep, + serviceBootstrapStep, kubernetesCredentialsStep) ++ - maybeDependencyResolutionStep.toSeq + dependencyResolutionStep ++ + initContainerBootstrapStep ++ + mountSecretsStep + } + + private def existNonContainerLocalFiles(files: Seq[String]): Boolean = { + files.exists { uri => + Utils.resolveURI(uri).getScheme != "local" + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 240a1144577b0..5884348cb3e41 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -80,22 +80,22 @@ private[spark] object ClientArguments { * spark.kubernetes.submission.waitAppCompletion is true. * * @param submissionSteps steps that collectively configure the driver - * @param submissionSparkConf the submission client Spark configuration + * @param sparkConf the submission client Spark configuration * @param kubernetesClient the client to talk to the Kubernetes API server * @param waitForAppCompletion a flag indicating whether the client should wait for the application * to complete * @param appName the application name - * @param loggingPodStatusWatcher a watcher that monitors and logs the application status + * @param watcher a watcher that monitors and logs the application status */ private[spark] class Client( submissionSteps: Seq[DriverConfigurationStep], - submissionSparkConf: SparkConf, + sparkConf: SparkConf, kubernetesClient: KubernetesClient, waitForAppCompletion: Boolean, appName: String, - loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { + watcher: LoggingPodStatusWatcher) extends Logging { - private val driverJavaOptions = submissionSparkConf.get( + private val driverJavaOptions = sparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) /** @@ -104,7 +104,7 @@ private[spark] class Client( * will be used to build the Driver Container, Driver Pod, and Kubernetes Resources */ def run(): Unit = { - var currentDriverSpec = KubernetesDriverSpec.initialSpec(submissionSparkConf) + var currentDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf) // submissionSteps contain steps necessary to take, to resolve varying // client arguments that are passed in, created by orchestrator for (nextStep <- submissionSteps) { @@ -141,7 +141,7 @@ private[spark] class Client( kubernetesClient .pods() .withName(resolvedDriverPod.getMetadata.getName) - .watch(loggingPodStatusWatcher)) { _ => + .watch(watcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { if (currentDriverSpec.otherKubernetesResources.nonEmpty) { @@ -157,7 +157,7 @@ private[spark] class Client( if (waitForAppCompletion) { logInfo(s"Waiting for application $appName to finish...") - loggingPodStatusWatcher.awaitCompletion() + watcher.awaitCompletion() logInfo(s"Application $appName finished.") } else { logInfo(s"Deployed Spark application $appName into Kubernetes.") @@ -207,11 +207,9 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val master = sparkConf.get("spark.master").substring("k8s://".length) val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None - val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( - kubernetesAppId, loggingInterval) + val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) - val configurationStepsOrchestrator = new DriverConfigurationStepsOrchestrator( - namespace, + val orchestrator = new DriverConfigOrchestrator( kubernetesAppId, launchTime, clientArguments.mainAppResource, @@ -228,12 +226,12 @@ private[spark] class KubernetesClientApplication extends SparkApplication { None, None)) { kubernetesClient => val client = new Client( - configurationStepsOrchestrator.getAllConfigurationSteps(), + orchestrator.getAllConfigurationSteps, sparkConf, kubernetesClient, waitForAppCompletion, appName, - loggingPodStatusWatcher) + watcher) client.run() } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala similarity index 70% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala index c335fcce4036e..b7a69a7dfd472 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala @@ -22,49 +22,46 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, EnvVarS import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec import org.apache.spark.internal.config.{DRIVER_CLASS_PATH, DRIVER_MEMORY, DRIVER_MEMORY_OVERHEAD} /** - * Represents the initial setup required for the driver. + * Performs basic configuration for the driver pod. */ -private[spark] class BaseDriverConfigurationStep( +private[spark] class BasicDriverConfigurationStep( kubernetesAppId: String, - kubernetesResourceNamePrefix: String, + resourceNamePrefix: String, driverLabels: Map[String, String], imagePullPolicy: String, appName: String, mainClass: String, appArgs: Array[String], - submissionSparkConf: SparkConf) extends DriverConfigurationStep { + sparkConf: SparkConf) extends DriverConfigurationStep { - private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"$kubernetesResourceNamePrefix-driver") + private val driverPodName = sparkConf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$resourceNamePrefix-driver") - private val driverExtraClasspath = submissionSparkConf.get( - DRIVER_CLASS_PATH) + private val driverExtraClasspath = sparkConf.get(DRIVER_CLASS_PATH) - private val driverContainerImage = submissionSparkConf + private val driverContainerImage = sparkConf .get(DRIVER_CONTAINER_IMAGE) .getOrElse(throw new SparkException("Must specify the driver container image")) // CPU settings - private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") - private val driverLimitCores = submissionSparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) + private val driverCpuCores = sparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = sparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) // Memory settings - private val driverMemoryMiB = submissionSparkConf.get( - DRIVER_MEMORY) - private val driverMemoryString = submissionSparkConf.get( - DRIVER_MEMORY.key, - DRIVER_MEMORY.defaultValueString) - private val memoryOverheadMiB = submissionSparkConf + private val driverMemoryMiB = sparkConf.get(DRIVER_MEMORY) + private val driverMemoryString = sparkConf.get( + DRIVER_MEMORY.key, DRIVER_MEMORY.defaultValueString) + private val memoryOverheadMiB = sparkConf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) - private val driverContainerMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) + private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => @@ -74,15 +71,13 @@ private[spark] class BaseDriverConfigurationStep( .build() } - val driverCustomAnnotations = ConfigurationUtils - .parsePrefixedKeyValuePairs( - submissionSparkConf, - KUBERNETES_DRIVER_ANNOTATION_PREFIX) + val driverCustomAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + " Spark bookkeeping operations.") - val driverCustomEnvs = submissionSparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq + val driverCustomEnvs = sparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq .map { env => new EnvVarBuilder() .withName(env._1) @@ -90,10 +85,10 @@ private[spark] class BaseDriverConfigurationStep( .build() } - val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + val driverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) - val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( - submissionSparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + val nodeSelector = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) val driverCpuQuantity = new QuantityBuilder(false) .withAmount(driverCpuCores) @@ -102,7 +97,7 @@ private[spark] class BaseDriverConfigurationStep( .withAmount(s"${driverMemoryMiB}Mi") .build() val driverMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${driverContainerMemoryWithOverheadMiB}Mi") + .withAmount(s"${driverMemoryWithOverheadMiB}Mi") .build() val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) @@ -142,9 +137,9 @@ private[spark] class BaseDriverConfigurationStep( val baseDriverPod = new PodBuilder(driverSpec.driverPod) .editOrNewMetadata() - .withName(kubernetesDriverPodName) + .withName(driverPodName) .addToLabels(driverLabels.asJava) - .addToAnnotations(allDriverAnnotations.asJava) + .addToAnnotations(driverAnnotations.asJava) .endMetadata() .withNewSpec() .withRestartPolicy("Never") @@ -153,9 +148,9 @@ private[spark] class BaseDriverConfigurationStep( .build() val resolvedSparkConf = driverSpec.driverSparkConf.clone() - .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + .setIfMissing(KUBERNETES_DRIVER_POD_NAME, driverPodName) .set("spark.app.id", kubernetesAppId) - .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, resourceNamePrefix) driverSpec.copy( driverPod = baseDriverPod, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index 44e0ecffc0e93..d4b83235b4e3b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -21,7 +21,8 @@ import java.io.File import io.fabric8.kubernetes.api.model.ContainerBuilder import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, KubernetesFileUtils} +import org.apache.spark.deploy.k8s.KubernetesUtils +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec /** * Step that configures the classpath, spark.jars, and spark.files for the driver given that the @@ -31,21 +32,22 @@ private[spark] class DependencyResolutionStep( sparkJars: Seq[String], sparkFiles: Seq[String], jarsDownloadPath: String, - localFilesDownloadPath: String) extends DriverConfigurationStep { + filesDownloadPath: String) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val resolvedSparkJars = KubernetesFileUtils.resolveFileUris(sparkJars, jarsDownloadPath) - val resolvedSparkFiles = KubernetesFileUtils.resolveFileUris( - sparkFiles, localFilesDownloadPath) - val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() + val resolvedSparkJars = KubernetesUtils.resolveFileUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesUtils.resolveFileUris(sparkFiles, filesDownloadPath) + + val sparkConf = driverSpec.driverSparkConf.clone() if (resolvedSparkJars.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) + sparkConf.set("spark.jars", resolvedSparkJars.mkString(",")) } if (resolvedSparkFiles.nonEmpty) { - sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) + sparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) } - val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) - val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { + + val resolvedClasspath = KubernetesUtils.resolveFilePaths(sparkJars, jarsDownloadPath) + val resolvedDriverContainer = if (resolvedClasspath.nonEmpty) { new ContainerBuilder(driverSpec.driverContainer) .addNewEnv() .withName(ENV_MOUNTED_CLASSPATH) @@ -55,8 +57,9 @@ private[spark] class DependencyResolutionStep( } else { driverSpec.driverContainer } + driverSpec.copy( - driverContainer = driverContainerWithResolvedClasspath, - driverSparkConf = sparkConfResolvedSparkDependencies) + driverContainer = resolvedDriverContainer, + driverSparkConf = sparkConf) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala index c99c0436cf25f..17614e040e587 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit.steps import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec /** - * Represents a step in preparing the Kubernetes driver. + * Represents a step in configuring the Spark driver pod. */ private[spark] trait DriverConfigurationStep { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala new file mode 100644 index 0000000000000..9fb3dafdda540 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStep.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import java.io.StringWriter +import java.util.Properties + +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, ContainerBuilder, HasMetadata} + +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.KubernetesUtils +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} + +/** + * Configures the driver init-container that localizes remote dependencies into the driver pod. + * It applies the given InitContainerConfigurationSteps in the given order to produce a final + * InitContainerSpec that is then used to configure the driver pod with the init-container attached. + * It also builds a ConfigMap that will be mounted into the init-container. The ConfigMap carries + * configuration properties for the init-container. + */ +private[spark] class DriverInitContainerBootstrapStep( + steps: Seq[InitContainerConfigurationStep], + configMapName: String, + configMapKey: String) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var initContainerSpec = InitContainerSpec( + properties = Map.empty[String, String], + driverSparkConf = Map.empty[String, String], + initContainer = new ContainerBuilder().build(), + driverContainer = driverSpec.driverContainer, + driverPod = driverSpec.driverPod, + dependentResources = Seq.empty[HasMetadata]) + for (nextStep <- steps) { + initContainerSpec = nextStep.configureInitContainer(initContainerSpec) + } + + val configMap = buildConfigMap( + configMapName, + configMapKey, + initContainerSpec.properties) + val resolvedDriverSparkConf = driverSpec.driverSparkConf + .clone() + .set(INIT_CONTAINER_CONFIG_MAP_NAME, configMapName) + .set(INIT_CONTAINER_CONFIG_MAP_KEY_CONF, configMapKey) + .setAll(initContainerSpec.driverSparkConf) + val resolvedDriverPod = KubernetesUtils.appendInitContainer( + initContainerSpec.driverPod, initContainerSpec.initContainer) + + driverSpec.copy( + driverPod = resolvedDriverPod, + driverContainer = initContainerSpec.driverContainer, + driverSparkConf = resolvedDriverSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + initContainerSpec.dependentResources ++ + Seq(configMap)) + } + + private def buildConfigMap( + configMapName: String, + configMapKey: String, + config: Map[String, String]): ConfigMap = { + val properties = new Properties() + config.foreach { entry => + properties.setProperty(entry._1, entry._2) + } + val propertiesWriter = new StringWriter() + properties.store(propertiesWriter, + s"Java properties built from Kubernetes config map with name: $configMapName " + + s"and config map key: $configMapKey") + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData(configMapKey, propertiesWriter.toString) + .build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.scala new file mode 100644 index 0000000000000..f872e0f4b65d1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStep.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.deploy.k8s.submit.steps + +import org.apache.spark.deploy.k8s.MountSecretsBootstrap +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +/** + * A driver configuration step for mounting user-specified secrets onto user-specified paths. + * + * @param bootstrap a utility actually handling mounting of the secrets. + */ +private[spark] class DriverMountSecretsStep( + bootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val (pod, container) = bootstrap.mountSecrets( + driverSpec.driverPod, driverSpec.driverContainer) + driverSpec.copy( + driverPod = pod, + driverContainer = container + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala index 696d11f15ed95..eb594e4f16ec0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala @@ -32,21 +32,22 @@ import org.apache.spark.util.Clock * ports should correspond to the ports that the executor will reach the pod at for RPC. */ private[spark] class DriverServiceBootstrapStep( - kubernetesResourceNamePrefix: String, + resourceNamePrefix: String, driverLabels: Map[String, String], - submissionSparkConf: SparkConf, + sparkConf: SparkConf, clock: Clock) extends DriverConfigurationStep with Logging { + import DriverServiceBootstrapStep._ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, + require(sparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's bind " + "address is managed and set to the driver pod's IP address.") - require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, + require(sparkConf.getOption(DRIVER_HOST_KEY).isEmpty, s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + "managed via a Kubernetes service.") - val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" + val preferredServiceName = s"$resourceNamePrefix$DRIVER_SVC_POSTFIX" val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { preferredServiceName } else { @@ -58,8 +59,8 @@ private[spark] class DriverServiceBootstrapStep( shorterServiceName } - val driverPort = submissionSparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) - val driverBlockManagerPort = submissionSparkConf.getInt( + val driverPort = sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverBlockManagerPort = sparkConf.getInt( org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) val driverService = new ServiceBuilder() .withNewMetadata() @@ -81,7 +82,7 @@ private[spark] class DriverServiceBootstrapStep( .endSpec() .build() - val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" val resolvedSparkConf = driverSpec.driverSparkConf.clone() .set(DRIVER_HOST_KEY, driverHostname) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..01469853dacc2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStep.scala @@ -0,0 +1,67 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.KubernetesUtils + +/** + * Performs basic configuration for the driver init-container with most of the work delegated to + * the given InitContainerBootstrap. + */ +private[spark] class BasicInitContainerConfigurationStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + bootstrap: InitContainerBootstrap) + extends InitContainerConfigurationStep { + + override def configureInitContainer(spec: InitContainerSpec): InitContainerSpec = { + val remoteJarsToDownload = KubernetesUtils.getOnlyRemoteFiles(sparkJars) + val remoteFilesToDownload = KubernetesUtils.getOnlyRemoteFiles(sparkFiles) + val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(",")) + } else { + Map() + } + val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(",")) + } else { + Map() + } + + val baseInitContainerConfig = Map( + JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath, + FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ + remoteJarsConf ++ + remoteFilesConf + + val bootstrapped = bootstrap.bootstrapInitContainer( + PodWithDetachedInitContainer( + spec.driverPod, + spec.initContainer, + spec.driverContainer)) + + spec.copy( + initContainer = bootstrapped.initContainer, + driverContainer = bootstrapped.mainContainer, + driverPod = bootstrapped.pod, + properties = spec.properties ++ baseInitContainerConfig) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala new file mode 100644 index 0000000000000..f2c29c7ce1076 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestrator.scala @@ -0,0 +1,79 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Figures out and returns the complete ordered list of InitContainerConfigurationSteps required to + * configure the driver init-container. The returned steps will be applied in the given order to + * produce a final InitContainerSpec that is used to construct the driver init-container in + * DriverInitContainerBootstrapStep. This class is only used when an init-container is needed, i.e., + * when there are remote application dependencies to localize. + */ +private[spark] class InitContainerConfigOrchestrator( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + imagePullPolicy: String, + configMapName: String, + configMapKey: String, + sparkConf: SparkConf) { + + private val initContainerImage = sparkConf + .get(INIT_CONTAINER_IMAGE) + .getOrElse(throw new SparkException( + "Must specify the init-container image when there are remote dependencies")) + + def getAllConfigurationSteps: Seq[InitContainerConfigurationStep] = { + val initContainerBootstrap = new InitContainerBootstrap( + initContainerImage, + imagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + configMapName, + configMapKey, + SPARK_POD_DRIVER_ROLE, + sparkConf) + val baseStep = new BasicInitContainerConfigurationStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + initContainerBootstrap) + + val secretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX) + // Mount user-specified driver secrets also into the driver's init-container. The + // init-container may need credentials in the secrets to be able to download remote + // dependencies. The driver's main container and its init-container share the secrets + // because the init-container is sort of an implementation details and this sharing + // avoids introducing a dedicated configuration property just for the init-container. + val mountSecretsStep = if (secretNamesToMountPaths.nonEmpty) { + Seq(new InitContainerMountSecretsStep(new MountSecretsBootstrap(secretNamesToMountPaths))) + } else { + Nil + } + + Seq(baseStep) ++ mountSecretsStep + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..0372ad5270951 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigurationStep.scala @@ -0,0 +1,25 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +/** + * Represents a step in configuring the driver init-container. + */ +private[spark] trait InitContainerConfigurationStep { + + def configureInitContainer(spec: InitContainerSpec): InitContainerSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala new file mode 100644 index 0000000000000..c0e7bb20cce8c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStep.scala @@ -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.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.deploy.k8s.MountSecretsBootstrap + +/** + * An init-container configuration step for mounting user-specified secrets onto user-specified + * paths. + * + * @param bootstrap a utility actually handling mounting of the secrets + */ +private[spark] class InitContainerMountSecretsStep( + bootstrap: MountSecretsBootstrap) extends InitContainerConfigurationStep { + + override def configureInitContainer(spec: InitContainerSpec) : InitContainerSpec = { + val (driverPod, initContainer) = bootstrap.mountSecrets( + spec.driverPod, + spec.initContainer) + spec.copy( + driverPod = driverPod, + initContainer = initContainer + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala new file mode 100644 index 0000000000000..b52c343f0c0ed --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerSpec.scala @@ -0,0 +1,37 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} + +/** + * Represents a specification of the init-container for the driver pod. + * + * @param properties properties that should be set on the init-container + * @param driverSparkConf Spark configuration properties that will be carried back to the driver + * @param initContainer the init-container object + * @param driverContainer the driver container object + * @param driverPod the driver pod object + * @param dependentResources resources the init-container depends on to work + */ +private[spark] case class InitContainerSpec( + properties: Map[String, String], + driverSparkConf: Map[String, String], + initContainer: Container, + driverContainer: Container, + driverPod: Pod, + dependentResources: Seq[HasMetadata]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala new file mode 100644 index 0000000000000..4a4b628aedbbf --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainer.scala @@ -0,0 +1,116 @@ +/* + * 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.deploy.rest.k8s + +import java.io.File +import java.util.concurrent.TimeUnit + +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Process that fetches files from a resource staging server and/or arbitrary remote locations. + * + * The init-container can handle fetching files from any of those sources, but not all of the + * sources need to be specified. This allows for composing multiple instances of this container + * with different configurations for different download sources, or using the same container to + * download everything at once. + */ +private[spark] class SparkPodInitContainer( + sparkConf: SparkConf, + fileFetcher: FileFetcher) extends Logging { + + private val maxThreadPoolSize = sparkConf.get(INIT_CONTAINER_MAX_THREAD_POOL_SIZE) + private implicit val downloadExecutor = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("download-executor", maxThreadPoolSize)) + + private val jarsDownloadDir = new File(sparkConf.get(JARS_DOWNLOAD_LOCATION)) + private val filesDownloadDir = new File(sparkConf.get(FILES_DOWNLOAD_LOCATION)) + + private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS) + private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES) + + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + + def run(): Unit = { + logInfo(s"Downloading remote jars: $remoteJars") + downloadFiles( + remoteJars, + jarsDownloadDir, + s"Remote jars download directory specified at $jarsDownloadDir does not exist " + + "or is not a directory.") + + logInfo(s"Downloading remote files: $remoteFiles") + downloadFiles( + remoteFiles, + filesDownloadDir, + s"Remote files download directory specified at $filesDownloadDir does not exist " + + "or is not a directory.") + + downloadExecutor.shutdown() + downloadExecutor.awaitTermination(downloadTimeoutMinutes, TimeUnit.MINUTES) + } + + private def downloadFiles( + filesCommaSeparated: Option[String], + downloadDir: File, + errMessage: String): Unit = { + filesCommaSeparated.foreach { files => + require(downloadDir.isDirectory, errMessage) + Utils.stringToSeq(files).foreach { file => + Future[Unit] { + fileFetcher.fetchFile(file, downloadDir) + } + } + } + } +} + +private class FileFetcher(sparkConf: SparkConf, securityManager: SparkSecurityManager) { + + def fetchFile(uri: String, targetDir: File): Unit = { + Utils.fetchFile( + url = uri, + targetDir = targetDir, + conf = sparkConf, + securityMgr = securityManager, + hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf), + timestamp = System.currentTimeMillis(), + useCache = false) + } +} + +object SparkPodInitContainer extends Logging { + + def main(args: Array[String]): Unit = { + logInfo("Starting init-container to download Spark application dependencies.") + val sparkConf = new SparkConf(true) + if (args.nonEmpty) { + Utils.loadDefaultSparkProperties(sparkConf, args(0)) + } + + val securityManager = new SparkSecurityManager(sparkConf) + val fileFetcher = new FileFetcher(sparkConf, securityManager) + new SparkPodInitContainer(sparkConf, fileFetcher).run() + logInfo("Finished downloading application dependencies.") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 70226157dd68b..ba5d891f4c77e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -21,35 +21,35 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.util.Utils /** - * A factory class for configuring and creating executor pods. + * A factory class for bootstrapping and creating executor pods with the given bootstrapping + * components. + * + * @param sparkConf Spark configuration + * @param mountSecretsBootstrap an optional component for mounting user-specified secrets onto + * user-specified paths into the executor container + * @param initContainerBootstrap an optional component for bootstrapping the executor init-container + * if one is needed, i.e., when there are remote dependencies to + * localize + * @param initContainerMountSecretsBootstrap an optional component for mounting user-specified + * secrets onto user-specified paths into the executor + * init-container */ -private[spark] trait ExecutorPodFactory { - - /** - * Configure and construct an executor pod with the given parameters. - */ - def createExecutorPod( - executorId: String, - applicationId: String, - driverUrl: String, - executorEnvs: Seq[(String, String)], - driverPod: Pod, - nodeToLocalTaskCount: Map[String, Int]): Pod -} - -private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) - extends ExecutorPodFactory { +private[spark] class ExecutorPodFactory( + sparkConf: SparkConf, + mountSecretsBootstrap: Option[MountSecretsBootstrap], + initContainerBootstrap: Option[InitContainerBootstrap], + initContainerMountSecretsBootstrap: Option[MountSecretsBootstrap]) { private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) - private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( + private val executorLabels = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) require( @@ -64,11 +64,11 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") private val executorAnnotations = - ConfigurationUtils.parsePrefixedKeyValuePairs( + KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) private val nodeSelector = - ConfigurationUtils.parsePrefixedKeyValuePairs( + KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) @@ -94,7 +94,10 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorCores = sparkConf.getDouble("spark.executor.cores", 1) private val executorLimitCores = sparkConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) - override def createExecutorPod( + /** + * Configure and construct an executor pod with the given parameters. + */ + def createExecutorPod( executorId: String, applicationId: String, driverUrl: String, @@ -198,7 +201,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .endSpec() .build() - val containerWithExecutorLimitCores = executorLimitCores.map { limitCores => + val containerWithLimitCores = executorLimitCores.map { limitCores => val executorCpuLimitQuantity = new QuantityBuilder(false) .withAmount(limitCores) .build() @@ -209,9 +212,33 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .build() }.getOrElse(executorContainer) - new PodBuilder(executorPod) + val (maybeSecretsMountedPod, maybeSecretsMountedContainer) = + mountSecretsBootstrap.map { bootstrap => + bootstrap.mountSecrets(executorPod, containerWithLimitCores) + }.getOrElse((executorPod, containerWithLimitCores)) + + val (bootstrappedPod, bootstrappedContainer) = + initContainerBootstrap.map { bootstrap => + val podWithInitContainer = bootstrap.bootstrapInitContainer( + PodWithDetachedInitContainer( + maybeSecretsMountedPod, + new ContainerBuilder().build(), + maybeSecretsMountedContainer)) + + val (pod, mayBeSecretsMountedInitContainer) = + initContainerMountSecretsBootstrap.map { bootstrap => + bootstrap.mountSecrets(podWithInitContainer.pod, podWithInitContainer.initContainer) + }.getOrElse((podWithInitContainer.pod, podWithInitContainer.initContainer)) + + val bootstrappedPod = KubernetesUtils.appendInitContainer( + pod, mayBeSecretsMountedInitContainer) + + (bootstrappedPod, podWithInitContainer.mainContainer) + }.getOrElse((maybeSecretsMountedPod, maybeSecretsMountedContainer)) + + new PodBuilder(bootstrappedPod) .editSpec() - .addToContainers(containerWithExecutorLimitCores) + .addToContainers(bootstrappedContainer) .endSpec() .build() } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b8bb152d17910..a942db6ae02db 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -21,9 +21,9 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, KubernetesUtils, MountSecretsBootstrap, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.ThreadUtils @@ -45,6 +45,59 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit masterURL: String, scheduler: TaskScheduler): SchedulerBackend = { val sparkConf = sc.getConf + val initContainerConfigMap = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_NAME) + val initContainerConfigMapKey = sparkConf.get(INIT_CONTAINER_CONFIG_MAP_KEY_CONF) + + if (initContainerConfigMap.isEmpty) { + logWarning("The executor's init-container config map is not specified. Executors will " + + "therefore not attempt to fetch remote or submitted dependencies.") + } + + if (initContainerConfigMapKey.isEmpty) { + logWarning("The executor's init-container config map key is not specified. Executors will " + + "therefore not attempt to fetch remote or submitted dependencies.") + } + + // Only set up the bootstrap if they've provided both the config map key and the config map + // name. The config map might not be provided if init-containers aren't being used to + // bootstrap dependencies. + val initContainerBootstrap = for { + configMap <- initContainerConfigMap + configMapKey <- initContainerConfigMapKey + } yield { + val initContainerImage = sparkConf + .get(INIT_CONTAINER_IMAGE) + .getOrElse(throw new SparkException( + "Must specify the init-container image when there are remote dependencies")) + new InitContainerBootstrap( + initContainerImage, + sparkConf.get(CONTAINER_IMAGE_PULL_POLICY), + sparkConf.get(JARS_DOWNLOAD_LOCATION), + sparkConf.get(FILES_DOWNLOAD_LOCATION), + configMap, + configMapKey, + SPARK_POD_EXECUTOR_ROLE, + sparkConf) + } + + val executorSecretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) + val mountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) + } else { + None + } + // Mount user-specified executor secrets also into the executor's init-container. The + // init-container may need credentials in the secrets to be able to download remote + // dependencies. The executor's main container and its init-container share the secrets + // because the init-container is sort of an implementation details and this sharing + // avoids introducing a dedicated configuration property just for the init-container. + val initContainerMountSecretsBootstrap = if (initContainerBootstrap.nonEmpty && + executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrap(executorSecretNamesToMountPaths)) + } else { + None + } val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, @@ -54,7 +107,12 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) - val executorPodFactory = new ExecutorPodFactoryImpl(sparkConf) + val executorPodFactory = new ExecutorPodFactory( + sparkConf, + mountSecretBootstrap, + initContainerBootstrap, + initContainerMountSecretsBootstrap) + val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( 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/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala similarity index 51% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala index 98f9f27da5cde..f193b1f4d3664 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigOrchestratorSuite.scala @@ -17,25 +17,27 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.Config.DRIVER_CONTAINER_IMAGE +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.submit.steps._ -class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { +class DriverConfigOrchestratorSuite extends SparkFunSuite { - private val NAMESPACE = "default" private val DRIVER_IMAGE = "driver-image" + private val IC_IMAGE = "init-container-image" private val APP_ID = "spark-app-id" private val LAUNCH_TIME = 975256L private val APP_NAME = "spark" private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" private val APP_ARGS = Array("arg1", "arg2") + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" test("Base submission steps with a main app resource.") { val sparkConf = new SparkConf(false) .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") - val orchestrator = new DriverConfigurationStepsOrchestrator( - NAMESPACE, + val orchestrator = new DriverConfigOrchestrator( APP_ID, LAUNCH_TIME, Some(mainAppResource), @@ -45,7 +47,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep] @@ -55,8 +57,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { test("Base submission steps without a main app resource.") { val sparkConf = new SparkConf(false) .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) - val orchestrator = new DriverConfigurationStepsOrchestrator( - NAMESPACE, + val orchestrator = new DriverConfigOrchestrator( APP_ID, LAUNCH_TIME, Option.empty, @@ -66,16 +67,62 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { sparkConf) validateStepTypes( orchestrator, - classOf[BaseDriverConfigurationStep], + classOf[BasicDriverConfigurationStep], classOf[DriverServiceBootstrapStep], classOf[DriverKubernetesCredentialsStep] ) } + test("Submission steps with an init-container.") { + val sparkConf = new SparkConf(false) + .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) + .set(INIT_CONTAINER_IMAGE, IC_IMAGE) + .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigOrchestrator( + APP_ID, + LAUNCH_TIME, + Some(mainAppResource), + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BasicDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[DriverInitContainerBootstrapStep]) + } + + test("Submission steps with driver secrets to mount") { + val sparkConf = new SparkConf(false) + .set(DRIVER_CONTAINER_IMAGE, DRIVER_IMAGE) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigOrchestrator( + APP_ID, + LAUNCH_TIME, + Some(mainAppResource), + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BasicDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[DriverMountSecretsStep]) + } + private def validateStepTypes( - orchestrator: DriverConfigurationStepsOrchestrator, + orchestrator: DriverConfigOrchestrator, types: Class[_ <: DriverConfigurationStep]*): Unit = { - val steps = orchestrator.getAllConfigurationSteps() + val steps = orchestrator.getAllConfigurationSteps assert(steps.size === types.size) assert(steps.map(_.getClass) === types) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala new file mode 100644 index 0000000000000..8388c16ded268 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.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.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] object SecretVolumeUtils { + + def podHasVolume(driverPod: Pod, volumeName: String): Boolean = { + driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName) + } + + def containerHasVolume( + driverContainer: Container, + volumeName: String, + mountPath: String): Boolean = { + driverContainer.getVolumeMounts.asScala.exists(volumeMount => + volumeMount.getName == volumeName && volumeMount.getMountPath == mountPath) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala index f7c1b3142cf71..e864c6a16eeb1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec -class BaseDriverConfigurationStepSuite extends SparkFunSuite { +class BasicDriverConfigurationStepSuite extends SparkFunSuite { private val APP_ID = "spark-app-id" private val RESOURCE_NAME_PREFIX = "spark" @@ -52,7 +52,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite { .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2") - val submissionStep = new BaseDriverConfigurationStep( + val submissionStep = new BasicDriverConfigurationStep( APP_ID, RESOURCE_NAME_PREFIX, DRIVER_LABELS, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala new file mode 100644 index 0000000000000..758871e2ba356 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverInitContainerBootstrapStepSuite.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import java.io.StringReader +import java.util.Properties + +import scala.collection.JavaConverters._ + +import com.google.common.collect.Maps +import io.fabric8.kubernetes.api.model.{ConfigMap, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.deploy.k8s.submit.steps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.util.Utils + +class DriverInitContainerBootstrapStepSuite extends SparkFunSuite { + + private val CONFIG_MAP_NAME = "spark-init-config-map" + private val CONFIG_MAP_KEY = "spark-init-config-map-key" + + test("The init container bootstrap step should use all of the init container steps") { + val baseDriverSpec = KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val initContainerSteps = Seq( + FirstTestInitContainerConfigurationStep, + SecondTestInitContainerConfigurationStep) + val bootstrapStep = new DriverInitContainerBootstrapStep( + initContainerSteps, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + + val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec) + + assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala === + FirstTestInitContainerConfigurationStep.additionalLabels) + val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(additionalDriverEnv.size === 1) + assert(additionalDriverEnv.head.getName === + FirstTestInitContainerConfigurationStep.additionalMainContainerEnvKey) + assert(additionalDriverEnv.head.getValue === + FirstTestInitContainerConfigurationStep.additionalMainContainerEnvValue) + + assert(preparedDriverSpec.otherKubernetesResources.size === 2) + assert(preparedDriverSpec.otherKubernetesResources.contains( + FirstTestInitContainerConfigurationStep.additionalKubernetesResource)) + assert(preparedDriverSpec.otherKubernetesResources.exists { + case configMap: ConfigMap => + val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME + val configMapData = configMap.getData.asScala + val hasCorrectNumberOfEntries = configMapData.size == 1 + val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY) + val initContainerProperties = new Properties() + Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) { + initContainerProperties.load(_) + } + val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala + val expectedInitContainerProperties = Map( + SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyKey -> + SecondTestInitContainerConfigurationStep.additionalInitContainerPropertyValue) + val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties + hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties + + case _ => false + }) + + val initContainers = preparedDriverSpec.driverPod.getSpec.getInitContainers + assert(initContainers.size() === 1) + val initContainerEnv = initContainers.get(0).getEnv.asScala + assert(initContainerEnv.size === 1) + assert(initContainerEnv.head.getName === + SecondTestInitContainerConfigurationStep.additionalInitContainerEnvKey) + assert(initContainerEnv.head.getValue === + SecondTestInitContainerConfigurationStep.additionalInitContainerEnvValue) + + val expectedSparkConf = Map( + INIT_CONTAINER_CONFIG_MAP_NAME.key -> CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY_CONF.key -> CONFIG_MAP_KEY, + SecondTestInitContainerConfigurationStep.additionalDriverSparkConfKey -> + SecondTestInitContainerConfigurationStep.additionalDriverSparkConfValue) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + } +} + +private object FirstTestInitContainerConfigurationStep extends InitContainerConfigurationStep { + + val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue") + val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY" + val additionalMainContainerEnvValue = "TEST_ENV_MAIN_VALUE" + val additionalKubernetesResource = new SecretBuilder() + .withNewMetadata() + .withName("test-secret") + .endMetadata() + .addToData("secret-key", "secret-value") + .build() + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val driverPod = new PodBuilder(initContainerSpec.driverPod) + .editOrNewMetadata() + .addToLabels(additionalLabels.asJava) + .endMetadata() + .build() + val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer) + .addNewEnv() + .withName(additionalMainContainerEnvKey) + .withValue(additionalMainContainerEnvValue) + .endEnv() + .build() + initContainerSpec.copy( + driverPod = driverPod, + driverContainer = mainContainer, + dependentResources = initContainerSpec.dependentResources ++ + Seq(additionalKubernetesResource)) + } +} + +private object SecondTestInitContainerConfigurationStep extends InitContainerConfigurationStep { + val additionalInitContainerEnvKey = "TEST_ENV_INIT_KEY" + val additionalInitContainerEnvValue = "TEST_ENV_INIT_VALUE" + val additionalInitContainerPropertyKey = "spark.initcontainer.testkey" + val additionalInitContainerPropertyValue = "testvalue" + val additionalDriverSparkConfKey = "spark.driver.testkey" + val additionalDriverSparkConfValue = "spark.driver.testvalue" + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val initContainer = new ContainerBuilder(initContainerSpec.initContainer) + .addNewEnv() + .withName(additionalInitContainerEnvKey) + .withValue(additionalInitContainerEnvValue) + .endEnv() + .build() + val initContainerProperties = initContainerSpec.properties ++ + Map(additionalInitContainerPropertyKey -> additionalInitContainerPropertyValue) + val driverSparkConf = initContainerSpec.driverSparkConf ++ + Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue) + initContainerSpec.copy( + initContainer = initContainer, + properties = initContainerProperties, + driverSparkConf = driverSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.scala new file mode 100644 index 0000000000000..9ec0cb55de5aa --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverMountSecretsStepSuite.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.deploy.k8s.submit.steps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.MountSecretsBootstrap +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, SecretVolumeUtils} + +class DriverMountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("mounts all given secrets") { + val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false)) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths) + val mountSecretsStep = new DriverMountSecretsStep(mountSecretsBootstrap) + val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec) + val driverPodWithSecretsMounted = configuredDriverSpec.driverPod + val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach { volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName)) + } + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach { volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH)) + } + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala new file mode 100644 index 0000000000000..4553f9f6b1d45 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/BasicInitContainerConfigurationStepSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps.initcontainer + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, PodWithDetachedInitContainer} +import org.apache.spark.deploy.k8s.Config._ + +class BasicInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter { + + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val POD_LABEL = Map("bootstrap" -> "true") + private val INIT_CONTAINER_NAME = "init-container" + private val DRIVER_CONTAINER_NAME = "driver-container" + + @Mock + private var podAndInitContainerBootstrap : InitContainerBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(podAndInitContainerBootstrap.bootstrapInitContainer( + any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] { + override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer]) + pod.copy( + pod = new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + initContainer = new ContainerBuilder() + .withName(INIT_CONTAINER_NAME) + .build(), + mainContainer = new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME) + .build() + )}}) + } + + test("additionalDriverSparkConf with mix of remote files and jars") { + val baseInitStep = new BasicInitContainerConfigurationStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + podAndInitContainerBootstrap) + val expectedDriverSparkConf = Map( + JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH, + INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar", + INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt") + val initContainerSpec = InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod, + Seq.empty[HasMetadata]) + val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec) + assert(expectedDriverSparkConf === returnContainerSpec.properties) + assert(returnContainerSpec.initContainer.getName === INIT_CONTAINER_NAME) + assert(returnContainerSpec.driverContainer.getName === DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala new file mode 100644 index 0000000000000..20f2e5bc15df3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerConfigOrchestratorSuite.scala @@ -0,0 +1,80 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +class InitContainerConfigOrchestratorSuite extends SparkFunSuite { + + private val DOCKER_IMAGE = "init-container" + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent" + private val CUSTOM_LABEL_KEY = "customLabel" + private val CUSTOM_LABEL_VALUE = "customLabelValue" + private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" + private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" + + test("including basic configuration step") { + val sparkConf = new SparkConf(true) + .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE) + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + val orchestrator = new InitContainerConfigOrchestrator( + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps = orchestrator.getAllConfigurationSteps + assert(initSteps.lengthCompare(1) == 0) + assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) + } + + test("including step to mount user-specified secrets") { + val sparkConf = new SparkConf(false) + .set(INIT_CONTAINER_IMAGE, DOCKER_IMAGE) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + + val orchestrator = new InitContainerConfigOrchestrator( + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps = orchestrator.getAllConfigurationSteps + assert(initSteps.length === 2) + assert(initSteps.head.isInstanceOf[BasicInitContainerConfigurationStep]) + assert(initSteps(1).isInstanceOf[InitContainerMountSecretsStep]) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala new file mode 100644 index 0000000000000..eab4e17659456 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/initcontainer/InitContainerMountSecretsStepSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.deploy.k8s.submit.steps.initcontainer + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.MountSecretsBootstrap +import org.apache.spark.deploy.k8s.submit.SecretVolumeUtils + +class InitContainerMountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/init-container" + + test("mounts all given secrets") { + val baseInitContainerSpec = InitContainerSpec( + Map.empty, + Map.empty, + new ContainerBuilder().build(), + new ContainerBuilder().build(), + new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(), + Seq.empty) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrap(secretNamesToMountPaths) + val initContainerMountSecretsStep = new InitContainerMountSecretsStep(mountSecretsBootstrap) + val configuredInitContainerSpec = initContainerMountSecretsStep.configureInitContainer( + baseInitContainerSpec) + + val podWithSecretsMounted = configuredInitContainerSpec.driverPod + val initContainerWithSecretsMounted = configuredInitContainerSpec.initContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(podWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + initContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala new file mode 100644 index 0000000000000..6c557ec4a7c9a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/SparkPodInitContainerSuite.scala @@ -0,0 +1,86 @@ +/* + * 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.deploy.rest.k8s + +import java.io.File +import java.util.UUID + +import com.google.common.base.Charsets +import com.google.common.io.Files +import org.mockito.Mockito +import org.scalatest.BeforeAndAfter +import org.scalatest.mockito.MockitoSugar._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.util.Utils + +class SparkPodInitContainerSuite extends SparkFunSuite with BeforeAndAfter { + + private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") + private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") + + private var downloadJarsDir: File = _ + private var downloadFilesDir: File = _ + private var downloadJarsSecretValue: String = _ + private var downloadFilesSecretValue: String = _ + private var fileFetcher: FileFetcher = _ + + override def beforeAll(): Unit = { + downloadJarsSecretValue = Files.toString( + new File(DOWNLOAD_JARS_SECRET_LOCATION), Charsets.UTF_8) + downloadFilesSecretValue = Files.toString( + new File(DOWNLOAD_FILES_SECRET_LOCATION), Charsets.UTF_8) + } + + before { + downloadJarsDir = Utils.createTempDir() + downloadFilesDir = Utils.createTempDir() + fileFetcher = mock[FileFetcher] + } + + after { + downloadJarsDir.delete() + downloadFilesDir.delete() + } + + test("Downloads from remote server should invoke the file fetcher") { + val sparkConf = getSparkConfForRemoteFileDownloads + val initContainerUnderTest = new SparkPodInitContainer(sparkConf, fileFetcher) + initContainerUnderTest.run() + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/file.txt", downloadFilesDir) + } + + private def getSparkConfForRemoteFileDownloads: SparkConf = { + new SparkConf(true) + .set(INIT_CONTAINER_REMOTE_JARS, + "http://localhost:9000/jar1.jar,hdfs://localhost:9000/jar2.jar") + .set(INIT_CONTAINER_REMOTE_FILES, + "http://localhost:9000/file.txt") + .set(JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) + .set(FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath) + } + + private def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 3a55d7cb37b1f..7121a802c69c1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -18,15 +18,19 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{Pod, _} -import org.mockito.MockitoAnnotations +import io.fabric8.kubernetes.api.model._ +import org.mockito.{AdditionalAnswers, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{InitContainerBootstrap, MountSecretsBootstrap, PodWithDetachedInitContainer} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { + private val driverPodName: String = "driver-pod" private val driverPodUid: String = "driver-uid" private val executorPrefix: String = "base" @@ -54,7 +58,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef } test("basic executor pod has reasonable defaults") { - val factory = new ExecutorPodFactoryImpl(baseConf) + val factory = new ExecutorPodFactory(baseConf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -85,7 +89,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") - val factory = new ExecutorPodFactoryImpl(conf) + val factory = new ExecutorPodFactory(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -97,7 +101,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") - val factory = new ExecutorPodFactoryImpl(conf) + val factory = new ExecutorPodFactory(conf, None, None, None) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) @@ -108,6 +112,74 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef checkOwnerReferences(executor, driverPodUid) } + test("executor secrets get mounted") { + val conf = baseConf.clone() + + val secretsBootstrap = new MountSecretsBootstrap(Map("secret1" -> "/var/secret1")) + val factory = new ExecutorPodFactory( + conf, + Some(secretsBootstrap), + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + // check volume mounted. + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container bootstrap step adds an init container") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[InitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainer( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + + val factory = new ExecutorPodFactory( + conf, + None, + Some(initContainerBootstrap), + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getInitContainers.size() === 1) + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container with secrets mount bootstrap") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[InitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainer( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + val secretsBootstrap = new MountSecretsBootstrap(Map("secret1" -> "/var/secret1")) + + val factory = new ExecutorPodFactory( + conf, + None, + Some(initContainerBootstrap), + Some(secretsBootstrap)) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + assert(executor.getSpec.getInitContainers.size() === 1) + assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getInitContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + checkOwnerReferences(executor, driverPodUid) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) 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/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile index d16349559466d..45fbcd9cd0deb 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -22,7 +22,7 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark # distribution, the docker build command should be invoked from the top level directory # of the Spark distribution. E.g.: -# docker build -t spark-driver:latest -f dockerfiles/spark-base/Dockerfile . +# docker build -t spark-driver:latest -f kubernetes/dockerfiles/driver/Dockerfile . COPY examples /opt/spark/examples @@ -31,4 +31,5 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_DRIVER_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY -Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile index 0e38169b8efdc..0f806cf7e148e 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile @@ -22,7 +22,7 @@ FROM spark-base # If this docker file is being used in the context of building your images from a Spark # distribution, the docker build command should be invoked from the top level directory # of the Spark distribution. E.g.: -# docker build -t spark-executor:latest -f dockerfiles/spark-base/Dockerfile . +# docker build -t spark-executor:latest -f kubernetes/dockerfiles/executor/Dockerfile . COPY examples /opt/spark/examples @@ -31,4 +31,5 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp "$SPARK_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile new file mode 100644 index 0000000000000..055493188fcb7 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/init-container/Dockerfile @@ -0,0 +1,24 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-init:latest -f kubernetes/dockerfiles/init-container/Dockerfile . + +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.SparkPodInitContainer" ] diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile index 20316c9c5098a..222e777db3a82 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile @@ -22,7 +22,7 @@ FROM openjdk:8-alpine # If this docker file is being used in the context of building your images from a Spark # distribution, the docker build command should be invoked from the top level directory # of the Spark distribution. E.g.: -# docker build -t spark-base:latest -f dockerfiles/spark-base/Dockerfile . +# docker build -t spark-base:latest -f kubernetes/dockerfiles/spark-base/Dockerfile . RUN set -ex && \ apk upgrade --no-cache && \ @@ -38,7 +38,7 @@ COPY jars /opt/spark/jars COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin COPY conf /opt/spark/conf -COPY dockerfiles/spark-base/entrypoint.sh /opt/ +COPY kubernetes/dockerfiles/spark-base/entrypoint.sh /opt/ ENV SPARK_HOME /opt/spark diff --git a/sbin/build-push-docker-images.sh b/sbin/build-push-docker-images.sh new file mode 100755 index 0000000000000..4546e98dc2074 --- /dev/null +++ b/sbin/build-push-docker-images.sh @@ -0,0 +1,68 @@ +#!/usr/bin/env bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This script builds and pushes docker images when run from a release of Spark +# with Kubernetes support. + +declare -A path=( [spark-driver]=kubernetes/dockerfiles/driver/Dockerfile \ + [spark-executor]=kubernetes/dockerfiles/executor/Dockerfile ) + +function build { + docker build -t spark-base -f kubernetes/dockerfiles/spark-base/Dockerfile . + for image in "${!path[@]}"; do + docker build -t ${REPO}/$image:${TAG} -f ${path[$image]} . + done +} + + +function push { + for image in "${!path[@]}"; do + docker push ${REPO}/$image:${TAG} + done +} + +function usage { + echo "This script must be run from a runnable distribution of Apache Spark." + echo "Usage: ./sbin/build-push-docker-images.sh -r -t build" + echo " ./sbin/build-push-docker-images.sh -r -t push" + echo "for example: ./sbin/build-push-docker-images.sh -r docker.io/myrepo -t v2.3.0 push" +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage + exit 0 +fi + +while getopts r:t: option +do + case "${option}" + in + r) REPO=${OPTARG};; + t) TAG=${OPTARG};; + esac +done + +if [ -z "$REPO" ] || [ -z "$TAG" ]; then + usage +else + case "${@: -1}" in + build) build;; + push) push;; + *) usage;; + esac +fi diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 784eaf8195194..11fb579dfa88c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -66,7 +66,8 @@ case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterminis override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val countTerm = ctx.addMutableState(ctx.JAVA_LONG, "count") - val partitionMaskTerm = ctx.addMutableState(ctx.JAVA_LONG, "partitionMask") + val partitionMaskTerm = "partitionMask" + ctx.addImmutableStateIfNotExists(ctx.JAVA_LONG, partitionMaskTerm) ctx.addPartitionInitializationStatement(s"$countTerm = 0L;") ctx.addPartitionInitializationStatement(s"$partitionMaskTerm = ((long) partitionIndex) << 33;") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 736ca37c6d54a..a160b9b275290 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -43,7 +43,8 @@ case class SparkPartitionID() extends LeafExpression with Nondeterministic { override protected def evalInternal(input: InternalRow): Int = partitionId override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val idTerm = ctx.addMutableState(ctx.JAVA_INT, "partitionId") + val idTerm = "partitionId" + ctx.addImmutableStateIfNotExists(ctx.JAVA_INT, idTerm) ctx.addPartitionInitializationStatement(s"$idTerm = partitionIndex;") ev.copy(code = s"final ${ctx.javaType(dataType)} ${ev.value} = $idTerm;", isNull = "false") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 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..d6eccadcfb63e 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 @@ -195,6 +207,14 @@ class CodegenContext { } + /** + * A map containing the mutable states which have been defined so far using + * `addImmutableStateIfNotExists`. Each entry contains the name of the mutable state as key and + * its Java type and init code as value. + */ + private val immutableStates: mutable.Map[String, (String, String)] = + mutable.Map.empty[String, (String, String)] + /** * Add a mutable state as a field to the generated class. c.f. the comments above. * @@ -241,6 +261,7 @@ class CodegenContext { val initCode = initFunc(varName) inlinedMutableStates += ((javaType, varName)) mutableStateInitCode += initCode + mutableStateNames += varName varName } else { val arrays = arrayCompactedMutableStates.getOrElseUpdate(javaType, new MutableStateArrays) @@ -252,6 +273,38 @@ class CodegenContext { } } + /** + * Add an immutable state as a field to the generated class only if it does not exist yet a field + * with that name. This helps reducing the number of the generated class' fields, since the same + * variable can be reused by many functions. + * + * Even though the added variables are not declared as final, they should never be reassigned in + * the generated code to prevent errors and unexpected behaviors. + * + * Internally, this method calls `addMutableState`. + * + * @param javaType Java type of the field. + * @param variableName Name of the field. + * @param initFunc Function includes statement(s) to put into the init() method to initialize + * this field. The argument is the name of the mutable state variable. + */ + def addImmutableStateIfNotExists( + javaType: String, + variableName: String, + initFunc: String => String = _ => ""): Unit = { + val existingImmutableState = immutableStates.get(variableName) + if (existingImmutableState.isEmpty) { + addMutableState(javaType, variableName, initFunc, useFreshName = false, forceInline = true) + immutableStates(variableName) = (javaType, initFunc(variableName)) + } else { + val (prevJavaType, prevInitCode) = existingImmutableState.get + assert(prevJavaType == javaType, s"$variableName has already been defined with type " + + s"$prevJavaType and now it is tried to define again with type $javaType.") + assert(prevInitCode == initFunc(variableName), s"$variableName has already been defined " + + s"with different initialization statements.") + } + } + /** * Add buffer variable which stores data coming from an [[InternalRow]]. This methods guarantees * that the variable is safely stored, which is important for (potentially) byte array backed @@ -930,6 +983,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/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 59c3e3d9947a3..7a674ea7f4d76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -443,7 +443,8 @@ case class DayOfWeek(child: Expression) extends UnaryExpression with ImplicitCas nullSafeCodeGen(ctx, ev, time => { val cal = classOf[Calendar].getName val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val c = ctx.addMutableState(cal, "cal", + val c = "calDayOfWeek" + ctx.addImmutableStateIfNotExists(cal, c, v => s"""$v = $cal.getInstance($dtu.getTimeZone("UTC"));""") s""" $c.setTimeInMillis($time * 1000L * 3600L * 24L); @@ -484,8 +485,9 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, time => { val cal = classOf[Calendar].getName + val c = "calWeekOfYear" val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val c = ctx.addMutableState(cal, "cal", v => + ctx.addImmutableStateIfNotExists(cal, c, v => s""" |$v = $cal.getInstance($dtu.getTimeZone("UTC")); |$v.setFirstDayOfWeek($cal.MONDAY); @@ -1017,7 +1019,8 @@ case class FromUTCTimestamp(left: Expression, right: Expression) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val tzTerm = ctx.addMutableState(tzClass, "tz", v => s"""$v = $dtu.getTimeZone("$tz");""") - val utcTerm = ctx.addMutableState(tzClass, "utc", + val utcTerm = "tzUTC" + ctx.addImmutableStateIfNotExists(tzClass, utcTerm, v => s"""$v = $dtu.getTimeZone("UTC");""") val eval = left.genCode(ctx) ev.copy(code = s""" @@ -1193,7 +1196,8 @@ case class ToUTCTimestamp(left: Expression, right: Expression) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val tzTerm = ctx.addMutableState(tzClass, "tz", v => s"""$v = $dtu.getTimeZone("$tz");""") - val utcTerm = ctx.addMutableState(tzClass, "utc", + val utcTerm = "tzUTC" + ctx.addImmutableStateIfNotExists(tzClass, utcTerm, v => s"""$v = $dtu.getTimeZone("UTC");""") val eval = left.genCode(ctx) ev.copy(code = s""" 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/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index a59aad5be8715..4af813456b790 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -1148,17 +1148,21 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { // Code to initialize the serializer. - val (serializerClass, serializerInstanceClass) = { + val (serializer, serializerClass, serializerInstanceClass) = { if (kryo) { - (classOf[KryoSerializer].getName, classOf[KryoSerializerInstance].getName) + ("kryoSerializer", + classOf[KryoSerializer].getName, + classOf[KryoSerializerInstance].getName) } else { - (classOf[JavaSerializer].getName, classOf[JavaSerializerInstance].getName) + ("javaSerializer", + classOf[JavaSerializer].getName, + classOf[JavaSerializerInstance].getName) } } // try conf from env, otherwise create a new one val env = s"${classOf[SparkEnv].getName}.get()" val sparkConf = s"new ${classOf[SparkConf].getName}()" - val serializer = ctx.addMutableState(serializerInstanceClass, "serializerForEncode", v => + ctx.addImmutableStateIfNotExists(serializerInstanceClass, serializer, v => s""" |if ($env == null) { | $v = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance(); @@ -1193,17 +1197,21 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { // Code to initialize the serializer. - val (serializerClass, serializerInstanceClass) = { + val (serializer, serializerClass, serializerInstanceClass) = { if (kryo) { - (classOf[KryoSerializer].getName, classOf[KryoSerializerInstance].getName) + ("kryoSerializer", + classOf[KryoSerializer].getName, + classOf[KryoSerializerInstance].getName) } else { - (classOf[JavaSerializer].getName, classOf[JavaSerializerInstance].getName) + ("javaSerializer", + classOf[JavaSerializer].getName, + classOf[JavaSerializerInstance].getName) } } // try conf from env, otherwise create a new one val env = s"${classOf[SparkEnv].getName}.get()" val sparkConf = s"new ${classOf[SparkConf].getName}()" - val serializer = ctx.addMutableState(serializerInstanceClass, "serializerForDecode", v => + ctx.addImmutableStateIfNotExists(serializerInstanceClass, serializer, v => s""" |if ($env == null) { | $v = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance(); 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/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index b1a44528e64d7..676ba3956ddc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -424,4 +424,16 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx2.arrayCompactedMutableStates("InternalRow[]").getCurrentIndex == 10) assert(ctx2.mutableStateInitCode.size == CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) } + + test("SPARK-22750: addImmutableStateIfNotExists") { + val ctx = new CodegenContext + val mutableState1 = "field1" + val mutableState2 = "field2" + ctx.addImmutableStateIfNotExists("int", mutableState1) + ctx.addImmutableStateIfNotExists("int", mutableState1) + ctx.addImmutableStateIfNotExists("String", mutableState2) + ctx.addImmutableStateIfNotExists("int", mutableState1) + ctx.addImmutableStateIfNotExists("String", mutableState2) + assert(ctx.inlinedMutableStates.length == 2) + } } 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/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index ef00562672a7e..209b800fdc6f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -401,6 +401,10 @@ class Dataset[T] private[sql]( * If the schema of the Dataset does not match the desired `U` type, you can use `select` * along with `alias` or `as` to rearrange or rename as required. * + * Note that `as[]` only changes the view of the data that is passed into typed operations, + * such as `map()`, and does not eagerly project away any columns that are not present in + * the specified class. + * * @group basic * @since 1.6.0 */ 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 cf0000c6393a3..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,15 +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 -import org.apache.spark.util.kvstore.KVStore -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). @@ -50,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) @@ -88,7 +87,7 @@ private[sql] class SQLAppStatusListener( } exec.jobs = exec.jobs + (jobId -> JobExecutionStatus.RUNNING) - exec.stages = event.stageIds.toSet + exec.stages ++= event.stageIds.toSet update(exec) } @@ -230,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 @@ -389,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 66% 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 932950687942c..7d84f45d36bee 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) } - sqlStoreTest("basic") { (store, bus) => + 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)) + } + + 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,127 +268,180 @@ 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)) + } + + test("handle one execution with multiple jobs") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + + val executionId = 0 + val df = createTestDataFrame + listener.onOtherEvent(SparkListenerSQLExecutionStart( + executionId, + "test", + "test", + df.queryExecution.toString, + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) + + var stageId = 0 + def twoStageJob(jobId: Int): Unit = { + val stages = Seq(stageId, stageId + 1).map { id => createStageInfo(id, 0)} + stageId += 2 + listener.onJobStart(SparkListenerJobStart( + jobId = jobId, + time = System.currentTimeMillis(), + stageInfos = stages, + createProperties(executionId))) + stages.foreach { s => + listener.onStageSubmitted(SparkListenerStageSubmitted(s)) + listener.onStageCompleted(SparkListenerStageCompleted(s)) + } + listener.onJobEnd(SparkListenerJobEnd( + jobId = jobId, + time = System.currentTimeMillis(), + JobSucceeded + )) + } + // submit two jobs with the same executionId + twoStageJob(0) + twoStageJob(1) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) + + assertJobs(statusStore.execution(0), completed = 0 to 1) + assert(statusStore.execution(0).get.stages === (0 to 3).toSet) } 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 @@ -387,7 +452,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) } @@ -405,30 +471,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) @@ -494,7 +536,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() @@ -522,9 +564,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!")