From 7570eab6bee57172ee3746207261307690a57b72 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Dec 2017 11:31:11 -0600 Subject: [PATCH 01/18] [SPARK-22788][STREAMING] Use correct hadoop config for fs append support. Still look at the old one in case any Spark user is setting it explicitly, though. Author: Marcelo Vanzin Closes #19983 from vanzin/SPARK-22788. --- .../scala/org/apache/spark/streaming/util/HdfsUtils.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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!") From 7798c9e6ef55dbadfc9eb896fa3f366c76dc187b Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 20 Dec 2017 10:43:10 -0800 Subject: [PATCH 02/18] [SPARK-22824] Restore old offset for binary compatibility ## What changes were proposed in this pull request? Some users depend on source compatibility with the org.apache.spark.sql.execution.streaming.Offset class. Although this is not a stable interface, we can keep it in place for now to simplify upgrades to 2.3. Author: Jose Torres Closes #20012 from joseph-torres/binary-compat. --- .../spark/sql/kafka010/KafkaSource.scala | 1 - .../sql/kafka010/KafkaSourceOffset.scala | 3 +- .../spark/sql/kafka010/KafkaSourceSuite.scala | 1 - .../spark/sql/sources/v2/reader/Offset.java | 6 +- .../streaming/FileStreamSource.scala | 1 - .../streaming/FileStreamSourceOffset.scala | 2 - .../sql/execution/streaming/LongOffset.scala | 2 - .../streaming/MicroBatchExecution.scala | 1 - .../spark/sql/execution/streaming/Offset.java | 61 +++++++++++++++++++ .../sql/execution/streaming/OffsetSeq.scala | 1 - .../execution/streaming/OffsetSeqLog.scala | 1 - .../streaming/RateSourceProvider.scala | 3 +- .../streaming/RateStreamOffset.scala | 4 +- .../{Offset.scala => SerializedOffset.scala} | 3 - .../sql/execution/streaming/Source.scala | 1 - .../execution/streaming/StreamExecution.scala | 1 - .../execution/streaming/StreamProgress.scala | 2 - .../ContinuousRateStreamSource.scala | 3 +- .../sql/execution/streaming/memory.scala | 1 - .../sql/execution/streaming/socket.scala | 1 - .../{ => sources}/RateStreamSourceV2.scala | 7 ++- .../streaming/{ => sources}/memoryV2.scala | 3 +- .../streaming/MemorySinkV2Suite.scala | 1 + .../streaming/RateSourceV2Suite.scala | 1 + .../sql/streaming/FileStreamSourceSuite.scala | 1 - .../spark/sql/streaming/OffsetSuite.scala | 3 +- .../spark/sql/streaming/StreamSuite.scala | 1 - .../spark/sql/streaming/StreamTest.scala | 1 - .../streaming/StreamingAggregationSuite.scala | 1 - .../StreamingQueryListenerSuite.scala | 1 - .../sql/streaming/StreamingQuerySuite.scala | 2 - .../test/DataStreamReaderWriterSuite.scala | 1 - .../sql/streaming/util/BlockingSource.scala | 3 +- 33 files changed, 82 insertions(+), 43 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{Offset.scala => SerializedOffset.scala} (95%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{ => sources}/RateStreamSourceV2.scala (96%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{ => sources}/memoryV2.scala (98%) 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/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/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/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/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} From d762d110d410b8d67d74eb4d2950cc556ac74123 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Wed, 20 Dec 2017 12:50:03 -0600 Subject: [PATCH 03/18] [SPARK-22832][ML] BisectingKMeans unpersist unused datasets ## What changes were proposed in this pull request? unpersist unused datasets ## How was this patch tested? existing tests and local check in Spark-Shell Author: Zheng RuiFeng Closes #20017 from zhengruifeng/bkm_unpersist. --- .../spark/mllib/clustering/BisectingKMeans.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) 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) From c89b43118347677f122db190c9033394c15cee30 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 20 Dec 2017 11:19:57 -0800 Subject: [PATCH 04/18] [SPARK-22849] ivy.retrieve pattern should also consider `classifier` ## What changes were proposed in this pull request? In the previous PR https://github.com/apache/spark/pull/5755#discussion_r157848354, we dropped `(-[classifier])` from the retrieval pattern. We should add it back; otherwise, > If this pattern for instance doesn't has the [type] or [classifier] token, Ivy will download the source/javadoc artifacts to the same file as the regular jar. ## How was this patch tested? The existing tests Author: gatorsmile Closes #20037 from gatorsmile/addClassifier. --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 { From 792915c8449b606cfdd50401fb349194a2558c36 Mon Sep 17 00:00:00 2001 From: chetkhatri Date: Wed, 20 Dec 2017 14:47:49 -0600 Subject: [PATCH 05/18] [SPARK-22830] Scala Coding style has been improved in Spark Examples ## What changes were proposed in this pull request? * Under Spark Scala Examples: Some of the syntax were written like Java way, It has been re-written as per scala style guide. * Most of all changes are followed to println() statement. ## How was this patch tested? Since, All changes proposed are re-writing println statements in scala way, manual run used to test println. Author: chetkhatri Closes #20016 from chetkhatri/scala-style-spark-examples. --- .../apache/spark/examples/BroadcastTest.scala | 2 +- .../spark/examples/DFSReadWriteTest.scala | 24 ++++++------ .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/examples/LocalALS.scala | 3 +- .../apache/spark/examples/LocalFileLR.scala | 6 +-- .../apache/spark/examples/LocalKMeans.scala | 4 +- .../org/apache/spark/examples/LocalLR.scala | 6 +-- .../org/apache/spark/examples/LocalPi.scala | 2 +- .../examples/SimpleSkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SparkALS.scala | 4 +- .../apache/spark/examples/SparkHdfsLR.scala | 6 +-- .../apache/spark/examples/SparkKMeans.scala | 2 +- .../org/apache/spark/examples/SparkLR.scala | 6 +-- .../apache/spark/examples/SparkPageRank.scala | 2 +- .../org/apache/spark/examples/SparkPi.scala | 2 +- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/examples/graphx/Analytics.scala | 37 ++++++++++--------- 17 files changed, 54 insertions(+), 58 deletions(-) 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, From b176014857b056e45cbc60ac77f1a95bc31cb0fa Mon Sep 17 00:00:00 2001 From: wuyi Date: Wed, 20 Dec 2017 14:27:56 -0800 Subject: [PATCH 06/18] [SPARK-22847][CORE] Remove redundant code in AppStatusListener while assigning schedulingPool for stage ## What changes were proposed in this pull request? In AppStatusListener's onStageSubmitted(event: SparkListenerStageSubmitted) method, there are duplicate code: ``` // schedulingPool was assigned twice with the same code stage.schedulingPool = Option(event.properties).flatMap { p => Option(p.getProperty("spark.scheduler.pool")) }.getOrElse(SparkUI.DEFAULT_POOL_NAME) ... ... ... stage.schedulingPool = Option(event.properties).flatMap { p => Option(p.getProperty("spark.scheduler.pool")) }.getOrElse(SparkUI.DEFAULT_POOL_NAME) ``` But, it does not make any sense to do this and there are no comment to explain for this. ## How was this patch tested? N/A Author: wuyi Closes #20033 from Ngone51/dev-spark-22847. --- .../scala/org/apache/spark/status/AppStatusListener.scala | 4 ---- 1 file changed, 4 deletions(-) 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..87eb84d94c005 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -329,10 +329,6 @@ private[spark] class AppStatusListener( .toSeq stage.jobIds = stage.jobs.map(_.jobId).toSet - stage.schedulingPool = Option(event.properties).flatMap { p => - Option(p.getProperty("spark.scheduler.pool")) - }.getOrElse(SparkUI.DEFAULT_POOL_NAME) - stage.description = Option(event.properties).flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } From 0114c89d049724b95f7823b957bf33790216316b Mon Sep 17 00:00:00 2001 From: foxish Date: Wed, 20 Dec 2017 16:14:36 -0800 Subject: [PATCH 07/18] [SPARK-22845][SCHEDULER] Modify spark.kubernetes.allocation.batch.delay to take time instead of int ## What changes were proposed in this pull request? Fixing configuration that was taking an int which should take time. Discussion in https://github.com/apache/spark/pull/19946#discussion_r156682354 Made the granularity milliseconds as opposed to seconds since there's a use-case for sub-second reactions to scale-up rapidly especially with dynamic allocation. ## How was this patch tested? TODO: manual run of integration tests against this PR. PTAL cc/ mccheah liyinan926 kimoonkim vanzin mridulm jiangxb1987 ueshin Author: foxish Closes #20032 from foxish/fix-time-conf. --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 8 ++++---- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 2 +- .../k8s/KubernetesClusterSchedulerBackendSuite.scala | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 04aadb4b06af4..45f527959cbe1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -102,10 +102,10 @@ private[spark] object Config extends Logging { val KUBERNETES_ALLOCATION_BATCH_DELAY = ConfigBuilder("spark.kubernetes.allocation.batch.delay") - .doc("Number of seconds to wait between each round of executor allocation.") - .longConf - .checkValue(value => value > 0, "Allocation batch delay should be a positive integer") - .createWithDefault(1) + .doc("Time to wait between each round of executor allocation.") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(value => value > 0, "Allocation batch delay must be a positive time value.") + .createWithDefaultString("1s") val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS = ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index e79c987852db2..9de4b16c30d3c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -217,7 +217,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .watch(new ExecutorPodsWatcher())) allocatorExecutor.scheduleWithFixedDelay( - allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.MILLISECONDS) if (!Utils.isDynamicAllocationEnabled(conf)) { doRequestTotalExecutors(initialExecutors) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 13c09033a50ee..b2f26f205a329 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -46,7 +46,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private val NAMESPACE = "test-namespace" private val SPARK_DRIVER_HOST = "localhost" private val SPARK_DRIVER_PORT = 7077 - private val POD_ALLOCATION_INTERVAL = 60L + private val POD_ALLOCATION_INTERVAL = "1m" private val DRIVER_URL = RpcEndpointAddress( SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val FIRST_EXECUTOR_POD = new PodBuilder() @@ -144,7 +144,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .set(KUBERNETES_NAMESPACE, NAMESPACE) .set("spark.driver.host", SPARK_DRIVER_HOST) .set("spark.driver.port", SPARK_DRIVER_PORT.toString) - .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY.key, POD_ALLOCATION_INTERVAL) executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) @@ -162,8 +162,8 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(allocatorExecutor.scheduleWithFixedDelay( allocatorRunnable.capture(), mockitoEq(0L), - mockitoEq(POD_ALLOCATION_INTERVAL), - mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + mockitoEq(TimeUnit.MINUTES.toMillis(1)), + mockitoEq(TimeUnit.MILLISECONDS))).thenReturn(null) // Creating Futures in Scala backed by a Java executor service resolves to running // ExecutorService#execute (as opposed to submit) doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) From fb0562f34605cd27fd39d09e6664a46e55eac327 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 20 Dec 2017 17:51:42 -0800 Subject: [PATCH 08/18] [SPARK-22810][ML][PYSPARK] Expose Python API for LinearRegression with huber loss. ## What changes were proposed in this pull request? Expose Python API for _LinearRegression_ with _huber_ loss. ## How was this patch tested? Unit test. Author: Yanbo Liang Closes #19994 from yanboliang/spark-22810. --- .../ml/param/_shared_params_code_gen.py | 3 +- python/pyspark/ml/param/shared.py | 23 +++++++ python/pyspark/ml/regression.py | 64 +++++++++++++++---- python/pyspark/ml/tests.py | 21 ++++++ 4 files changed, 96 insertions(+), 15 deletions(-) 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): From 9c289a5cb46e00cd60db4794357f070dfdf80691 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 21 Dec 2017 10:02:30 +0800 Subject: [PATCH 09/18] [SPARK-22387][SQL] Propagate session configs to data source read/write options ## What changes were proposed in this pull request? Introduce a new interface `SessionConfigSupport` for `DataSourceV2`, it can help to propagate session configs with the specified key-prefix to all data source operations in this session. ## How was this patch tested? Add new test suite `DataSourceV2UtilsSuite`. Author: Xingbo Jiang Closes #19861 from jiangxb1987/datasource-configs. --- .../sql/sources/v2/SessionConfigSupport.java | 39 +++++++++++++ .../apache/spark/sql/DataFrameReader.scala | 11 +++- .../apache/spark/sql/DataFrameWriter.scala | 15 +++-- .../datasources/v2/DataSourceV2Utils.scala | 58 +++++++++++++++++++ .../sources/v2/DataSourceV2UtilsSuite.scala | 49 ++++++++++++++++ 5 files changed, 164 insertions(+), 8 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala 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/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 39fec8f983b65..c43ee91294a27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, ReadSupport, ReadSupportWithSchema} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -184,9 +185,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val options = new DataSourceV2Options(extraOptions.asJava) + val ds = cls.newInstance() + val options = new DataSourceV2Options((extraOptions ++ + DataSourceV2Utils.extractSessionConfigs( + ds = ds.asInstanceOf[DataSourceV2], + conf = sparkSession.sessionState.conf)).asJava) - val reader = (cls.newInstance(), userSpecifiedSchema) match { + val reader = (ds, userSpecifiedSchema) match { case (ds: ReadSupportWithSchema, Some(schema)) => ds.createReader(schema, options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 59a01e61124f7..7ccda0ad36d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -30,9 +30,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.sources.v2.{DataSourceV2, DataSourceV2Options, WriteSupport} +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.types.StructType /** @@ -236,14 +237,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { - cls.newInstance() match { - case ds: WriteSupport => - val options = new DataSourceV2Options(extraOptions.asJava) + val ds = cls.newInstance() + ds match { + case ws: WriteSupport => + val options = new DataSourceV2Options((extraOptions ++ + DataSourceV2Utils.extractSessionConfigs( + ds = ds.asInstanceOf[DataSourceV2], + conf = df.sparkSession.sessionState.conf)).asJava) // Using a timestamp and a random UUID to distinguish different writing jobs. This is good // enough as there won't be tons of writing jobs created at the same second. val jobId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) .format(new Date()) + "-" + UUID.randomUUID() - val writer = ds.createWriter(jobId, df.logicalPlan.schema, mode, options) + val writer = ws.createWriter(jobId, df.logicalPlan.schema, mode, options) if (writer.isPresent) { runCommand(df.sparkSession, "save") { WriteToDataSourceV2(writer.get(), df.logicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/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/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" +} From d3ae3e1e894f88a8500752d9633fe9ad00da5f20 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 20 Dec 2017 19:53:35 -0800 Subject: [PATCH 10/18] [SPARK-19634][SQL][ML][FOLLOW-UP] Improve interface of dataframe vectorized summarizer ## What changes were proposed in this pull request? Make several improvements in dataframe vectorized summarizer. 1. Make the summarizer return `Vector` type for all metrics (except "count"). It will return "WrappedArray" type before which won't be very convenient. 2. Make `MetricsAggregate` inherit `ImplicitCastInputTypes` trait. So it can check and implicitly cast input values. 3. Add "weight" parameter for all single metric method. 4. Update doc and improve the example code in doc. 5. Simplified test cases. ## How was this patch tested? Test added and simplified. Author: WeichenXu Closes #19156 from WeichenXu123/improve_vec_summarizer. --- .../org/apache/spark/ml/stat/Summarizer.scala | 128 ++++--- .../spark/ml/stat/JavaSummarizerSuite.java | 64 ++++ .../spark/ml/stat/SummarizerSuite.scala | 362 ++++++++++-------- 3 files changed, 341 insertions(+), 213 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java 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/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java new file mode 100644 index 0000000000000..38ab39aa0f492 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/stat/JavaSummarizerSuite.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Dataset; +import static org.apache.spark.sql.functions.col; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; + +public class JavaSummarizerSuite extends SharedSparkSession { + + private transient Dataset dataset; + + @Override + public void setUp() throws IOException { + super.setUp(); + List points = new ArrayList(); + points.add(new LabeledPoint(0.0, Vectors.dense(1.0, 2.0))); + points.add(new LabeledPoint(0.0, Vectors.dense(3.0, 4.0))); + + dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); + } + + @Test + public void testSummarizer() { + dataset.select(col("features")); + Row result = dataset + .select(Summarizer.metrics("mean", "max", "count").summary(col("features"))) + .first().getStruct(0); + Vector meanVec = result.getAs("mean"); + Vector maxVec = result.getAs("max"); + long count = result.getAs("count"); + + assertEquals(2L, count); + assertArrayEquals(new double[]{2.0, 3.0}, meanVec.toArray(), 0.0); + assertArrayEquals(new double[]{3.0, 4.0}, maxVec.toArray(), 0.0); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala index 1ea851ef2d676..5e4f402989697 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml.stat -import org.scalatest.exceptions.TestFailedException - import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, Statistics} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.Row class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -35,237 +32,262 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { import SummaryBuilderImpl._ private case class ExpectedMetrics( - mean: Seq[Double], - variance: Seq[Double], + mean: Vector, + variance: Vector, count: Long, - numNonZeros: Seq[Long], - max: Seq[Double], - min: Seq[Double], - normL2: Seq[Double], - normL1: Seq[Double]) + numNonZeros: Vector, + max: Vector, + min: Vector, + normL2: Vector, + normL1: Vector) /** - * The input is expected to be either a sparse vector, a dense vector or an array of doubles - * (which will be converted to a dense vector) - * The expected is the list of all the known metrics. + * The input is expected to be either a sparse vector, a dense vector. * - * The tests take an list of input vectors and a list of all the summary values that - * are expected for this input. They currently test against some fixed subset of the - * metrics, but should be made fuzzy in the future. + * The tests take an list of input vectors, and compare results with + * `mllib.stat.MultivariateOnlineSummarizer`. They currently test against some fixed subset + * of the metrics, but should be made fuzzy in the future. */ - private def testExample(name: String, input: Seq[Any], exp: ExpectedMetrics): Unit = { + private def testExample(name: String, inputVec: Seq[(Vector, Double)], + exp: ExpectedMetrics, expWithoutWeight: ExpectedMetrics): Unit = { - def inputVec: Seq[Vector] = input.map { - case x: Array[Double @unchecked] => Vectors.dense(x) - case x: Seq[Double @unchecked] => Vectors.dense(x.toArray) - case x: Vector => x - case x => throw new Exception(x.toString) + val summarizer = { + val _summarizer = new MultivariateOnlineSummarizer + inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v._1), v._2)) + _summarizer } - val summarizer = { + val summarizerWithoutWeight = { val _summarizer = new MultivariateOnlineSummarizer - inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v))) + inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v._1))) _summarizer } // Because the Spark context is reset between tests, we cannot hold a reference onto it. def wrappedInit() = { - val df = inputVec.map(Tuple1.apply).toDF("features") - val col = df.col("features") - (df, col) + val df = inputVec.toDF("features", "weight") + val featuresCol = df.col("features") + val weightCol = df.col("weight") + (df, featuresCol, weightCol) } registerTest(s"$name - mean only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("mean").summary(c), mean(c)), Seq(Row(exp.mean), summarizer.mean)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("mean").summary(c, w), mean(c, w)).first(), + Row(Row(summarizer.mean), exp.mean)) } - registerTest(s"$name - mean only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(mean(c)), Seq(exp.mean)) + registerTest(s"$name - mean only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("mean").summary(c), mean(c)).first(), + Row(Row(summarizerWithoutWeight.mean), expWithoutWeight.mean)) } registerTest(s"$name - variance only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("variance").summary(c), variance(c)), - Seq(Row(exp.variance), summarizer.variance)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("variance").summary(c, w), variance(c, w)).first(), + Row(Row(summarizer.variance), exp.variance)) } - registerTest(s"$name - variance only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(variance(c)), Seq(summarizer.variance)) + registerTest(s"$name - variance only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("variance").summary(c), variance(c)).first(), + Row(Row(summarizerWithoutWeight.variance), expWithoutWeight.variance)) } registerTest(s"$name - count only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("count").summary(c), count(c)), - Seq(Row(exp.count), exp.count)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("count").summary(c, w), count(c, w)).first(), + Row(Row(summarizer.count), exp.count)) } - registerTest(s"$name - count only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(count(c)), - Seq(exp.count)) + registerTest(s"$name - count only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("count").summary(c), count(c)).first(), + Row(Row(summarizerWithoutWeight.count), expWithoutWeight.count)) } registerTest(s"$name - numNonZeros only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("numNonZeros").summary(c), numNonZeros(c)), - Seq(Row(exp.numNonZeros), exp.numNonZeros)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("numNonZeros").summary(c, w), numNonZeros(c, w)).first(), + Row(Row(summarizer.numNonzeros), exp.numNonZeros)) } - registerTest(s"$name - numNonZeros only (direct)") { - val (df, c) = wrappedInit() - compare(df.select(numNonZeros(c)), - Seq(exp.numNonZeros)) + registerTest(s"$name - numNonZeros only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("numNonZeros").summary(c), numNonZeros(c)).first(), + Row(Row(summarizerWithoutWeight.numNonzeros), expWithoutWeight.numNonZeros)) } registerTest(s"$name - min only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("min").summary(c), min(c)), - Seq(Row(exp.min), exp.min)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("min").summary(c, w), min(c, w)).first(), + Row(Row(summarizer.min), exp.min)) + } + + registerTest(s"$name - min only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("min").summary(c), min(c)).first(), + Row(Row(summarizerWithoutWeight.min), expWithoutWeight.min)) } registerTest(s"$name - max only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("max").summary(c), max(c)), - Seq(Row(exp.max), exp.max)) + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("max").summary(c, w), max(c, w)).first(), + Row(Row(summarizer.max), exp.max)) } - registerTest(s"$name - normL1 only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("normL1").summary(c), normL1(c)), - Seq(Row(exp.normL1), exp.normL1)) + registerTest(s"$name - max only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("max").summary(c), max(c)).first(), + Row(Row(summarizerWithoutWeight.max), expWithoutWeight.max)) } - registerTest(s"$name - normL2 only") { - val (df, c) = wrappedInit() - compare(df.select(metrics("normL2").summary(c), normL2(c)), - Seq(Row(exp.normL2), exp.normL2)) + registerTest(s"$name - normL1 only") { + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("normL1").summary(c, w), normL1(c, w)).first(), + Row(Row(summarizer.normL1), exp.normL1)) } - registerTest(s"$name - all metrics at once") { - val (df, c) = wrappedInit() - compare(df.select( - metrics("mean", "variance", "count", "numNonZeros").summary(c), - mean(c), variance(c), count(c), numNonZeros(c)), - Seq(Row(exp.mean, exp.variance, exp.count, exp.numNonZeros), - exp.mean, exp.variance, exp.count, exp.numNonZeros)) + registerTest(s"$name - normL1 only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("normL1").summary(c), normL1(c)).first(), + Row(Row(summarizerWithoutWeight.normL1), expWithoutWeight.normL1)) } - } - private def denseData(input: Seq[Seq[Double]]): DataFrame = { - input.map(_.toArray).map(Vectors.dense).map(Tuple1.apply).toDF("features") - } + registerTest(s"$name - normL2 only") { + val (df, c, w) = wrappedInit() + compareRow(df.select(metrics("normL2").summary(c, w), normL2(c, w)).first(), + Row(Row(summarizer.normL2), exp.normL2)) + } - private def compare(df: DataFrame, exp: Seq[Any]): Unit = { - val coll = df.collect().toSeq - val Seq(row) = coll - val res = row.toSeq - val names = df.schema.fieldNames.zipWithIndex.map { case (n, idx) => s"$n ($idx)" } - assert(res.size === exp.size, (res.size, exp.size)) - for (((x1, x2), name) <- res.zip(exp).zip(names)) { - compareStructures(x1, x2, name) + registerTest(s"$name - normL2 only w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select(metrics("normL2").summary(c), normL2(c)).first(), + Row(Row(summarizerWithoutWeight.normL2), expWithoutWeight.normL2)) } - } - // Compares structured content. - private def compareStructures(x1: Any, x2: Any, name: String): Unit = (x1, x2) match { - case (y1: Seq[Double @unchecked], v1: OldVector) => - compareStructures(y1, v1.toArray.toSeq, name) - case (d1: Double, d2: Double) => - assert2(Vectors.dense(d1) ~== Vectors.dense(d2) absTol 1e-4, name) - case (r1: GenericRowWithSchema, r2: Row) => - assert(r1.size === r2.size, (r1, r2)) - for (((fname, x1), x2) <- r1.schema.fieldNames.zip(r1.toSeq).zip(r2.toSeq)) { - compareStructures(x1, x2, s"$name.$fname") - } - case (r1: Row, r2: Row) => - assert(r1.size === r2.size, (r1, r2)) - for ((x1, x2) <- r1.toSeq.zip(r2.toSeq)) { compareStructures(x1, x2, name) } - case (v1: Vector, v2: Vector) => - assert2(v1 ~== v2 absTol 1e-4, name) - case (l1: Long, l2: Long) => assert(l1 === l2) - case (s1: Seq[_], s2: Seq[_]) => - assert(s1.size === s2.size, s"$name ${(s1, s2)}") - for (((x1, idx), x2) <- s1.zipWithIndex.zip(s2)) { - compareStructures(x1, x2, s"$name.$idx") - } - case (arr1: Array[_], arr2: Array[_]) => - assert(arr1.toSeq === arr2.toSeq) - case _ => throw new Exception(s"$name: ${x1.getClass} ${x2.getClass} $x1 $x2") - } + registerTest(s"$name - multiple metrics at once") { + val (df, c, w) = wrappedInit() + compareRow(df.select( + metrics("mean", "variance", "count", "numNonZeros").summary(c, w)).first(), + Row(Row(exp.mean, exp.variance, exp.count, exp.numNonZeros)) + ) + } - private def assert2(x: => Boolean, hint: String): Unit = { - try { - assert(x, hint) - } catch { - case tfe: TestFailedException => - throw new TestFailedException(Some(s"Failure with hint $hint"), Some(tfe), 1) + registerTest(s"$name - multiple metrics at once w/o weight") { + val (df, c, _) = wrappedInit() + compareRow(df.select( + metrics("mean", "variance", "count", "numNonZeros").summary(c)).first(), + Row(Row(expWithoutWeight.mean, expWithoutWeight.variance, + expWithoutWeight.count, expWithoutWeight.numNonZeros)) + ) } } - test("debugging test") { - val df = denseData(Nil) - val c = df.col("features") - val c1 = metrics("mean").summary(c) - val res = df.select(c1) - intercept[SparkException] { - compare(res, Seq.empty) + private def compareRow(r1: Row, r2: Row): Unit = { + assert(r1.size === r2.size, (r1, r2)) + r1.toSeq.zip(r2.toSeq).foreach { + case (v1: Vector, v2: Vector) => + assert(v1 ~== v2 absTol 1e-4) + case (v1: Vector, v2: OldVector) => + assert(v1 ~== v2.asML absTol 1e-4) + case (l1: Long, l2: Long) => + assert(l1 === l2) + case (r1: Row, r2: Row) => + compareRow(r1, r2) + case (x1: Any, x2: Any) => + throw new Exception(s"type mismatch: ${x1.getClass} ${x2.getClass} $x1 $x2") } } - test("basic error handling") { - val df = denseData(Nil) + test("no element") { + val df = Seq[Tuple1[Vector]]().toDF("features") val c = df.col("features") - val res = df.select(metrics("mean").summary(c), mean(c)) intercept[SparkException] { - compare(res, Seq.empty) + df.select(metrics("mean").summary(c), mean(c)).first() } + compareRow(df.select(metrics("count").summary(c), count(c)).first(), + Row(Row(0L), 0L)) } - test("no element, working metrics") { - val df = denseData(Nil) - val c = df.col("features") - val res = df.select(metrics("count").summary(c), count(c)) - compare(res, Seq(Row(0L), 0L)) - } + val singleElem = Vectors.dense(0.0, 1.0, 2.0) + testExample("single element", Seq((singleElem, 2.0)), + ExpectedMetrics( + mean = singleElem, + variance = Vectors.dense(0.0, 0.0, 0.0), + count = 1L, + numNonZeros = Vectors.dense(0.0, 1.0, 1.0), + max = singleElem, + min = singleElem, + normL1 = Vectors.dense(0.0, 2.0, 4.0), + normL2 = Vectors.dense(0.0, 1.414213, 2.828427) + ), + ExpectedMetrics( + mean = singleElem, + variance = Vectors.dense(0.0, 0.0, 0.0), + count = 1L, + numNonZeros = Vectors.dense(0.0, 1.0, 1.0), + max = singleElem, + min = singleElem, + normL1 = singleElem, + normL2 = singleElem + ) + ) + + testExample("multiple elements (dense)", + Seq( + (Vectors.dense(-1.0, 0.0, 6.0), 0.5), + (Vectors.dense(3.0, -3.0, 0.0), 2.8), + (Vectors.dense(1.0, -3.0, 0.0), 0.0) + ), + ExpectedMetrics( + mean = Vectors.dense(2.393939, -2.545454, 0.909090), + variance = Vectors.dense(8.0, 4.5, 18.0), + count = 2L, + numNonZeros = Vectors.dense(2.0, 1.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(8.9, 8.4, 3.0), + normL2 = Vectors.dense(5.069516, 5.019960, 4.242640) + ), + ExpectedMetrics( + mean = Vectors.dense(1.0, -2.0, 2.0), + variance = Vectors.dense(4.0, 3.0, 12.0), + count = 3L, + numNonZeros = Vectors.dense(3.0, 2.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(5.0, 6.0, 6.0), + normL2 = Vectors.dense(3.316624, 4.242640, 6.0) + ) + ) - val singleElem = Seq(0.0, 1.0, 2.0) - testExample("single element", Seq(singleElem), ExpectedMetrics( - mean = singleElem, - variance = Seq(0.0, 0.0, 0.0), - count = 1, - numNonZeros = Seq(0, 1, 1), - max = singleElem, - min = singleElem, - normL1 = singleElem, - normL2 = singleElem - )) - - testExample("two elements", Seq(Seq(0.0, 1.0, 2.0), Seq(0.0, -1.0, -2.0)), ExpectedMetrics( - mean = Seq(0.0, 0.0, 0.0), - // TODO: I have a doubt about these values, they are not normalized. - variance = Seq(0.0, 2.0, 8.0), - count = 2, - numNonZeros = Seq(0, 2, 2), - max = Seq(0.0, 1.0, 2.0), - min = Seq(0.0, -1.0, -2.0), - normL1 = Seq(0.0, 2.0, 4.0), - normL2 = Seq(0.0, math.sqrt(2.0), math.sqrt(2.0) * 2.0) - )) - - testExample("dense vector input", - Seq(Seq(-1.0, 0.0, 6.0), Seq(3.0, -3.0, 0.0)), + testExample("multiple elements (sparse)", + Seq( + (Vectors.dense(-1.0, 0.0, 6.0).toSparse, 0.5), + (Vectors.dense(3.0, -3.0, 0.0).toSparse, 2.8), + (Vectors.dense(1.0, -3.0, 0.0).toSparse, 0.0) + ), + ExpectedMetrics( + mean = Vectors.dense(2.393939, -2.545454, 0.909090), + variance = Vectors.dense(8.0, 4.5, 18.0), + count = 2L, + numNonZeros = Vectors.dense(2.0, 1.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(8.9, 8.4, 3.0), + normL2 = Vectors.dense(5.069516, 5.019960, 4.242640) + ), ExpectedMetrics( - mean = Seq(1.0, -1.5, 3.0), - variance = Seq(8.0, 4.5, 18.0), - count = 2, - numNonZeros = Seq(2, 1, 1), - max = Seq(3.0, 0.0, 6.0), - min = Seq(-1.0, -3, 0.0), - normL1 = Seq(4.0, 3.0, 6.0), - normL2 = Seq(math.sqrt(10), 3, 6.0) + mean = Vectors.dense(1.0, -2.0, 2.0), + variance = Vectors.dense(4.0, 3.0, 12.0), + count = 3L, + numNonZeros = Vectors.dense(3.0, 2.0, 1.0), + max = Vectors.dense(3.0, 0.0, 6.0), + min = Vectors.dense(-1.0, -3.0, 0.0), + normL1 = Vectors.dense(5.0, 6.0, 6.0), + normL2 = Vectors.dense(3.316624, 4.242640, 6.0) ) ) From cb9fc8d9b6d385c9d193801edaea2d52e29a90fb Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 21 Dec 2017 14:54:38 +0800 Subject: [PATCH 11/18] [SPARK-22848][SQL] Eliminate mutable state from Stack ## What changes were proposed in this pull request? This PR eliminates mutable states from the generated code for `Stack`. ## How was this patch tested? Existing test suites Author: Kazuaki Ishizaki Closes #20035 from kiszk/SPARK-22848. --- .../spark/sql/catalyst/expressions/generators.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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") } } From 59d52631eb86394f1d981419cb744c20bd4e0b87 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 21 Dec 2017 20:43:56 +0900 Subject: [PATCH 12/18] [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0 ## What changes were proposed in this pull request? Upgrade Spark to Arrow 0.8.0 for Java and Python. Also includes an upgrade of Netty to 4.1.17 to resolve dependency requirements. The highlights that pertain to Spark for the update from Arrow versoin 0.4.1 to 0.8.0 include: * Java refactoring for more simple API * Java reduced heap usage and streamlined hot code paths * Type support for DecimalType, ArrayType * Improved type casting support in Python * Simplified type checking in Python ## How was this patch tested? Existing tests Author: Bryan Cutler Author: Shixiong Zhu Closes #19884 from BryanCutler/arrow-upgrade-080-SPARK-22324. --- .../spark/network/crypto/TransportCipher.java | 41 +- .../network/protocol/MessageWithHeader.java | 39 +- .../spark/network/sasl/SaslEncryption.java | 41 +- .../network/util/AbstractFileRegion.java | 53 ++ .../apache/spark/network/ProtocolSuite.java | 4 +- .../protocol/MessageWithHeaderSuite.java | 7 +- .../org/apache/spark/storage/DiskStore.scala | 9 +- dev/deps/spark-deps-hadoop-2.6 | 10 +- dev/deps/spark-deps-hadoop-2.7 | 10 +- pom.xml | 12 +- python/pyspark/serializers.py | 27 +- python/pyspark/sql/dataframe.py | 2 + python/pyspark/sql/functions.py | 13 +- python/pyspark/sql/group.py | 2 +- python/pyspark/sql/session.py | 3 + python/pyspark/sql/tests.py | 12 +- python/pyspark/sql/types.py | 25 +- python/pyspark/sql/udf.py | 16 +- python/pyspark/sql/utils.py | 9 + .../vectorized/ArrowColumnVector.java | 136 ++--- .../sql/execution/arrow/ArrowConverters.scala | 13 +- .../sql/execution/arrow/ArrowWriter.scala | 132 ++-- .../execution/python/ArrowPythonRunner.scala | 27 +- .../arrow/ArrowConvertersSuite.scala | 571 ++---------------- .../vectorized/ArrowColumnVectorSuite.scala | 150 +++-- .../vectorized/ColumnarBatchSuite.scala | 20 +- 26 files changed, 515 insertions(+), 869 deletions(-) create mode 100644 common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java 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/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/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/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/python/pyspark/serializers.py b/python/pyspark/serializers.py index 37e7cf3fa662e..88d6a191babca 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -223,27 +223,14 @@ def _create_batch(series, timezone): series = [series] series = ((s, None) if not isinstance(s, (list, tuple)) else s for s in series) - # If a nullable integer series has been promoted to floating point with NaNs, need to cast - # NOTE: this is not necessary with Arrow >= 0.7 - def cast_series(s, t): - if type(t) == pa.TimestampType: - # NOTE: convert to 'us' with astype here, unit ignored in `from_pandas` see ARROW-1680 - return _check_series_convert_timestamps_internal(s.fillna(0), timezone)\ - .values.astype('datetime64[us]', copy=False) - # NOTE: can not compare None with pyarrow.DataType(), fixed with Arrow >= 0.7.1 - elif t is not None and t == pa.date32(): - # TODO: this converts the series to Python objects, possibly avoid with Arrow >= 0.8 - return s.dt.date - elif t is None or s.dtype == t.to_pandas_dtype(): - return s - else: - return s.fillna(0).astype(t.to_pandas_dtype(), copy=False) - - # Some object types don't support masks in Arrow, see ARROW-1721 def create_array(s, t): - casted = cast_series(s, t) - mask = None if casted.dtype == 'object' else s.isnull() - return pa.Array.from_pandas(casted, mask=mask, type=t) + mask = s.isnull() + # Ensure timestamp series are in expected form for Spark internal representation + if t is not None and pa.types.is_timestamp(t): + s = _check_series_convert_timestamps_internal(s.fillna(0), timezone) + # TODO: need cast after Arrow conversion, ns values cause error with pandas 0.19.2 + return pa.Array.from_pandas(s, mask=mask).cast(t, safe=False) + return pa.Array.from_pandas(s, mask=mask, type=t) arrs = [create_array(s, t) for s, t in series] return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in xrange(len(arrs))]) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 75395a754a831..440684d3edfa6 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1906,7 +1906,9 @@ def toPandas(self): if self.sql_ctx.getConf("spark.sql.execution.arrow.enabled", "false").lower() == "true": try: from pyspark.sql.types import _check_dataframe_localize_timestamps + from pyspark.sql.utils import _require_minimum_pyarrow_version import pyarrow + _require_minimum_pyarrow_version() tables = self._collectAsArrow() if tables: table = pyarrow.concat_tables(tables) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 54530055dfa85..ddd8df3b15bf6 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2159,16 +2159,17 @@ def pandas_udf(f=None, returnType=None, functionType=None): >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> from pyspark.sql.types import IntegerType, StringType - >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) - >>> @pandas_udf(StringType()) + >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) # doctest: +SKIP + >>> @pandas_udf(StringType()) # doctest: +SKIP ... def to_upper(s): ... return s.str.upper() ... - >>> @pandas_udf("integer", PandasUDFType.SCALAR) + >>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP ... def add_one(x): ... return x + 1 ... - >>> df = spark.createDataFrame([(1, "John Doe", 21)], ("id", "name", "age")) + >>> df = spark.createDataFrame([(1, "John Doe", 21)], + ... ("id", "name", "age")) # doctest: +SKIP >>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")) \\ ... .show() # doctest: +SKIP +----------+--------------+------------+ @@ -2189,8 +2190,8 @@ def pandas_udf(f=None, returnType=None, functionType=None): >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) - >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) + ... ("id", "v")) # doctest: +SKIP + >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP ... def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 4d47dd6a3e878..09fae46adf014 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -218,7 +218,7 @@ def apply(self, udf): >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ... ("id", "v")) - >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) + >>> @pandas_udf("id long, v double", PandasUDFType.GROUP_MAP) # doctest: +SKIP ... def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index e2435e09af23d..86db16eca7889 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -495,11 +495,14 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): from pyspark.serializers import ArrowSerializer, _create_batch from pyspark.sql.types import from_arrow_schema, to_arrow_type, \ _old_pandas_exception_message, TimestampType + from pyspark.sql.utils import _require_minimum_pyarrow_version try: from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype except ImportError as e: raise ImportError(_old_pandas_exception_message(e)) + _require_minimum_pyarrow_version() + # Determine arrow types to coerce data when creating batches if isinstance(schema, StructType): arrow_types = [to_arrow_type(f.dataType) for f in schema.fields] diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b4d32d8de8a22..6fdfda1cc831b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -3339,10 +3339,11 @@ def test_createDataFrame_with_single_data_type(self): self.spark.createDataFrame(pd.DataFrame({"a": [1]}), schema="int") def test_createDataFrame_does_not_modify_input(self): + import pandas as pd # Some series get converted for Spark to consume, this makes sure input is unchanged pdf = self.create_pandas_data_frame() # Use a nanosecond value to make sure it is not truncated - pdf.ix[0, '7_timestamp_t'] = 1 + pdf.ix[0, '7_timestamp_t'] = pd.Timestamp(1) # Integers with nulls will get NaNs filled with 0 and will be casted pdf.ix[1, '2_int_t'] = None pdf_copy = pdf.copy(deep=True) @@ -3356,6 +3357,7 @@ def test_schema_conversion_roundtrip(self): self.assertEquals(self.schema, schema_rt) +@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed") class PandasUDFTests(ReusedSQLTestCase): def test_pandas_udf_basic(self): from pyspark.rdd import PythonEvalType @@ -3671,9 +3673,9 @@ def test_vectorized_udf_chained(self): def test_vectorized_udf_wrong_return_type(self): from pyspark.sql.functions import pandas_udf, col df = self.spark.range(10) - f = pandas_udf(lambda x: x * 1.0, StringType()) + f = pandas_udf(lambda x: x * 1.0, ArrayType(LongType())) with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, 'Invalid.*type'): + with self.assertRaisesRegexp(Exception, 'Unsupported.*type.*conversion'): df.select(f(col('id'))).collect() def test_vectorized_udf_return_scalar(self): @@ -3974,12 +3976,12 @@ def test_wrong_return_type(self): foo = pandas_udf( lambda pdf: pdf, - 'id long, v string', + 'id long, v array', PandasUDFType.GROUP_MAP ) with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, 'Invalid.*type'): + with self.assertRaisesRegexp(Exception, 'Unsupported.*type.*conversion'): df.groupby('id').apply(foo).sort('id').toPandas() def test_wrong_args(self): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 78abc32a35a1c..46d9a417414b5 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1642,29 +1642,28 @@ def to_arrow_schema(schema): def from_arrow_type(at): """ Convert pyarrow type to Spark data type. """ - # TODO: newer pyarrow has is_boolean(at) functions that would be better to check type - import pyarrow as pa - if at == pa.bool_(): + import pyarrow.types as types + if types.is_boolean(at): spark_type = BooleanType() - elif at == pa.int8(): + elif types.is_int8(at): spark_type = ByteType() - elif at == pa.int16(): + elif types.is_int16(at): spark_type = ShortType() - elif at == pa.int32(): + elif types.is_int32(at): spark_type = IntegerType() - elif at == pa.int64(): + elif types.is_int64(at): spark_type = LongType() - elif at == pa.float32(): + elif types.is_float32(at): spark_type = FloatType() - elif at == pa.float64(): + elif types.is_float64(at): spark_type = DoubleType() - elif type(at) == pa.DecimalType: + elif types.is_decimal(at): spark_type = DecimalType(precision=at.precision, scale=at.scale) - elif at == pa.string(): + elif types.is_string(at): spark_type = StringType() - elif at == pa.date32(): + elif types.is_date32(at): spark_type = DateType() - elif type(at) == pa.TimestampType: + elif types.is_timestamp(at): spark_type = TimestampType() else: raise TypeError("Unsupported type in conversion from Arrow: " + str(at)) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index c3301a41ccd5a..50c87ba1ac882 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -33,19 +33,23 @@ def _wrap_function(sc, func, returnType): def _create_udf(f, returnType, evalType): - if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF: + + if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF or \ + evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF: import inspect + from pyspark.sql.utils import _require_minimum_pyarrow_version + + _require_minimum_pyarrow_version() argspec = inspect.getargspec(f) - if len(argspec.args) == 0 and argspec.varargs is None: + + if evalType == PythonEvalType.SQL_PANDAS_SCALAR_UDF and len(argspec.args) == 0 and \ + argspec.varargs is None: raise ValueError( "Invalid function: 0-arg pandas_udfs are not supported. " "Instead, create a 1-arg pandas_udf and ignore the arg in your function." ) - elif evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF: - import inspect - argspec = inspect.getargspec(f) - if len(argspec.args) != 1: + if evalType == PythonEvalType.SQL_PANDAS_GROUP_MAP_UDF and len(argspec.args) != 1: raise ValueError( "Invalid function: pandas_udfs with function type GROUP_MAP " "must take a single arg that is a pandas DataFrame." diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 7bc6a59ad3b26..cc7dabb64b3ec 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -110,3 +110,12 @@ def toJArray(gateway, jtype, arr): for i in range(0, len(arr)): jarr[i] = arr[i] return jarr + + +def _require_minimum_pyarrow_version(): + """ Raise ImportError if minimum version of pyarrow is not installed + """ + from distutils.version import LooseVersion + import pyarrow + if LooseVersion(pyarrow.__version__) < LooseVersion('0.8.0'): + raise ImportError("pyarrow >= 0.8.0 must be installed on calling Python process") diff --git a/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/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/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/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/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)) From 0abaf31be7ab9e030ea9433938b9123596954814 Mon Sep 17 00:00:00 2001 From: Erik LaBianca Date: Thu, 21 Dec 2017 09:38:21 -0600 Subject: [PATCH 13/18] [SPARK-22852][BUILD] Exclude -Xlint:unchecked from sbt javadoc flags ## What changes were proposed in this pull request? Moves the -Xlint:unchecked flag in the sbt build configuration from Compile to (Compile, compile) scope, allowing publish and publishLocal commands to work. ## How was this patch tested? Successfully published the spark-launcher subproject from within sbt successfully, where it fails without this patch. Author: Erik LaBianca Closes #20040 from easel/javadoc-xlint. --- project/SparkBuild.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 75703380cdb4a..83054945afcf0 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( From 4c2efde9314a5f67052ac87bfa1472ebb9aca74a Mon Sep 17 00:00:00 2001 From: Erik LaBianca Date: Thu, 21 Dec 2017 10:08:38 -0600 Subject: [PATCH 14/18] [SPARK-22855][BUILD] Add -no-java-comments to sbt docs/scalacOptions Prevents Scala 2.12 scaladoc from blowing up attempting to parse java comments. ## What changes were proposed in this pull request? Adds -no-java-comments to docs/scalacOptions under Scala 2.12. Also moves scaladoc configs out of the TestSettings and into the standard sharedSettings section in SparkBuild.scala. ## How was this patch tested? SBT_OPTS=-Dscala-2.12 sbt ++2.12.4 tags/publishLocal Author: Erik LaBianca Closes #20042 from easel/scaladoc-212. --- project/SparkBuild.scala | 28 ++++++++++++++++------------ 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 83054945afcf0..7469f11df0294 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -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) ) } From 8a0ed5a5ee64a6e854c516f80df5a9729435479b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Dec 2017 00:21:27 +0800 Subject: [PATCH 15/18] [SPARK-22668][SQL] Ensure no global variables in arguments of method split by CodegenContext.splitExpressions() ## What changes were proposed in this pull request? Passing global variables to the split method is dangerous, as any mutating to it is ignored and may lead to unexpected behavior. To prevent this, one approach is to make sure no expression would output global variables: Localizing lifetime of mutable states in expressions. Another approach is, when calling `ctx.splitExpression`, make sure we don't use children's output as parameter names. Approach 1 is actually hard to do, as we need to check all expressions and operators that support whole-stage codegen. Approach 2 is easier as the callers of `ctx.splitExpressions` are not too many. Besides, approach 2 is more flexible, as children's output may be other stuff that can't be parameter name: literal, inlined statement(a + 1), etc. close https://github.com/apache/spark/pull/19865 close https://github.com/apache/spark/pull/19938 ## How was this patch tested? existing tests Author: Wenchen Fan Closes #20021 from cloud-fan/codegen. --- .../sql/catalyst/expressions/arithmetic.scala | 18 +++++------ .../expressions/codegen/CodeGenerator.scala | 32 ++++++++++++++++--- .../expressions/conditionalExpressions.scala | 8 ++--- .../expressions/nullExpressions.scala | 9 +++--- .../sql/catalyst/expressions/predicates.scala | 2 +- 5 files changed, 43 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index d3a8cb5804717..8bb14598a6d7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -602,13 +602,13 @@ case class Least(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val evalChildren = children.map(_.genCode(ctx)) - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "leastTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) val evals = evalChildren.map(eval => s""" |${eval.code} - |if (!${eval.isNull} && ($tmpIsNull || + |if (!${eval.isNull} && (${ev.isNull} || | ${ctx.genGreater(dataType, ev.value, eval.value)})) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; |} """.stripMargin @@ -628,10 +628,9 @@ case class Least(children: Seq[Expression]) extends Expression { foldFunctions = _.map(funcCall => s"${ev.value} = $funcCall;").mkString("\n")) ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; |$codes - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } @@ -682,13 +681,13 @@ case class Greatest(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val evalChildren = children.map(_.genCode(ctx)) - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "greatestTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) val evals = evalChildren.map(eval => s""" |${eval.code} - |if (!${eval.isNull} && ($tmpIsNull || + |if (!${eval.isNull} && (${ev.isNull} || | ${ctx.genGreater(dataType, eval.value, ev.value)})) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; |} """.stripMargin @@ -708,10 +707,9 @@ case class Greatest(children: Seq[Expression]) extends Expression { foldFunctions = _.map(funcCall => s"${ev.value} = $funcCall;").mkString("\n")) ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; |$codes - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 41a920ba3d677..9adf632ddcde8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -128,7 +128,7 @@ class CodegenContext { * `currentVars` to null, or set `currentVars(i)` to null for certain columns, before calling * `Expression.genCode`. */ - final var INPUT_ROW = "i" + var INPUT_ROW = "i" /** * Holding a list of generated columns as input of current operator, will be used by @@ -146,22 +146,30 @@ class CodegenContext { * as a member variable * * They will be kept as member variables in generated classes like `SpecificProjection`. + * + * Exposed for tests only. */ - val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] = + private[catalyst] val inlinedMutableStates: mutable.ArrayBuffer[(String, String)] = mutable.ArrayBuffer.empty[(String, String)] /** * The mapping between mutable state types and corrseponding compacted arrays. * The keys are java type string. The values are [[MutableStateArrays]] which encapsulates * the compacted arrays for the mutable states with the same java type. + * + * Exposed for tests only. */ - val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] = + private[catalyst] val arrayCompactedMutableStates: mutable.Map[String, MutableStateArrays] = mutable.Map.empty[String, MutableStateArrays] // An array holds the code that will initialize each state - val mutableStateInitCode: mutable.ArrayBuffer[String] = + // Exposed for tests only. + private[catalyst] val mutableStateInitCode: mutable.ArrayBuffer[String] = mutable.ArrayBuffer.empty[String] + // Tracks the names of all the mutable states. + private val mutableStateNames: mutable.HashSet[String] = mutable.HashSet.empty + /** * This class holds a set of names of mutableStateArrays that is used for compacting mutable * states for a certain type, and holds the next available slot of the current compacted array. @@ -172,7 +180,11 @@ class CodegenContext { private[this] var currentIndex = 0 - private def createNewArray() = arrayNames.append(freshName("mutableStateArray")) + private def createNewArray() = { + val newArrayName = freshName("mutableStateArray") + mutableStateNames += newArrayName + arrayNames.append(newArrayName) + } def getCurrentIndex: Int = currentIndex @@ -241,6 +253,7 @@ class CodegenContext { val initCode = initFunc(varName) inlinedMutableStates += ((javaType, varName)) mutableStateInitCode += initCode + mutableStateNames += varName varName } else { val arrays = arrayCompactedMutableStates.getOrElseUpdate(javaType, new MutableStateArrays) @@ -930,6 +943,15 @@ class CodegenContext { // inline execution if only one block blocks.head } else { + if (Utils.isTesting) { + // Passing global variables to the split method is dangerous, as any mutating to it is + // ignored and may lead to unexpected behavior. + arguments.foreach { case (_, name) => + assert(!mutableStateNames.contains(name), + s"split function argument $name cannot be a global variable.") + } + } + val func = freshName(funcName) val argString = arguments.map { case (t, name) => s"$t $name" }.mkString(", ") val functions = blocks.zipWithIndex.map { case (body, i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 1a9b68222a7f4..142dfb02be0a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -190,7 +190,7 @@ case class CaseWhen( // It is initialized to `NOT_MATCHED`, and if it's set to `HAS_NULL` or `HAS_NONNULL`, // We won't go on anymore on the computation. val resultState = ctx.freshName("caseWhenResultState") - val tmpResult = ctx.addMutableState(ctx.javaType(dataType), "caseWhenTmpResult") + ev.value = ctx.addMutableState(ctx.javaType(dataType), ev.value) // these blocks are meant to be inside a // do { @@ -205,7 +205,7 @@ case class CaseWhen( |if (!${cond.isNull} && ${cond.value}) { | ${res.code} | $resultState = (byte)(${res.isNull} ? $HAS_NULL : $HAS_NONNULL); - | $tmpResult = ${res.value}; + | ${ev.value} = ${res.value}; | continue; |} """.stripMargin @@ -216,7 +216,7 @@ case class CaseWhen( s""" |${res.code} |$resultState = (byte)(${res.isNull} ? $HAS_NULL : $HAS_NONNULL); - |$tmpResult = ${res.value}; + |${ev.value} = ${res.value}; """.stripMargin } @@ -264,13 +264,11 @@ case class CaseWhen( ev.copy(code = s""" |${ctx.JAVA_BYTE} $resultState = $NOT_MATCHED; - |$tmpResult = ${ctx.defaultValue(dataType)}; |do { | $codes |} while (false); |// TRUE if any condition is met and the result is null, or no any condition is met. |final boolean ${ev.isNull} = ($resultState != $HAS_NONNULL); - |final ${ctx.javaType(dataType)} ${ev.value} = $tmpResult; """.stripMargin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index b4f895fffda38..470d5da041ea5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -72,7 +72,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tmpIsNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, "coalesceTmpIsNull") + ev.isNull = ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) // all the evals are meant to be in a do { ... } while (false); loop val evals = children.map { e => @@ -80,7 +80,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { s""" |${eval.code} |if (!${eval.isNull}) { - | $tmpIsNull = false; + | ${ev.isNull} = false; | ${ev.value} = ${eval.value}; | continue; |} @@ -103,7 +103,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { foldFunctions = _.map { funcCall => s""" |${ev.value} = $funcCall; - |if (!$tmpIsNull) { + |if (!${ev.isNull}) { | continue; |} """.stripMargin @@ -112,12 +112,11 @@ case class Coalesce(children: Seq[Expression]) extends Expression { ev.copy(code = s""" - |$tmpIsNull = true; + |${ev.isNull} = true; |$resultType ${ev.value} = ${ctx.defaultValue(dataType)}; |do { | $codes |} while (false); - |final boolean ${ev.isNull} = $tmpIsNull; """.stripMargin) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index ac9f56f78eb2e..f4ee3d10f3f43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -285,7 +285,7 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { |${valueGen.code} |byte $tmpResult = $HAS_NULL; |if (!${valueGen.isNull}) { - | $tmpResult = 0; + | $tmpResult = $NOT_MATCHED; | $javaDataType $valueArg = ${valueGen.value}; | do { | $codes From d3a1d9527bcd6675cc45773f01d4558cf4b46b3d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Dec 2017 01:08:13 +0800 Subject: [PATCH 16/18] [SPARK-22786][SQL] only use AppStatusPlugin in history server ## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/19681 we introduced a new interface called `AppStatusPlugin`, to register listeners and set up the UI for both live and history UI. However I think it's an overkill for live UI. For example, we should not register `SQLListener` if users are not using SQL functions. Previously we register the `SQLListener` and set up SQL tab when `SparkSession` is firstly created, which indicates users are going to use SQL functions. But in #19681 , we register the SQL functions during `SparkContext` creation. The same thing should apply to streaming too. I think we should keep the previous behavior, and only use this new interface for history server. To reflect this change, I also rename the new interface to `SparkHistoryUIPlugin` This PR also refines the tests for sql listener. ## How was this patch tested? existing tests Author: Wenchen Fan Closes #19981 from cloud-fan/listener. --- .../scala/org/apache/spark/SparkContext.scala | 16 +- .../deploy/history/FsHistoryProvider.scala | 14 +- .../spark/status/AppHistoryServerPlugin.scala | 38 +++ .../spark/status/AppStatusListener.scala | 2 +- .../apache/spark/status/AppStatusPlugin.scala | 71 ------ .../apache/spark/status/AppStatusStore.scala | 17 +- .../org/apache/spark/ui/StagePageSuite.scala | 20 +- ...apache.spark.status.AppHistoryServerPlugin | 1 + .../org.apache.spark.status.AppStatusPlugin | 1 - .../execution/ui/SQLAppStatusListener.scala | 23 +- .../sql/execution/ui/SQLAppStatusStore.scala | 62 +---- .../execution/ui/SQLHistoryServerPlugin.scala | 36 +++ .../spark/sql/internal/SharedState.scala | 18 +- .../execution/metric/SQLMetricsSuite.scala | 8 - .../metric/SQLMetricsTestUtils.scala | 10 +- ....scala => SQLAppStatusListenerSuite.scala} | 238 +++++++++--------- 16 files changed, 256 insertions(+), 319 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/AppStatusPlugin.scala create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin delete mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.status.AppStatusPlugin create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala rename sql/core/src/test/scala/org/apache/spark/sql/execution/ui/{SQLListenerSuite.scala => SQLAppStatusListenerSuite.scala} (71%) 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/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index fa2c5194aa41b..a299b79850613 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -44,7 +44,6 @@ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.status._ import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} -import org.apache.spark.status.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -322,15 +321,18 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) (new InMemoryStore(), true) } + val plugins = ServiceLoader.load( + classOf[AppHistoryServerPlugin], Utils.getContextOrSparkClassLoader).asScala val trackingStore = new ElementTrackingStore(kvstore, conf) if (needReplay) { val replayBus = new ReplayListenerBus() val listener = new AppStatusListener(trackingStore, conf, false, lastUpdateTime = Some(attempt.info.lastUpdated.getTime())) replayBus.addListener(listener) - AppStatusPlugin.loadPlugins().foreach { plugin => - plugin.setupListeners(conf, trackingStore, l => replayBus.addListener(l), false) - } + for { + plugin <- plugins + listener <- plugin.createListeners(conf, trackingStore) + } replayBus.addListener(listener) try { val fileStatus = fs.getFileStatus(new Path(logDir, attempt.logPath)) replay(fileStatus, isApplicationCompleted(fileStatus), replayBus) @@ -353,9 +355,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - AppStatusPlugin.loadPlugins().foreach { plugin => - plugin.setupUI(ui) - } + plugins.foreach(_.setupUI(ui)) val loadedUI = LoadedAppUI(ui) diff --git a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala new file mode 100644 index 0000000000000..d144a0e998fa1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.ui.SparkUI + +/** + * An interface for creating history listeners(to replay event logs) defined in other modules like + * SQL, and setup the UI of the plugin to rebuild the history UI. + */ +private[spark] trait AppHistoryServerPlugin { + /** + * Creates listeners to replay the event logs. + */ + def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] + + /** + * Sets up UI of this plugin to rebuild the history UI. + */ + def setupUI(ui: SparkUI): Unit +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 87eb84d94c005..4db797e1d24c6 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -48,7 +48,7 @@ private[spark] class AppStatusListener( import config._ - private var sparkVersion = SPARK_VERSION + private val sparkVersion = SPARK_VERSION private var appInfo: v1.ApplicationInfo = null private var appSummary = new AppSummary(0, 0) private var coresPerTask: Int = 1 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/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/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/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index cf0000c6393a3..aa78fa015dbef 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) @@ -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/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/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala similarity index 71% 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..5ebbeb4a7cb40 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,137 @@ 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("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 +409,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 +428,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 +493,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 +521,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()) } } } From 4e107fdb7463a67d9c77c4a3434dfe70c72982f4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 21 Dec 2017 09:18:27 -0800 Subject: [PATCH 17/18] [SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPrecision ## What changes were proposed in this pull request? Test Coverage for `WindowFrameCoercion` and `DecimalPrecision`, this is a Sub-tasks for [SPARK-22722](https://issues.apache.org/jira/browse/SPARK-22722). ## How was this patch tested? N/A Author: Yuming Wang Closes #20008 from wangyum/SPARK-22822. --- .../expressions/windowExpressions.scala | 4 +- .../typeCoercion/native/decimalPrecision.sql | 1448 +++ .../native/windowFrameCoercion.sql | 44 + .../native/decimalPrecision.sql.out | 9514 +++++++++++++++++ .../native/windowFrameCoercion.sql.out | 206 + .../sql-tests/results/window.sql.out | 2 +- 6 files changed, 11215 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index e11e3a105f597..220cc4f885d7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -251,8 +251,8 @@ case class SpecifiedWindowFrame( TypeCheckFailure(s"Window frame $location bound '$e' is not a literal.") case e: Expression if !frameType.inputType.acceptsType(e.dataType) => TypeCheckFailure( - s"The data type of the $location bound '${e.dataType} does not match " + - s"the expected data type '${frameType.inputType}'.") + s"The data type of the $location bound '${e.dataType}' does not match " + + s"the expected data type '${frameType.inputType.simpleString}'.") case _ => TypeCheckSuccess } diff --git a/sql/core/src/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 From fe65361b0579777c360dee1d7f633f28df0c6aeb Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Thu, 21 Dec 2017 09:22:08 -0800 Subject: [PATCH 18/18] [SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when child's partitioning is not decided ## What changes were proposed in this pull request? This is a followup PR of https://github.com/apache/spark/pull/19257 where gatorsmile had left couple comments wrt code style. ## How was this patch tested? Doesn't change any functionality. Will depend on build to see if no checkstyle rules are violated. Author: Tejas Patil Closes #20041 from tejasapatil/followup_19257. --- .../exchange/EnsureRequirements.scala | 82 ++++++++++--------- .../spark/sql/sources/BucketedReadSuite.scala | 4 +- 2 files changed, 44 insertions(+), 42 deletions(-) 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/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)) } } }