/
TypeUtils.scala
124 lines (111 loc) · 4.59 KB
/
TypeUtils.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst.util
import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion}
import org.apache.spark.sql.catalyst.expressions.RowOrdering
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.types._
/**
* Functions to help with checking for valid data types and value comparison of various types.
*/
object TypeUtils {
def checkForNumericExpr(dt: DataType, caller: String): TypeCheckResult = {
if (dt.isInstanceOf[NumericType] || dt == NullType) {
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure(s"$caller requires numeric types, not ${dt.catalogString}")
}
}
def checkForOrderingExpr(dt: DataType, caller: String): TypeCheckResult = {
if (RowOrdering.isOrderable(dt)) {
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure(
s"$caller does not support ordering on type ${dt.catalogString}")
}
}
def checkForSameTypeInputExpr(types: Seq[DataType], caller: String): TypeCheckResult = {
if (TypeCoercion.haveSameType(types)) {
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure(
s"input to $caller should all be the same type, but it's " +
types.map(_.catalogString).mkString("[", ", ", "]"))
}
}
def checkForMapKeyType(keyType: DataType): TypeCheckResult = {
if (keyType.existsRecursively(_.isInstanceOf[MapType])) {
TypeCheckResult.TypeCheckFailure("The key of map cannot be/contain map.")
} else {
TypeCheckResult.TypeCheckSuccess
}
}
def checkForAnsiIntervalOrNumericType(
dt: DataType, funcName: String): TypeCheckResult = dt match {
case YearMonthIntervalType | DayTimeIntervalType | NullType => TypeCheckResult.TypeCheckSuccess
case dt if dt.isInstanceOf[NumericType] => TypeCheckResult.TypeCheckSuccess
case other => TypeCheckResult.TypeCheckFailure(
s"function $funcName requires numeric or interval types, not ${other.catalogString}")
}
def getNumeric(t: DataType, exactNumericRequired: Boolean = false): Numeric[Any] = {
if (exactNumericRequired) {
t.asInstanceOf[NumericType].exactNumeric.asInstanceOf[Numeric[Any]]
} else {
t.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]]
}
}
def getInterpretedOrdering(t: DataType): Ordering[Any] = {
t match {
case i: AtomicType => i.ordering.asInstanceOf[Ordering[Any]]
case a: ArrayType => a.interpretedOrdering.asInstanceOf[Ordering[Any]]
case s: StructType => s.interpretedOrdering.asInstanceOf[Ordering[Any]]
case udt: UserDefinedType[_] => getInterpretedOrdering(udt.sqlType)
}
}
def compareBinary(x: Array[Byte], y: Array[Byte]): Int = {
val limit = if (x.length <= y.length) x.length else y.length
var i = 0
while (i < limit) {
val res = (x(i) & 0xff) - (y(i) & 0xff)
if (res != 0) return res
i += 1
}
x.length - y.length
}
/**
* Returns true if the equals method of the elements of the data type is implemented properly.
* This also means that they can be safely used in collections relying on the equals method,
* as sets or maps.
*/
def typeWithProperEquals(dataType: DataType): Boolean = dataType match {
case BinaryType => false
case _: AtomicType => true
case _ => false
}
def failWithIntervalType(dataType: DataType): Unit = {
invokeOnceForInterval(dataType) {
throw QueryCompilationErrors.cannotUseIntervalTypeInTableSchemaError()
}
}
def invokeOnceForInterval(dataType: DataType)(f: => Unit): Unit = {
def isInterval(dataType: DataType): Boolean = dataType match {
case CalendarIntervalType | DayTimeIntervalType | YearMonthIntervalType => true
case _ => false
}
if (dataType.existsRecursively(isInterval)) f
}
}