-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-52482][SQL][CORE] ZStandard support for file data source reader #51182
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
sandip-db
wants to merge
13
commits into
apache:master
Choose a base branch
from
sandip-db:zstd-file-source-support
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
a9093ee
[SPARK-52482][SQL] Inline Hadoop's LineRecordReader
sandip-db 16b4bf8
Add support for ZSTD decompression and non-standard extensions like .…
sandip-db 78c8699
zstd support in Hadoop line reader; Add test in TextSuite.scala
sandip-db 1ada4a9
More tests
sandip-db 48e1d93
linter fix
sandip-db c158673
Add comments
sandip-db 313eb83
Debug only changes to debug GitHub Action Failure - Will revert soon …
sandip-db 58da804
debug
sandip-db d9268d9
test fix
sandip-db 8566580
remove SparkConf
sandip-db 4b7cf40
indent
sandip-db fa0bfc5
Add tests with files compressed using ZSTD version 1.4.4+dfsg-3ubuntu0.1
sandip-db 66c3533
Add a test case with a file compressed using Hadoop native ZSTD codec
sandip-db File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
94 changes: 94 additions & 0 deletions
94
core/src/main/scala/org/apache/spark/io/HadoopCodecStreams.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
/* | ||
* 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.io | ||
|
||
import java.io.InputStream | ||
import java.util.Locale | ||
|
||
import scala.collection.Seq | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.io.compress._ | ||
|
||
import org.apache.spark.{SparkConf, SparkEnv} | ||
import org.apache.spark.io.{CompressionCodec => SparkCompressionCodec} | ||
|
||
/** | ||
* An utility object to look up Hadoop compression codecs and create input streams. | ||
* In addition to standard Hadoop codecs, it also supports Spark's Zstandard codec | ||
* if Hadopp is not compiled with Zstandard support. Additionally, it supports | ||
* non-standard file extensions like `.zstd` and `.gzip` for Zstandard and Gzip codecs. | ||
*/ | ||
object HadoopCodecStreams { | ||
private val ZSTD_EXTENSIONS = Seq(".zstd", ".zst") | ||
|
||
// get codec based on file name extension | ||
def getDecompressionCodec( | ||
config: Configuration, | ||
file: Path): Option[CompressionCodec] = { | ||
val factory = new CompressionCodecFactory(config) | ||
Option(factory.getCodec(file)).orElse { | ||
// Try some non-standards extensions for Zstandard and Gzip | ||
file.getName.toLowerCase() match { | ||
sandip-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
case name if name.endsWith(".zstd") => | ||
Option(factory.getCodecByName(classOf[ZStandardCodec].getName)) | ||
case name if name.endsWith(".gzip") => | ||
Option(factory.getCodecByName(classOf[GzipCodec].getName)) | ||
case _ => None | ||
} | ||
} | ||
} | ||
|
||
def createZstdInputStream( | ||
file: Path, | ||
inputStream: InputStream): Option[InputStream] = { | ||
val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf) | ||
val fileName = file.getName.toLowerCase(Locale.ROOT) | ||
|
||
val isOpt = if (ZSTD_EXTENSIONS.exists(fileName.endsWith)) { | ||
Some( | ||
SparkCompressionCodec | ||
.createCodec(sparkConf, SparkCompressionCodec.ZSTD) | ||
.compressedInputStream(inputStream) | ||
) | ||
} else { | ||
None | ||
} | ||
isOpt | ||
} | ||
|
||
def createInputStream( | ||
config: Configuration, | ||
file: Path): InputStream = { | ||
val fs = file.getFileSystem(config) | ||
val inputStream: InputStream = fs.open(file) | ||
|
||
getDecompressionCodec(config, file) | ||
.map { codec => | ||
try { | ||
codec.createInputStream(inputStream) | ||
} catch { | ||
case e: RuntimeException => | ||
// createInputStream may fail for ZSTD if hadoop is not already compiled with ZSTD | ||
// support. In that case, we try to use Spark's Zstandard codec. | ||
createZstdInputStream(file, inputStream).getOrElse(throw e) | ||
} | ||
}.getOrElse(inputStream) | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.