-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-53633][SQL] Reuse InputStream in vectorized Parquet reader #52384
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
Closed
Closed
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
3cc3dc2
[SPARK-53633][SQL] Reuse InputStream in vectorized Parquet reader
pan3793 2377158
break long method buildReaderWithPartitionValues
pan3793 ff3682a
restore comment
pan3793 e21983e
OpenedParquetFooter
pan3793 9344c0f
Fix assertion
pan3793 cc9f9e0
covert ParquetFooterReader back to Java
pan3793 d9e8b5e
Fix import order
pan3793 9266816
fix style
pan3793 c9ee046
comment
pan3793 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
34 changes: 34 additions & 0 deletions
34
...src/main/java/org/apache/spark/sql/execution/datasources/parquet/OpenedParquetFooter.java
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,34 @@ | ||
| /* | ||
| * 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.execution.datasources.parquet; | ||
|
|
||
| import java.util.Optional; | ||
|
|
||
| import org.apache.parquet.hadoop.metadata.ParquetMetadata; | ||
| import org.apache.parquet.hadoop.util.HadoopInputFile; | ||
| import org.apache.parquet.io.SeekableInputStream; | ||
|
|
||
| public record OpenedParquetFooter( | ||
| ParquetMetadata footer, | ||
| HadoopInputFile inputFile, | ||
| Optional<SeekableInputStream> inputStreamOpt) { | ||
|
|
||
| public SeekableInputStream inputStream() { | ||
| return inputStreamOpt.get(); | ||
| } | ||
| } |
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 |
|---|---|---|
|
|
@@ -18,10 +18,9 @@ | |
| package org.apache.spark.sql.execution.datasources.parquet; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Optional; | ||
|
|
||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.FileStatus; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.parquet.HadoopReadOptions; | ||
| import org.apache.parquet.ParquetReadOptions; | ||
| import org.apache.parquet.format.converter.ParquetMetadataConverter; | ||
|
|
@@ -37,53 +36,77 @@ | |
| */ | ||
| public class ParquetFooterReader { | ||
|
|
||
| public static final boolean SKIP_ROW_GROUPS = true; | ||
| public static final boolean WITH_ROW_GROUPS = false; | ||
|
|
||
| /** | ||
| * Reads footer for the input Parquet file 'split'. If 'skipRowGroup' is true, | ||
| * this will skip reading the Parquet row group metadata. | ||
| * Build a filter for reading footer of the input Parquet file 'split'. | ||
| * If 'skipRowGroup' is true, this will skip reading the Parquet row group metadata. | ||
| * | ||
| * @param file a part (i.e. "block") of a single file that should be read | ||
| * @param configuration hadoop configuration of file | ||
| * @param hadoopConf hadoop configuration of file | ||
| * @param skipRowGroup If true, skip reading row groups; | ||
| * if false, read row groups according to the file split range | ||
| */ | ||
| public static ParquetMetadata readFooter( | ||
| Configuration configuration, | ||
| PartitionedFile file, | ||
| boolean skipRowGroup) throws IOException { | ||
| long fileStart = file.start(); | ||
| ParquetMetadataConverter.MetadataFilter filter; | ||
| public static ParquetMetadataConverter.MetadataFilter buildFilter( | ||
| Configuration hadoopConf, PartitionedFile file, boolean skipRowGroup) { | ||
| if (skipRowGroup) { | ||
| filter = ParquetMetadataConverter.SKIP_ROW_GROUPS; | ||
| return ParquetMetadataConverter.SKIP_ROW_GROUPS; | ||
| } else { | ||
| filter = HadoopReadOptions.builder(configuration, file.toPath()) | ||
| long fileStart = file.start(); | ||
| return HadoopReadOptions.builder(hadoopConf, file.toPath()) | ||
| .withRange(fileStart, fileStart + file.length()) | ||
| .build() | ||
| .getMetadataFilter(); | ||
| } | ||
| return readFooter(configuration, file.toPath(), filter); | ||
| } | ||
|
|
||
| public static ParquetMetadata readFooter(Configuration configuration, | ||
| Path file, ParquetMetadataConverter.MetadataFilter filter) throws IOException { | ||
| return readFooter(HadoopInputFile.fromPath(file, configuration), filter); | ||
| } | ||
|
|
||
| public static ParquetMetadata readFooter(Configuration configuration, | ||
| FileStatus fileStatus, ParquetMetadataConverter.MetadataFilter filter) throws IOException { | ||
| return readFooter(HadoopInputFile.fromStatus(fileStatus, configuration), filter); | ||
| } | ||
|
|
||
| private static ParquetMetadata readFooter(HadoopInputFile inputFile, | ||
| public static ParquetMetadata readFooter( | ||
| HadoopInputFile inputFile, | ||
| ParquetMetadataConverter.MetadataFilter filter) throws IOException { | ||
| ParquetReadOptions readOptions = | ||
| HadoopReadOptions.builder(inputFile.getConfiguration(), inputFile.getPath()) | ||
| ParquetReadOptions readOptions = HadoopReadOptions | ||
| .builder(inputFile.getConfiguration(), inputFile.getPath()) | ||
| .withMetadataFilter(filter).build(); | ||
| // Use try-with-resources to ensure fd is closed. | ||
| try (ParquetFileReader fileReader = ParquetFileReader.open(inputFile, readOptions)) { | ||
| try (var fileReader = ParquetFileReader.open(inputFile, readOptions)) { | ||
| return fileReader.getFooter(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Decoding Parquet files generally involves two steps: | ||
| * 1. read and resolve the metadata (footer), | ||
| * 2. read and decode the row groups/column chunks. | ||
| * <p> | ||
| * It's possible to avoid opening the file twice by resuing the SeekableInputStream. | ||
| * When keepInputStreamOpen is true, the caller takes responsibility to close the | ||
| * SeekableInputStream. Currently, this is only supported by parquet vectorized reader. | ||
| * | ||
| * @param hadoopConf hadoop configuration of file | ||
| * @param file a part (i.e. "block") of a single file that should be read | ||
| * @param keepInputStreamOpen when true, keep the SeekableInputStream of file being open | ||
| * @return if keepInputStreamOpen is true, the returned OpenedParquetFooter carries | ||
| * Some(SeekableInputStream), otherwise None. | ||
| */ | ||
| public static OpenedParquetFooter openFileAndReadFooter( | ||
| Configuration hadoopConf, | ||
| PartitionedFile file, | ||
| boolean keepInputStreamOpen) throws IOException { | ||
| var readOptions = HadoopReadOptions.builder(hadoopConf, file.toPath()) | ||
| // `keepInputStreamOpen` is true only when parquet vectorized reader is used | ||
| // on the caller side, in such a case, the footer will be resued later on | ||
| // reading row groups, so here must read row groups metadata ahead. | ||
| // when false, the caller uses parquet-mr to read the file, only file metadata | ||
| // is required on planning phase, and parquet-mr will read the footer again | ||
| // on reading row groups. | ||
| .withMetadataFilter(buildFilter(hadoopConf, file, !keepInputStreamOpen)) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maybe worth adding some comments here to explain why we choose to skip row groups when
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. added a comment |
||
| .build(); | ||
| var inputFile = HadoopInputFile.fromPath(file.toPath(), hadoopConf); | ||
| var inputStream = inputFile.newStream(); | ||
| try (var fileReader = ParquetFileReader.open(inputFile, readOptions, inputStream)) { | ||
| var footer = fileReader.getFooter(); | ||
| if (keepInputStreamOpen) { | ||
| fileReader.detachFileInputStream(); | ||
| return new OpenedParquetFooter(footer, inputFile, Optional.of(inputStream)); | ||
| } else { | ||
| return new OpenedParquetFooter(footer, inputFile, Optional.empty()); | ||
| } | ||
| } | ||
| } | ||
| } | ||
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I think the doc is out-dated - there is no
'split'There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thing has not changed, the 'split' represents
PartitionedFilehere