aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorhyukjinkwon <gurwls223@gmail.com>2016-03-09 19:12:46 -0800
committerReynold Xin <rxin@databricks.com>2016-03-09 19:12:46 -0800
commitaa0eba2c354dc57dd83a427daa68d6171f292a83 (patch)
tree72d0d85b3c530be8f564f99ea1f9b3fd1d1d63e4 /sql/hive
parent790646125edd8b853e2ad2425112590e78799bd3 (diff)
downloadspark-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/hive')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala17
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala2
2 files changed, 17 insertions, 2 deletions
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))