-
Notifications
You must be signed in to change notification settings - Fork 414
Expand file tree
/
Copy pathTables.scala
More file actions
357 lines (310 loc) · 12.3 KB
/
Tables.scala
File metadata and controls
357 lines (310 loc) · 12.3 KB
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
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
/*
* Copyright 2015 Databricks Inc.
*
* Licensed 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 com.databricks.spark.sql.perf
import java.util.concurrent.LinkedBlockingQueue
import scala.collection.immutable.Stream
import scala.sys.process._
import org.slf4j.LoggerFactory
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
import org.apache.spark.sql.{Row, SQLContext, SaveMode, SparkSession}
/**
* Using ProcessBuilder.lineStream produces a stream, that uses
* a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE.
*
* This causes OOM if the consumer cannot keep up with the producer.
*
* See scala.sys.process.ProcessBuilderImpl.lineStream
*/
object BlockingLineStream {
// See scala.sys.process.Streamed
private final class BlockingStreamed[T](
val process: T => Unit,
val done: Int => Unit,
val stream: () => Stream[T]
)
// See scala.sys.process.Streamed
private object BlockingStreamed {
// scala.process.sys.Streamed uses default of Integer.MAX_VALUE,
// which causes OOMs if the consumer cannot keep up with producer.
val maxQueueSize = 65536
def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = {
val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize)
def next(): Stream[T] = q.take match {
case Left(0) => Stream.empty
case Left(code) =>
if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) else Stream.empty
case Right(s) => Stream.cons(s, next())
}
new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), () => next())
}
}
// See scala.sys.process.ProcessImpl.Spawn
private object Spawn {
def apply(f: => Unit): Thread = apply(f, daemon = false)
def apply(f: => Unit, daemon: Boolean): Thread = {
val thread = new Thread() { override def run() = { f } }
thread.setDaemon(daemon)
thread.start()
thread
}
}
def apply(command: Seq[String]): Stream[String] = {
val streamed = BlockingStreamed[String](true)
val process = command.run(BasicIO(false, streamed.process, None))
Spawn(streamed.done(process.exitValue()))
streamed.stream()
}
}
trait DataGenerator extends Serializable {
def generate(
sparkContext: SparkContext,
name: String,
partitions: Int,
scaleFactor: String): RDD[String]
}
abstract class Tables(sqlSession SparkSession, sqlContext: SQLContext, scaleFactor: String,
useDoubleForDecimal: Boolean = false, useStringForDate: Boolean = false)
extends Serializable {
def dataGenerator: DataGenerator
def tables: Seq[Table]
private val log = LoggerFactory.getLogger(getClass)
def sparkContext = sqlContext.sparkContext
case class Table(name: String, partitionColumns: Seq[String], fields: StructField*) {
val schema = StructType(fields)
def nonPartitioned: Table = {
Table(name, Nil, fields : _*)
}
/**
* If convertToSchema is true, the data from generator will be parsed into columns and
* converted to `schema`. Otherwise, it just outputs the raw data (as a single STRING column).
*/
def df(convertToSchema: Boolean, numPartition: Int) = {
val generatedData = dataGenerator.generate(sparkContext, name, numPartition, scaleFactor)
val rows = generatedData.mapPartitions { iter =>
iter.map { l =>
if (convertToSchema) {
val values = l.split("\\|", -1).dropRight(1).map { v =>
if (v.equals("")) {
// If the string value is an empty string, we turn it to a null
null
} else {
v
}
}
Row.fromSeq(values)
} else {
Row.fromSeq(Seq(l))
}
}
}
if (convertToSchema) {
val stringData =
sqlContext.createDataFrame(
rows,
StructType(schema.fields.map(f => StructField(f.name, StringType))))
val convertedData = {
val columns = schema.fields.map { f =>
col(f.name).cast(f.dataType).as(f.name)
}
stringData.select(columns: _*)
}
convertedData
} else {
sqlContext.createDataFrame(rows, StructType(Seq(StructField("value", StringType))))
}
}
def convertTypes(): Table = {
val newFields = fields.map { field =>
val newDataType = field.dataType match {
case decimal: DecimalType if useDoubleForDecimal => DoubleType
case date: DateType if useStringForDate => StringType
case other => other
}
field.copy(dataType = newDataType)
}
Table(name, partitionColumns, newFields:_*)
}
def genData(
location: String,
format: String,
overwrite: Boolean,
clusterByPartitionColumns: Boolean,
filterOutNullPartitionValues: Boolean,
numPartitions: Int): Unit = {
val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Ignore
val data = df(format != "text", numPartitions)
val tempTableName = s"${name}_text"
data.createOrReplaceTempView(tempTableName)
val writer = if (partitionColumns.nonEmpty) {
if (clusterByPartitionColumns) {
val columnString = data.schema.fields.map { field =>
field.name
}.mkString(",")
val partitionColumnString = partitionColumns.mkString(",")
val predicates = if (filterOutNullPartitionValues) {
partitionColumns.map(col => s"$col IS NOT NULL").mkString("WHERE ", " AND ", "")
} else {
""
}
val query =
s"""
|SELECT
| $columnString
|FROM
| $tempTableName
|$predicates
|DISTRIBUTE BY
| $partitionColumnString
""".stripMargin
val grouped = sqlContext.sql(query)
println(s"Pre-clustering with partitioning columns with query $query.")
log.info(s"Pre-clustering with partitioning columns with query $query.")
grouped.write
} else {
data.write
}
} else {
// treat non-partitioned tables as "one partition" that we want to coalesce
if (clusterByPartitionColumns) {
// in case data has more than maxRecordsPerFile, split into multiple writers to improve datagen speed
// files will be truncated to maxRecordsPerFile value, so the final result will be the same
val numRows = data.count
val maxRecordPerFile = util.Try(sqlContext.getConf("spark.sql.files.maxRecordsPerFile").toInt).getOrElse(0)
println(s"Data has $numRows rows clustered $clusterByPartitionColumns for $maxRecordPerFile")
log.info(s"Data has $numRows rows clustered $clusterByPartitionColumns for $maxRecordPerFile")
if (maxRecordPerFile > 0 && numRows > maxRecordPerFile) {
val numFiles = (numRows.toDouble/maxRecordPerFile).ceil.toInt
println(s"Coalescing into $numFiles files")
log.info(s"Coalescing into $numFiles files")
data.coalesce(numFiles).write
} else {
data.coalesce(1).write
}
} else {
data.write
}
}
writer.format(format).mode(mode)
if (partitionColumns.nonEmpty) {
writer.partitionBy(partitionColumns : _*)
}
println(s"Generating table $name in database to $location with save mode $mode.")
log.info(s"Generating table $name in database to $location with save mode $mode.")
writer.save(location)
sqlContext.dropTempTable(tempTableName)
}
def createExternalTable(location: String, format: String, databaseName: String,
overwrite: Boolean, discoverPartitions: Boolean = true): Unit = {
val qualifiedTableName = databaseName + "." + name
val tableExists = sqlContext.tableNames(databaseName).contains(name)
if (overwrite) {
sqlContext.sql(s"DROP TABLE IF EXISTS $databaseName.$name")
}
if (!tableExists || overwrite) {
println(s"Creating external table $name in database $databaseName using data stored in $location.")
log.info(s"Creating external table $name in database $databaseName using data stored in $location.")
sqlSession.createTable(qualifiedTableName, location, format)
}
if (partitionColumns.nonEmpty && discoverPartitions) {
println(s"Discovering partitions for table $name.")
log.info(s"Discovering partitions for table $name.")
sqlContext.sql(s"ALTER TABLE $databaseName.$name RECOVER PARTITIONS")
}
}
def createTemporaryTable(location: String, format: String): Unit = {
println(s"Creating temporary table $name using data stored in $location.")
log.info(s"Creating temporary table $name using data stored in $location.")
sqlContext.read.format(format).load(location).createOrReplaceTempView(name)
}
def analyzeTable(databaseName: String, analyzeColumns: Boolean = false): Unit = {
println(s"Analyzing table $name.")
log.info(s"Analyzing table $name.")
sqlContext.sql(s"ANALYZE TABLE $databaseName.$name COMPUTE STATISTICS")
if (analyzeColumns) {
val allColumns = fields.map(_.name).mkString(", ")
println(s"Analyzing table $name columns $allColumns.")
log.info(s"Analyzing table $name columns $allColumns.")
sqlContext.sql(s"ANALYZE TABLE $databaseName.$name COMPUTE STATISTICS FOR COLUMNS $allColumns")
}
}
}
def genData(
location: String,
format: String,
overwrite: Boolean,
partitionTables: Boolean,
clusterByPartitionColumns: Boolean,
filterOutNullPartitionValues: Boolean,
tableFilter: String = "",
numPartitions: Int = 100): Unit = {
var tablesToBeGenerated = if (partitionTables) {
tables
} else {
tables.map(_.nonPartitioned)
}
if (!tableFilter.isEmpty) {
tablesToBeGenerated = tablesToBeGenerated.filter(_.name == tableFilter)
if (tablesToBeGenerated.isEmpty) {
throw new RuntimeException("Bad table name filter: " + tableFilter)
}
}
tablesToBeGenerated.foreach { table =>
val tableLocation = s"$location/${table.name}"
table.genData(tableLocation, format, overwrite, clusterByPartitionColumns,
filterOutNullPartitionValues, numPartitions)
}
}
def createExternalTables(location: String, format: String, databaseName: String,
overwrite: Boolean, discoverPartitions: Boolean, tableFilter: String = ""): Unit = {
val filtered = if (tableFilter.isEmpty) {
tables
} else {
tables.filter(_.name == tableFilter)
}
sqlContext.sql(s"CREATE DATABASE IF NOT EXISTS $databaseName")
filtered.foreach { table =>
val tableLocation = s"$location/${table.name}"
table.createExternalTable(tableLocation, format, databaseName, overwrite, discoverPartitions)
}
sqlContext.sql(s"USE $databaseName")
println(s"The current database has been set to $databaseName.")
log.info(s"The current database has been set to $databaseName.")
}
def createTemporaryTables(location: String, format: String, tableFilter: String = ""): Unit = {
val filtered = if (tableFilter.isEmpty) {
tables
} else {
tables.filter(_.name == tableFilter)
}
filtered.foreach { table =>
val tableLocation = s"$location/${table.name}"
table.createTemporaryTable(tableLocation, format)
}
}
def analyzeTables(databaseName: String, analyzeColumns: Boolean = false, tableFilter: String = ""): Unit = {
val filtered = if (tableFilter.isEmpty) {
tables
} else {
tables.filter(_.name == tableFilter)
}
filtered.foreach { table =>
table.analyzeTable(databaseName, analyzeColumns)
}
}
}