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

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

andersonm-1 commented on a change in pull request #782:
URL: https://github.com/apache/parquet-mr/pull/782#discussion_r437544524



##########
File path: 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestEncryptionOptions.java
##########
@@ -125,150 +133,337 @@
   @Rule
   public ErrorCollector errorCollector = new ErrorCollector();
 
+  private static String PARQUET_TESTING_PATH = 
"../submodules/parquet-testing/data";
+  private static final int RANDOM_SEED = 42;
+  private static final int FIXED_LENGTH = 10;
+  private static final Random RANDOM = new Random(RANDOM_SEED);
+  private static final RandomValues.IntGenerator intGenerator
+    = new RandomValues.IntGenerator(RANDOM_SEED);
+  private static final RandomValues.FloatGenerator floatGenerator
+    = new RandomValues.FloatGenerator(RANDOM_SEED);
+  private static final RandomValues.DoubleGenerator doubleGenerator
+    = new RandomValues.DoubleGenerator(RANDOM_SEED);
+  private static final RandomValues.BinaryGenerator binaryGenerator
+    = new RandomValues.BinaryGenerator(RANDOM_SEED);
+  private static final RandomValues.FixedGenerator fixedBinaryGenerator
+    = new RandomValues.FixedGenerator(RANDOM_SEED, FIXED_LENGTH);
+
   private static final byte[] FOOTER_ENCRYPTION_KEY = 
"0123456789012345".getBytes();
-  private static final byte[] COLUMN_ENCRYPTION_KEY1 = 
"1234567890123450".getBytes();
-  private static final byte[] COLUMN_ENCRYPTION_KEY2 = 
"1234567890123451".getBytes();
+  private static final byte[][] COLUMN_ENCRYPTION_KEYS = { 
"1234567890123450".getBytes(),
+    "1234567890123451".getBytes(), "1234567890123452".getBytes(), 
"1234567890123453".getBytes(),
+    "1234567890123454".getBytes(), "1234567890123455".getBytes()};
+  private static final String[] COLUMN_ENCRYPTION_KEY_IDS = { "kc1", "kc2", 
"kc3", "kc4", "kc5", "kc6"};
   private static final String FOOTER_ENCRYPTION_KEY_ID = "kf";
-  private static final String COLUMN_ENCRYPTION_KEY1_ID = "kc1";
-  private static final String COLUMN_ENCRYPTION_KEY2_ID = "kc2";
   private static final String AAD_PREFIX_STRING = "tester";
   private static final String BOOLEAN_FIELD_NAME = "boolean_field";
   private static final String INT32_FIELD_NAME = "int32_field";
   private static final String FLOAT_FIELD_NAME = "float_field";
   private static final String DOUBLE_FIELD_NAME = "double_field";
+  private static final String BINARY_FIELD_NAME = "ba_field";
+  private static final String FIXED_LENGTH_BINARY_FIELD_NAME = "flba_field";
+  private static final String PLAINTEXT_INT32_FIELD_NAME = "plain_int32_field";
+
+  private static final byte[] footerKeyMetadata = 
FOOTER_ENCRYPTION_KEY_ID.getBytes(StandardCharsets.UTF_8);
+  private static final byte[] AADPrefix = 
AAD_PREFIX_STRING.getBytes(StandardCharsets.UTF_8);
+
+  private static final int ROW_COUNT = 10000;
+  private static final List<SingleRow> DATA = 
Collections.unmodifiableList(generateRandomData(ROW_COUNT));
+  private static final List<SingleRow> LINEAR_DATA = 
Collections.unmodifiableList(generateLinearData(250));
+
+  private static final MessageType SCHEMA =
+    new MessageType("schema",
+      new PrimitiveType(REQUIRED, BOOLEAN, BOOLEAN_FIELD_NAME),
+      new PrimitiveType(REQUIRED, INT32, INT32_FIELD_NAME),
+      new PrimitiveType(REQUIRED, FLOAT, FLOAT_FIELD_NAME),
+      new PrimitiveType(REQUIRED, DOUBLE, DOUBLE_FIELD_NAME),
+      new PrimitiveType(OPTIONAL, BINARY, BINARY_FIELD_NAME),
+      
Types.required(FIXED_LEN_BYTE_ARRAY).length(FIXED_LENGTH).named(FIXED_LENGTH_BINARY_FIELD_NAME),
+      new PrimitiveType(OPTIONAL, INT32, PLAINTEXT_INT32_FIELD_NAME));
+
+  private static final DecryptionKeyRetrieverMock decryptionKeyRetrieverMock = 
new DecryptionKeyRetrieverMock()
+    .putKey(FOOTER_ENCRYPTION_KEY_ID, FOOTER_ENCRYPTION_KEY)
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[0], COLUMN_ENCRYPTION_KEYS[0])
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[1], COLUMN_ENCRYPTION_KEYS[1])
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[2], COLUMN_ENCRYPTION_KEYS[2])
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[3], COLUMN_ENCRYPTION_KEYS[3])
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[4], COLUMN_ENCRYPTION_KEYS[4])
+    .putKey(COLUMN_ENCRYPTION_KEY_IDS[5], COLUMN_ENCRYPTION_KEYS[5]);
 
   public enum EncryptionConfiguration {
-    UNIFORM_ENCRYPTION("UNIFORM_ENCRYPTION"),
-    ENCRYPT_COLUMNS_AND_FOOTER("ENCRYPT_COLUMNS_AND_FOOTER"),
-    ENCRYPT_COLUMNS_PLAINTEXT_FOOTER("ENCRYPT_COLUMNS_PLAINTEXT_FOOTER"),
-    ENCRYPT_COLUMNS_AND_FOOTER_AAD("ENCRYPT_COLUMNS_AND_FOOTER_AAD"),
-    
ENCRYPT_COLUMNS_AND_FOOTER_DISABLE_AAD_STORAGE("ENCRYPT_COLUMNS_AND_FOOTER_DISABLE_AAD_STORAGE"),
-    ENCRYPT_COLUMNS_AND_FOOTER_CTR("ENCRYPT_COLUMNS_AND_FOOTER_CTR"),
-    NO_ENCRYPTION("NO_ENCRYPTION");
-
-    private final String configurationName;
-
-    EncryptionConfiguration(String configurationName) {
-      this.configurationName = configurationName;
-    }
+    UNIFORM_ENCRYPTION {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getUniformEncryptionEncryptionProperties();
+      }
+    },
+    ENCRYPT_COLUMNS_AND_FOOTER {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getEncryptColumnsAndFooterEncryptionProperties();
+      }
+    },
+    ENCRYPT_COLUMNS_PLAINTEXT_FOOTER {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getPlaintextFooterEncryptionProperties();
+      }
+    },
+    ENCRYPT_COLUMNS_AND_FOOTER_AAD {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getEncryptWithAADEncryptionProperties();
+      }
+    },
+    ENCRYPT_COLUMNS_AND_FOOTER_DISABLE_AAD_STORAGE {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getDisableAADStorageEncryptionProperties();
+      }
+    },
+    ENCRYPT_COLUMNS_AND_FOOTER_CTR {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return getCTREncryptionProperties();
+      }
+    },
+    NO_ENCRYPTION {
+      public FileEncryptionProperties getEncryptionProperties() {
+        return null;
+      }
+    };
 
-    @Override
-    public String toString() {
-      return configurationName;
-    }
+    abstract public FileEncryptionProperties getEncryptionProperties();
   }
 
-
   public enum DecryptionConfiguration {
-    DECRYPT_WITH_KEY_RETRIEVER("DECRYPT_WITH_KEY_RETRIEVER"),
-    DECRYPT_WITH_KEY_RETRIEVER_AAD("DECRYPT_WITH_KEY_RETRIEVER_AAD"),
-    DECRYPT_WITH_EXPLICIT_KEYS("DECRYPT_WITH_EXPLICIT_KEYS"),
-    NO_DECRYPTION("NO_DECRYPTION");
-
-    private final String configurationName;
-
-    DecryptionConfiguration(String configurationName) {
-      this.configurationName = configurationName;
-    }
+    DECRYPT_WITH_KEY_RETRIEVER {
+      public FileDecryptionProperties getDecryptionProperties() {
+        return getKeyRetrieverDecryptionProperties();
+      }
+    },
+    DECRYPT_WITH_KEY_RETRIEVER_AAD {
+      public FileDecryptionProperties getDecryptionProperties() {
+        return getKeyRetrieverAADDecryptionProperties();
+      }
+    },
+    DECRYPT_WITH_EXPLICIT_KEYS {
+      public FileDecryptionProperties getDecryptionProperties() {
+        return getExplicitKeysDecryptionProperties();
+      }
+    },
+    NO_DECRYPTION {
+      public FileDecryptionProperties getDecryptionProperties() {
+        return null;
+      }
+    };
 
-    @Override
-    public String toString() {
-      return configurationName;
-    }
+    abstract public FileDecryptionProperties getDecryptionProperties();
   }
 
   @Test
   public void testWriteReadEncryptedParquetFiles() throws IOException {
     Path rootPath = new Path(temporaryFolder.getRoot().getPath());
     LOG.info("======== testWriteReadEncryptedParquetFiles {} ========", 
rootPath.toString());
     byte[] AADPrefix = AAD_PREFIX_STRING.getBytes(StandardCharsets.UTF_8);
-    // This array will hold various encryption configuraions.
-    Map<EncryptionConfiguration, FileEncryptionProperties> 
encryptionPropertiesMap =
-      getEncryptionConfigurations(AADPrefix);
-    testWriteEncryptedParquetFiles(rootPath, encryptionPropertiesMap);
-    // This array will hold various decryption configurations.
-    Map<DecryptionConfiguration, FileDecryptionProperties> 
decryptionPropertiesMap =
-      getDecryptionConfigurations(AADPrefix);
-    testReadEncryptedParquetFiles(rootPath, decryptionPropertiesMap);
+    // Write using various encryption configuraions
+    testWriteEncryptedParquetFiles(rootPath, DATA);
+    // Read using various decryption configurations.
+    testReadEncryptedParquetFiles(rootPath, DATA);
   }
 
   @Test
   public void testInteropReadEncryptedParquetFiles() throws IOException {
-    Path rootPath = new Path("submodules/parquet-testing/data");
+    Path rootPath = new Path(PARQUET_TESTING_PATH);
     LOG.info("======== testInteropReadEncryptedParquetFiles {} ========", 
rootPath.toString());
     byte[] AADPrefix = AAD_PREFIX_STRING.getBytes(StandardCharsets.UTF_8);
-    // This array will hold various decryption configurations.
-    Map<DecryptionConfiguration, FileDecryptionProperties> 
decryptionPropertiesMap =
-      getDecryptionConfigurations(AADPrefix);
-    testReadEncryptedParquetFiles(rootPath, decryptionPropertiesMap);
+    // Read using various decryption configurations.
+    testInteropReadEncryptedParquetFiles(rootPath, true/*readOnlyEncrypted*/, 
LINEAR_DATA);
   }
 
-  private void testWriteEncryptedParquetFiles(Path root, 
Map<EncryptionConfiguration, FileEncryptionProperties> encryptionPropertiesMap) 
throws IOException {
-    Configuration conf = new Configuration();
-    int numberOfEncryptionModes = encryptionPropertiesMap.size();
+  private static List<SingleRow> generateRandomData(int rowCount) {
+    List<SingleRow> dataList = new ArrayList<>(rowCount);
+    for (int row = 0; row < rowCount; ++row) {
+      SingleRow newRow = new SingleRow(RANDOM.nextBoolean(),
+        intGenerator.nextValue(),  floatGenerator.nextValue(),
+        doubleGenerator.nextValue(), binaryGenerator.nextValue().getBytes(),
+        fixedBinaryGenerator.nextValue().getBytes(), intGenerator.nextValue());
+      dataList.add(newRow);
+    }
+    return dataList;
+  }
+
+  private static List<SingleRow> generateLinearData(int rowCount) {
+    List<SingleRow> dataList = new ArrayList<>(rowCount);
+    String baseStr = "parquet";
+    for (int row = 0; row < rowCount; ++row) {
+      boolean boolean_val = ((row % 2) == 0) ? true : false;
+      float float_val = (float) row * 1.1f;
+      double double_val = (row * 1.1111111);
+
+      byte[] binary_val = null;
+      if ((row % 2) == 0) {
+        char firstChar = (char) ((int) '0' + row / 100);
+        char secondChar = (char) ((int) '0' + (row / 10) % 10);
+        char thirdChar = (char) ((int) '0' + row % 10);
+        binary_val = (baseStr + firstChar + secondChar + 
thirdChar).getBytes(StandardCharsets.UTF_8);
+      }
+      char[] fixed = new char[FIXED_LENGTH];
+      char[] aChar = Character.toChars(row);
+      Arrays.fill(fixed, aChar[0]);
+
+      SingleRow newRow = new SingleRow(boolean_val,
+        row, float_val, double_val,
+        binary_val, new String(fixed).getBytes(StandardCharsets.UTF_8), 
null/*plaintext_int32_field*/);
+      dataList.add(newRow);
+    }
+    return dataList;
+  }
 
-    MessageType schema = parseMessageType(
-      "message test { "
-        + "required boolean " + BOOLEAN_FIELD_NAME + "; "
-        + "required int32 " + INT32_FIELD_NAME + "; "
-        + "required float " + FLOAT_FIELD_NAME + "; "
-        + "required double " + DOUBLE_FIELD_NAME + "; "
-        + "} ");
+  public static class SingleRow {
+    public final boolean boolean_field;
+    public final int int32_field;
+    public final float float_field;
+    public final double double_field;
+    public final byte[] ba_field;
+    public final byte[] flba_field;
+    public final Integer plaintext_int32_field; // Can be null, since it 
doesn't exist in C++-created files yet.
+
+    public SingleRow(boolean boolean_field,
+                     int int32_field,
+                     float float_field,
+                     double double_field,
+                     byte[] ba_field,
+                     byte[] flba_field,
+                     Integer plaintext_int32_field) {
+      this.boolean_field = boolean_field;
+      this.int32_field = int32_field;
+      this.float_field = float_field;
+      this.double_field = double_field;
+      this.ba_field = ba_field;
+      this.flba_field = flba_field;
+      this.plaintext_int32_field = plaintext_int32_field;
+    }
+  }
 
-    GroupWriteSupport.setSchema(schema, conf);
-    SimpleGroupFactory f = new SimpleGroupFactory(schema);
+  private void testWriteEncryptedParquetFiles(Path root, List<SingleRow> data) 
throws IOException {
+    Configuration conf = new Configuration();
 
+    int pageSize = data.size() / 10;     // Ensure that several pages will be 
created
+    int rowGroupSize = pageSize * 6 * 5; // Ensure that there are more 
row-groups created
 
-    for (Map.Entry<EncryptionConfiguration, FileEncryptionProperties> 
encryptionConfigurationEntry : encryptionPropertiesMap.entrySet()) {
-      EncryptionConfiguration encryptionConfiguration = 
encryptionConfigurationEntry.getKey();
-      Path file = new Path(root, encryptionConfiguration.toString() + 
".parquet.encrypted");
+    SimpleGroupFactory f = new SimpleGroupFactory(SCHEMA);
 
+    EncryptionConfiguration[] encryptionConfigurations = 
EncryptionConfiguration.values();
+    for (EncryptionConfiguration encryptionConfiguration : 
encryptionConfigurations) {
+      Path file = new Path(root, getFileName(encryptionConfiguration));
+      FileEncryptionProperties encryptionProperties = 
encryptionConfiguration.getEncryptionProperties();
       LOG.info("\nWrite " + file.toString());
-      ParquetWriter<Group> writer = ExampleParquetWriter.builder(file)
+      try (ParquetWriter<Group> writer = ExampleParquetWriter.builder(file)
         .withWriteMode(OVERWRITE)
-        .withType(schema)
-        .withEncryption(encryptionConfigurationEntry.getValue())
-        .build();
-
-      for (int i = 0; i < 100; i++) {
-        boolean expect = false;
-        if ((i % 2) == 0)
-          expect = true;
-        float float_val = (float) i * 1.1f;
-        double double_val = (i * 1.1111111);
-
-        writer.write(
-          f.newGroup()
-            .append(BOOLEAN_FIELD_NAME, expect)
-            .append(INT32_FIELD_NAME, i)
-            .append(FLOAT_FIELD_NAME, float_val)
-            .append(DOUBLE_FIELD_NAME, double_val));
+        .withRowGroupSize(rowGroupSize)
+        .withPageSize(pageSize)
+        .withType(SCHEMA)
+        .withConf(conf)
+        .withEncryption(encryptionProperties)
+        .build()) {
+
+        for (SingleRow singleRow : data) {
+          writer.write(
+            f.newGroup()
+              .append(BOOLEAN_FIELD_NAME, singleRow.boolean_field)
+              .append(INT32_FIELD_NAME, singleRow.int32_field)
+              .append(FLOAT_FIELD_NAME, singleRow.float_field)
+              .append(DOUBLE_FIELD_NAME, singleRow.double_field)
+              .append(BINARY_FIELD_NAME, 
Binary.fromConstantByteArray(singleRow.ba_field))
+              .append(FIXED_LENGTH_BINARY_FIELD_NAME, 
Binary.fromConstantByteArray(singleRow.flba_field))
+              .append(PLAINTEXT_INT32_FIELD_NAME, 
singleRow.plaintext_int32_field));
 
+        }
       }
-      writer.close();
     }
   }
 
-  private void testReadEncryptedParquetFiles(Path root, 
Map<DecryptionConfiguration, FileDecryptionProperties> decryptionPropertiesMap) 
throws IOException {
+  private String getFileName(EncryptionConfiguration encryptionConfiguration) {
+    return encryptionConfiguration.toString().toLowerCase() + 
".parquet.encrypted";
+  }
+
+  private void testReadEncryptedParquetFiles(Path root, List<SingleRow> data) {
     Configuration conf = new Configuration();
+    DecryptionConfiguration[] decryptionConfigurations = 
DecryptionConfiguration.values();
+    for (DecryptionConfiguration decryptionConfiguration : 
decryptionConfigurations) {
+      EncryptionConfiguration[] encryptionConfigurations = 
EncryptionConfiguration.values();
+      for (EncryptionConfiguration encryptionConfiguration : 
encryptionConfigurations) {
+        Path file = new Path(root, getFileName(encryptionConfiguration));
+        LOG.info("==> Decryption configuration {}", decryptionConfiguration);
+        FileDecryptionProperties fileDecryptionProperties = 
decryptionConfiguration.getDecryptionProperties();
 
-    for (Map.Entry<DecryptionConfiguration, FileDecryptionProperties> 
decryptionConfigurationEntry : decryptionPropertiesMap.entrySet()) {
-      DecryptionConfiguration decryptionConfiguration = 
decryptionConfigurationEntry.getKey();
-      LOG.info("==> Decryption configuration {}", decryptionConfiguration);
-      FileDecryptionProperties fileDecryptionProperties = 
decryptionConfigurationEntry.getValue();
+        LOG.info("--> Read file {} {}", file.toString(), 
encryptionConfiguration);
 
-      File folder = new File(root.toString());
-      File[] listOfFiles = folder.listFiles();
+        // Read only the non-encrypted columns
+        if ((decryptionConfiguration == DecryptionConfiguration.NO_DECRYPTION) 
&&
+          (encryptionConfiguration == 
EncryptionConfiguration.ENCRYPT_COLUMNS_PLAINTEXT_FOOTER)) {
+          conf.set("parquet.read.schema", Types.buildMessage()
+            .optional(INT32).named(PLAINTEXT_INT32_FIELD_NAME)
+            .named("FormatTestObject").toString());
+        }
 
-      for (int fileNum = 0; fileNum < listOfFiles.length; fileNum++) {
-        Path file = new Path(listOfFiles[fileNum].getAbsolutePath());
-        if (!file.getName().endsWith("parquet.encrypted")) { // Skip non 
encrypted files
-          continue;
+        int rowNum = 0;
+        try (ParquetReader<Group> reader = ParquetReader.builder(new 
GroupReadSupport(), file)
+          .withConf(conf)
+          .withDecryption(fileDecryptionProperties)
+          .build()) {
+          for (Group group = reader.read(); group != null; group = 
reader.read()) {
+            SingleRow rowExpected = data.get(rowNum++);
+            // plaintext columns
+            if ((null != rowExpected.plaintext_int32_field) &&
+              rowExpected.plaintext_int32_field != 
group.getInteger(PLAINTEXT_INT32_FIELD_NAME, 0)) {
+              addErrorToErrorCollectorAndLog("Wrong int", 
encryptionConfiguration, decryptionConfiguration);
+            }

Review comment:
       @gszadovszky, If you don't mind, I'd rather leave the method as it is, 
since it both adds an error to errorCollector for the final result and logs it 
for a more readable test progress log. If I let errorCollector.checkThat() do 
the comparison, I will need the current check anyway in order to add the test 
progress logging.
   
   Thanks, I've removed the null check.




----------------------------------------------------------------
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:
[email protected]


> Encryption: Interop and Function test suite for Java version
> ------------------------------------------------------------
>
>                 Key: PARQUET-1807
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1807
>             Project: Parquet
>          Issue Type: Sub-task
>          Components: parquet-mr
>            Reporter: Gidon Gershinsky
>            Assignee: Maya Anderson
>            Priority: Major
>              Labels: pull-request-available
>
> # Interop: test parquet-mr reading of encrypted files produced by parquet-cpp 
> (fetched from parquet-testing)
>  # Function: test writing/reading in a number of encryption and decryption 
> configurations



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

Reply via email to