-
Notifications
You must be signed in to change notification settings - Fork 28k
/
DataFrameWriter.scala
624 lines (565 loc) · 23.3 KB
/
DataFrameWriter.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
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
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
/*
* 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
import java.util.Properties
import scala.collection.JavaConverters._
import org.apache.spark.annotation.InterfaceStability
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation}
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogRelation, CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable
import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation}
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types.StructType
/**
* Interface used to write a [[Dataset]] to external storage systems (e.g. file systems,
* key-value stores, etc). Use `Dataset.write` to access this.
*
* @since 1.4.0
*/
@InterfaceStability.Stable
final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
private val df = ds.toDF()
/**
* Specifies the behavior when data or table already exists. Options include:
* - `SaveMode.Overwrite`: overwrite the existing data.
* - `SaveMode.Append`: append the data.
* - `SaveMode.Ignore`: ignore the operation (i.e. no-op).
* - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
*
* @since 1.4.0
*/
def mode(saveMode: SaveMode): DataFrameWriter[T] = {
this.mode = saveMode
this
}
/**
* Specifies the behavior when data or table already exists. Options include:
* - `overwrite`: overwrite the existing data.
* - `append`: append the data.
* - `ignore`: ignore the operation (i.e. no-op).
* - `error`: default option, throw an exception at runtime.
*
* @since 1.4.0
*/
def mode(saveMode: String): DataFrameWriter[T] = {
this.mode = saveMode.toLowerCase match {
case "overwrite" => SaveMode.Overwrite
case "append" => SaveMode.Append
case "ignore" => SaveMode.Ignore
case "error" | "default" => SaveMode.ErrorIfExists
case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " +
"Accepted save modes are 'overwrite', 'append', 'ignore', 'error'.")
}
this
}
/**
* Specifies the underlying output data source. Built-in options include "parquet", "json", etc.
*
* @since 1.4.0
*/
def format(source: String): DataFrameWriter[T] = {
this.source = source
this
}
/**
* Adds an output option for the underlying data source.
*
* @since 1.4.0
*/
def option(key: String, value: String): DataFrameWriter[T] = {
this.extraOptions += (key -> value)
this
}
/**
* Adds an output option for the underlying data source.
*
* @since 2.0.0
*/
def option(key: String, value: Boolean): DataFrameWriter[T] = option(key, value.toString)
/**
* Adds an output option for the underlying data source.
*
* @since 2.0.0
*/
def option(key: String, value: Long): DataFrameWriter[T] = option(key, value.toString)
/**
* Adds an output option for the underlying data source.
*
* @since 2.0.0
*/
def option(key: String, value: Double): DataFrameWriter[T] = option(key, value.toString)
/**
* (Scala-specific) Adds output options for the underlying data source.
*
* @since 1.4.0
*/
def options(options: scala.collection.Map[String, String]): DataFrameWriter[T] = {
this.extraOptions ++= options
this
}
/**
* Adds output options for the underlying data source.
*
* @since 1.4.0
*/
def options(options: java.util.Map[String, String]): DataFrameWriter[T] = {
this.options(options.asScala)
this
}
/**
* Partitions the output by the given columns on the file system. If specified, the output is
* laid out on the file system similar to Hive's partitioning scheme. As an example, when we
* partition a dataset by year and then month, the directory layout would look like:
*
* - year=2016/month=01/
* - year=2016/month=02/
*
* Partitioning is one of the most widely used techniques to optimize physical data layout.
* It provides a coarse-grained index for skipping unnecessary data reads when queries have
* predicates on the partitioned columns. In order for partitioning to work well, the number
* of distinct values in each column should typically be less than tens of thousands.
*
* This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0.
*
* @since 1.4.0
*/
@scala.annotation.varargs
def partitionBy(colNames: String*): DataFrameWriter[T] = {
this.partitioningColumns = Option(colNames)
this
}
/**
* Buckets the output by the given columns. If specified, the output is laid out on the file
* system similar to Hive's bucketing scheme.
*
* This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0.
*
* @since 2.0
*/
@scala.annotation.varargs
def bucketBy(numBuckets: Int, colName: String, colNames: String*): DataFrameWriter[T] = {
this.numBuckets = Option(numBuckets)
this.bucketColumnNames = Option(colName +: colNames)
this
}
/**
* Sorts the output in each bucket by the given columns.
*
* This is applicable for all file-based data sources (e.g. Parquet, JSON) staring Spark 2.1.0.
*
* @since 2.0
*/
@scala.annotation.varargs
def sortBy(colName: String, colNames: String*): DataFrameWriter[T] = {
this.sortColumnNames = Option(colName +: colNames)
this
}
/**
* Saves the content of the `DataFrame` at the specified path.
*
* @since 1.4.0
*/
def save(path: String): Unit = {
this.extraOptions += ("path" -> path)
save()
}
/**
* Saves the content of the `DataFrame` as the specified table.
*
* @since 1.4.0
*/
def save(): Unit = {
if (source.toLowerCase == DDLUtils.HIVE_PROVIDER) {
throw new AnalysisException("Hive data source can only be used with tables, you can not " +
"write files of Hive data source directly.")
}
assertNotBucketed("save")
val dataSource = DataSource(
df.sparkSession,
className = source,
partitionColumns = partitioningColumns.getOrElse(Nil),
bucketSpec = getBucketSpec,
options = extraOptions.toMap)
dataSource.write(mode, df)
}
/**
* Inserts the content of the `DataFrame` to the specified table. It requires that
* the schema of the `DataFrame` is the same as the schema of the table.
*
* @note Unlike `saveAsTable`, `insertInto` ignores the column names and just uses position-based
* resolution. For example:
*
* {{{
* scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1")
* scala> Seq((3, 4)).toDF("j", "i").write.insertInto("t1")
* scala> Seq((5, 6)).toDF("a", "b").write.insertInto("t1")
* scala> sql("select * from t1").show
* +---+---+
* | i| j|
* +---+---+
* | 5| 6|
* | 3| 4|
* | 1| 2|
* +---+---+
* }}}
*
* Because it inserts data to an existing table, format or options will be ignored.
*
* @since 1.4.0
*/
def insertInto(tableName: String): Unit = {
insertInto(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName))
}
private def insertInto(tableIdent: TableIdentifier): Unit = {
assertNotBucketed("insertInto")
if (partitioningColumns.isDefined) {
throw new AnalysisException(
"insertInto() can't be used together with partitionBy(). " +
"Partition columns have already be defined for the table. " +
"It is not necessary to use partitionBy()."
)
}
df.sparkSession.sessionState.executePlan(
InsertIntoTable(
table = UnresolvedRelation(tableIdent),
partition = Map.empty[String, Option[String]],
child = df.logicalPlan,
overwrite = mode == SaveMode.Overwrite,
ifNotExists = false)).toRdd
}
private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols =>
cols.map(normalize(_, "Partition"))
}
private def normalizedBucketColNames: Option[Seq[String]] = bucketColumnNames.map { cols =>
cols.map(normalize(_, "Bucketing"))
}
private def normalizedSortColNames: Option[Seq[String]] = sortColumnNames.map { cols =>
cols.map(normalize(_, "Sorting"))
}
private def getBucketSpec: Option[BucketSpec] = {
if (sortColumnNames.isDefined) {
require(numBuckets.isDefined, "sortBy must be used together with bucketBy")
}
for {
n <- numBuckets
} yield {
require(n > 0 && n < 100000, "Bucket number must be greater than 0 and less than 100000.")
// partitionBy columns cannot be used in bucketBy
if (normalizedParCols.nonEmpty &&
normalizedBucketColNames.get.toSet.intersect(normalizedParCols.get.toSet).nonEmpty) {
throw new AnalysisException(
s"bucketBy columns '${bucketColumnNames.get.mkString(", ")}' should not be part of " +
s"partitionBy columns '${partitioningColumns.get.mkString(", ")}'")
}
BucketSpec(n, normalizedBucketColNames.get, normalizedSortColNames.getOrElse(Nil))
}
}
/**
* The given column name may not be equal to any of the existing column names if we were in
* case-insensitive context. Normalize the given column name to the real one so that we don't
* need to care about case sensitivity afterwards.
*/
private def normalize(columnName: String, columnType: String): String = {
val validColumnNames = df.logicalPlan.output.map(_.name)
validColumnNames.find(df.sparkSession.sessionState.analyzer.resolver(_, columnName))
.getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " +
s"existing columns (${validColumnNames.mkString(", ")})"))
}
private def assertNotBucketed(operation: String): Unit = {
if (numBuckets.isDefined || sortColumnNames.isDefined) {
throw new AnalysisException(s"'$operation' does not support bucketing right now")
}
}
private def assertNotPartitioned(operation: String): Unit = {
if (partitioningColumns.isDefined) {
throw new AnalysisException( s"'$operation' does not support partitioning")
}
}
/**
* Saves the content of the `DataFrame` as the specified table.
*
* In the case the table already exists, behavior of this function depends on the
* save mode, specified by the `mode` function (default to throwing an exception).
* When `mode` is `Overwrite`, the schema of the `DataFrame` does not need to be
* the same as that of the existing table.
*
* When `mode` is `Append`, if there is an existing table, we will use the format and options of
* the existing table. The column order in the schema of the `DataFrame` doesn't need to be same
* as that of the existing table. Unlike `insertInto`, `saveAsTable` will use the column names to
* find the correct column positions. For example:
*
* {{{
* scala> Seq((1, 2)).toDF("i", "j").write.mode("overwrite").saveAsTable("t1")
* scala> Seq((3, 4)).toDF("j", "i").write.mode("append").saveAsTable("t1")
* scala> sql("select * from t1").show
* +---+---+
* | i| j|
* +---+---+
* | 1| 2|
* | 4| 3|
* +---+---+
* }}}
*
* When the DataFrame is created from a non-partitioned `HadoopFsRelation` with a single input
* path, and the data source provider can be mapped to an existing Hive builtin SerDe (i.e. ORC
* and Parquet), the table is persisted in a Hive compatible format, which means other systems
* like Hive will be able to read this table. Otherwise, the table is persisted in a Spark SQL
* specific format.
*
* @since 1.4.0
*/
def saveAsTable(tableName: String): Unit = {
saveAsTable(df.sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName))
}
private def saveAsTable(tableIdent: TableIdentifier): Unit = {
val catalog = df.sparkSession.sessionState.catalog
val tableExists = catalog.tableExists(tableIdent)
val db = tableIdent.database.getOrElse(catalog.getCurrentDatabase)
val tableIdentWithDB = tableIdent.copy(database = Some(db))
val tableName = tableIdentWithDB.unquotedString
(tableExists, mode) match {
case (true, SaveMode.Ignore) =>
// Do nothing
case (true, SaveMode.ErrorIfExists) =>
throw new AnalysisException(s"Table $tableIdent already exists.")
case (true, SaveMode.Overwrite) =>
// Get all input data source or hive relations of the query.
val srcRelations = df.logicalPlan.collect {
case LogicalRelation(src: BaseRelation, _, _) => src
case relation: CatalogRelation if DDLUtils.isHiveTable(relation.catalogTable) =>
relation.catalogTable.identifier
}
EliminateSubqueryAliases(catalog.lookupRelation(tableIdentWithDB)) match {
// check if the table is a data source table (the relation is a BaseRelation).
case LogicalRelation(dest: BaseRelation, _, _) if srcRelations.contains(dest) =>
throw new AnalysisException(
s"Cannot overwrite table $tableName that is also being read from")
// check hive table relation when overwrite mode
case relation: CatalogRelation if DDLUtils.isHiveTable(relation.catalogTable)
&& srcRelations.contains(relation.catalogTable.identifier) =>
throw new AnalysisException(
s"Cannot overwrite table $tableName that is also being read from")
case _ => // OK
}
// Drop the existing table
catalog.dropTable(tableIdentWithDB, ignoreIfNotExists = true, purge = false)
createTable(tableIdentWithDB)
// Refresh the cache of the table in the catalog.
catalog.refreshTable(tableIdentWithDB)
case _ => createTable(tableIdent)
}
}
private def createTable(tableIdent: TableIdentifier): Unit = {
val storage = DataSource.buildStorageFormatFromOptions(extraOptions.toMap)
val tableType = if (storage.locationUri.isDefined) {
CatalogTableType.EXTERNAL
} else {
CatalogTableType.MANAGED
}
val tableDesc = CatalogTable(
identifier = tableIdent,
tableType = tableType,
storage = storage,
schema = new StructType,
provider = Some(source),
partitionColumnNames = partitioningColumns.getOrElse(Nil),
bucketSpec = getBucketSpec
)
df.sparkSession.sessionState.executePlan(
CreateTable(tableDesc, mode, Some(df.logicalPlan))).toRdd
}
/**
* Saves the content of the `DataFrame` to an external database table via JDBC. In the case the
* table already exists in the external database, behavior of this function depends on the
* save mode, specified by the `mode` function (default to throwing an exception).
*
* Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash
* your external database systems.
*
* You can set the following JDBC-specific option(s) for storing JDBC:
* <ul>
* <li>`truncate` (default `false`): use `TRUNCATE TABLE` instead of `DROP TABLE`.</li>
* </ul>
*
* In case of failures, users should turn off `truncate` option to use `DROP TABLE` again. Also,
* due to the different behavior of `TRUNCATE TABLE` among DBMS, it's not always safe to use this.
* MySQLDialect, DB2Dialect, MsSqlServerDialect, DerbyDialect, and OracleDialect supports this
* while PostgresDialect and default JDBCDirect doesn't. For unknown and unsupported JDBCDirect,
* the user option `truncate` is ignored.
*
* @param url JDBC database url of the form `jdbc:subprotocol:subname`
* @param table Name of the table in the external database.
* @param connectionProperties JDBC database connection arguments, a list of arbitrary string
* tag/value. Normally at least a "user" and "password" property
* should be included. "batchsize" can be used to control the
* number of rows per insert. "isolationLevel" can be one of
* "NONE", "READ_COMMITTED", "READ_UNCOMMITTED", "REPEATABLE_READ",
* or "SERIALIZABLE", corresponding to standard transaction
* isolation levels defined by JDBC's Connection object, with default
* of "READ_UNCOMMITTED".
* @since 1.4.0
*/
def jdbc(url: String, table: String, connectionProperties: Properties): Unit = {
assertNotPartitioned("jdbc")
assertNotBucketed("jdbc")
// connectionProperties should override settings in extraOptions.
this.extraOptions ++= connectionProperties.asScala
// explicit url and dbtable should override all
this.extraOptions += ("url" -> url, "dbtable" -> table)
format("jdbc").save()
}
/**
* Saves the content of the `DataFrame` in JSON format (<a href="http://jsonlines.org/">
* JSON Lines text format or newline-delimited JSON</a>) at the specified path.
* This is equivalent to:
* {{{
* format("json").save(path)
* }}}
*
* You can set the following JSON-specific option(s) for writing JSON files:
* <ul>
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
* `snappy` and `deflate`). </li>
* <li>`dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format.
* Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to
* date type.</li>
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
* indicates a timestamp format. Custom date formats follow the formats at
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
* </ul>
*
* @since 1.4.0
*/
def json(path: String): Unit = {
format("json").save(path)
}
/**
* Saves the content of the `DataFrame` in Parquet format at the specified path.
* This is equivalent to:
* {{{
* format("parquet").save(path)
* }}}
*
* You can set the following Parquet-specific option(s) for writing Parquet files:
* <ul>
* <li>`compression` (default is the value specified in `spark.sql.parquet.compression.codec`):
* compression codec to use when saving to file. This can be one of the known case-insensitive
* shorten names(none, `snappy`, `gzip`, and `lzo`). This will override
* `spark.sql.parquet.compression.codec`.</li>
* </ul>
*
* @since 1.4.0
*/
def parquet(path: String): Unit = {
format("parquet").save(path)
}
/**
* Saves the content of the `DataFrame` in ORC format at the specified path.
* This is equivalent to:
* {{{
* format("orc").save(path)
* }}}
*
* You can set the following ORC-specific option(s) for writing ORC files:
* <ul>
* <li>`compression` (default `snappy`): compression codec to use when saving to file. This can be
* one of the known case-insensitive shorten names(`none`, `snappy`, `zlib`, and `lzo`).
* This will override `orc.compress`.</li>
* </ul>
*
* @since 1.5.0
* @note Currently, this method can only be used after enabling Hive support
*/
def orc(path: String): Unit = {
format("orc").save(path)
}
/**
* Saves the content of the `DataFrame` in a text file at the specified path.
* The DataFrame must have only one column that is of string type.
* Each row becomes a new line in the output file. For example:
* {{{
* // Scala:
* df.write.text("/path/to/output")
*
* // Java:
* df.write().text("/path/to/output")
* }}}
*
* You can set the following option(s) for writing text files:
* <ul>
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
* `snappy` and `deflate`). </li>
* </ul>
*
* @since 1.6.0
*/
def text(path: String): Unit = {
format("text").save(path)
}
/**
* Saves the content of the `DataFrame` in CSV format at the specified path.
* This is equivalent to:
* {{{
* format("csv").save(path)
* }}}
*
* You can set the following CSV-specific option(s) for writing CSV files:
* <ul>
* <li>`sep` (default `,`): sets the single character as a separator for each
* field and value.</li>
* <li>`quote` (default `"`): sets the single character used for escaping quoted values where
* the separator can be part of the value.</li>
* <li>`escape` (default `\`): sets the single character used for escaping quotes inside
* an already quoted value.</li>
* <li>`escapeQuotes` (default `true`): a flag indicating whether values containing
* quotes should always be enclosed in quotes. Default is to escape all values containing
* a quote character.</li>
* <li>`quoteAll` (default `false`): A flag indicating whether all values should always be
* enclosed in quotes. Default is to only escape values containing a quote character.</li>
* <li>`header` (default `false`): writes the names of columns as the first line.</li>
* <li>`nullValue` (default empty string): sets the string representation of a null value.</li>
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
* `snappy` and `deflate`). </li>
* <li>`dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format.
* Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to
* date type.</li>
* <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
* indicates a timestamp format. Custom date formats follow the formats at
* `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
* </ul>
*
* @since 2.0.0
*/
def csv(path: String): Unit = {
format("csv").save(path)
}
///////////////////////////////////////////////////////////////////////////////////////
// Builder pattern config options
///////////////////////////////////////////////////////////////////////////////////////
private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName
private var mode: SaveMode = SaveMode.ErrorIfExists
private val extraOptions = new scala.collection.mutable.HashMap[String, String]
private var partitioningColumns: Option[Seq[String]] = None
private var bucketColumnNames: Option[Seq[String]] = None
private var numBuckets: Option[Int] = None
private var sortColumnNames: Option[Seq[String]] = None
}