This is an automated email from the ASF dual-hosted git repository.

gershinsky pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new 9fcf4cea0 PARQUET-2103: Fix crypto exception in print toPrettyJSON 
(#1019)
9fcf4cea0 is described below

commit 9fcf4cea0505aab216d5cd9528f51b083d5e761d
Author: ggershinsky <ggershin...@users.noreply.github.com>
AuthorDate: Tue Feb 28 10:54:24 2023 +0200

    PARQUET-2103: Fix crypto exception in print toPrettyJSON (#1019)
    
    * fix bug
    
    * recover constructor
    
    * test fix
    
    * Update 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/FileMetaData.java
    
    Co-authored-by: Gang Wu <ust...@gmail.com>
    
    * deprecate and document API
    
    ---------
    
    Co-authored-by: Gang Wu <ust...@gmail.com>
---
 .../format/converter/ParquetMetadataConverter.java | 12 ++++++-
 .../parquet/hadoop/metadata/FileMetaData.java      | 38 +++++++++++++++-------
 .../parquet/hadoop/metadata/ParquetMetadata.java   | 10 +++++-
 3 files changed, 46 insertions(+), 14 deletions(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
index 4409d5f68..73f77ff9e 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
@@ -116,6 +116,7 @@ import org.apache.parquet.format.UUIDType;
 import org.apache.parquet.hadoop.metadata.BlockMetaData;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.hadoop.metadata.FileMetaData.EncryptionType;
 import org.apache.parquet.column.EncodingStats;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.internal.column.columnindex.BinaryTruncator;
@@ -1625,8 +1626,17 @@ public class ParquetMetadataConverter {
         keyValueMetaData.put(keyValue.key, keyValue.value);
       }
     }
+    EncryptionType encryptionType;
+    if (encryptedFooter) {
+      encryptionType = EncryptionType.ENCRYPTED_FOOTER;
+    } else if (parquetMetadata.isSetEncryption_algorithm()) {
+      encryptionType = EncryptionType.PLAINTEXT_FOOTER;
+    } else {
+      encryptionType = EncryptionType.UNENCRYPTED;
+    }
     return new ParquetMetadata(
-        new org.apache.parquet.hadoop.metadata.FileMetaData(messageType, 
keyValueMetaData, parquetMetadata.getCreated_by(), fileDecryptor),
+        new org.apache.parquet.hadoop.metadata.FileMetaData(messageType, 
keyValueMetaData,
+          parquetMetadata.getCreated_by(), encryptionType, fileDecryptor),
         blocks);
   }
 
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/FileMetaData.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/FileMetaData.java
index 1668a7f73..b8e231887 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/FileMetaData.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/FileMetaData.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -23,26 +23,25 @@ import static java.util.Collections.unmodifiableMap;
 import java.io.Serializable;
 import java.util.Map;
 import java.util.Objects;
-
 import org.apache.parquet.crypto.InternalFileDecryptor;
 import org.apache.parquet.schema.MessageType;
 
-
 /**
  * File level meta data (Schema, codec, ...)
  */
 public final class FileMetaData implements Serializable {
   private static final long serialVersionUID = 1L;
 
-  private final MessageType schema;
+  public enum EncryptionType {UNENCRYPTED, PLAINTEXT_FOOTER, ENCRYPTED_FOOTER}
 
+  private final MessageType schema;
   private final Map<String, String> keyValueMetaData;
-
   private final String createdBy;
-  
   private final InternalFileDecryptor fileDecryptor;
+  private final EncryptionType encryptionType;
 
   /**
+   * FileMetaData for writers.
    * @param schema the schema for the file
    * @param keyValueMetaData the app specific metadata
    * @param createdBy the description of the library that created the file
@@ -50,16 +49,27 @@ public final class FileMetaData implements Serializable {
    * @throws NullPointerException if schema or keyValueMetaData is {@code null}
    */
   public FileMetaData(MessageType schema, Map<String, String> 
keyValueMetaData, String createdBy) {
-    this(schema, keyValueMetaData, createdBy, null);
+    this(schema, keyValueMetaData, createdBy, null, null);
+  }
+
+  @Deprecated
+  public FileMetaData(MessageType schema, Map<String, String> 
keyValueMetaData, String createdBy,
+                      InternalFileDecryptor fileDecryptor) {
+    this(schema, keyValueMetaData, createdBy, null, fileDecryptor);
   }
-  
-  public FileMetaData(MessageType schema, Map<String, String> 
keyValueMetaData, String createdBy, InternalFileDecryptor fileDecryptor) {
+
+  /**
+   * FileMetaData for readers (decryptors).
+   */
+  public FileMetaData(MessageType schema, Map<String, String> 
keyValueMetaData, String createdBy,
+                      EncryptionType encryptionType, InternalFileDecryptor 
fileDecryptor) {
     super();
     this.schema = Objects.requireNonNull(schema, "schema cannot be null");
     this.keyValueMetaData = unmodifiableMap(Objects
         .requireNonNull(keyValueMetaData, "keyValueMetaData cannot be null"));
     this.createdBy = createdBy;
     this.fileDecryptor = fileDecryptor;
+    this.encryptionType = encryptionType;
   }
 
   /**
@@ -71,7 +81,7 @@ public final class FileMetaData implements Serializable {
 
   @Override
   public String toString() {
-    return "FileMetaData{schema: "+schema+ ", metadata: " + keyValueMetaData + 
"}";
+    return "FileMetaData{schema: " + schema + ", metadata: " + 
keyValueMetaData + "}";
   }
 
   /**
@@ -91,4 +101,8 @@ public final class FileMetaData implements Serializable {
   public InternalFileDecryptor getFileDecryptor() {
     return fileDecryptor;
   }
+
+  public EncryptionType getEncryptionType() {
+    return encryptionType;
+  }
 }
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java
index 00d02a488..25c215d86 100755
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/metadata/ParquetMetadata.java
@@ -63,9 +63,17 @@ public class ParquetMetadata {
 
   private static String toJSON(ParquetMetadata parquetMetaData, boolean 
isPrettyPrint) {
     StringWriter stringWriter = new StringWriter();
+
     try {
       if (isPrettyPrint) {
-        objectMapper.writerWithDefaultPrettyPrinter().writeValue(stringWriter, 
parquetMetaData);
+        Object objectToPrint;
+        if (parquetMetaData.getFileMetaData() == null ||
+            parquetMetaData.getFileMetaData().getEncryptionType() == 
FileMetaData.EncryptionType.UNENCRYPTED) {
+          objectToPrint = parquetMetaData;
+        } else {
+          objectToPrint = parquetMetaData.getFileMetaData();
+        }
+        objectMapper.writerWithDefaultPrettyPrinter().writeValue(stringWriter, 
objectToPrint);
       } else {
         objectMapper.writeValue(stringWriter, parquetMetaData);
       }

Reply via email to