[ 
https://issues.apache.org/jira/browse/PARQUET-1229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17103604#comment-17103604
 ] 

ASF GitHub Bot commented on PARQUET-1229:
-----------------------------------------

shangxinli commented on a change in pull request #776:
URL: https://github.com/apache/parquet-mr/pull/776#discussion_r422575326



##########
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
##########
@@ -1185,70 +1275,189 @@ static long getOffset(ColumnChunk columnChunk) {
     return offset;
   }
 
+  private static void verifyFooterIntegrity(InputStream from, 
InternalFileDecryptor fileDecryptor, 
+      int combinedFooterLength) throws IOException {
+    
+    byte[] nonce = new byte[AesCipher.NONCE_LENGTH];
+    from.read(nonce);
+    byte[] gcmTag = new byte[AesCipher.GCM_TAG_LENGTH];
+    from.read(gcmTag);
+    
+    AesGcmEncryptor footerSigner =  
fileDecryptor.createSignedFooterEncryptor();
+    
+    byte[] footerAndSignature = ((ByteBufferInputStream) 
from).slice(0).array();
+    int footerSignatureLength = AesCipher.NONCE_LENGTH + 
AesCipher.GCM_TAG_LENGTH;
+    byte[] serializedFooter = new byte[combinedFooterLength - 
footerSignatureLength];
+    System.arraycopy(footerAndSignature, 0, serializedFooter, 0, 
serializedFooter.length);
+
+    byte[] signedFooterAAD = 
AesCipher.createFooterAAD(fileDecryptor.getFileAAD());
+    byte[] encryptedFooterBytes = footerSigner.encrypt(false, 
serializedFooter, nonce, signedFooterAAD);
+    byte[] calculatedTag = new byte[AesCipher.GCM_TAG_LENGTH];
+    System.arraycopy(encryptedFooterBytes, encryptedFooterBytes.length - 
AesCipher.GCM_TAG_LENGTH, 
+        calculatedTag, 0, AesCipher.GCM_TAG_LENGTH);
+    if (!Arrays.equals(gcmTag, calculatedTag)) {
+      throw new TagVerificationException("Signature mismatch in plaintext 
footer");
+    }
+  }
+
   public ParquetMetadata readParquetMetadata(final InputStream from, 
MetadataFilter filter) throws IOException {
+    return readParquetMetadata(from, filter, null, false, 0);
+  }
+
+  public ParquetMetadata readParquetMetadata(final InputStream from, 
MetadataFilter filter,
+      final InternalFileDecryptor fileDecryptor, final boolean 
encryptedFooter, 
+      final int combinedFooterLength) throws IOException {
+    
+    final BlockCipher.Decryptor footerDecryptor = (encryptedFooter? 
fileDecryptor.fetchFooterDecryptor() : null);
+    final byte[] encryptedFooterAAD = (encryptedFooter? 
AesCipher.createFooterAAD(fileDecryptor.getFileAAD()) : null);
+    
     FileMetaData fileMetaData = filter.accept(new 
MetadataFilterVisitor<FileMetaData, IOException>() {
       @Override
       public FileMetaData visit(NoFilter filter) throws IOException {
-        return readFileMetaData(from);
+        return readFileMetaData(from, footerDecryptor, encryptedFooterAAD);
       }
 
       @Override
       public FileMetaData visit(SkipMetadataFilter filter) throws IOException {
-        return readFileMetaData(from, true);
+        return readFileMetaData(from, true, footerDecryptor, 
encryptedFooterAAD);
       }
 
       @Override
       public FileMetaData visit(OffsetMetadataFilter filter) throws 
IOException {
-        return filterFileMetaDataByStart(readFileMetaData(from), filter);
+        return filterFileMetaDataByStart(readFileMetaData(from, 
footerDecryptor, encryptedFooterAAD), filter);
       }
 
       @Override
       public FileMetaData visit(RangeMetadataFilter filter) throws IOException 
{
-        return filterFileMetaDataByMidpoint(readFileMetaData(from), filter);
+        return filterFileMetaDataByMidpoint(readFileMetaData(from, 
footerDecryptor, encryptedFooterAAD), filter);
       }
     });
     LOG.debug("{}", fileMetaData);
-    ParquetMetadata parquetMetadata = fromParquetMetadata(fileMetaData);
+    
+    if (!encryptedFooter && null != fileDecryptor) {
+      if (!fileMetaData.isSetEncryption_algorithm()) { // Plaintext file

Review comment:
       No. I see most of them on up line but a few on the same line. It is not 
a must. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> parquet-mr code changes for encryption support
> ----------------------------------------------
>
>                 Key: PARQUET-1229
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1229
>             Project: Parquet
>          Issue Type: Sub-task
>          Components: parquet-mr
>            Reporter: Gidon Gershinsky
>            Assignee: Gidon Gershinsky
>            Priority: Major
>              Labels: pull-request-available
>
> Addition of encryption/decryption support to the existing Parquet classes and 
> APIs



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to