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 7be98076f PARQUET-1126: Write unencrypted Parquet files without Hadoop 
(#1376)
7be98076f is described below

commit 7be98076f30faad3626155666a8e2a9ebe922c6a
Author: David Venable <d...@amazon.com>
AuthorDate: Mon Jul 29 21:04:09 2024 -0500

    PARQUET-1126: Write unencrypted Parquet files without Hadoop (#1376)
    
    * Internal changes to allow writing unencrypted Parquet without needing 
Hadoop in the classpath. Relates to PARQUET-1126.
    * Avoid loading encryption factory twice.
---
 .../crypto/EncryptionPropertiesFactory.java        | 17 ++++++-
 .../parquet/hadoop/EncryptionPropertiesHelper.java | 54 ++++++++++++++++++++++
 .../apache/parquet/hadoop/ParquetOutputFormat.java |  7 +--
 .../org/apache/parquet/hadoop/ParquetWriter.java   |  6 +--
 .../crypto/EncryptionPropertiesFactoryTest.java    | 23 +++++++++
 5 files changed, 97 insertions(+), 10 deletions(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
index c5425931a..a49ae2db1 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/EncryptionPropertiesFactory.java
@@ -20,6 +20,8 @@ package org.apache.parquet.crypto;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.parquet.conf.HadoopParquetConfiguration;
+import org.apache.parquet.conf.ParquetConfiguration;
 import org.apache.parquet.hadoop.BadConfigurationException;
 import org.apache.parquet.hadoop.api.WriteSupport.WriteContext;
 import org.apache.parquet.hadoop.util.ConfigurationUtil;
@@ -53,12 +55,25 @@ public interface EncryptionPropertiesFactory {
    * Load EncryptionPropertiesFactory class specified by 
CRYPTO_FACTORY_CLASS_PROPERTY_NAME as the path in the
    * configuration
    *
-   * @param conf Configuration where user specifies the class path
+   * @param conf Hadoop Configuration where user specifies the class path
    * @return object with class EncryptionPropertiesFactory if user specified 
the class path and invoking of
    * the class succeeds. Null if user doesn't specify the class path (no 
encryption then).
    * @throws BadConfigurationException if the instantiation of the configured 
class fails
    */
   static EncryptionPropertiesFactory loadFactory(Configuration conf) {
+    return loadFactory(new HadoopParquetConfiguration(conf));
+  }
+
+  /**
+   * Load EncryptionPropertiesFactory class specified by 
CRYPTO_FACTORY_CLASS_PROPERTY_NAME as the path in the
+   * configuration
+   *
+   * @param conf ParquetConfiguration where user specifies the class path
+   * @return object with class EncryptionPropertiesFactory if user specified 
the class path and invoking of
+   * the class succeeds. Null if user doesn't specify the class path (no 
encryption then).
+   * @throws BadConfigurationException if the instantiation of the configured 
class fails
+   */
+  static EncryptionPropertiesFactory loadFactory(ParquetConfiguration conf) {
     final Class<?> encryptionPropertiesFactoryClass = 
ConfigurationUtil.getClassFromConfig(
         conf, CRYPTO_FACTORY_CLASS_PROPERTY_NAME, 
EncryptionPropertiesFactory.class);
 
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
new file mode 100644
index 000000000..8d4f6f7ca
--- /dev/null
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+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;
+
+final class EncryptionPropertiesHelper {
+  static FileEncryptionProperties createEncryptionProperties(
+      ParquetConfiguration fileParquetConfig, Path tempFilePath, 
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);
+  }
+
+  static FileEncryptionProperties createEncryptionProperties(
+      Configuration fileHadoopConfig,
+      org.apache.hadoop.fs.Path tempFilePath,
+      WriteSupport.WriteContext fileWriteContext) {
+    EncryptionPropertiesFactory cryptoFactory = 
EncryptionPropertiesFactory.loadFactory(fileHadoopConfig);
+    if (null == cryptoFactory) {
+      return null;
+    }
+    return cryptoFactory.getFileEncryptionProperties(fileHadoopConfig, 
tempFilePath, fileWriteContext);
+  }
+}
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
index 37a551cde..801da0501 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.column.ParquetProperties.WriterVersion;
-import org.apache.parquet.crypto.EncryptionPropertiesFactory;
 import org.apache.parquet.crypto.FileEncryptionProperties;
 import org.apache.parquet.hadoop.ParquetFileWriter.Mode;
 import org.apache.parquet.hadoop.api.WriteSupport;
@@ -571,10 +570,6 @@ public class ParquetOutputFormat<T> extends 
FileOutputFormat<Void, T> {
 
   public static FileEncryptionProperties createEncryptionProperties(
       Configuration fileHadoopConfig, Path tempFilePath, WriteContext 
fileWriteContext) {
-    EncryptionPropertiesFactory cryptoFactory = 
EncryptionPropertiesFactory.loadFactory(fileHadoopConfig);
-    if (null == cryptoFactory) {
-      return null;
-    }
-    return cryptoFactory.getFileEncryptionProperties(fileHadoopConfig, 
tempFilePath, fileWriteContext);
+    return 
EncryptionPropertiesHelper.createEncryptionProperties(fileHadoopConfig, 
tempFilePath, fileWriteContext);
   }
 }
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 911288988..ac868df8b 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,6 +20,7 @@ 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 +388,8 @@ public class ParquetWriter<T> implements Closeable {
     // attached.
     if (encryptionProperties == null) {
       String path = file == null ? null : file.getPath();
-      Configuration hadoopConf = 
ConfigurationUtil.createHadoopConfiguration(conf);
-      encryptionProperties = ParquetOutputFormat.createEncryptionProperties(
-          hadoopConf, path == null ? null : new Path(path), writeContext);
+      encryptionProperties = 
EncryptionPropertiesHelper.createEncryptionProperties(
+          conf, path == null ? null : Paths.get(path), writeContext);
     }
 
     ParquetFileWriter fileWriter = new ParquetFileWriter(
diff --git 
a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
 
b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
index 1fdc98965..51133251d 100644
--- 
a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
+++ 
b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/EncryptionPropertiesFactoryTest.java
@@ -23,6 +23,9 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.conf.ParquetConfiguration;
+import org.apache.parquet.conf.PlainParquetConfiguration;
+import org.apache.parquet.hadoop.util.ConfigurationUtil;
 import org.junit.Test;
 
 public class EncryptionPropertiesFactoryTest {
@@ -45,4 +48,24 @@ public class EncryptionPropertiesFactoryTest {
         
encryptionProperties.getColumnProperties(SampleEncryptionPropertiesFactory.COL2),
         SampleEncryptionPropertiesFactory.COL2_ENCR_PROPERTIES);
   }
+
+  @Test
+  public void testLoadEncPropertiesFactoryParquetConfiguration() {
+    ParquetConfiguration conf = new PlainParquetConfiguration();
+    conf.set(
+        EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME,
+        "org.apache.parquet.crypto.SampleEncryptionPropertiesFactory");
+
+    EncryptionPropertiesFactory encryptionPropertiesFactory = 
EncryptionPropertiesFactory.loadFactory(conf);
+    FileEncryptionProperties encryptionProperties = 
encryptionPropertiesFactory.getFileEncryptionProperties(
+        ConfigurationUtil.createHadoopConfiguration(conf), null, null);
+
+    assertArrayEquals(encryptionProperties.getFooterKey(), 
SampleEncryptionPropertiesFactory.FOOTER_KEY);
+    assertEquals(
+        
encryptionProperties.getColumnProperties(SampleEncryptionPropertiesFactory.COL1),
+        SampleEncryptionPropertiesFactory.COL1_ENCR_PROPERTIES);
+    assertEquals(
+        
encryptionProperties.getColumnProperties(SampleEncryptionPropertiesFactory.COL2),
+        SampleEncryptionPropertiesFactory.COL2_ENCR_PROPERTIES);
+  }
 }

Reply via email to