This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new d2128afda GH-3029: Fix EncryptionPropertiesHelper not to use
java.nio.file.Path (#3038)
d2128afda is described below
commit d2128afda4ba53667e95128f9de50518b555c96d
Author: Gang Wu <[email protected]>
AuthorDate: Fri Nov 1 11:12:34 2024 +0800
GH-3029: Fix EncryptionPropertiesHelper not to use java.nio.file.Path
(#3038)
---
.../org/apache/parquet/hadoop/EncryptionPropertiesHelper.java | 8 +++-----
.../src/main/java/org/apache/parquet/hadoop/ParquetWriter.java | 5 +----
2 files changed, 4 insertions(+), 9 deletions(-)
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java
index 8d4f6f7ca..090f29d30 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java
@@ -18,27 +18,25 @@
*/
package org.apache.parquet.hadoop;
-import java.net.URI;
-import java.nio.file.Path;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.conf.ParquetConfiguration;
import org.apache.parquet.crypto.EncryptionPropertiesFactory;
import org.apache.parquet.crypto.FileEncryptionProperties;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.util.ConfigurationUtil;
+import org.apache.parquet.io.OutputFile;
final class EncryptionPropertiesHelper {
static FileEncryptionProperties createEncryptionProperties(
- ParquetConfiguration fileParquetConfig, Path tempFilePath,
WriteSupport.WriteContext fileWriteContext) {
+ ParquetConfiguration fileParquetConfig, OutputFile file,
WriteSupport.WriteContext fileWriteContext) {
EncryptionPropertiesFactory cryptoFactory =
EncryptionPropertiesFactory.loadFactory(fileParquetConfig);
if (null == cryptoFactory) {
return null;
}
Configuration hadoopConf =
ConfigurationUtil.createHadoopConfiguration(fileParquetConfig);
- URI path = tempFilePath == null ? null : tempFilePath.toUri();
return cryptoFactory.getFileEncryptionProperties(
- hadoopConf, path == null ? null : new org.apache.hadoop.fs.Path(path),
fileWriteContext);
+ hadoopConf, file == null ? null : new
org.apache.hadoop.fs.Path(file.getPath()), fileWriteContext);
}
static FileEncryptionProperties createEncryptionProperties(
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index 60867aa93..4fa6d96be 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -20,7 +20,6 @@ package org.apache.parquet.hadoop;
import java.io.Closeable;
import java.io.IOException;
-import java.nio.file.Paths;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
@@ -387,9 +386,7 @@ public class ParquetWriter<T> implements Closeable {
// encryptionProperties could be built from the implementation of
EncryptionPropertiesFactory when it is
// attached.
if (encryptionProperties == null) {
- String path = file == null ? null : file.getPath();
- encryptionProperties =
EncryptionPropertiesHelper.createEncryptionProperties(
- conf, path == null ? null : Paths.get(path), writeContext);
+ encryptionProperties =
EncryptionPropertiesHelper.createEncryptionProperties(conf, file, writeContext);
}
ParquetFileWriter fileWriter = new ParquetFileWriter(