From 39ccabacf11abdd9afc8f9895084c6707ff35c85 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Oct 2014 11:50:42 -0700 Subject: [PATCH 01/12] [SPARK-3861][SQL] Avoid rebuilding hash tables for broadcast joins on each partition Author: Reynold Xin Closes #2727 from rxin/SPARK-3861-broadcast-hash-2 and squashes the following commits: 9c7b1a2 [Reynold Xin] Revert "Reuse CompactBuffer in UniqueKeyHashedRelation." 97626a1 [Reynold Xin] Reuse CompactBuffer in UniqueKeyHashedRelation. 7fcffb5 [Reynold Xin] Make UniqueKeyHashedRelation private[joins]. 18eb214 [Reynold Xin] Merge branch 'SPARK-3861-broadcast-hash' into SPARK-3861-broadcast-hash-1 4b9d0c9 [Reynold Xin] UniqueKeyHashedRelation.get should return null if the value is null. e0ebdd1 [Reynold Xin] Added a test case. 90b58c0 [Reynold Xin] [SPARK-3861] Avoid rebuilding hash tables on each partition 0c0082b [Reynold Xin] Fix line length. cbc664c [Reynold Xin] Rename join -> joins package. a070d44 [Reynold Xin] Fix line length in HashJoin a39be8c [Reynold Xin] [SPARK-3857] Create a join package for various join operators. --- .../execution/joins/BroadcastHashJoin.scala | 8 +- .../spark/sql/execution/joins/HashJoin.scala | 34 ++---- .../sql/execution/joins/HashedRelation.scala | 109 ++++++++++++++++++ .../execution/joins/ShuffledHashJoin.scala | 5 +- .../execution/joins/HashedRelationSuite.scala | 63 ++++++++++ 5 files changed, 187 insertions(+), 32 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index d88ab6367a1b3..8fd35880eedfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -22,7 +22,7 @@ import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Row, Expression} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} @@ -49,14 +49,16 @@ case class BroadcastHashJoin( @transient private val broadcastFuture = future { - sparkContext.broadcast(buildPlan.executeCollect()) + val input: Array[Row] = buildPlan.executeCollect() + val hashed = HashedRelation(input.iterator, buildSideKeyGenerator, input.length) + sparkContext.broadcast(hashed) } override def execute() = { val broadcastRelation = Await.result(broadcastFuture, 5.minute) streamedPlan.execute().mapPartitions { streamedIter => - joinIterators(broadcastRelation.value.iterator, streamedIter) + hashJoin(streamedIter, broadcastRelation.value) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 472b2e6ca6b4a..4012d757d5f9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow2, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.collection.CompactBuffer @@ -43,34 +43,14 @@ trait HashJoin { override def output = left.output ++ right.output - @transient protected lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) - @transient protected lazy val streamSideKeyGenerator = + @transient protected lazy val buildSideKeyGenerator: Projection = + newProjection(buildKeys, buildPlan.output) + + @transient protected lazy val streamSideKeyGenerator: () => MutableProjection = newMutableProjection(streamedKeys, streamedPlan.output) - protected def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = + protected def hashJoin(streamIter: Iterator[Row], hashedRelation: HashedRelation): Iterator[Row] = { - // TODO: Use Spark's HashMap implementation. - - val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() - var currentRow: Row = null - - // Create a mapping of buildKeys -> rows - while (buildIter.hasNext) { - currentRow = buildIter.next() - val rowKey = buildSideKeyGenerator(currentRow) - if (!rowKey.anyNull) { - val existingMatchList = hashTable.get(rowKey) - val matchList = if (existingMatchList == null) { - val newMatchList = new CompactBuffer[Row]() - hashTable.put(rowKey, newMatchList) - newMatchList - } else { - existingMatchList - } - matchList += currentRow.copy() - } - } - new Iterator[Row] { private[this] var currentStreamedRow: Row = _ private[this] var currentHashMatches: CompactBuffer[Row] = _ @@ -107,7 +87,7 @@ trait HashJoin { while (currentHashMatches == null && streamIter.hasNext) { currentStreamedRow = streamIter.next() if (!joinKeys(currentStreamedRow).anyNull) { - currentHashMatches = hashTable.get(joinKeys.currentValue) + currentHashMatches = hashedRelation.get(joinKeys.currentValue) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala new file mode 100644 index 0000000000000..38b8993b03f82 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -0,0 +1,109 @@ +/* + * 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.joins + +import java.util.{HashMap => JavaHashMap} + +import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.util.collection.CompactBuffer + + +/** + * Interface for a hashed relation by some key. Use [[HashedRelation.apply]] to create a concrete + * object. + */ +private[joins] sealed trait HashedRelation { + def get(key: Row): CompactBuffer[Row] +} + + +/** + * A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values. + */ +private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) + extends HashedRelation with Serializable { + + override def get(key: Row) = hashTable.get(key) +} + + +/** + * A specialized [[HashedRelation]] that maps key into a single value. This implementation + * assumes the key is unique. + */ +private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) + extends HashedRelation with Serializable { + + override def get(key: Row) = { + val v = hashTable.get(key) + if (v eq null) null else CompactBuffer(v) + } + + def getValue(key: Row): Row = hashTable.get(key) +} + + +// TODO(rxin): a version of [[HashedRelation]] backed by arrays for consecutive integer keys. + + +private[joins] object HashedRelation { + + def apply( + input: Iterator[Row], + keyGenerator: Projection, + sizeEstimate: Int = 64): HashedRelation = { + + // TODO: Use Spark's HashMap implementation. + val hashTable = new JavaHashMap[Row, CompactBuffer[Row]](sizeEstimate) + var currentRow: Row = null + + // Whether the join key is unique. If the key is unique, we can convert the underlying + // hash map into one specialized for this. + var keyIsUnique = true + + // Create a mapping of buildKeys -> rows + while (input.hasNext) { + currentRow = input.next() + val rowKey = keyGenerator(currentRow) + if (!rowKey.anyNull) { + val existingMatchList = hashTable.get(rowKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new CompactBuffer[Row]() + hashTable.put(rowKey, newMatchList) + newMatchList + } else { + keyIsUnique = false + existingMatchList + } + matchList += currentRow.copy() + } + } + + if (keyIsUnique) { + val uniqHashTable = new JavaHashMap[Row, Row](hashTable.size) + val iter = hashTable.entrySet().iterator() + while (iter.hasNext) { + val entry = iter.next() + uniqHashTable.put(entry.getKey, entry.getValue()(0)) + } + new UniqueKeyHashedRelation(uniqHashTable) + } else { + new GeneralHashedRelation(hashTable) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 8247304c1dc2c..418c1c23e5546 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -42,8 +42,9 @@ case class ShuffledHashJoin( ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil override def execute() = { - buildPlan.execute().zipPartitions(streamedPlan.execute()) { - (buildIter, streamIter) => joinIterators(buildIter, streamIter) + buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => + val hashed = HashedRelation(buildIter, buildSideKeyGenerator) + hashJoin(streamIter, hashed) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala new file mode 100644 index 0000000000000..2aad01ded1acf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.joins + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.util.collection.CompactBuffer + + +class HashedRelationSuite extends FunSuite { + + // Key is simply the record itself + private val keyProjection = new Projection { + override def apply(row: Row): Row = row + } + + test("GeneralHashedRelation") { + val data = Array(Row(0), Row(1), Row(2), Row(2)) + val hashed = HashedRelation(data.iterator, keyProjection) + assert(hashed.isInstanceOf[GeneralHashedRelation]) + + assert(hashed.get(data(0)) == CompactBuffer[Row](data(0))) + assert(hashed.get(data(1)) == CompactBuffer[Row](data(1))) + assert(hashed.get(Row(10)) === null) + + val data2 = CompactBuffer[Row](data(2)) + data2 += data(2) + assert(hashed.get(data(2)) == data2) + } + + test("UniqueKeyHashedRelation") { + val data = Array(Row(0), Row(1), Row(2)) + val hashed = HashedRelation(data.iterator, keyProjection) + assert(hashed.isInstanceOf[UniqueKeyHashedRelation]) + + assert(hashed.get(data(0)) == CompactBuffer[Row](data(0))) + assert(hashed.get(data(1)) == CompactBuffer[Row](data(1))) + assert(hashed.get(data(2)) == CompactBuffer[Row](data(2))) + assert(hashed.get(Row(10)) === null) + + val uniqHashed = hashed.asInstanceOf[UniqueKeyHashedRelation] + assert(uniqHashed.getValue(data(0)) == data(0)) + assert(uniqHashed.getValue(data(1)) == data(1)) + assert(uniqHashed.getValue(data(2)) == data(2)) + assert(uniqHashed.getValue(Row(10)) == null) + } +} From 49bbdcb660edff7522430b329a300765164ccc44 Mon Sep 17 00:00:00 2001 From: yingjieMiao Date: Mon, 13 Oct 2014 13:11:55 -0700 Subject: [PATCH 02/12] [Spark] RDD take() method: overestimate too much In the comment (Line 1083), it says: "Otherwise, interpolate the number of partitions we need to try, but overestimate it by 50%." `(1.5 * num * partsScanned / buf.size).toInt` is the guess of "num of total partitions needed". In every iteration, we should consider the increment `(1.5 * num * partsScanned / buf.size).toInt - partsScanned` Existing implementation 'exponentially' grows `partsScanned ` ( roughly: `x_{n+1} >= (1.5 + 1) x_n`) This could be a performance problem. (unless this is the intended behavior) Author: yingjieMiao Closes #2648 from yingjieMiao/rdd_take and squashes the following commits: d758218 [yingjieMiao] scala style fix a8e74bb [yingjieMiao] python style fix 4b6e777 [yingjieMiao] infix operator style fix 4391d3b [yingjieMiao] typo fix. 692f4e6 [yingjieMiao] cap numPartsToTry c4483dc [yingjieMiao] style fix 1d2c410 [yingjieMiao] also change in rdd.py and AsyncRDD d31ff7e [yingjieMiao] handle the edge case after 1 iteration a2aa36b [yingjieMiao] RDD take method: overestimate too much --- .../scala/org/apache/spark/rdd/AsyncRDDActions.scala | 12 +++++++----- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 8 +++++--- python/pyspark/rdd.py | 5 ++++- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index b62f3fbdc4a15..ede5568493cc0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -78,16 +78,18 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the first iteration, just try all partitions next. + // If we didn't find any rows after the previous iteration, quadruple and retry. // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. + // by 50%. We also cap the estimation in the end. if (results.size == 0) { - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 } else { - numPartsToTry = (1.5 * num * partsScanned / results.size).toInt + // the left side of max is >=1 whenever partsScanned >= 2 + numPartsToTry = Math.max(1, + (1.5 * num * partsScanned / results.size).toInt - partsScanned) + numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) } } - numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions val left = num - results.size val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 2aba40d152e3e..71cabf61d4ee0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1079,15 +1079,17 @@ abstract class RDD[T: ClassTag]( // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, + // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, // interpolate the number of partitions we need to try, but overestimate it by 50%. + // We also cap the estimation in the end. if (buf.size == 0) { numPartsToTry = partsScanned * 4 } else { - numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt + // the left side of max is >=1 whenever partsScanned >= 2 + numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1) + numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) } } - numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions val left = num - buf.size val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e13bab946c44a..15be4bfec92f9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1070,10 +1070,13 @@ def take(self, num): # If we didn't find any rows after the previous iteration, # quadruple and retry. Otherwise, interpolate the number of # partitions we need to try, but overestimate it by 50%. + # We also cap the estimation in the end. if len(items) == 0: numPartsToTry = partsScanned * 4 else: - numPartsToTry = int(1.5 * num * partsScanned / len(items)) + # the first paramter of max is >=1 whenever partsScanned >= 2 + numPartsToTry = int(1.5 * num * partsScanned / len(items)) - partsScanned + numPartsToTry = min(max(numPartsToTry, 1), partsScanned * 4) left = num - len(items) From 46db277cc14bf3c1e4c4779baa8a40189b332d89 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Oct 2014 13:31:14 -0700 Subject: [PATCH 03/12] [SPARK-3892][SQL] remove redundant type name Author: Daoyuan Wang Closes #2747 from adrian-wang/typename and squashes the following commits: 2824216 [Daoyuan Wang] remove redundant typeName fbaf340 [Daoyuan Wang] typename --- .../org/apache/spark/sql/catalyst/types/dataTypes.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 1d375b8754182..5bdacab664f8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -349,7 +349,6 @@ case object FloatType extends FractionalType { object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) - def typeName: String = "array" } /** @@ -395,8 +394,6 @@ case class StructField(name: String, dataType: DataType, nullable: Boolean) { object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) - - def typeName = "struct" } case class StructType(fields: Seq[StructField]) extends DataType { @@ -459,8 +456,6 @@ object MapType { */ def apply(keyType: DataType, valueType: DataType): MapType = MapType(keyType: DataType, valueType: DataType, true) - - def simpleName = "map" } /** From 2ac40da3f9fa6d45a59bb45b41606f1931ac5e81 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Oct 2014 13:33:12 -0700 Subject: [PATCH 04/12] [SPARK-3407][SQL]Add Date type support Author: Daoyuan Wang Closes #2344 from adrian-wang/date and squashes the following commits: f15074a [Daoyuan Wang] remove outdated lines 2038085 [Daoyuan Wang] update return type 00fe81f [Daoyuan Wang] address lian cheng's comments 0df6ea1 [Daoyuan Wang] rebase and remove simple string bb1b1ef [Daoyuan Wang] remove failing test aa96735 [Daoyuan Wang] not cast for same type compare 30bf48b [Daoyuan Wang] resolve rebase conflict 617d1a8 [Daoyuan Wang] add date_udf case to white list c37e848 [Daoyuan Wang] comment update 5429212 [Daoyuan Wang] change to long f8f219f [Daoyuan Wang] revise according to Cheng Hao 0e0a4f5 [Daoyuan Wang] minor format 4ddcb92 [Daoyuan Wang] add java api for date 0e3110e [Daoyuan Wang] try to fix timezone issue 17fda35 [Daoyuan Wang] set test list 2dfbb5b [Daoyuan Wang] support date type --- .../spark/sql/catalyst/ScalaReflection.scala | 5 +- .../catalyst/analysis/HiveTypeCoercion.scala | 29 +++- .../spark/sql/catalyst/dsl/package.scala | 6 +- .../spark/sql/catalyst/expressions/Cast.scala | 98 +++++++++++-- .../sql/catalyst/expressions/literals.scala | 3 +- .../spark/sql/catalyst/types/dataTypes.scala | 12 +- .../ExpressionEvaluationSuite.scala | 35 ++++- .../apache/spark/sql/api/java/DataType.java | 5 + .../apache/spark/sql/api/java/DateType.java | 27 ++++ .../spark/sql/columnar/ColumnAccessor.scala | 4 + .../spark/sql/columnar/ColumnBuilder.scala | 3 + .../spark/sql/columnar/ColumnStats.scala | 20 ++- .../spark/sql/columnar/ColumnType.scala | 28 +++- .../scala/org/apache/spark/sql/package.scala | 10 ++ .../sql/types/util/DataTypeConversions.scala | 3 + .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 1 + .../spark/sql/columnar/ColumnTypeSuite.scala | 7 +- .../sql/columnar/ColumnarTestUtils.scala | 3 +- .../NullableColumnAccessorSuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 10 ++ .../apache/spark/sql/hive/HiveContext.scala | 6 +- .../spark/sql/hive/HiveInspectors.scala | 9 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + .../org/apache/spark/sql/hive/HiveQl.scala | 8 + .../date_1-0-23edf29bf7376c70d5ecf12720f4b1eb | 0 .../date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b | 0 ...date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + ...date_1-11-480c5f024a28232b7857be327c992509 | 1 + ...date_1-12-4c0ed7fcb75770d8790575b586bf14f4 | 1 + .../date_1-13-44fc74c1993062c0a9522199ff27fea | 1 + ...date_1-14-4855a66124b16d1d0d003235995ac06b | 1 + ...date_1-15-8bc190dba0f641840b5e1e198a14c55b | 1 + ...date_1-16-23edf29bf7376c70d5ecf12720f4b1eb | 0 .../date_1-2-abdce0c0d14d3fc7441b7c134b02f99a | 0 .../date_1-3-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + .../date_1-5-5e70fc74158fbfca38134174360de12d | 0 .../date_1-6-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + .../date_1-8-1d5c58095cd52ea539d869f2ab1ab67d | 0 .../date_1-9-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_2-3-eedb73e0a622c2ab760b524f395dd4ba | 137 ++++++++++++++++++ .../date_2-4-3618dfde8da7c26f03bca72970db9ef7 | 137 ++++++++++++++++++ .../date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c | 12 ++ .../date_2-6-f4edce7cb20f325e8b69e787b2ae8882 | 0 .../date_3-3-4cf49e71b636df754871a675f9e4e24 | 0 .../date_3-4-e009f358964f6d1236cfc03283e2b06f | 1 + .../date_3-5-c26de4559926ddb0127d2dc5ea154774 | 0 .../date_4-0-b84f7e931d710dcbe3c5126d998285a8 | 0 .../date_4-1-6272f5e518f6a20bc96a5870ff315c4f | 0 .../date_4-2-4a0e7bde447ef616b98e0f55d2886de0 | 0 .../date_4-3-a23faa56b5d3ca9063a21f72b4278b00 | 0 .../date_4-4-bee09a7384666043621f68297cee2e68 | 1 + .../date_4-5-b84f7e931d710dcbe3c5126d998285a8 | 0 ...parison-0-69eec445bd045c9dc899fafa348d8495 | 1 + ...parison-1-fcc400871a502009c8680509e3869ec1 | 1 + ...arison-10-a9f2560c273163e11306d4f1dd1d9d54 | 1 + ...arison-11-4a7bac9ddcf40db6329faaec8e426543 | 1 + ...parison-2-b8598a4d0c948c2ddcf3eeef0abf2264 | 1 + ...parison-3-14d35f266be9cceb11a2ae09ec8b3835 | 1 + ...parison-4-c8865b14d53f2c2496fb69ee8191bf37 | 1 + ...parison-5-f2c907e64da8166a731ddc0ed19bad6c | 1 + ...parison-6-5606505a92bad10023ad9a3ef77eacc9 | 1 + ...mparison-7-47913d4aaf0d468ab3764cc3bfd68eb | 1 + ...parison-8-1e5ce4f833b6fba45618437c8fb7643c | 1 + ...parison-9-bcd987341fc1c38047a27d29dac6ae7c | 1 + ...e_join1-3-f71c7be760fb4de4eff8225f2c6614b2 | 22 +++ ...te_join1-4-70b9b49c55699fe94cfde069f5d197c | 0 ..._serde-10-d80e681519dcd8f5078c5602bb5befa9 | 0 ..._serde-11-29540200936bba47f17553547b409af7 | 0 ..._serde-12-c3c3275658b89d31fc504db31ae9f99c | 0 ..._serde-13-6c546456c81e635b6753e1552fac9129 | 1 + ..._serde-14-f8ba18cc7b0225b4022299c44d435101 | 1 + ..._serde-15-66fadc9bcea7d107a610758aa6f50ff3 | 0 ..._serde-16-1bd3345b46f77e17810978e56f9f7c6b | 0 ..._serde-17-a0df43062f8ab676ef728c9968443f12 | 0 ..._serde-18-b50ecc72ce9018ab12fb17568fef038a | 1 + ..._serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 | 1 + ..._serde-20-588516368d8c1533cb7bfb2157fd58c1 | 0 ..._serde-21-dfe166fe053468e738dca23ebe043091 | 0 ..._serde-22-45240a488fb708e432d2f45b74ef7e63 | 0 ..._serde-23-1742a51e4967a8d263572d890cd8d4a8 | 1 + ...e_serde-24-14fd49bd6fee907c1699f7b4e26685b | 1 + ..._serde-25-a199cf185184a25190d65c123d0694ee | 0 ..._serde-26-c5fa68d9aff36f22e5edc1b54332d0ab | 0 ..._serde-27-4d86c79f858866acec3c37f6598c2638 | 0 ..._serde-28-16a41fc9e0f51eb417c763bae8e9cadb | 1 + ..._serde-29-bd1cb09aacd906527b0bbf43bbded812 | 1 + ..._serde-30-7c80741f9f485729afc68609c55423a0 | 0 ...e_serde-31-da36cd1654aee055cb3650133c9d11f | 0 ..._serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 | 0 ..._serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 | 1 + ..._serde-34-6485841336c097895ad5b34f42c0745f | 1 + ..._serde-35-8651a7c351cbc07fb1af6193f6885de8 | 0 ..._serde-36-36e6041f53433482631018410bb62a99 | 0 ..._serde-37-3ddfd8ecb28991aeed588f1ea852c427 | 0 ..._serde-38-e6167e27465514356c557a77d956ea46 | 0 ..._serde-39-c1e17c93582656c12970c37bac153bf2 | 0 ..._serde-40-4a17944b9ec8999bb20c5ba5d4cb877c | 0 ...e_serde-8-cace4f60a08342f58fbe816a9c3a73cf | 137 ++++++++++++++++++ ...e_serde-9-436c3c61cc4278b54ac79c53c88ff422 | 12 ++ ...ate_udf-0-84604a42a5d7f2842f1eec10c689d447 | 0 ...ate_udf-1-5e8136f6a6503ae9bef9beca80fada13 | 0 ...te_udf-10-988ad9744096a29a3672a2d4c121299b | 1 + ...te_udf-11-a5100dd42201b5bc035a9d684cc21bdc | 1 + ...te_udf-12-eb7280a1f191344a99eaa0f805e8faff | 1 + ...te_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 | 1 + ...ate_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 | 1 + ...te_udf-15-d031ee50c119d7c6acafd53543dbd0c4 | 1 + ...te_udf-16-dc59f69e1685e8d923b187ec50d80f06 | 1 + ...te_udf-17-7d046d4efc568049cf3792470b6feab9 | 1 + ...te_udf-18-84604a42a5d7f2842f1eec10c689d447 | 0 ...te_udf-19-5e8136f6a6503ae9bef9beca80fada13 | 0 ...ate_udf-2-10e337c34d1e82a360b8599988f4b266 | 0 ...te_udf-20-10e337c34d1e82a360b8599988f4b266 | 0 ...ate_udf-3-29e406e613c0284b3e16a8943a4d31bd | 0 ...ate_udf-4-23653315213f578856ab5c3bd80c0264 | 0 ...ate_udf-5-891fd92a4787b9789f6d1f51c1eddc8a | 0 ...ate_udf-6-3473c118d20783eafb456043a2ee5d5b | 0 ...ate_udf-7-9fb5165824e161074565e7500959c1b2 | 0 ...ate_udf-8-badfe833681362092fc6345f888b1c21 | 1 + ...ate_udf-9-a8cbb039661d796beaa0d1564c58c563 | 1 + ...on_date-0-7ec1f3a845e2c49191460e15af30aa30 | 0 ...on_date-1-916193405ce5e020dcd32c58325db6fe | 0 ...n_date-10-a8dde9c0b5746dd770c9c262d23ffb10 | 1 + ...n_date-11-fdface2fb6eef67f15bb7d0de2294957 | 1 + ...n_date-12-9b945f8ece6e09ad28c866ff3a10cc24 | 1 + ...on_date-13-b7cb91c7c459798078a79071d329dbf | 1 + ...n_date-14-e4366325f3a0c4a8e92be59f4de73fce | 1 + ...n_date-15-a062a6e87867d8c8cfbdad97bedcbe5f | 1 + ...n_date-16-22a5627d9ac112665eae01d07a91c89c | 1 + ...on_date-17-b9ce94ef93cb16d629af7d7f8ee637e | 1 + ...n_date-18-72c6e9a4e0b434cef67144825346c687 | 1 + ...n_date-19-44e5165eb210559e420105073bc96125 | 1 + ...on_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 | 0 ...n_date-20-7ec1f3a845e2c49191460e15af30aa30 | 0 ...on_date-3-c938b08f57d588926a5d5fbfa4531012 | 0 ...on_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 | 0 ...on_date-5-a855aba47876561fd4fb095e09580686 | 0 ...on_date-6-1405c311915f27b0cc616c83d39eaacc | 2 + ...on_date-7-2ac950d8d5656549dd453e5464cb8530 | 5 + ...on_date-8-a425c11c12c9ce4c9c43d4fbccee5347 | 1 + ...on_date-9-aad6078a09b7bd8f5141437e86bb229f | 1 + ..._check-12-7e053ba4f9dea1e74c1d04c557c3adac | 6 + ..._check-13-45fb706ff448da1fe609c7ff76a80d4d | 0 ...on_date-6-f4d5c71145a9b7464685aa7d09cd4dfd | 40 +++++ ...on_date-7-a0bade1c77338d4f72962389a1f5bea2 | 0 ...on_date-8-21306adbd8be8ad75174ad9d3e42b73c | 0 150 files changed, 872 insertions(+), 42 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java create mode 100644 sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb create mode 100644 sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b create mode 100644 sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 create mode 100644 sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 create mode 100644 sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea create mode 100644 sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b create mode 100644 sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b create mode 100644 sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb create mode 100644 sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a create mode 100644 sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d create mode 100644 sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d create mode 100644 sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba create mode 100644 sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 create mode 100644 sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c create mode 100644 sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 create mode 100644 sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 create mode 100644 sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f create mode 100644 sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 create mode 100644 sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 create mode 100644 sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f create mode 100644 sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 create mode 100644 sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 create mode 100644 sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 create mode 100644 sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c create mode 100644 sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb create mode 100644 sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c create mode 100644 sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c create mode 100644 sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 create mode 100644 sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c create mode 100644 sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 create mode 100644 sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 create mode 100644 sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c create mode 100644 sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 create mode 100644 sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 create mode 100644 sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 create mode 100644 sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b create mode 100644 sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 create mode 100644 sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a create mode 100644 sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 create mode 100644 sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 create mode 100644 sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 create mode 100644 sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 create mode 100644 sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 create mode 100644 sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b create mode 100644 sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee create mode 100644 sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab create mode 100644 sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 create mode 100644 sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb create mode 100644 sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 create mode 100644 sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 create mode 100644 sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f create mode 100644 sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 create mode 100644 sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 create mode 100644 sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f create mode 100644 sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 create mode 100644 sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 create mode 100644 sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 create mode 100644 sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 create mode 100644 sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 create mode 100644 sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c create mode 100644 sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf create mode 100644 sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 create mode 100644 sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 create mode 100644 sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 create mode 100644 sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b create mode 100644 sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc create mode 100644 sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff create mode 100644 sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 create mode 100644 sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 create mode 100644 sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 create mode 100644 sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 create mode 100644 sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 create mode 100644 sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 create mode 100644 sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 create mode 100644 sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 create mode 100644 sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 create mode 100644 sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd create mode 100644 sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 create mode 100644 sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a create mode 100644 sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b create mode 100644 sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 create mode 100644 sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 create mode 100644 sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 create mode 100644 sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 create mode 100644 sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe create mode 100644 sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 create mode 100644 sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 create mode 100644 sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 create mode 100644 sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf create mode 100644 sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce create mode 100644 sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f create mode 100644 sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c create mode 100644 sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e create mode 100644 sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 create mode 100644 sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 create mode 100644 sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 create mode 100644 sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 create mode 100644 sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 create mode 100644 sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 create mode 100644 sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 create mode 100644 sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc create mode 100644 sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 create mode 100644 sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 create mode 100644 sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd create mode 100644 sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 create mode 100644 sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index b3ae8e6779700..3d4296f9d7068 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -77,8 +77,9 @@ object ScalaReflection { val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 64881854df7a5..7c480de107e7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -220,20 +220,39 @@ trait HiveTypeCoercion { case a: BinaryArithmetic if a.right.dataType == StringType => a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) + // we should cast all timestamp/date/string compare into string compare + case p: BinaryPredicate if p.left.dataType == StringType + && p.right.dataType == DateType => + p.makeCopy(Array(p.left, Cast(p.right, StringType))) + case p: BinaryPredicate if p.left.dataType == DateType + && p.right.dataType == StringType => + p.makeCopy(Array(Cast(p.left, StringType), p.right)) case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType == TimestampType => - p.makeCopy(Array(Cast(p.left, TimestampType), p.right)) + p.makeCopy(Array(p.left, Cast(p.right, StringType))) case p: BinaryPredicate if p.left.dataType == TimestampType && p.right.dataType == StringType => - p.makeCopy(Array(p.left, Cast(p.right, TimestampType))) + p.makeCopy(Array(Cast(p.left, StringType), p.right)) + case p: BinaryPredicate if p.left.dataType == TimestampType + && p.right.dataType == DateType => + p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) + case p: BinaryPredicate if p.left.dataType == DateType + && p.right.dataType == TimestampType => + p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) - case i @ In(a,b) if a.dataType == TimestampType && b.forall(_.dataType == StringType) => - i.makeCopy(Array(a,b.map(Cast(_,TimestampType)))) + case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == StringType) => + i.makeCopy(Array(Cast(a, StringType), b)) + case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == StringType) => + i.makeCopy(Array(Cast(a, StringType), b)) + case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == TimestampType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) + case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == DateType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) case Sum(e) if e.dataType == StringType => Sum(Cast(e, DoubleType)) @@ -283,6 +302,8 @@ trait HiveTypeCoercion { // Skip if the type is boolean type already. Note that this extra cast should be removed // by optimizer.SimplifyCasts. case Cast(e, BooleanType) if e.dataType == BooleanType => e + // DateType should be null if be cast to boolean. + case Cast(e, BooleanType) if e.dataType == DateType => Cast(e, BooleanType) // If the data type is not boolean and is being cast boolean, turn it into a comparison // with the numeric value, i.e. x != 0. This will coerce the type into numeric type. case Cast(e, BooleanType) if e.dataType != BooleanType => Not(EqualTo(e, Literal(0))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index deb622c39faf5..75b6e37c2a1f9 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.language.implicitConversions @@ -119,6 +119,7 @@ package object dsl { implicit def floatToLiteral(f: Float) = Literal(f) implicit def doubleToLiteral(d: Double) = Literal(d) implicit def stringToLiteral(s: String) = Literal(s) + implicit def dateToLiteral(d: Date) = Literal(d) implicit def decimalToLiteral(d: BigDecimal) = Literal(d) implicit def timestampToLiteral(t: Timestamp) = Literal(t) implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) @@ -174,6 +175,9 @@ package object dsl { /** Creates a new AttributeReference of type string */ def string = AttributeReference(s, StringType, nullable = true)() + /** Creates a new AttributeReference of type date */ + def date = AttributeReference(s, DateType, nullable = true)() + /** Creates a new AttributeReference of type decimal */ def decimal = AttributeReference(s, DecimalType, nullable = true)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index f626d09f037bc..8e5ee12e314bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -17,18 +17,21 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ -case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { +case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { override def foldable = child.foldable override def nullable = (child.dataType, dataType) match { case (StringType, _: NumericType) => true case (StringType, TimestampType) => true + case (StringType, DateType) => true case _ => child.nullable } @@ -42,6 +45,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // UDFToString private[this] def castToString: Any => Any = child.dataType match { case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) + case DateType => buildCast[Date](_, dateToString) case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -56,7 +60,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => buildCast[String](_, _.length() != 0) case TimestampType => - buildCast[Timestamp](_, b => b.getTime() != 0 || b.getNanos() != 0) + buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) + case DateType => + // Hive would return null when cast from date to boolean + buildCast[Date](_, d => null) case LongType => buildCast[Long](_, _ != 0) case IntegerType => @@ -95,6 +102,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { buildCast[Short](_, s => new Timestamp(s)) case ByteType => buildCast[Byte](_, b => new Timestamp(b)) + case DateType => + buildCast[Date](_, d => new Timestamp(d.getTime)) // TimestampWritable.decimalToTimestamp case DecimalType => buildCast[BigDecimal](_, d => decimalToTimestamp(d)) @@ -130,7 +139,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // Converts Timestamp to string according to Hive TimestampWritable convention private[this] def timestampToString(ts: Timestamp): String = { val timestampString = ts.toString - val formatted = Cast.threadLocalDateFormat.get.format(ts) + val formatted = Cast.threadLocalTimestampFormat.get.format(ts) if (timestampString.length > 19 && timestampString.substring(19) != ".0") { formatted + timestampString.substring(19) @@ -139,6 +148,39 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } } + // Converts Timestamp to string according to Hive TimestampWritable convention + private[this] def timestampToDateString(ts: Timestamp): String = { + Cast.threadLocalDateFormat.get.format(ts) + } + + // DateConverter + private[this] def castToDate: Any => Any = child.dataType match { + case StringType => + buildCast[String](_, s => + try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null } + ) + case TimestampType => + // throw valid precision more than seconds, according to Hive. + // Timestamp.nanos is in 0 to 999,999,999, no more than a second. + buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000)) + // Hive throws this exception as a Semantic Exception + // It is never possible to compare result when hive return with exception, so we can return null + // NULL is more reasonable here, since the query itself obeys the grammar. + case _ => _ => null + } + + // Date cannot be cast to long, according to hive + private[this] def dateToLong(d: Date) = null + + // Date cannot be cast to double, according to hive + private[this] def dateToDouble(d: Date) = null + + // Converts Date to string according to Hive DateWritable convention + private[this] def dateToString(d: Date): String = { + Cast.threadLocalDateFormat.get.format(d) + } + + // LongConverter private[this] def castToLong: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toLong catch { @@ -146,6 +188,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0L) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => @@ -154,6 +198,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } + // IntConverter private[this] def castToInt: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toInt catch { @@ -161,6 +206,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1 else 0) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) case DecimalType => @@ -169,6 +216,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } + // ShortConverter private[this] def castToShort: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toShort catch { @@ -176,6 +224,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => @@ -184,6 +234,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } + // ByteConverter private[this] def castToByte: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toByte catch { @@ -191,6 +242,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => @@ -199,6 +252,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } + // DecimalConverter private[this] def castToDecimal: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try BigDecimal(s.toDouble) catch { @@ -206,6 +260,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) BigDecimal(1) else BigDecimal(0)) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) @@ -213,6 +269,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) } + // DoubleConverter private[this] def castToDouble: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toDouble catch { @@ -220,6 +277,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) case DecimalType => @@ -228,6 +287,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) } + // FloatConverter private[this] def castToFloat: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toFloat catch { @@ -235,6 +295,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) case DecimalType => @@ -245,17 +307,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { private[this] lazy val cast: Any => Any = dataType match { case dt if dt == child.dataType => identity[Any] - case StringType => castToString - case BinaryType => castToBinary - case DecimalType => castToDecimal + case StringType => castToString + case BinaryType => castToBinary + case DecimalType => castToDecimal + case DateType => castToDate case TimestampType => castToTimestamp - case BooleanType => castToBoolean - case ByteType => castToByte - case ShortType => castToShort - case IntegerType => castToInt - case FloatType => castToFloat - case LongType => castToLong - case DoubleType => castToDouble + case BooleanType => castToBoolean + case ByteType => castToByte + case ShortType => castToShort + case IntegerType => castToInt + case FloatType => castToFloat + case LongType => castToLong + case DoubleType => castToDouble } override def eval(input: Row): Any = { @@ -267,6 +330,13 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { object Cast { // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + override def initialValue() = { + new SimpleDateFormat("yyyy-MM-dd") + } + } + + // `SimpleDateFormat` is not thread-safe. + private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue() = { new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 78a0c55e4bbe5..ba240233cae61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.types._ @@ -33,6 +33,7 @@ object Literal { case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(d, DecimalType) case t: Timestamp => Literal(t, TimestampType) + case d: Date => Literal(d, DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 5bdacab664f8b..0cf139ebde417 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.types -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.math.Numeric.{BigDecimalAsIfIntegral, DoubleAsIfIntegral, FloatAsIfIntegral} import scala.reflect.ClassTag @@ -250,6 +250,16 @@ case object TimestampType extends NativeType { } } +case object DateType extends NativeType { + private[sql] type JvmType = Date + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Date, y: Date) = x.compareTo(y) + } +} + abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 692ed78a7292c..6dc5942023f9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.collection.immutable.HashSet @@ -252,8 +252,11 @@ class ExpressionEvaluationSuite extends FunSuite { test("data type casting") { - val sts = "1970-01-01 00:00:01.1" - val ts = Timestamp.valueOf(sts) + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = Timestamp.valueOf(nts) checkEvaluation("abdef" cast StringType, "abdef") checkEvaluation("abdef" cast DecimalType, null) @@ -266,8 +269,15 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) - checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) + checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) + checkEvaluation(Cast(Literal(d) cast StringType, DateType), d) + checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) + // all convert to string type to check + checkEvaluation( + Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) + checkEvaluation( + Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts) checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") @@ -316,6 +326,12 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } + test("date") { + val d1 = Date.valueOf("1970-01-01") + val d2 = Date.valueOf("1970-01-02") + checkEvaluation(Literal(d1) < Literal(d2), true) + } + test("timestamp") { val ts1 = new Timestamp(12) val ts2 = new Timestamp(123) @@ -323,6 +339,17 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(ts1) < Literal(ts2), true) } + test("date casting") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(Cast(d, ShortType), null) + checkEvaluation(Cast(d, IntegerType), null) + checkEvaluation(Cast(d, LongType), null) + checkEvaluation(Cast(d, FloatType), null) + checkEvaluation(Cast(d, DoubleType), null) + checkEvaluation(Cast(d, StringType), "1970-01-01") + checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + } + test("timestamp casting") { val millis = 15 * 1000 + 2 val seconds = millis * 1000 + 2 diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 37b4c8ffcba0b..37e88d72b9172 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -44,6 +44,11 @@ public abstract class DataType { */ public static final BooleanType BooleanType = new BooleanType(); + /** + * Gets the DateType object. + */ + public static final DateType DateType = new DateType(); + /** * Gets the TimestampType object. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java new file mode 100644 index 0000000000000..6677793baa365 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java @@ -0,0 +1,27 @@ +/* + * 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.api.java; + +/** + * The data type representing java.sql.Date values. + * + * {@code DateType} is represented by the singleton object {@link DataType#DateType}. + */ +public class DateType extends DataType { + protected DateType() {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index c9faf0852142a..538dd5b734664 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -92,6 +92,9 @@ private[sql] class FloatColumnAccessor(buffer: ByteBuffer) private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) +private[sql] class DateColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, DATE) + private[sql] class TimestampColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, TIMESTAMP) @@ -118,6 +121,7 @@ private[sql] object ColumnAccessor { case BYTE.typeId => new ByteColumnAccessor(dup) case SHORT.typeId => new ShortColumnAccessor(dup) case STRING.typeId => new StringColumnAccessor(dup) + case DATE.typeId => new DateColumnAccessor(dup) case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) case BINARY.typeId => new BinaryColumnAccessor(dup) case GENERIC.typeId => new GenericColumnAccessor(dup) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 2e61a981375aa..300cef15bf8a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -107,6 +107,8 @@ private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColum private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) +private[sql] class DateColumnBuilder extends NativeColumnBuilder(new DateColumnStats, DATE) + private[sql] class TimestampColumnBuilder extends NativeColumnBuilder(new TimestampColumnStats, TIMESTAMP) @@ -151,6 +153,7 @@ private[sql] object ColumnBuilder { case STRING.typeId => new StringColumnBuilder case BINARY.typeId => new BinaryColumnBuilder case GENERIC.typeId => new GenericColumnBuilder + case DATE.typeId => new DateColumnBuilder case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 203a714e03c97..b34ab255d084a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} @@ -190,6 +190,24 @@ private[sql] class StringColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount) } +private[sql] class DateColumnStats extends ColumnStats { + var upper: Date = null + var lower: Date = null + var nullCount = 0 + + override def gatherStats(row: Row, ordinal: Int) { + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Date] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 + } + } + + def collectedStatistics = Row(lower, upper, nullCount) +} + private[sql] class TimestampColumnStats extends ColumnStats { var upper: Timestamp = null var lower: Timestamp = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 198b5756676aa..ab66c85c4f242 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.TypeTag @@ -335,7 +335,26 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } } -private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { +private[sql] object DATE extends NativeColumnType(DateType, 8, 8) { + override def extract(buffer: ByteBuffer) = { + val date = new Date(buffer.getLong()) + date + } + + override def append(v: Date, buffer: ByteBuffer): Unit = { + buffer.putLong(v.getTime) + } + + override def getField(row: Row, ordinal: Int) = { + row(ordinal).asInstanceOf[Date] + } + + override def setField(row: MutableRow, ordinal: Int, value: Date): Unit = { + row(ordinal) = value + } +} + +private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { override def extract(buffer: ByteBuffer) = { val timestamp = new Timestamp(buffer.getLong()) timestamp.setNanos(buffer.getInt()) @@ -376,7 +395,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -387,7 +406,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -407,6 +426,7 @@ private[sql] object ColumnType { case ShortType => SHORT case StringType => STRING case BinaryType => BINARY + case DateType => DATE case TimestampType => TIMESTAMP case _ => GENERIC } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index f513eae9c2d13..e98d151286818 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -165,6 +165,16 @@ package object sql { @DeveloperApi val TimestampType = catalyst.types.TimestampType + /** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Date` values. + * + * @group dataType + */ + @DeveloperApi + val DateType = catalyst.types.DateType + /** * :: DeveloperApi :: * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 77353f4eb0227..e44cb08309523 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -41,6 +41,7 @@ protected[sql] object DataTypeConversions { case StringType => JDataType.StringType case BinaryType => JDataType.BinaryType case BooleanType => JDataType.BooleanType + case DateType => JDataType.DateType case TimestampType => JDataType.TimestampType case DecimalType => JDataType.DecimalType case DoubleType => JDataType.DoubleType @@ -80,6 +81,8 @@ protected[sql] object DataTypeConversions { BinaryType case booleanType: org.apache.spark.sql.api.java.BooleanType => BooleanType + case dateType: org.apache.spark.sql.api.java.DateType => + DateType case timestampType: org.apache.spark.sql.api.java.TimestampType => TimestampType case decimalType: org.apache.spark.sql.api.java.DecimalType => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index e24c521d24c7a..bfa9ea416266d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -34,6 +34,7 @@ case class ReflectData( byteField: Byte, booleanField: Boolean, decimalField: BigDecimal, + date: Date, timestampField: Timestamp, seqInt: Seq[Int]) @@ -76,7 +77,7 @@ case class ComplexReflectData( class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - BigDecimal(1), new Timestamp(12345), Seq(1,2,3)) + BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.registerTempTable("reflectData") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 0cdbb3167ce36..6bdf741134e2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -30,6 +30,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) + testColumnStats(classOf[DateColumnStats], DATE, Row(null, null, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) def testColumnStats[T <: NativeType, U <: ColumnStats]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 4fb1ecf1d532b..3f3f35d50188b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -33,8 +33,8 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( - INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, - BOOLEAN -> 1, STRING -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, BOOLEAN -> 1, + STRING -> 8, DATE -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -64,6 +64,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(DATE, new Date(0L), 8) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 38b04dd959f70..a1f21219eaf2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar import scala.collection.immutable.HashSet import scala.util.Random -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow @@ -50,6 +50,7 @@ object ColumnarTestUtils { case STRING => Random.nextString(Random.nextInt(32)) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) + case DATE => new Date(Random.nextLong()) case TIMESTAMP => val timestamp = new Timestamp(Random.nextLong()) timestamp.setNanos(Random.nextInt(999999999)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 6c9a9ab6c3418..21906e3fdcc6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -41,7 +41,9 @@ object TestNullableColumnAccessor { class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { + Seq( + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + ).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index f54a21eb4fbb1..cb73f3da81e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -37,7 +37,9 @@ object TestNullableColumnBuilder { class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { + Seq( + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + ).foreach { testNullableColumnBuilder(_) } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 35e9c9939d4b7..463888551a359 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -343,6 +343,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ct_case_insensitive", "database_location", "database_properties", + "date_2", + "date_3", + "date_4", + "date_comparison", + "date_join1", + "date_serde", + "date_udf", "decimal_1", "decimal_4", "decimal_join", @@ -604,8 +611,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partition_date", "partition_schema1", "partition_serde_format", + "partition_type_check", "partition_varchar1", "partition_wise_fileformat4", "partition_wise_fileformat5", @@ -904,6 +913,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union7", "union8", "union9", + "union_date", "union_lateralview", "union_ppr", "union_remove_11", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index fad3b39f81413..8b5a90159e1bb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.io.TimestampWritable +import org.apache.hadoop.hive.serde2.io.DateWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -357,7 +358,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType, TimestampType, BinaryType) + ShortType, DecimalType, DateType, TimestampType, BinaryType) protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => @@ -372,6 +373,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" + case (d: Date, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (other, tpe) if primitiveTypes contains tpe => other.toString diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index d633c42c6bd67..1977618b4c9f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -39,6 +39,7 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType @@ -49,6 +50,7 @@ private[hive] trait HiveInspectors { // java class case c: Class[_] if c == classOf[java.lang.String] => StringType + case c: Class[_] if c == classOf[java.sql.Date] => DateType case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType case c: Class[_] if c == classOf[HiveDecimal] => DecimalType case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType @@ -93,6 +95,7 @@ private[hive] trait HiveInspectors { System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) bytes } + case d: hiveIo.DateWritable => d.get case t: hiveIo.TimestampWritable => t.getTimestamp case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) case list: java.util.List[_] => list.map(unwrap) @@ -108,6 +111,7 @@ private[hive] trait HiveInspectors { case str: String => str case p: java.math.BigDecimal => p case p: Array[Byte] => p + case p: java.sql.Date => p case p: java.sql.Timestamp => p } @@ -147,6 +151,7 @@ private[hive] trait HiveInspectors { case l: Byte => l: java.lang.Byte case b: BigDecimal => new HiveDecimal(b.underlying()) case b: Array[Byte] => b + case d: java.sql.Date => d case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => @@ -173,6 +178,7 @@ private[hive] trait HiveInspectors { case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => @@ -211,6 +217,8 @@ private[hive] trait HiveInspectors { case _: JavaBinaryObjectInspector => BinaryType case _: WritableHiveDecimalObjectInspector => DecimalType case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableDateObjectInspector => DateType + case _: JavaDateObjectInspector => DateType case _: WritableTimestampObjectInspector => TimestampType case _: JavaTimestampObjectInspector => TimestampType case _: WritableVoidObjectInspector => NullType @@ -238,6 +246,7 @@ private[hive] trait HiveInspectors { case ShortType => shortTypeInfo case StringType => stringTypeInfo case DecimalType => decimalTypeInfo + case DateType => dateTypeInfo case TimestampType => timestampTypeInfo case NullType => voidTypeInfo } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index addd5bed8426d..c5fee5e4702f6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -186,6 +186,7 @@ object HiveMetastoreTypes extends RegexParsers { "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | "decimal" ^^^ DecimalType | + "date" ^^^ DateType | "timestamp" ^^^ TimestampType | "varchar\\((\\d+)\\)".r ^^^ StringType @@ -235,6 +236,7 @@ object HiveMetastoreTypes extends RegexParsers { case LongType => "bigint" case BinaryType => "binary" case BooleanType => "boolean" + case DateType => "date" case DecimalType => "decimal" case TimestampType => "timestamp" case NullType => "void" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 7cc14dc7a9c9e..2b599157d15d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.sql.Date + import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils @@ -317,6 +319,7 @@ private[hive] object HiveQl { case Token("TOK_STRING", Nil) => StringType case Token("TOK_FLOAT", Nil) => FloatType case Token("TOK_DOUBLE", Nil) => DoubleType + case Token("TOK_DATE", Nil) => DateType case Token("TOK_TIMESTAMP", Nil) => TimestampType case Token("TOK_BINARY", Nil) => BinaryType case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) @@ -924,6 +927,8 @@ private[hive] object HiveQl { Cast(nodeToExpr(arg), DecimalType) case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), TimestampType) + case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DateType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) @@ -1047,6 +1052,9 @@ private[hive] object HiveQl { case ast: ASTNode if ast.getType == HiveParser.StringLiteral => Literal(BaseSemanticAnalyzer.unescapeSQLString(ast.getText)) + case ast: ASTNode if ast.getType == HiveParser.TOK_DATELITERAL => + Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1))) + case a: ASTNode => throw new NotImplementedError( s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : diff --git a/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b b/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 b/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 new file mode 100644 index 0000000000000..5a368ab170261 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 @@ -0,0 +1 @@ +2012-01-01 2011-01-01 2011-01-01 00:00:00 2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 b/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 new file mode 100644 index 0000000000000..edb4b1f84001b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea b/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea new file mode 100644 index 0000000000000..2af0b9ed3a68c --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea @@ -0,0 +1 @@ +true true true true true true true true true true diff --git a/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b b/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b new file mode 100644 index 0000000000000..d8dfbf60007bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b @@ -0,0 +1 @@ +2001-01-28 2001-02-28 2001-03-28 2001-04-28 2001-05-28 2001-06-28 2001-07-28 2001-08-28 2001-09-28 2001-10-28 2001-11-28 2001-12-28 diff --git a/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b b/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b new file mode 100644 index 0000000000000..4f6a1bc4273e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b @@ -0,0 +1 @@ +true true true true true true true true true true true true diff --git a/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a b/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d b/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d b/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba b/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba new file mode 100644 index 0000000000000..db973ab292d5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba @@ -0,0 +1,137 @@ +2010-10-20 7291 +2010-10-20 3198 +2010-10-20 3014 +2010-10-20 2630 +2010-10-20 1610 +2010-10-20 1599 +2010-10-20 1531 +2010-10-20 1142 +2010-10-20 1064 +2010-10-20 897 +2010-10-20 361 +2010-10-21 7291 +2010-10-21 3198 +2010-10-21 3014 +2010-10-21 2646 +2010-10-21 2630 +2010-10-21 1610 +2010-10-21 1599 +2010-10-21 1531 +2010-10-21 1142 +2010-10-21 1064 +2010-10-21 897 +2010-10-21 361 +2010-10-22 3198 +2010-10-22 3014 +2010-10-22 2646 +2010-10-22 2630 +2010-10-22 1610 +2010-10-22 1599 +2010-10-22 1531 +2010-10-22 1142 +2010-10-22 1064 +2010-10-22 897 +2010-10-22 361 +2010-10-23 7274 +2010-10-23 5917 +2010-10-23 5904 +2010-10-23 5832 +2010-10-23 3171 +2010-10-23 3085 +2010-10-23 2932 +2010-10-23 1805 +2010-10-23 650 +2010-10-23 426 +2010-10-23 384 +2010-10-23 272 +2010-10-24 7282 +2010-10-24 3198 +2010-10-24 3014 +2010-10-24 2646 +2010-10-24 2630 +2010-10-24 2571 +2010-10-24 2254 +2010-10-24 1610 +2010-10-24 1599 +2010-10-24 1531 +2010-10-24 897 +2010-10-24 361 +2010-10-25 7291 +2010-10-25 3198 +2010-10-25 3014 +2010-10-25 2646 +2010-10-25 2630 +2010-10-25 1610 +2010-10-25 1599 +2010-10-25 1531 +2010-10-25 1142 +2010-10-25 1064 +2010-10-25 897 +2010-10-25 361 +2010-10-26 7291 +2010-10-26 3198 +2010-10-26 3014 +2010-10-26 2662 +2010-10-26 2646 +2010-10-26 2630 +2010-10-26 1610 +2010-10-26 1599 +2010-10-26 1531 +2010-10-26 1142 +2010-10-26 1064 +2010-10-26 897 +2010-10-26 361 +2010-10-27 7291 +2010-10-27 3198 +2010-10-27 3014 +2010-10-27 2630 +2010-10-27 1610 +2010-10-27 1599 +2010-10-27 1531 +2010-10-27 1142 +2010-10-27 1064 +2010-10-27 897 +2010-10-27 361 +2010-10-28 7291 +2010-10-28 3198 +2010-10-28 3014 +2010-10-28 2646 +2010-10-28 2630 +2010-10-28 1610 +2010-10-28 1599 +2010-10-28 1531 +2010-10-28 1142 +2010-10-28 1064 +2010-10-28 897 +2010-10-28 361 +2010-10-29 7291 +2010-10-29 3198 +2010-10-29 3014 +2010-10-29 2646 +2010-10-29 2630 +2010-10-29 1610 +2010-10-29 1599 +2010-10-29 1531 +2010-10-29 1142 +2010-10-29 1064 +2010-10-29 897 +2010-10-29 361 +2010-10-30 5917 +2010-10-30 5904 +2010-10-30 3171 +2010-10-30 3085 +2010-10-30 2932 +2010-10-30 2018 +2010-10-30 1805 +2010-10-30 650 +2010-10-30 426 +2010-10-30 384 +2010-10-30 272 +2010-10-31 7282 +2010-10-31 3198 +2010-10-31 2571 +2010-10-31 1610 +2010-10-31 1599 +2010-10-31 1531 +2010-10-31 897 +2010-10-31 361 diff --git a/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 b/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 new file mode 100644 index 0000000000000..1b0ea7b9eec84 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 @@ -0,0 +1,137 @@ +2010-10-31 361 +2010-10-31 897 +2010-10-31 1531 +2010-10-31 1599 +2010-10-31 1610 +2010-10-31 2571 +2010-10-31 3198 +2010-10-31 7282 +2010-10-30 272 +2010-10-30 384 +2010-10-30 426 +2010-10-30 650 +2010-10-30 1805 +2010-10-30 2018 +2010-10-30 2932 +2010-10-30 3085 +2010-10-30 3171 +2010-10-30 5904 +2010-10-30 5917 +2010-10-29 361 +2010-10-29 897 +2010-10-29 1064 +2010-10-29 1142 +2010-10-29 1531 +2010-10-29 1599 +2010-10-29 1610 +2010-10-29 2630 +2010-10-29 2646 +2010-10-29 3014 +2010-10-29 3198 +2010-10-29 7291 +2010-10-28 361 +2010-10-28 897 +2010-10-28 1064 +2010-10-28 1142 +2010-10-28 1531 +2010-10-28 1599 +2010-10-28 1610 +2010-10-28 2630 +2010-10-28 2646 +2010-10-28 3014 +2010-10-28 3198 +2010-10-28 7291 +2010-10-27 361 +2010-10-27 897 +2010-10-27 1064 +2010-10-27 1142 +2010-10-27 1531 +2010-10-27 1599 +2010-10-27 1610 +2010-10-27 2630 +2010-10-27 3014 +2010-10-27 3198 +2010-10-27 7291 +2010-10-26 361 +2010-10-26 897 +2010-10-26 1064 +2010-10-26 1142 +2010-10-26 1531 +2010-10-26 1599 +2010-10-26 1610 +2010-10-26 2630 +2010-10-26 2646 +2010-10-26 2662 +2010-10-26 3014 +2010-10-26 3198 +2010-10-26 7291 +2010-10-25 361 +2010-10-25 897 +2010-10-25 1064 +2010-10-25 1142 +2010-10-25 1531 +2010-10-25 1599 +2010-10-25 1610 +2010-10-25 2630 +2010-10-25 2646 +2010-10-25 3014 +2010-10-25 3198 +2010-10-25 7291 +2010-10-24 361 +2010-10-24 897 +2010-10-24 1531 +2010-10-24 1599 +2010-10-24 1610 +2010-10-24 2254 +2010-10-24 2571 +2010-10-24 2630 +2010-10-24 2646 +2010-10-24 3014 +2010-10-24 3198 +2010-10-24 7282 +2010-10-23 272 +2010-10-23 384 +2010-10-23 426 +2010-10-23 650 +2010-10-23 1805 +2010-10-23 2932 +2010-10-23 3085 +2010-10-23 3171 +2010-10-23 5832 +2010-10-23 5904 +2010-10-23 5917 +2010-10-23 7274 +2010-10-22 361 +2010-10-22 897 +2010-10-22 1064 +2010-10-22 1142 +2010-10-22 1531 +2010-10-22 1599 +2010-10-22 1610 +2010-10-22 2630 +2010-10-22 2646 +2010-10-22 3014 +2010-10-22 3198 +2010-10-21 361 +2010-10-21 897 +2010-10-21 1064 +2010-10-21 1142 +2010-10-21 1531 +2010-10-21 1599 +2010-10-21 1610 +2010-10-21 2630 +2010-10-21 2646 +2010-10-21 3014 +2010-10-21 3198 +2010-10-21 7291 +2010-10-20 361 +2010-10-20 897 +2010-10-20 1064 +2010-10-20 1142 +2010-10-20 1531 +2010-10-20 1599 +2010-10-20 1610 +2010-10-20 2630 +2010-10-20 3014 +2010-10-20 3198 +2010-10-20 7291 diff --git a/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c b/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c new file mode 100644 index 0000000000000..0f2a6f7a99237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c @@ -0,0 +1,12 @@ +2010-10-20 11 +2010-10-21 12 +2010-10-22 11 +2010-10-23 12 +2010-10-24 12 +2010-10-25 12 +2010-10-26 13 +2010-10-27 11 +2010-10-28 12 +2010-10-29 12 +2010-10-30 11 +2010-10-31 8 diff --git a/sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 b/sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 b/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f b/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f new file mode 100644 index 0000000000000..66d2220d06de2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f @@ -0,0 +1 @@ +1 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 b/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 b/sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f b/sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 b/sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 b/sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 b/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 new file mode 100644 index 0000000000000..b61affde4ffce --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 @@ -0,0 +1 @@ +2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 b/sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 b/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 b/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 b/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 b/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 b/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 b/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 b/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c b/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 b/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb b/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c b/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c b/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 b/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 new file mode 100644 index 0000000000000..b7305b903edca --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 @@ -0,0 +1,22 @@ +1064 2010-10-20 1064 2010-10-20 +1142 2010-10-21 1142 2010-10-21 +1599 2010-10-22 1599 2010-10-22 +361 2010-10-23 361 2010-10-23 +897 2010-10-24 897 2010-10-24 +1531 2010-10-25 1531 2010-10-25 +1610 2010-10-26 1610 2010-10-26 +3198 2010-10-27 3198 2010-10-27 +1064 2010-10-28 1064 2010-10-28 +1142 2010-10-29 1142 2010-10-29 +1064 2000-11-20 1064 2000-11-20 +1142 2000-11-21 1142 2000-11-21 +1599 2000-11-22 1599 2000-11-22 +361 2000-11-23 361 2000-11-23 +897 2000-11-24 897 2000-11-24 +1531 2000-11-25 1531 2000-11-25 +1610 2000-11-26 1610 2000-11-26 +3198 2000-11-27 3198 2000-11-27 +1064 2000-11-28 1064 2000-11-28 +1142 2000-11-28 1064 2000-11-28 +1064 2000-11-28 1142 2000-11-28 +1142 2000-11-28 1142 2000-11-28 diff --git a/sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c b/sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 b/sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 b/sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c b/sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 b/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 b/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 b/sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b b/sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 b/sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a b/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 b/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 b/sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 b/sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 b/sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 b/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b b/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee b/sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab b/sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 b/sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb b/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 b/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 b/sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f b/sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 b/sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 b/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f b/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 b/sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 b/sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 b/sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 b/sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 b/sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c b/sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf b/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf new file mode 100644 index 0000000000000..16c03e7276fec --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf @@ -0,0 +1,137 @@ +Baltimore New York 2010-10-20 -30.0 1064 +Baltimore New York 2010-10-20 23.0 1142 +Baltimore New York 2010-10-20 6.0 1599 +Chicago New York 2010-10-20 42.0 361 +Chicago New York 2010-10-20 24.0 897 +Chicago New York 2010-10-20 15.0 1531 +Chicago New York 2010-10-20 -6.0 1610 +Chicago New York 2010-10-20 -2.0 3198 +Baltimore New York 2010-10-21 17.0 1064 +Baltimore New York 2010-10-21 105.0 1142 +Baltimore New York 2010-10-21 28.0 1599 +Chicago New York 2010-10-21 142.0 361 +Chicago New York 2010-10-21 77.0 897 +Chicago New York 2010-10-21 53.0 1531 +Chicago New York 2010-10-21 -5.0 1610 +Chicago New York 2010-10-21 51.0 3198 +Baltimore New York 2010-10-22 -12.0 1064 +Baltimore New York 2010-10-22 54.0 1142 +Baltimore New York 2010-10-22 18.0 1599 +Chicago New York 2010-10-22 2.0 361 +Chicago New York 2010-10-22 24.0 897 +Chicago New York 2010-10-22 16.0 1531 +Chicago New York 2010-10-22 -6.0 1610 +Chicago New York 2010-10-22 -11.0 3198 +Baltimore New York 2010-10-23 18.0 272 +Baltimore New York 2010-10-23 -10.0 1805 +Baltimore New York 2010-10-23 6.0 3171 +Chicago New York 2010-10-23 3.0 384 +Chicago New York 2010-10-23 32.0 426 +Chicago New York 2010-10-23 1.0 650 +Chicago New York 2010-10-23 11.0 3085 +Baltimore New York 2010-10-24 12.0 1599 +Baltimore New York 2010-10-24 20.0 2571 +Chicago New York 2010-10-24 10.0 361 +Chicago New York 2010-10-24 113.0 897 +Chicago New York 2010-10-24 -5.0 1531 +Chicago New York 2010-10-24 -17.0 1610 +Chicago New York 2010-10-24 -3.0 3198 +Baltimore New York 2010-10-25 -25.0 1064 +Baltimore New York 2010-10-25 92.0 1142 +Baltimore New York 2010-10-25 106.0 1599 +Chicago New York 2010-10-25 31.0 361 +Chicago New York 2010-10-25 -1.0 897 +Chicago New York 2010-10-25 43.0 1531 +Chicago New York 2010-10-25 6.0 1610 +Chicago New York 2010-10-25 -16.0 3198 +Baltimore New York 2010-10-26 -22.0 1064 +Baltimore New York 2010-10-26 123.0 1142 +Baltimore New York 2010-10-26 90.0 1599 +Chicago New York 2010-10-26 12.0 361 +Chicago New York 2010-10-26 0.0 897 +Chicago New York 2010-10-26 29.0 1531 +Chicago New York 2010-10-26 -17.0 1610 +Chicago New York 2010-10-26 6.0 3198 +Baltimore New York 2010-10-27 -18.0 1064 +Baltimore New York 2010-10-27 49.0 1142 +Baltimore New York 2010-10-27 92.0 1599 +Chicago New York 2010-10-27 148.0 361 +Chicago New York 2010-10-27 -11.0 897 +Chicago New York 2010-10-27 70.0 1531 +Chicago New York 2010-10-27 8.0 1610 +Chicago New York 2010-10-27 21.0 3198 +Baltimore New York 2010-10-28 -4.0 1064 +Baltimore New York 2010-10-28 -14.0 1142 +Baltimore New York 2010-10-28 -14.0 1599 +Chicago New York 2010-10-28 2.0 361 +Chicago New York 2010-10-28 2.0 897 +Chicago New York 2010-10-28 -11.0 1531 +Chicago New York 2010-10-28 3.0 1610 +Chicago New York 2010-10-28 -18.0 3198 +Baltimore New York 2010-10-29 -24.0 1064 +Baltimore New York 2010-10-29 21.0 1142 +Baltimore New York 2010-10-29 -2.0 1599 +Chicago New York 2010-10-29 -12.0 361 +Chicago New York 2010-10-29 -11.0 897 +Chicago New York 2010-10-29 15.0 1531 +Chicago New York 2010-10-29 -18.0 1610 +Chicago New York 2010-10-29 -4.0 3198 +Baltimore New York 2010-10-30 14.0 272 +Baltimore New York 2010-10-30 -1.0 1805 +Baltimore New York 2010-10-30 5.0 3171 +Chicago New York 2010-10-30 -6.0 384 +Chicago New York 2010-10-30 -10.0 426 +Chicago New York 2010-10-30 -5.0 650 +Chicago New York 2010-10-30 -5.0 3085 +Baltimore New York 2010-10-31 -1.0 1599 +Baltimore New York 2010-10-31 -14.0 2571 +Chicago New York 2010-10-31 -25.0 361 +Chicago New York 2010-10-31 -18.0 897 +Chicago New York 2010-10-31 -4.0 1531 +Chicago New York 2010-10-31 -22.0 1610 +Chicago New York 2010-10-31 -15.0 3198 +Cleveland New York 2010-10-30 -23.0 2018 +Cleveland New York 2010-10-30 -12.0 2932 +Cleveland New York 2010-10-29 -4.0 2630 +Cleveland New York 2010-10-29 -19.0 2646 +Cleveland New York 2010-10-29 -12.0 3014 +Cleveland New York 2010-10-28 3.0 2630 +Cleveland New York 2010-10-28 -6.0 2646 +Cleveland New York 2010-10-28 1.0 3014 +Cleveland New York 2010-10-27 16.0 2630 +Cleveland New York 2010-10-27 27.0 3014 +Cleveland New York 2010-10-26 4.0 2630 +Cleveland New York 2010-10-26 -27.0 2646 +Cleveland New York 2010-10-26 -11.0 2662 +Cleveland New York 2010-10-26 13.0 3014 +Cleveland New York 2010-10-25 -4.0 2630 +Cleveland New York 2010-10-25 81.0 2646 +Cleveland New York 2010-10-25 42.0 3014 +Cleveland New York 2010-10-24 5.0 2254 +Cleveland New York 2010-10-24 -11.0 2630 +Cleveland New York 2010-10-24 -20.0 2646 +Cleveland New York 2010-10-24 -9.0 3014 +Cleveland New York 2010-10-23 -21.0 2932 +Cleveland New York 2010-10-22 1.0 2630 +Cleveland New York 2010-10-22 -25.0 2646 +Cleveland New York 2010-10-22 -3.0 3014 +Cleveland New York 2010-10-21 3.0 2630 +Cleveland New York 2010-10-21 29.0 2646 +Cleveland New York 2010-10-21 72.0 3014 +Cleveland New York 2010-10-20 -8.0 2630 +Cleveland New York 2010-10-20 -15.0 3014 +Washington New York 2010-10-23 -25.0 5832 +Washington New York 2010-10-23 -21.0 5904 +Washington New York 2010-10-23 -18.0 5917 +Washington New York 2010-10-30 -27.0 5904 +Washington New York 2010-10-30 -16.0 5917 +Washington New York 2010-10-20 -2.0 7291 +Washington New York 2010-10-21 22.0 7291 +Washington New York 2010-10-23 -16.0 7274 +Washington New York 2010-10-24 -26.0 7282 +Washington New York 2010-10-25 9.0 7291 +Washington New York 2010-10-26 4.0 7291 +Washington New York 2010-10-27 26.0 7291 +Washington New York 2010-10-28 45.0 7291 +Washington New York 2010-10-29 1.0 7291 +Washington New York 2010-10-31 -18.0 7282 diff --git a/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 b/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 new file mode 100644 index 0000000000000..0f2a6f7a99237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 @@ -0,0 +1,12 @@ +2010-10-20 11 +2010-10-21 12 +2010-10-22 11 +2010-10-23 12 +2010-10-24 12 +2010-10-25 12 +2010-10-26 13 +2010-10-27 11 +2010-10-28 12 +2010-10-29 12 +2010-10-30 11 +2010-10-31 8 diff --git a/sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 b/sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 b/sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b b/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b new file mode 100644 index 0000000000000..83c33400edb47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b @@ -0,0 +1 @@ +0 3333 -3333 -3332 3332 diff --git a/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc b/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc new file mode 100644 index 0000000000000..4a2462bb3929b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc @@ -0,0 +1 @@ +NULL 2011 5 6 6 18 2011-05-06 diff --git a/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff b/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff new file mode 100644 index 0000000000000..19497254f8f7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff @@ -0,0 +1 @@ +2011-05-11 2011-04-26 diff --git a/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 b/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 new file mode 100644 index 0000000000000..977f0d24c58cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 @@ -0,0 +1 @@ +0 3333 -3333 -3333 3333 diff --git a/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 b/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 new file mode 100644 index 0000000000000..44d1f45e4eb73 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 @@ -0,0 +1 @@ +1970-01-01 08:00:00 1969-12-31 16:00:00 2013-06-19 07:00:00 2013-06-18 17:00:00 diff --git a/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 b/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 new file mode 100644 index 0000000000000..645b71d8d61e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 @@ -0,0 +1 @@ +true true true true diff --git a/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 b/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 new file mode 100644 index 0000000000000..51863e9a14e4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 @@ -0,0 +1 @@ +2010-10-20 diff --git a/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 b/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 new file mode 100644 index 0000000000000..4043ee1cbdd40 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 @@ -0,0 +1 @@ +2010-10-31 diff --git a/sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 b/sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 b/sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 b/sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 b/sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd b/sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 b/sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a b/sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b b/sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 b/sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 b/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 new file mode 100644 index 0000000000000..18d17ea11b53e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 @@ -0,0 +1 @@ +1304665200 2011 5 6 6 18 2011-05-06 diff --git a/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 b/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 new file mode 100644 index 0000000000000..19497254f8f7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 @@ -0,0 +1 @@ +2011-05-11 2011-04-26 diff --git a/sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 b/sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe b/sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 b/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 b/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 @@ -0,0 +1 @@ +11 diff --git a/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 b/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 @@ -0,0 +1 @@ +30 diff --git a/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf b/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce b/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f b/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c b/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e b/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 b/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 b/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 b/sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 b/sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 b/sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 b/sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 b/sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc b/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc new file mode 100644 index 0000000000000..051ca3d3c28e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc @@ -0,0 +1,2 @@ +2000-01-01 +2013-08-08 diff --git a/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 b/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 @@ -0,0 +1,5 @@ +165 val_165 2000-01-01 2 +238 val_238 2000-01-01 2 +27 val_27 2000-01-01 2 +311 val_311 2000-01-01 2 +86 val_86 2000-01-01 2 diff --git a/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 b/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f b/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac b/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac new file mode 100644 index 0000000000000..91ba621412d72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac @@ -0,0 +1,6 @@ +1 11 2008-01-01 +2 12 2008-01-01 +3 13 2008-01-01 +7 17 2008-01-01 +8 18 2008-01-01 +8 28 2008-01-01 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d b/sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd b/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd new file mode 100644 index 0000000000000..7941f53d8d4c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd @@ -0,0 +1,40 @@ +1064 2000-11-20 +1064 2000-11-20 +1142 2000-11-21 +1142 2000-11-21 +1599 2000-11-22 +1599 2000-11-22 +361 2000-11-23 +361 2000-11-23 +897 2000-11-24 +897 2000-11-24 +1531 2000-11-25 +1531 2000-11-25 +1610 2000-11-26 +1610 2000-11-26 +3198 2000-11-27 +3198 2000-11-27 +1064 2000-11-28 +1064 2000-11-28 +1142 2000-11-28 +1142 2000-11-28 +1064 2010-10-20 +1064 2010-10-20 +1142 2010-10-21 +1142 2010-10-21 +1599 2010-10-22 +1599 2010-10-22 +361 2010-10-23 +361 2010-10-23 +897 2010-10-24 +897 2010-10-24 +1531 2010-10-25 +1531 2010-10-25 +1610 2010-10-26 +1610 2010-10-26 +3198 2010-10-27 +3198 2010-10-27 +1064 2010-10-28 +1064 2010-10-28 +1142 2010-10-29 +1142 2010-10-29 diff --git a/sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 b/sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c b/sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c new file mode 100644 index 0000000000000..e69de29bb2d1d From 56102dc2d849c221f325a7888cd66abb640ec887 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 13 Oct 2014 13:36:39 -0700 Subject: [PATCH 05/12] [SPARK-2066][SQL] Adds checks for non-aggregate attributes with aggregation This PR adds a new rule `CheckAggregation` to the analyzer to provide better error message for non-aggregate attributes with aggregation. Author: Cheng Lian Closes #2774 from liancheng/non-aggregate-attr and squashes the following commits: 5246004 [Cheng Lian] Passes test suites bf1878d [Cheng Lian] Adds checks for non-aggregate attributes with aggregation --- .../sql/catalyst/analysis/Analyzer.scala | 36 ++++++++++++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 26 ++++++++++++++ 2 files changed, 57 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fe83eb12502dc..82553063145b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -63,7 +63,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, - CheckResolution), + CheckResolution, + CheckAggregation), Batch("AnalysisOperators", fixedPoint, EliminateAnalysisOperators) ) @@ -88,6 +89,32 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * Checks for non-aggregated attributes with aggregation + */ + object CheckAggregation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + plan.transform { + case aggregatePlan @ Aggregate(groupingExprs, aggregateExprs, child) => + def isValidAggregateExpression(expr: Expression): Boolean = expr match { + case _: AggregateExpression => true + case e: Attribute => groupingExprs.contains(e) + case e if groupingExprs.contains(e) => true + case e if e.references.isEmpty => true + case e => e.children.forall(isValidAggregateExpression) + } + + aggregateExprs.foreach { e => + if (!isValidAggregateExpression(e)) { + throw new TreeNodeException(plan, s"Expression not in GROUP BY: $e") + } + } + + aggregatePlan + } + } + } + /** * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ @@ -204,18 +231,17 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) + case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) if aggregate.resolved && containsAggregate(havingCondition) => { val evaluatedCondition = Alias(havingCondition, "havingCondition")() val aggExprsWithHaving = evaluatedCondition +: originalAggExprs - + Project(aggregate.output, Filter(evaluatedCondition.toAttribute, aggregate.copy(aggregateExpressions = aggExprsWithHaving))) } - } - + protected def containsAggregate(condition: Expression): Boolean = condition .collect { case ae: AggregateExpression => ae } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a94022c0cf6e3..15f6ba4f72bbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.joins.BroadcastHashJoin import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll @@ -694,4 +695,29 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer( sql("SELECT CASE WHEN key = 1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"), 1) } + + test("throw errors for non-aggregate attributes with aggregation") { + def checkAggregation(query: String, isInvalidQuery: Boolean = true) { + val logicalPlan = sql(query).queryExecution.logical + + if (isInvalidQuery) { + val e = intercept[TreeNodeException[LogicalPlan]](sql(query).queryExecution.analyzed) + assert( + e.getMessage.startsWith("Expression not in GROUP BY"), + "Non-aggregate attribute(s) not detected\n" + logicalPlan) + } else { + // Should not throw + sql(query).queryExecution.analyzed + } + } + + checkAggregation("SELECT key, COUNT(*) FROM testData") + checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", false) + + checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") + checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) + + checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") + checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) + } } From d3cdf9128ae828dc7f1893439f66a0de68c6e527 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 13 Oct 2014 13:40:20 -0700 Subject: [PATCH 06/12] [SPARK-3529] [SQL] Delete the temp files after test exit There are lots of temporal files created by TestHive under the /tmp by default, which may cause potential performance issue for testing. This PR will automatically delete them after test exit. Author: Cheng Hao Closes #2393 from chenghao-intel/delete_temp_on_exit and squashes the following commits: 3a6511f [Cheng Hao] Remove the temp dir after text exit --- .../main/scala/org/apache/spark/sql/hive/TestHive.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a4354c1379c63..9a9e2eda6bcd4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ @@ -71,11 +72,14 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") setConf("hive.metastore.warehouse.dir", warehousePath) + Utils.registerShutdownDeleteDir(new File(warehousePath)) + Utils.registerShutdownDeleteDir(new File(metastorePath)) } val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp") testTempDir.delete() testTempDir.mkdir() + Utils.registerShutdownDeleteDir(testTempDir) // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) @@ -121,8 +125,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() - hiveFilesTemp.deleteOnExit() - + Utils.registerShutdownDeleteDir(hiveFilesTemp) val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) From 73da9c26b0e2e8bf0ab055906211727a7097c963 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 13 Oct 2014 13:43:41 -0700 Subject: [PATCH 07/12] [SPARK-3771][SQL] AppendingParquetOutputFormat should use reflection to prevent from breaking binary-compatibility. Original problem is [SPARK-3764](https://issues.apache.org/jira/browse/SPARK-3764). `AppendingParquetOutputFormat` uses a binary-incompatible method `context.getTaskAttemptID`. This causes binary-incompatible of Spark itself, i.e. if Spark itself is built against hadoop-1, the artifact is for only hadoop-1, and vice versa. Author: Takuya UESHIN Closes #2638 from ueshin/issues/SPARK-3771 and squashes the following commits: efd3784 [Takuya UESHIN] Add a comment to explain the reason to use reflection. ec213c1 [Takuya UESHIN] Use reflection to prevent breaking binary-compatibility. --- .../spark/sql/parquet/ParquetTableOperations.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ffb732347d30a..1f4237d7ede65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -331,13 +331,21 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) // override to choose output filename so not overwrite existing ones override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val taskId: TaskID = context.getTaskAttemptID.getTaskID + val taskId: TaskID = getTaskAttemptID(context).getTaskID val partition: Int = taskId.getId val filename = s"part-r-${partition + offset}.parquet" val committer: FileOutputCommitter = getOutputCommitter(context).asInstanceOf[FileOutputCommitter] new Path(committer.getWorkPath, filename) } + + // The TaskAttemptContext is a class in hadoop-1 but is an interface in hadoop-2. + // The signatures of the method TaskAttemptContext.getTaskAttemptID for the both versions + // are the same, so the method calls are source-compatible but NOT binary-compatible because + // the opcode of method call for class is INVOKEVIRTUAL and for interface is INVOKEINTERFACE. + private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { + context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] + } } /** From e10d71e7e58bf2ec0f1942cb2f0602396ab866b4 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Mon, 13 Oct 2014 13:45:34 -0700 Subject: [PATCH 08/12] [SPARK-3559][SQL] Remove unnecessary columns from List of needed Column Ids in Hive Conf Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #2713 from gvramana/remove_unnecessary_columns and squashes the following commits: b7ba768 [Venkata Ramana Gollamudi] Added comment and checkstyle fix 6a93459 [Venkata Ramana Gollamudi] cloned hiveconf for each TableScanOperators so that only required columns are added --- .../scala/org/apache/spark/sql/hive/TableReader.scala | 6 ++++-- .../spark/sql/hive/execution/HiveTableScan.scala | 10 ++++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 84fafcde63d05..0de29d5cffd0e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} @@ -52,7 +53,8 @@ private[hive] class HadoopTableReader( @transient attributes: Seq[Attribute], @transient relation: MetastoreRelation, - @transient sc: HiveContext) + @transient sc: HiveContext, + @transient hiveExtraConf: HiveConf) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -63,7 +65,7 @@ class HadoopTableReader( // TODO: set aws s3 credentials. private val _broadcastedHiveConf = - sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) + sc.sparkContext.broadcast(new SerializableWritable(hiveExtraConf)) def broadcastedHiveConf = _broadcastedHiveConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 577ca928b43b6..a32147584f6f4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -64,8 +64,14 @@ case class HiveTableScan( BindReferences.bindReference(pred, relation.partitionKeys) } + // Create a local copy of hiveconf,so that scan specific modifications should not impact + // other queries @transient - private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context) + private[this] val hiveExtraConf = new HiveConf(context.hiveconf) + + @transient + private[this] val hadoopReader = + new HadoopTableReader(attributes, relation, context, hiveExtraConf) private[this] def castFromString(value: String, dataType: DataType) = { Cast(Literal(value), dataType).eval(null) @@ -97,7 +103,7 @@ case class HiveTableScan( hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } - addColumnMetadataToConf(context.hiveconf) + addColumnMetadataToConf(hiveExtraConf) /** * Prunes partitions not involve the query plan. From 371321cadee7df39258bd374eb59c1e32451d96b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 13 Oct 2014 13:46:34 -0700 Subject: [PATCH 09/12] [SQL] Add type checking debugging functions Adds some functions that were very useful when trying to track down the bug from #2656. This change also changes the tree output for query plans to include the `'` prefix to unresolved nodes and `!` prefix to nodes that refer to non-existent attributes. Author: Michael Armbrust Closes #2657 from marmbrus/debugging and squashes the following commits: 654b926 [Michael Armbrust] Clean-up, add tests 763af15 [Michael Armbrust] Add typeChecking debugging functions 8c69303 [Michael Armbrust] Add inputSet, references to QueryPlan. Improve tree string with a prefix to denote invalid or unresolved nodes. fbeab54 [Michael Armbrust] Better toString, factories for AttributeSet. --- .../catalyst/expressions/AttributeSet.scala | 23 +++-- .../sql/catalyst/expressions/Projection.scala | 2 + .../expressions/namedExpressions.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 23 +++++ .../catalyst/plans/logical/LogicalPlan.scala | 8 +- .../plans/logical/basicOperators.scala | 5 -- .../spark/sql/execution/debug/package.scala | 85 +++++++++++++++++++ .../sql/execution/debug/DebuggingSuite.scala | 33 +++++++ 8 files changed, 163 insertions(+), 20 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index c3a08bbdb6bc7..2b4969b7cfec0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -17,19 +17,26 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.analysis.Star + protected class AttributeEquals(val a: Attribute) { override def hashCode() = a.exprId.hashCode() - override def equals(other: Any) = other match { - case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId - case otherAttribute => false + override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match { + case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId + case (a1, a2) => a1 == a2 } } object AttributeSet { - /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ - def apply(baseSet: Seq[Attribute]) = { - new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) - } + def apply(a: Attribute) = + new AttributeSet(Set(new AttributeEquals(a))) + + /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ + def apply(baseSet: Seq[Expression]) = + new AttributeSet( + baseSet + .flatMap(_.references) + .map(new AttributeEquals(_)).toSet) } /** @@ -103,4 +110,6 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all // sorts of things in its closure. override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq + + override def toString = "{" + baseSet.map(_.a).mkString(", ") + "}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 204904ecf04db..e7e81a21fdf03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -39,6 +39,8 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { } new GenericRow(outputArray) } + + override def toString = s"Row => [${exprArray.mkString(",")}]" } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index e5a958d599393..d023db44d8543 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -57,6 +57,8 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => + override def references = AttributeSet(this) + def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute @@ -116,8 +118,6 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { - override def references = AttributeSet(this :: Nil) - override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index af9e4d86e995a..dcbbb62c0aca4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -31,6 +31,25 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy */ def outputSet: AttributeSet = AttributeSet(output) + /** + * All Attributes that appear in expressions from this operator. Note that this set does not + * include attributes that are implicitly referenced by being passed through to the output tuple. + */ + def references: AttributeSet = AttributeSet(expressions.flatMap(_.references)) + + /** + * The set of all attributes that are input to this operator by its children. + */ + def inputSet: AttributeSet = + AttributeSet(children.flatMap(_.asInstanceOf[QueryPlan[PlanType]].output)) + + /** + * Attributes that are referenced by expressions but not provided by this nodes children. + * Subclasses should override this method if they produce attributes internally as it is used by + * assertions designed to prevent the construction of invalid plans. + */ + def missingInput: AttributeSet = references -- inputSet + /** * Runs [[transform]] with `rule` on all expressions present in this query operator. * Users should not expect a specific directionality. If a specific directionality is needed, @@ -132,4 +151,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** Prints out the schema in the tree format */ def printSchema(): Unit = println(schemaString) + + protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" + + override def simpleString = statePrefix + super.simpleString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 4f8ad8a7e0223..882e9c6110089 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -53,12 +53,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) } - /** - * Returns the set of attributes that this node takes as - * input from its children. - */ - lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) - /** * Returns true if this expression and all its children have been resolved to a specific schema * and false if it still contains any unresolved placeholders. Implementations of LogicalPlan @@ -68,6 +62,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { */ lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved + override protected def statePrefix = if (!resolved) "'" else super.statePrefix + /** * Returns true if all its children of this query plan have been resolved. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index f8e9930ac270d..14b03c7445c13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -138,11 +138,6 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { - /** The set of all AttributeReferences required for this aggregation. */ - def references = - AttributeSet( - groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) - override def output = aggregateExpressions.map(_.toAttribute) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index a9535a750bcd7..61be5ed2db65c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext._ import org.apache.spark.sql.{SchemaRDD, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.types._ /** * :: DeveloperApi :: @@ -56,6 +57,23 @@ package object debug { case _ => } } + + def typeCheck(): Unit = { + val plan = query.queryExecution.executedPlan + val visited = new collection.mutable.HashSet[TreeNodeRef]() + val debugPlan = plan transform { + case s: SparkPlan if !visited.contains(new TreeNodeRef(s)) => + visited += new TreeNodeRef(s) + TypeCheck(s) + } + try { + println(s"Results returned: ${debugPlan.execute().count()}") + } catch { + case e: Exception => + def unwrap(e: Throwable): Throwable = if (e.getCause == null) e else unwrap(e.getCause) + println(s"Deepest Error: ${unwrap(e)}") + } + } } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { @@ -115,4 +133,71 @@ package object debug { } } } + + /** + * :: DeveloperApi :: + * Helper functions for checking that runtime types match a given schema. + */ + @DeveloperApi + object TypeCheck { + def typeCheck(data: Any, schema: DataType): Unit = (data, schema) match { + case (null, _) => + + case (row: Row, StructType(fields)) => + row.zip(fields.map(_.dataType)).foreach { case(d,t) => typeCheck(d,t) } + case (s: Seq[_], ArrayType(elemType, _)) => + s.foreach(typeCheck(_, elemType)) + case (m: Map[_, _], MapType(keyType, valueType, _)) => + m.keys.foreach(typeCheck(_, keyType)) + m.values.foreach(typeCheck(_, valueType)) + + case (_: Long, LongType) => + case (_: Int, IntegerType) => + case (_: String, StringType) => + case (_: Float, FloatType) => + case (_: Byte, ByteType) => + case (_: Short, ShortType) => + case (_: Boolean, BooleanType) => + case (_: Double, DoubleType) => + + case (d, t) => sys.error(s"Invalid data found: got $d (${d.getClass}) expected $t") + } + } + + /** + * :: DeveloperApi :: + * Augments SchemaRDDs with debug methods. + */ + @DeveloperApi + private[sql] case class TypeCheck(child: SparkPlan) extends SparkPlan { + import TypeCheck._ + + override def nodeName = "" + + /* Only required when defining this class in a REPL. + override def makeCopy(args: Array[Object]): this.type = + TypeCheck(args(0).asInstanceOf[SparkPlan]).asInstanceOf[this.type] + */ + + def output = child.output + + def children = child :: Nil + + def execute() = { + child.execute().map { row => + try typeCheck(row, child.schema) catch { + case e: Exception => + sys.error( + s""" + |ERROR WHEN TYPE CHECKING QUERY + |============================== + |$e + |======== BAD TREE ============ + |$child + """.stripMargin) + } + row + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala new file mode 100644 index 0000000000000..87c28c334d228 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -0,0 +1,33 @@ +/* + * 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.debug + +import org.scalatest.FunSuite + +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext._ + +class DebuggingSuite extends FunSuite { + test("SchemaRDD.debug()") { + testData.debug() + } + + test("SchemaRDD.typeCheck()") { + testData.typeCheck() + } +} \ No newline at end of file From e6e37701f12be82fa77dc28d825ddd36a1ab7594 Mon Sep 17 00:00:00 2001 From: chirag Date: Mon, 13 Oct 2014 13:47:26 -0700 Subject: [PATCH 10/12] SPARK-3807: SparkSql does not work for tables created using custom serde SparkSql crashes on selecting tables using custom serde. Example: ---------------- CREATE EXTERNAL TABLE table_name PARTITIONED BY ( a int) ROW FORMAT 'SERDE "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" with serdeproperties("serialization.format"="org.apache.thrift.protocol.TBinaryProtocol","serialization.class"="ser_class") STORED AS SEQUENCEFILE; The following exception is seen on running a query like 'select * from table_name limit 1': ERROR CliDriver: org.apache.hadoop.hive.serde2.SerDeException: java.lang.NullPointerException at org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer.initialize(ThriftDeserializer.java:68) at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializer(TableDesc.java:80) at org.apache.spark.sql.hive.execution.HiveTableScan.addColumnMetadataToConf(HiveTableScan.scala:86) at org.apache.spark.sql.hive.execution.HiveTableScan.(HiveTableScan.scala:100) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.SQLContext$SparkPlanner.pruneFilterProject(SQLContext.scala:364) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$.apply(HiveStrategies.scala:184) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:280) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:402) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:400) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:406) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:406) at org.apache.spark.sql.hive.HiveContext$QueryExecution.stringResult(HiveContext.scala:406) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:59) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:291) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:226) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source) at java.lang.reflect.Method.invoke(Unknown Source) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:328) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:75) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.NullPointerException Author: chirag Closes #2674 from chiragaggarwal/branch-1.1 and squashes the following commits: 370c31b [chirag] SPARK-3807: Add a test case to validate the fix. 1f26805 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (Incorporated Review Comments) ba4bc0c [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde 5c73b72 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (cherry picked from commit 925e22d3132b983a2fcee31e3878b680c7ff92da) Signed-off-by: Michael Armbrust --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/execution/HiveTableScan.scala | 6 +++++- .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 3 +++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index c5fee5e4702f6..75a19656af110 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -305,7 +305,7 @@ private[hive] case class MetastoreRelation val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) /** Non-partitionKey attributes */ - val attributes = table.getSd.getCols.map(_.toAttribute) + val attributes = hiveQlTable.getCols.map(_.toAttribute) val output = attributes ++ partitionKeys } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index a32147584f6f4..5b83b77d80a22 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -86,10 +86,14 @@ case class HiveTableScan( ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + val tableDesc = relation.tableDesc + val deserializer = tableDesc.getDeserializerClass.newInstance + deserializer.initialize(hiveConf, tableDesc.getProperties) + // Specifies types and object inspectors of columns to be scanned. val structOI = ObjectInspectorUtils .getStandardObjectInspector( - relation.tableDesc.getDeserializer.getObjectInspector, + deserializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2829105f43716..3e100775e4981 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -802,6 +802,9 @@ class HiveQuerySuite extends HiveComparisonTest { clear() } + createQueryTest("select from thrift based table", + "SELECT * from src_thrift") + // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. } From 9d9ca91fef70eca6fc576be9c99aed5d8ce6e68b Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 13 Oct 2014 13:49:11 -0700 Subject: [PATCH 11/12] [SQL]Small bug in unresolved.scala name should throw exception with name instead of exprId. Author: Liquan Pei Closes #2758 from Ishiihara/SparkSQL-bug and squashes the following commits: aa36a3b [Liquan Pei] small bug --- .../org/apache/spark/sql/catalyst/analysis/unresolved.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 67570a6f73c36..77d84e1687e1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -88,7 +88,7 @@ case class Star( mapFunction: Attribute => Expression = identity[Attribute]) extends Attribute with trees.LeafNode[Expression] { - override def name = throw new UnresolvedException(this, "exprId") + override def name = throw new UnresolvedException(this, "name") override def exprId = throw new UnresolvedException(this, "exprId") override def dataType = throw new UnresolvedException(this, "dataType") override def nullable = throw new UnresolvedException(this, "nullable") From 9eb49d4134e23a15142fb592d54d920e89bd8786 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 13 Oct 2014 13:50:27 -0700 Subject: [PATCH 12/12] [SPARK-3809][SQL] Fixes test suites in hive-thriftserver As scwf pointed out, `HiveThriftServer2Suite` isn't effective anymore after the Thrift server was made a daemon. On the other hand, these test suites were known flaky, PR #2214 tried to fix them but failed because of unknown Jenkins build error. This PR fixes both sets of issues. In this PR, instead of watching `start-thriftserver.sh` output, the test code start a `tail` process to watch the log file. A `Thread.sleep` has to be introduced because the `kill` command used in `stop-thriftserver.sh` is not synchronous. As for the root cause of the mysterious Jenkins build failure. Please refer to [this comment](https://github.com/apache/spark/pull/2675#issuecomment-58464189) below for details. ---- (Copied from PR description of #2214) This PR fixes two issues of `HiveThriftServer2Suite` and brings 1 enhancement: 1. Although metastore, warehouse directories and listening port are randomly chosen, all test cases share the same configuration. Due to parallel test execution, one of the two test case is doomed to fail 2. We caught any exceptions thrown from a test case and print diagnosis information, but forgot to re-throw the exception... 3. When the forked server process ends prematurely (e.g., fails to start), the `serverRunning` promise is completed with a failure, preventing the test code to keep waiting until timeout. So, embarrassingly, this test suite was failing continuously for several days but no one had ever noticed it... Fortunately no bugs in the production code were covered under the hood. Author: Cheng Lian Author: wangfei Closes #2675 from liancheng/fix-thriftserver-tests and squashes the following commits: 1c384b7 [Cheng Lian] Minor code cleanup, restore the logging level hack in TestHive.scala 7805c33 [wangfei] reset SPARK_TESTING to avoid loading Log4J configurations in testing class paths af2b5a9 [Cheng Lian] Removes log level hacks from TestHiveContext d116405 [wangfei] make sure that log4j level is INFO ee92a82 [Cheng Lian] Relaxes timeout 7fd6757 [Cheng Lian] Fixes test suites in hive-thriftserver --- .../sql/hive/thriftserver/CliSuite.scala | 13 ++- .../thriftserver/HiveThriftServer2Suite.scala | 86 +++++++++++-------- 2 files changed, 60 insertions(+), 39 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d68dd090b5e6c..fc97a25be34be 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.sql.catalyst.util.getTempFilePath class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { @@ -62,8 +62,11 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { def captureOutput(source: String)(line: String) { buffer += s"$source> $line" + // If we haven't found all expected answers... if (next.get() < expectedAnswers.size) { + // If another expected answer is found... if (line.startsWith(expectedAnswers(next.get()))) { + // If all expected answers have been found... if (next.incrementAndGet() == expectedAnswers.size) { foundAllExpectedAnswers.trySuccess(()) } @@ -77,7 +80,8 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { Future { val exitValue = process.exitValue() - logInfo(s"Spark SQL CLI process exit value: $exitValue") + foundAllExpectedAnswers.tryFailure( + new SparkException(s"Spark SQL CLI process exit value: $exitValue")) } try { @@ -98,6 +102,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { |End CliSuite failure output |=========================== """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() @@ -109,7 +114,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - runCliWithin(1.minute)( + runCliWithin(3.minute)( "CREATE TABLE hive_test(key INT, val STRING);" -> "OK", "SHOW TABLES;" @@ -120,7 +125,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { -> "Time taken: ", "SELECT COUNT(*) FROM hive_test;" -> "5", - "DROP TABLE hive_test" + "DROP TABLE hive_test;" -> "Time taken: " ) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 38977ff162097..e3b4e45a3d68c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration._ -import scala.concurrent.{Await, Future, Promise} -import scala.sys.process.{Process, ProcessLogger} - import java.io.File import java.net.ServerSocket import java.sql.{DriverManager, Statement} import java.util.concurrent.TimeoutException +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.concurrent.{Await, Promise} +import scala.sys.process.{Process, ProcessLogger} +import scala.util.Try + import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver import org.scalatest.FunSuite @@ -41,25 +41,25 @@ import org.apache.spark.sql.catalyst.util.getTempFilePath class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) - private val listeningHost = "localhost" - private val listeningPort = { - // Let the system to choose a random available port to avoid collision with other parallel - // builds. - val socket = new ServerSocket(0) - val port = socket.getLocalPort - socket.close() - port - } - - private val warehousePath = getTempFilePath("warehouse") - private val metastorePath = getTempFilePath("metastore") - private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - - def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { - val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + def startThriftServerWithin(timeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) + + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + val listeningHost = "localhost" + val listeningPort = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } val command = - s"""$serverScript + s"""$startScript | --master local | --hiveconf hive.root.logger=INFO,console | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri @@ -68,29 +68,40 @@ class HiveThriftServer2Suite extends FunSuite with Logging { | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort """.stripMargin.split("\\s+").toSeq - val serverStarted = Promise[Unit]() + val serverRunning = Promise[Unit]() val buffer = new ArrayBuffer[String]() + val LOGGING_MARK = + s"starting ${HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$")}, logging to " + var logTailingProcess: Process = null + var logFilePath: String = null - def captureOutput(source: String)(line: String) { - buffer += s"$source> $line" + def captureLogOutput(line: String): Unit = { + buffer += line if (line.contains("ThriftBinaryCLIService listening on")) { - serverStarted.success(()) + serverRunning.success(()) } } - val process = Process(command).run( - ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - - Future { - val exitValue = process.exitValue() - logInfo(s"Spark SQL Thrift server process exit value: $exitValue") + def captureThriftServerOutput(source: String)(line: String): Unit = { + if (line.startsWith(LOGGING_MARK)) { + logFilePath = line.drop(LOGGING_MARK.length).trim + // Ensure that the log file is created so that the `tail' command won't fail + Try(new File(logFilePath).createNewFile()) + logTailingProcess = Process(s"/usr/bin/env tail -f $logFilePath") + .run(ProcessLogger(captureLogOutput, _ => ())) + } } + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + Process(command, None, "SPARK_TESTING" -> "0").run(ProcessLogger( + captureThriftServerOutput("stdout"), + captureThriftServerOutput("stderr"))) + val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" val user = System.getProperty("user.name") try { - Await.result(serverStarted.future, timeout) + Await.result(serverRunning.future, timeout) val connection = DriverManager.getConnection(jdbcUri, user, "") val statement = connection.createStatement() @@ -122,10 +133,15 @@ class HiveThriftServer2Suite extends FunSuite with Logging { |End HiveThriftServer2Suite failure output |========================================= """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() - process.destroy() + Process(stopScript).run().exitValue() + // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while. + Thread.sleep(3.seconds.toMillis) + Option(logTailingProcess).map(_.destroy()) + Option(logFilePath).map(new File(_).delete()) } }