Skip to content

Commit

Permalink
GH-3029: Fix EncryptionPropertiesHelper not to use java.nio.file.Path (
Browse files Browse the repository at this point in the history
  • Loading branch information
wgtmac authored and Fokko committed Nov 6, 2024
1 parent a30f9be commit 9584632
Show file tree
Hide file tree
Showing 2 changed files with 4 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

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;
Expand Down Expand Up @@ -387,9 +386,7 @@ public ParquetWriter(Path file, Configuration conf, WriteSupport<T> writeSupport
// 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(
Expand Down

0 comments on commit 9584632

Please sign in to comment.