8888import org .apache .parquet .hadoop .metadata .ColumnChunkMetaData ;
8989import org .apache .parquet .hadoop .metadata .FileMetaData ;
9090import org .apache .parquet .hadoop .metadata .ParquetMetadata ;
91+ import org .apache .parquet .hadoop .util .HadoopInputFile ;
9192import org .apache .parquet .hadoop .util .HiddenFileFilter ;
9293import org .apache .parquet .hadoop .util .HadoopStreams ;
9394import org .apache .parquet .io .SeekableInputStream ;
9495import org .apache .parquet .hadoop .util .counters .BenchmarkCounter ;
9596import org .apache .parquet .io .ParquetDecodingException ;
97+ import org .apache .parquet .io .InputFile ;
9698
9799/**
98100 * Internal implementation of the Parquet file reader as a block container
@@ -410,8 +412,7 @@ public static final ParquetMetadata readFooter(Configuration configuration, Path
410412 * @throws IOException if an error occurs while reading the file
411413 */
412414 public static ParquetMetadata readFooter (Configuration configuration , Path file , MetadataFilter filter ) throws IOException {
413- FileSystem fileSystem = file .getFileSystem (configuration );
414- return readFooter (configuration , fileSystem .getFileStatus (file ), filter );
415+ return readFooter (HadoopInputFile .fromPath (file , configuration ), filter );
415416 }
416417
417418 /**
@@ -431,12 +432,21 @@ public static final ParquetMetadata readFooter(Configuration configuration, File
431432 * @throws IOException if an error occurs while reading the file
432433 */
433434 public static final ParquetMetadata readFooter (Configuration configuration , FileStatus file , MetadataFilter filter ) throws IOException {
434- FileSystem fileSystem = file .getPath ().getFileSystem (configuration );
435- SeekableInputStream in = HadoopStreams .wrap (fileSystem .open (file .getPath ()));
436- try {
437- return readFooter (file .getLen (), file .getPath ().toString (), in , filter );
438- } finally {
439- in .close ();
435+ return readFooter (HadoopInputFile .fromStatus (file , configuration ), filter );
436+ }
437+
438+ /**
439+ * Reads the meta data block in the footer of the file using provided input stream
440+ * @param file a {@link InputFile} to read
441+ * @param filter the filter to apply to row groups
442+ * @return the metadata blocks in the footer
443+ * @throws IOException if an error occurs while reading the file
444+ */
445+ public static final ParquetMetadata readFooter (
446+ InputFile file , MetadataFilter filter ) throws IOException {
447+ try (SeekableInputStream in = file .newStream ()) {
448+ return readFooter (converter , file .getLength (), file .toString (),
449+ in , filter );
440450 }
441451 }
442452
@@ -449,7 +459,7 @@ public static final ParquetMetadata readFooter(Configuration configuration, File
449459 * @return the metadata blocks in the footer
450460 * @throws IOException if an error occurs while reading the file
451461 */
452- public static final ParquetMetadata readFooter (long fileLen , String filePath , SeekableInputStream f , MetadataFilter filter ) throws IOException {
462+ private static final ParquetMetadata readFooter (ParquetMetadataConverter converter , long fileLen , String filePath , SeekableInputStream f , MetadataFilter filter ) throws IOException {
453463 if (Log .DEBUG ) {
454464 LOG .debug ("File length " + fileLen );
455465 }
@@ -563,7 +573,7 @@ public ParquetFileReader(Configuration conf, Path file, MetadataFilter filter) t
563573 FileSystem fs = file .getFileSystem (conf );
564574 this .fileStatus = fs .getFileStatus (file );
565575 this .f = HadoopStreams .wrap (fs .open (file ));
566- this .footer = readFooter (fileStatus .getLen (), fileStatus .getPath ().toString (), f , filter );
576+ this .footer = readFooter (converter , fileStatus .getLen (), fileStatus .getPath ().toString (), f , filter );
567577 this .fileMetaData = footer .getFileMetaData ();
568578 this .blocks = footer .getBlocks ();
569579 for (ColumnDescriptor col : footer .getFileMetaData ().getSchema ().getColumns ()) {
@@ -602,7 +612,7 @@ public ParquetMetadata getFooter() {
602612 if (footer == null ) {
603613 try {
604614 // don't read the row groups because this.blocks is always set
605- this .footer = readFooter (fileStatus .getLen (), fileStatus .getPath ().toString (), f , SKIP_ROW_GROUPS );
615+ this .footer = readFooter (converter , fileStatus .getLen (), fileStatus .getPath ().toString (), f , SKIP_ROW_GROUPS );
606616 } catch (IOException e ) {
607617 throw new ParquetDecodingException ("Unable to read file footer" , e );
608618 }
0 commit comments