diff options
author | hyukjinkwon <gurwls223@gmail.com> | 2016-03-09 19:12:46 -0800 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2016-03-09 19:12:46 -0800 |
commit | aa0eba2c354dc57dd83a427daa68d6171f292a83 (patch) | |
tree | 72d0d85b3c530be8f564f99ea1f9b3fd1d1d63e4 /sql | |
parent | 790646125edd8b853e2ad2425112590e78799bd3 (diff) | |
download | spark-aa0eba2c354dc57dd83a427daa68d6171f292a83.tar.gz spark-aa0eba2c354dc57dd83a427daa68d6171f292a83.tar.bz2 spark-aa0eba2c354dc57dd83a427daa68d6171f292a83.zip |
[SPARK-13766][SQL] Consistent file extensions for files written by internal data sources
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-13766
This PR makes the file extensions (written by internal datasource) consistent.
**Before**
- TEXT, CSV and JSON
```
[.COMPRESSION_CODEC_NAME]
```
- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```
- ORC
```
.orc
```
**After**
- TEXT, CSV and JSON
```
.txt[.COMPRESSION_CODEC_NAME]
.csv[.COMPRESSION_CODEC_NAME]
.json[.COMPRESSION_CODEC_NAME]
```
- Parquet
```
[.COMPRESSION_CODEC_NAME].parquet
```
- ORC
```
[.COMPRESSION_CODEC_NAME].orc
```
When the compression codec is set,
- For Parquet and ORC, each still stays in Parquet and ORC format but just have compressed data internally. So, I think it is okay to name `.parquet` and `.orc` at the end.
- For Text, CSV and JSON, each does not stays in each format but it has different data format according to compression codec. So, each has the names `.json`, `.csv` and `.txt` before the compression extension.
## How was this patch tested?
Unit tests are used and `./dev/run_tests` for coding style tests.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #11604 from HyukjinKwon/SPARK-13766.
Diffstat (limited to 'sql')
9 files changed, 29 insertions, 11 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index d7ce9a0ce8..0e6b9855c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -140,7 +140,7 @@ private[sql] class CsvOutputWriter( val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.csv$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 497e3c59e9..05b44d1a2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -167,7 +167,7 @@ private[json] class JsonOutputWriter( val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString.json$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 82404b8499..f1060074d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -379,6 +379,9 @@ private[sql] class ParquetOutputWriter( val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + // It has the `.parquet` extension at the end because (de)compression tools + // such as gunzip would not be able to decompress this as the compression + // is not applied on this whole file but on each "page" in Parquet format. new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index b3297254cb..2869a6a1ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -140,7 +140,7 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") val taskAttemptId = context.getTaskAttemptID val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.txt$extension") } }.getRecordWriter(context) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 076fe5e041..680759d457 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -404,7 +404,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .save(csvDir) val compressedFiles = new File(csvDir).listFiles() - assert(compressedFiles.exists(_.getName.endsWith(".gz"))) + assert(compressedFiles.exists(_.getName.endsWith(".csv.gz"))) val carsCopy = sqlContext.read .format("csv") @@ -439,7 +439,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .save(csvDir) val compressedFiles = new File(csvDir).listFiles() - assert(compressedFiles.exists(!_.getName.endsWith(".gz"))) + assert(compressedFiles.exists(!_.getName.endsWith(".csv.gz"))) val carsCopy = sqlContext.read .format("csv") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 02b173d30a..097ece3525 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1441,7 +1441,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .save(jsonDir) val compressedFiles = new File(jsonDir).listFiles() - assert(compressedFiles.exists(_.getName.endsWith(".gz"))) + assert(compressedFiles.exists(_.getName.endsWith(".json.gz"))) val jsonCopy = sqlContext.read .format("json") @@ -1479,7 +1479,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { .save(jsonDir) val compressedFiles = new File(jsonDir).listFiles() - assert(compressedFiles.exists(!_.getName.endsWith(".gz"))) + assert(compressedFiles.exists(!_.getName.endsWith(".json.gz"))) val jsonCopy = sqlContext.read .format("json") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 9eb1016b64..ee398721c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -74,7 +74,7 @@ class TextSuite extends QueryTest with SharedSQLContext { val tempDirPath = tempDir.getAbsolutePath testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) val compressedFiles = new File(tempDirPath).listFiles() - assert(compressedFiles.exists(_.getName.endsWith(extension))) + assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) verifyFrame(sqlContext.read.text(tempDirPath)) } @@ -102,7 +102,7 @@ class TextSuite extends QueryTest with SharedSQLContext { val tempDirPath = tempDir.getAbsolutePath testDf.write.option("compression", "none").mode(SaveMode.Overwrite).text(tempDirPath) val compressedFiles = new File(tempDirPath).listFiles() - assert(compressedFiles.exists(!_.getName.endsWith(".gz"))) + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) verifyFrame(sqlContext.read.text(tempDirPath)) } finally { // Hadoop 1 doesn't have `Configuration.unset` diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 041e0fb477..8a39d95fc5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -161,7 +161,14 @@ private[orc] class OrcOutputWriter( val taskAttemptId = context.getTaskAttemptID val partition = taskAttemptId.getTaskID.getId val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - val filename = f"part-r-$partition%05d-$uniqueWriteJobId$bucketString.orc" + val compressionExtension = { + val name = conf.get(OrcTableProperties.COMPRESSION.getPropName) + OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + } + // It has the `.orc` extension at the end because (de)compression tools + // such as gunzip would not be able to decompress this as the compression + // is not applied on this whole file but on each "stream" in ORC format. + val filename = f"part-r-$partition%05d-$uniqueWriteJobId$bucketString$compressionExtension.orc" new OrcOutputFormat().getRecordWriter( new Path(path, filename).getFileSystem(conf), @@ -328,5 +335,13 @@ private[orc] object OrcRelation { "snappy" -> CompressionKind.SNAPPY, "zlib" -> CompressionKind.ZLIB, "lzo" -> CompressionKind.LZO) + + // The extensions for ORC compression codecs + val extensionsForCompressionCodecNames = Map( + CompressionKind.NONE.name -> "", + CompressionKind.SNAPPY.name -> ".snappy", + CompressionKind.ZLIB.name -> ".zlib", + CompressionKind.LZO.name -> ".lzo" + ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 823b52af1b..2345c1cf9c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -96,7 +96,7 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { // Check if this is compressed as ZLIB. val conf = sparkContext.hadoopConfiguration val fs = FileSystem.getLocal(conf) - val maybeOrcFile = new File(path).listFiles().find(_.getName.endsWith(".orc")) + val maybeOrcFile = new File(path).listFiles().find(_.getName.endsWith(".zlib.orc")) assert(maybeOrcFile.isDefined) val orcFilePath = new Path(maybeOrcFile.get.toPath.toString) val orcReader = OrcFile.createReader(orcFilePath, OrcFile.readerOptions(conf)) |