Skip to content

Commit 3b8d936

Browse files
committed
PARQUET-686: Add getConfiguration to HadoopInputFile.
This is used to avoiding extra public methods in ParquetFileReader.
1 parent 301bd3a commit 3b8d936

File tree

2 files changed

+19
-5
lines changed

2 files changed

+19
-5
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,7 @@
5252
import java.util.concurrent.Executors;
5353
import java.util.concurrent.Future;
5454

55+
import org.apache.hadoop.conf.Configurable;
5556
import org.apache.hadoop.conf.Configuration;
5657
import org.apache.hadoop.fs.FileStatus;
5758
import org.apache.hadoop.fs.FileSystem;
@@ -444,9 +445,16 @@ public static final ParquetMetadata readFooter(Configuration configuration, File
444445
*/
445446
public static final ParquetMetadata readFooter(
446447
InputFile file, MetadataFilter filter) throws IOException {
448+
ParquetMetadataConverter converter;
449+
if (file instanceof HadoopInputFile) {
450+
converter = new ParquetMetadataConverter(
451+
((HadoopInputFile) file).getConfiguration());
452+
} else {
453+
converter = new ParquetMetadataConverter();
454+
}
447455
try (SeekableInputStream in = file.newStream()) {
448-
return readFooter(new ParquetMetadataConverter(), file.getLength(),
449-
file.toString(), in, filter);
456+
457+
return readFooter(converter, file.getLength(), file.toString(), in, filter);
450458
}
451459
}
452460

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopInputFile.java

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,22 +31,28 @@ public class HadoopInputFile implements InputFile {
3131

3232
private final FileSystem fs;
3333
private final FileStatus stat;
34+
private final Configuration conf;
3435

3536
public static HadoopInputFile fromPath(Path path, Configuration conf)
3637
throws IOException {
3738
FileSystem fs = path.getFileSystem(conf);
38-
return new HadoopInputFile(fs, fs.getFileStatus(path));
39+
return new HadoopInputFile(fs, fs.getFileStatus(path), conf);
3940
}
4041

4142
public static HadoopInputFile fromStatus(FileStatus stat, Configuration conf)
4243
throws IOException {
4344
FileSystem fs = stat.getPath().getFileSystem(conf);
44-
return new HadoopInputFile(fs, stat);
45+
return new HadoopInputFile(fs, stat, conf);
4546
}
4647

47-
private HadoopInputFile(FileSystem fs, FileStatus stat) {
48+
private HadoopInputFile(FileSystem fs, FileStatus stat, Configuration conf) {
4849
this.fs = fs;
4950
this.stat = stat;
51+
this.conf = conf;
52+
}
53+
54+
public Configuration getConfiguration() {
55+
return conf;
5056
}
5157

5258
@Override

0 commit comments

Comments
 (0)