Skip to content

Commit baf7a63

Browse files
committed
Add some comments for consistent compression names and add none/uncompressed for test-based datasources
1 parent 2304bfb commit baf7a63

File tree

3 files changed

+25
-15
lines changed

3 files changed

+25
-15
lines changed

python/pyspark/sql/readwriter.py

Lines changed: 10 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -465,7 +465,8 @@ def json(self, path, mode=None, compression=None):
465465
* ``ignore``: Silently ignore this operation if data already exists.
466466
* ``error`` (default case): Throw an exception if data already exists.
467467
:param compression: compression codec to use when saving to file. This can be one of the
468-
known case-insensitive shorten names (bzip2, gzip, lz4, and snappy).
468+
known case-insensitive shorten names (none, bzip2, gzip, lz4,
469+
snappy and deflate).
469470
470471
>>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data'))
471472
"""
@@ -487,8 +488,8 @@ def parquet(self, path, mode=None, partitionBy=None, compression=None):
487488
* ``error`` (default case): Throw an exception if data already exists.
488489
:param partitionBy: names of partitioning columns
489490
:param compression: compression codec to use when saving to file. This can be one of the
490-
known case-insensitive shorten names (uncompressed, snappy, gzip, and
491-
lzo). This will overwrite ``spark.sql.parquet.compression.codec``.
491+
known case-insensitive shorten names (none, snappy, gzip, and lzo).
492+
This will overwrite ``spark.sql.parquet.compression.codec``.
492493
493494
>>> df.write.parquet(os.path.join(tempfile.mkdtemp(), 'data'))
494495
"""
@@ -505,7 +506,8 @@ def text(self, path, compression=None):
505506
506507
:param path: the path in any Hadoop supported file system
507508
:param compression: compression codec to use when saving to file. This can be one of the
508-
known case-insensitive shorten names (bzip2, gzip, lz4, and snappy).
509+
known case-insensitive shorten names (none, bzip2, gzip, lz4,
510+
snappy and deflate).
509511
510512
The DataFrame must have only one column that is of string type.
511513
Each row becomes a new line in the output file.
@@ -527,7 +529,8 @@ def csv(self, path, mode=None, compression=None):
527529
* ``error`` (default case): Throw an exception if data already exists.
528530
529531
:param compression: compression codec to use when saving to file. This can be one of the
530-
known case-insensitive shorten names (bzip2, gzip, lz4, and snappy).
532+
known case-insensitive shorten names (none, bzip2, gzip, lz4,
533+
snappy and deflate).
531534
532535
>>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data'))
533536
"""
@@ -552,8 +555,8 @@ def orc(self, path, mode=None, partitionBy=None, compression=None):
552555
* ``error`` (default case): Throw an exception if data already exists.
553556
:param partitionBy: names of partitioning columns
554557
:param compression: compression codec to use when saving to file. This can be one of the
555-
known case-insensitive shorten names (uncompressed, snappy, zlib, and
556-
lzo). This will overwrite ``orc.compress``.
558+
known case-insensitive shorten names (none, snappy, zlib, and lzo).
559+
This will overwrite ``orc.compress``.
557560
558561
>>> orc_df = hiveContext.read.orc('python/test_support/sql/orc_partitioned')
559562
>>> orc_df.write.orc(os.path.join(tempfile.mkdtemp(), 'data'))

sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala

Lines changed: 10 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -455,7 +455,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
455455
*
456456
* You can set the following JSON-specific option(s) for writing JSON files:
457457
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
458-
* one of the known case-insensitive shorten names (`bzip2`, `gzip`, `lz4`, and `snappy`). </li>
458+
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
459+
* `snappy` and `deflate`). </li>
459460
*
460461
* @since 1.4.0
461462
*/
@@ -470,8 +471,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
470471
*
471472
* You can set the following Parquet-specific option(s) for writing Parquet files:
472473
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
473-
* one of the known case-insensitive shorten names(`uncompressed`, `snappy`,`gzip`, and
474-
* `lzo`). This will overwrite `spark.sql.parquet.compression.codec`. </li>
474+
* one of the known case-insensitive shorten names(`none`, `snappy`, `gzip`, and `lzo`).
475+
* This will overwrite `spark.sql.parquet.compression.codec`. </li>
475476
*
476477
* @since 1.4.0
477478
*/
@@ -486,8 +487,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
486487
*
487488
* You can set the following ORC-specific option(s) for writing ORC files:
488489
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
489-
* one of the known case-insensitive shorten names(`uncompressed`, `snappy`, `zlib`, and
490-
* `lzo`). This will overwrite `orc.compress`. </li>
490+
* one of the known case-insensitive shorten names(`none`, `snappy`, `zlib`, and `lzo`).
491+
* This will overwrite `orc.compress`. </li>
491492
*
492493
* @since 1.5.0
493494
* @note Currently, this method can only be used together with `HiveContext`.
@@ -508,7 +509,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
508509
*
509510
* You can set the following option(s) for writing text files:
510511
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
511-
* one of the known case-insensitive shorten names (`bzip2`, `gzip`, `lz4`, and `snappy`). </li>
512+
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
513+
* `snappy` and `deflate`). </li>
512514
*
513515
* @since 1.6.0
514516
*/
@@ -523,7 +525,8 @@ final class DataFrameWriter private[sql](df: DataFrame) {
523525
*
524526
* You can set the following CSV-specific option(s) for writing CSV files:
525527
* <li>`compression` (default `null`): compression codec to use when saving to file. This can be
526-
* one of the known case-insensitive shorten names (`bzip2`, `gzip`, `lz4`, and `snappy`). </li>
528+
* one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`,
529+
* `snappy` and `deflate`). </li>
527530
*
528531
* @since 2.0.0
529532
*/

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CompressionCodecs.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,8 @@ import org.apache.spark.util.Utils
2525

2626
private[datasources] object CompressionCodecs {
2727
private val shortCompressionCodecNames = Map(
28+
"none" -> null,
29+
"uncompressed" -> null,
2830
"bzip2" -> classOf[BZip2Codec].getName,
2931
"deflate" -> classOf[DeflateCodec].getName,
3032
"gzip" -> classOf[GzipCodec].getName,
@@ -39,7 +41,9 @@ private[datasources] object CompressionCodecs {
3941
val codecName = shortCompressionCodecNames.getOrElse(name.toLowerCase, name)
4042
try {
4143
// Validate the codec name
42-
Utils.classForName(codecName)
44+
if (codecName != null) {
45+
Utils.classForName(codecName)
46+
}
4347
codecName
4448
} catch {
4549
case e: ClassNotFoundException =>

0 commit comments

Comments
 (0)