-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
[SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader #39950
Conversation
@@ -207,11 +207,11 @@ class ParquetFileFormat | |||
|
|||
lazy val footerFileMetaData = | |||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData | |||
val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( | |||
lazy val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( | |||
footerFileMetaData.getKeyValueMetaData.get, |
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.
footerFileMetaData
is lazy, but datetimeRebaseSpec
causes the footer to be read immediately.
Actually we can avoid this unnecessary footer reads and use footer metadata in VectorizedParquetRecordReader
@MaxGekk @gengliangwang If you have time, please take a look, thanks |
cc @sunchao , too. |
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.
Thanks @yabola . I feel this PR is not that useful though since in most cases there will be filters pushed down to Parquet. Instead, a better approach IMO is to introduce another constructor (there is one but is marked as deprecated) on ParquetFileReader
which takes a footer
as input, so it doesn't need to read it again. We can then pass the footer obtained in ParquetFileFormat
to it via the ctor of VectorizedParquetRecordReader
.
We also should support this for data source V2, in ParquetPartitionReaderFactory
.
@sunchao Thank you for your reply!
Actually the second So when in case that we need filter pushdown and also |
@yabola yes, we'll need to use The only problem is we need new non-deprecated API from |
@sunchao Sorry, it might be necessary to read footer twice if having filters. We should read schema in footer meta first to get which filters need to be pushed down. After that we set pushdown info ((https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L261) and read filtered RowGroups with the filter configuration. But we can reduce footer reading when no filter is needed pushdown. |
@sunchao Hi~ Could you take a look at this PR? I think it will be useful when there are joined tables and filter conditions are on few tables. |
(sorry for the late reply!) Hmm why? when we read the footer for the first time, the result already contain all the row groups. We just need to pass these to |
...re/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
Outdated
Show resolved
Hide resolved
@sunchao please take a look, thank you |
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.
We also need to support V2 data source, e.g., ParquetPartitionReaderFactory
@@ -205,11 +212,21 @@ class ParquetFileFormat | |||
|
|||
val sharedConf = broadcastedHadoopConf.value.value | |||
|
|||
lazy val footerFileMetaData = | |||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData | |||
val fileRange = HadoopReadOptions.builder(sharedConf, split.getPath) |
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.
can we add these in ParquetFooterReader
? we may need to use try-with-resources
clause to make sure resources are properly closed.
we can just obtain the footer here and use it later for footerFileMetaData
and pass it to VectorizedParquetRecordReader
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.
Yes, before I created and passed ParquetFileReader
because I wanted to create one less file.newStream()
in it (if there is no filter pushdown), but it doesn’t seem to make much sense. I have changed to pass footer here.
Please take a look, thank you!
@@ -279,7 +301,7 @@ class ParquetFileFormat | |||
// Instead, we use FileScanRDD's task completion listener to close this iterator. | |||
val iter = new RecordReaderIterator(vectorizedReader) | |||
try { | |||
vectorizedReader.initialize(split, hadoopAttemptContext) | |||
vectorizedReader.initialize(split, hadoopAttemptContext, fileReader) |
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.
can we pass footer to initialize
instead?
// Read all the row groups in advance and filter the row groups later if there are | ||
// filters that need push down. | ||
ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS) | ||
} else { |
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.
I think we can use
if (aggregation.isDefined || enableVectorizedReader) {
ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
}
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.
it looks we can, since when aggregation push down is enabled, ParquetScan.isSplitable
returns false, and we'll always read all the row groups in the file, so NO_FILTER
is the same as WITH_ROW_GROUPS
.
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.
Thanks @yabola ! looks much better now. Have a few more nits and I think we are close to merge after that.
} else { | ||
filter = HadoopReadOptions.builder(configuration, split.getPath()) | ||
.withRange(split.getStart(), split.getStart() + split.getLength()) | ||
.withCodecFactory(new ParquetCodecFactory(configuration, 0)) |
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.
hmm is this required? we don't need the codec factory when reading footer?
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.
Thank you for review. Yes, if just read footer, there is no need to bring
public static final boolean WITH_ROW_GROUPS = false; | ||
|
||
/** | ||
* method to read parquet file footer |
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: How about:
Reads footer for the input Parquet file 'split'. If 'skipRowGroup' is true, this will skip reading the Parquet row group metadata.
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.
done
* if false, read row groups according to the file range | ||
*/ | ||
public static ParquetMetadata readFooter( | ||
Configuration configuration, |
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: 4 space indent
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.
done
} | ||
|
||
public void initialize( | ||
InputSplit inputSplit, |
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: 4 space indent
@@ -181,6 +184,16 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont | |||
initializeInternal(); | |||
} | |||
|
|||
@Override | |||
public void initialize( | |||
InputSplit inputSplit, |
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: 4 space indent
lazy val footerFileMetaData = | ||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData | ||
val fileFooter = if (enableVectorizedReader) { | ||
// This can avoid reading the footer twice(currently only optimize for vectorized read). |
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: space after "twice"
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.
not addressed yet
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.
I updated comments, thank you
*/ | ||
public static ParquetMetadata readFooter( | ||
Configuration configuration, | ||
FileSplit 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.
instead of using FileSplit
, can we just pass PartitionedFile
here? FileSplit
is from org.apache.hadoop.mapred
which is really out-dated.
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.
done, I have changed to pass PartitionedFile
// Read all the row groups in advance and filter the row groups later if there are | ||
// filters that need push down. | ||
ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS) | ||
} else { |
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.
it looks we can, since when aggregation push down is enabled, ParquetScan.isSplitable
returns false, and we'll always read all the row groups in the file, so NO_FILTER
is the same as WITH_ROW_GROUPS
.
...java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
Show resolved
Hide resolved
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.
...java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
Show resolved
Hide resolved
lazy val footerFileMetaData = | ||
ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData | ||
val fileFooter = if (enableVectorizedReader) { | ||
// This can avoid reading the footer twice(currently only optimize for vectorized read). |
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.
not addressed yet
… vectorized reader
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.
LGTM except one nit
Configuration configuration, | ||
PartitionedFile partitionedFile, | ||
boolean skipRowGroup) throws IOException { | ||
FileSplit split = new FileSplit(partitionedFile.toPath(), partitionedFile.start(), |
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 there is no need to use FileSplit
and hence depend on org.apache.hadoop.mapred
here. Instead we can do:
long start = file.start();
long length = file.length();
Path filePath = new Path(new URI(file.filePath().toString()));
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.
Good idea, done
Merged to master, thanks! |
@sunchao Thank you for your detailed review! |
@yabola @sunchao Could you share any benchmark numbers for the second optimisation of reading all row groups for each task? My concern is that it could be suboptimal in performance when you have, let's say, 100 row groups in a file, you create 100 tasks for each row group but then you read the full footer with all of the row groups for every task just to process one row group. |
@sadikovi Thanks for your advice. I would do a benchmark in the scenario you describe latter. My understanding is that I didn't read all the RowGroups, but included the range filter (each task has its own start and end in the FileSplit, may contain multiple RowGroups). Lines 61 to 64 in 0515e6b
But it does not contain information of filter pushdown (block meta reading seems inevitable here, because without reading, it is impossible to know whether it is needed? ). I will follow up to confirm your concerns latter. |
Yea @yabola is correct, if we have 100 row groups in a file and there are 100 tasks to read them, each task will only be assigned a range (e.g., a single row group) in the file to read, so it won't read metadata for all the row groups in the file. |
@sadikovi I have tested the scenario as you said. The smaller the row group size (footer size will be larger), the higher the PR benefits. |
…der (apache#1877) Parquet footer metadata is now always read twice in vectorized parquet reader. When the NameNode is under high pressure, it will cost time to read twice. Actually we can avoid reading the footer twice by reading all row groups in advance and filter row groups according to filters that require push down (no need to read the footer metadata again the second time). Reduce the reading of footer in vectorized parquet reader no existing tests Closes apache#39950 from yabola/skip_footer. Authored-by: chenliang.lu <[email protected]> Signed-off-by: Chao Sun <[email protected]> Co-authored-by: chenliang.lu <[email protected]>
SPARK-48950 seems to report a correctness issue about this. When you have some time, could you check it, please? |
@dongjoon-hyun I will look into this later. |
What changes were proposed in this pull request?
Parquet footer metadata is now always read twice in vectorized parquet reader.
When the NameNode is under high pressure, it will cost time to read twice. Actually we can avoid reading the footer twice by reading all row groups in advance and filter row groups according to filters that require push down (no need to read the footer metadata again the second time).
Why are the changes needed?
Reduce the reading of footer in vectorized parquet reader
Does this PR introduce any user-facing change?
no
How was this patch tested?
existing tests