-
Notifications
You must be signed in to change notification settings - Fork 28k
/
SQLUtils.scala
287 lines (253 loc) · 10.2 KB
/
SQLUtils.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
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
/*
* 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.r
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream}
import java.util.{Map => JMap}
import scala.collection.JavaConverters._
import scala.util.matching.Regex
import org.apache.spark.internal.Logging
import org.apache.spark.SparkContext
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.r.SerDe
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
import org.apache.spark.sql.execution.command.ShowTablesCommand
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
import org.apache.spark.sql.types._
private[sql] object SQLUtils extends Logging {
SerDe.setSQLReadObject(readSqlObject).setSQLWriteObject(writeSqlObject)
private[this] def withHiveExternalCatalog(sc: SparkContext): SparkContext = {
sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive")
sc
}
def getOrCreateSparkSession(
jsc: JavaSparkContext,
sparkConfigMap: JMap[Object, Object],
enableHiveSupport: Boolean): SparkSession = {
val spark = if (SparkSession.hiveClassesArePresent && enableHiveSupport) {
SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate()
} else {
if (enableHiveSupport) {
logWarning("SparkR: enableHiveSupport is requested for SparkSession but " +
"Spark is not built with Hive; falling back to without Hive support.")
}
SparkSession.builder().sparkContext(jsc.sc).getOrCreate()
}
setSparkContextSessionConf(spark, sparkConfigMap)
spark
}
def setSparkContextSessionConf(
spark: SparkSession,
sparkConfigMap: JMap[Object, Object]): Unit = {
for ((name, value) <- sparkConfigMap.asScala) {
spark.sessionState.conf.setConfString(name.toString, value.toString)
}
for ((name, value) <- sparkConfigMap.asScala) {
spark.sparkContext.conf.set(name.toString, value.toString)
}
}
def getSessionConf(spark: SparkSession): JMap[String, String] = {
spark.conf.getAll.asJava
}
def getJavaSparkContext(spark: SparkSession): JavaSparkContext = {
new JavaSparkContext(spark.sparkContext)
}
def createStructType(fields : Seq[StructField]): StructType = {
StructType(fields)
}
// Support using regex in string interpolation
private[this] implicit class RegexContext(sc: StringContext) {
def r: Regex = new Regex(sc.parts.mkString, sc.parts.tail.map(_ => "x"): _*)
}
def getSQLDataType(dataType: String): DataType = {
dataType match {
case "byte" => org.apache.spark.sql.types.ByteType
case "integer" => org.apache.spark.sql.types.IntegerType
case "float" => org.apache.spark.sql.types.FloatType
case "double" => org.apache.spark.sql.types.DoubleType
case "numeric" => org.apache.spark.sql.types.DoubleType
case "character" => org.apache.spark.sql.types.StringType
case "string" => org.apache.spark.sql.types.StringType
case "binary" => org.apache.spark.sql.types.BinaryType
case "raw" => org.apache.spark.sql.types.BinaryType
case "logical" => org.apache.spark.sql.types.BooleanType
case "boolean" => org.apache.spark.sql.types.BooleanType
case "timestamp" => org.apache.spark.sql.types.TimestampType
case "date" => org.apache.spark.sql.types.DateType
case r"\Aarray<(.+)${elemType}>\Z" =>
org.apache.spark.sql.types.ArrayType(getSQLDataType(elemType))
case r"\Amap<(.+)${keyType},(.+)${valueType}>\Z" =>
if (keyType != "string" && keyType != "character") {
throw new IllegalArgumentException("Key type of a map must be string or character")
}
org.apache.spark.sql.types.MapType(getSQLDataType(keyType), getSQLDataType(valueType))
case r"\Astruct<(.+)${fieldsStr}>\Z" =>
if (fieldsStr(fieldsStr.length - 1) == ',') {
throw new IllegalArgumentException(s"Invalid type $dataType")
}
val fields = fieldsStr.split(",")
val structFields = fields.map { field =>
field match {
case r"\A(.+)${fieldName}:(.+)${fieldType}\Z" =>
createStructField(fieldName, fieldType, true)
case _ => throw new IllegalArgumentException(s"Invalid type $dataType")
}
}
createStructType(structFields)
case _ => throw new IllegalArgumentException(s"Invalid type $dataType")
}
}
def createStructField(name: String, dataType: String, nullable: Boolean): StructField = {
val dtObj = getSQLDataType(dataType)
StructField(name, dtObj, nullable)
}
def createDF(rdd: RDD[Array[Byte]], schema: StructType, sparkSession: SparkSession): DataFrame = {
val num = schema.fields.length
val rowRDD = rdd.map(bytesToRow(_, schema))
sparkSession.createDataFrame(rowRDD, schema)
}
def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = {
df.rdd.map(r => rowToRBytes(r))
}
private[this] def doConversion(data: Object, dataType: DataType): Object = {
data match {
case d: java.lang.Double if dataType == FloatType =>
new java.lang.Float(d)
// Scala Map is the only allowed external type of map type in Row.
case m: java.util.Map[_, _] => m.asScala
case _ => data
}
}
private[sql] def bytesToRow(bytes: Array[Byte], schema: StructType): Row = {
val bis = new ByteArrayInputStream(bytes)
val dis = new DataInputStream(bis)
val num = SerDe.readInt(dis)
Row.fromSeq((0 until num).map { i =>
doConversion(SerDe.readObject(dis, jvmObjectTracker = null), schema.fields(i).dataType)
})
}
private[sql] def rowToRBytes(row: Row): Array[Byte] = {
val bos = new ByteArrayOutputStream()
val dos = new DataOutputStream(bos)
val cols = (0 until row.length).map(row(_).asInstanceOf[Object]).toArray
SerDe.writeObject(dos, cols, jvmObjectTracker = null)
bos.toByteArray()
}
// Schema for DataFrame of serialized R data
// TODO: introduce a user defined type for serialized R data.
val SERIALIZED_R_DATA_SCHEMA = StructType(Seq(StructField("R", BinaryType)))
/**
* The helper function for dapply() on R side.
*/
def dapply(
df: DataFrame,
func: Array[Byte],
packageNames: Array[Byte],
broadcastVars: Array[Object],
schema: StructType): DataFrame = {
val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]])
val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema
df.mapPartitionsInR(func, packageNames, bv, realSchema)
}
/**
* The helper function for gapply() on R side.
*/
def gapply(
gd: RelationalGroupedDataset,
func: Array[Byte],
packageNames: Array[Byte],
broadcastVars: Array[Object],
schema: StructType): DataFrame = {
val bv = broadcastVars.map(_.asInstanceOf[Broadcast[Object]])
val realSchema = if (schema == null) SERIALIZED_R_DATA_SCHEMA else schema
gd.flatMapGroupsInR(func, packageNames, bv, realSchema)
}
def dfToCols(df: DataFrame): Array[Array[Any]] = {
val localDF: Array[Row] = df.collect()
val numCols = df.columns.length
val numRows = localDF.length
val colArray = new Array[Array[Any]](numCols)
for (colNo <- 0 until numCols) {
colArray(colNo) = new Array[Any](numRows)
for (rowNo <- 0 until numRows) {
colArray(colNo)(rowNo) = localDF(rowNo)(colNo)
}
}
colArray
}
def saveMode(mode: String): SaveMode = {
mode match {
case "append" => SaveMode.Append
case "overwrite" => SaveMode.Overwrite
case "error" => SaveMode.ErrorIfExists
case "ignore" => SaveMode.Ignore
}
}
def loadDF(
sparkSession: SparkSession,
source: String,
options: java.util.Map[String, String]): DataFrame = {
sparkSession.read.format(source).options(options).load()
}
def loadDF(
sparkSession: SparkSession,
source: String,
schema: StructType,
options: java.util.Map[String, String]): DataFrame = {
sparkSession.read.format(source).schema(schema).options(options).load()
}
def readSqlObject(dis: DataInputStream, dataType: Char): Object = {
dataType match {
case 's' =>
// Read StructType for DataFrame
val fields = SerDe.readList(dis, jvmObjectTracker = null).asInstanceOf[Array[Object]]
Row.fromSeq(fields)
case _ => null
}
}
def writeSqlObject(dos: DataOutputStream, obj: Object): Boolean = {
obj match {
// Handle struct type in DataFrame
case v: GenericRowWithSchema =>
dos.writeByte('s')
SerDe.writeObject(dos, v.schema.fieldNames, jvmObjectTracker = null)
SerDe.writeObject(dos, v.values, jvmObjectTracker = null)
true
case _ =>
false
}
}
def getTables(sparkSession: SparkSession, databaseName: String): DataFrame = {
databaseName match {
case n: String if n != null && n.trim.nonEmpty =>
Dataset.ofRows(sparkSession, ShowTablesCommand(Some(n), None))
case _ =>
Dataset.ofRows(sparkSession, ShowTablesCommand(None, None))
}
}
def getTableNames(sparkSession: SparkSession, databaseName: String): Array[String] = {
val db = databaseName match {
case _ if databaseName != null && databaseName.trim.nonEmpty =>
databaseName
case _ =>
sparkSession.catalog.currentDatabase
}
sparkSession.sessionState.catalog.listTables(db).map(_.table).toArray
}
}