-
Notifications
You must be signed in to change notification settings - Fork 1.5k
GH-3141: Add constructor to ParquetFileReader to allow passing in parquet footer and expose setRequestedSchema that accepts List<ColumnDescriptor>
#3262
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
Changes from all commits
6577dc8
0ea55b7
3038220
82daf68
9242b3e
7e28a23
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -36,6 +36,8 @@ | |
| import java.util.zip.CRC32; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.parquet.HadoopReadOptions; | ||
| import org.apache.parquet.ParquetReadOptions; | ||
| import org.apache.parquet.bytes.BytesInput; | ||
| import org.apache.parquet.bytes.HeapByteBufferAllocator; | ||
| import org.apache.parquet.bytes.TrackingByteBufferAllocator; | ||
|
|
@@ -688,8 +690,13 @@ private int getDataOffset(Page page) { | |
| */ | ||
| private ParquetFileReader getParquetFileReader(Path path, Configuration conf, List<ColumnDescriptor> columns) | ||
| throws IOException { | ||
| ParquetMetadata footer = ParquetFileReader.readFooter(conf, path); | ||
| return new ParquetFileReader(conf, footer.getFileMetaData(), path, footer.getBlocks(), columns); | ||
| HadoopInputFile inputFile = HadoopInputFile.fromPath(path, conf); | ||
| SeekableInputStream inputStream = inputFile.newStream(); | ||
| ParquetReadOptions readOptions = HadoopReadOptions.builder(conf).build(); | ||
| ParquetMetadata footer = ParquetFileReader.readFooter(inputFile, readOptions, inputStream); | ||
| ParquetFileReader reader = ParquetFileReader.open(inputFile, footer, readOptions, inputStream); | ||
| reader.setRequestedSchema(columns); | ||
| return reader; | ||
|
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. It would be better to explicitly add a new test case rather than silently changing old ones.
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. I understand that, but it seems there is no dedicated test suite for |
||
| } | ||
|
|
||
| /** | ||
|
|
||
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.
Why do we need this change? Does it exist in the original PR?
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.
the diff here is not smart, this actually adds a new method
public void setRequestedSchema(List<ColumnDescriptor> columns)there are many ctors of
ParquetFileReaderwere marked as deprecated but without providing the equivalent replacement, for example,this API is proposed to restore the ability to set read columns via
List<ColumnDescriptor>, instead ofMessageType