diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java index 22ec14c576..2c0cc85e30 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.hadoop.util.HadoopFileIO; public class HadoopFSKeyMaterialStore implements FileKeyMaterialStore { public static final String KEY_MATERIAL_FILE_PREFIX = "_KEY_MATERIAL_FOR_"; @@ -73,7 +74,7 @@ public String getKeyMaterial(String keyIDInFile) throws ParquetCryptoRuntimeExce } private void loadKeyMaterialMap() { - try (FSDataInputStream keyMaterialStream = hadoopFileSystem.open(keyMaterialFile)) { + try (FSDataInputStream keyMaterialStream = HadoopFileIO.openFile(hadoopFileSystem, keyMaterialFile, false)) { JsonNode keyMaterialJson = objectMapper.readTree(keyMaterialStream); keyMaterialMap = objectMapper.readValue(keyMaterialJson.traverse(), new TypeReference>() {}); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 1d8cce3d8c..911c05183b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -104,6 +104,7 @@ import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.hadoop.metadata.FileMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopFileIO; import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; import org.apache.parquet.hadoop.util.wrapped.io.FutureIO; @@ -438,19 +439,37 @@ public static List