/
HiveShim.scala
214 lines (190 loc) · 7.49 KB
/
HiveShim.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
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
/*
* 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.hive
import java.rmi.server.UID
import scala.jdk.CollectionConverters._
import com.google.common.base.Objects
import org.apache.avro.Schema
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.ql.exec.SerializationUtilities
import org.apache.hadoop.hive.ql.exec.UDF
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils}
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector
import org.apache.hadoop.io.Writable
import org.apache.spark.sql.types.Decimal
import org.apache.spark.util.Utils
private[hive] object HiveShim {
val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro"
/*
* This function in hive-0.13 become private, but we have to do this to work around hive bug
*/
private def appendReadColumnNames(conf: Configuration, cols: Seq[String]): Unit = {
val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "")
val result: StringBuilder = new StringBuilder(old)
var first: Boolean = old.isEmpty
for (col <- cols) {
if (first) {
first = false
} else {
result.append(',')
}
result.append(col)
}
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString)
}
/*
* Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null
*/
def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]): Unit = {
if (ids != null) {
ColumnProjectionUtils.appendReadColumns(conf, ids.asJava)
}
if (names != null) {
appendReadColumnNames(conf, names)
}
}
/*
* Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that
* is needed to initialize before serialization.
*/
def prepareWritable(w: Writable, serDeProps: Seq[(String, String)]): Writable = {
w match {
case w: AvroGenericRecordWritable =>
w.setRecordReaderID(new UID())
// In Hive 1.1, the record's schema may need to be initialized manually or a NPE will
// be thrown.
if (w.getFileSchema() == null) {
serDeProps
.find(_._1 == AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName())
.foreach { kv =>
w.setFileSchema(new Schema.Parser().parse(kv._2))
}
}
case _ =>
}
w
}
def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = {
if (hdoi.preferWritable()) {
val value = hdoi.getPrimitiveWritableObject(data)
if (value == null) {
null
} else {
Decimal(value.getHiveDecimal().bigDecimalValue, hdoi.precision(), hdoi.scale())
}
} else {
val value = hdoi.getPrimitiveJavaObject(data)
if (value == null) {
null
} else {
Decimal(value.bigDecimalValue(), hdoi.precision(), hdoi.scale())
}
}
}
/**
* This class provides the UDF creation and also the UDF instance serialization and
* de-serialization cross process boundary.
*
* Detail discussion can be found at https://github.com/apache/spark/pull/3640
*
* @param functionClassName UDF class name
* @param instance optional UDF instance which contains additional information (for macro)
* @param clazz optional class instance to create UDF instance
*/
private[hive] case class HiveFunctionWrapper(
var functionClassName: String,
private var instance: AnyRef = null,
private var clazz: Class[_ <: AnyRef] = null) extends java.io.Externalizable {
// for Serialization
def this() = this(null)
override def hashCode(): Int = {
if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) {
Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody())
} else {
functionClassName.hashCode()
}
}
override def equals(other: Any): Boolean = other match {
case a: HiveFunctionWrapper if functionClassName == a.functionClassName =>
// In case of udf macro, check to make sure they point to the same underlying UDF
if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) {
a.instance.asInstanceOf[GenericUDFMacro].getBody() ==
instance.asInstanceOf[GenericUDFMacro].getBody()
} else {
true
}
case _ => false
}
def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = {
SerializationUtilities.deserializePlan(is, clazz).asInstanceOf[UDFType]
}
def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = {
SerializationUtilities.serializePlan(function, out)
}
def writeExternal(out: java.io.ObjectOutput): Unit = {
// output the function name
out.writeUTF(functionClassName)
// Write a flag if instance is null or not
out.writeBoolean(instance != null)
if (instance != null) {
// Some of the UDF are serializable, but some others are not
// Hive Utilities can handle both cases
val baos = new java.io.ByteArrayOutputStream()
serializePlan(instance, baos)
val functionInBytes = baos.toByteArray
// output the function bytes
out.writeInt(functionInBytes.length)
out.write(functionInBytes, 0, functionInBytes.length)
}
}
def readExternal(in: java.io.ObjectInput): Unit = {
// read the function name
functionClassName = in.readUTF()
if (in.readBoolean()) {
// if the instance is not null
// read the function in bytes
val functionInBytesLength = in.readInt()
val functionInBytes = new Array[Byte](functionInBytesLength)
in.readFully(functionInBytes)
// deserialize the function object via Hive Utilities
clazz = Utils.getContextOrSparkClassLoader.loadClass(functionClassName)
.asInstanceOf[Class[_ <: AnyRef]]
instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes),
clazz)
}
}
def createFunction[UDFType <: AnyRef](): UDFType = {
if (instance != null) {
instance.asInstanceOf[UDFType]
} else {
if (clazz == null) {
clazz = Utils.getContextOrSparkClassLoader.loadClass(functionClassName)
.asInstanceOf[Class[_ <: AnyRef]]
}
val func = clazz.getConstructor().newInstance().asInstanceOf[UDFType]
if (!func.isInstanceOf[UDF]) {
// We cache the function if it's no the Simple UDF,
// as we always have to create new instance for Simple UDF
instance = func
}
func
}
}
}
}