|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.apache.parquet.cli.commands; |
| 20 | + |
| 21 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; |
| 22 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; |
| 23 | + |
| 24 | +import java.io.File; |
| 25 | +import java.io.IOException; |
| 26 | +import java.nio.charset.StandardCharsets; |
| 27 | +import java.util.HashMap; |
| 28 | +import java.util.Map; |
| 29 | +import org.apache.hadoop.conf.Configuration; |
| 30 | +import org.apache.hadoop.fs.Path; |
| 31 | +import org.apache.parquet.crypto.ColumnEncryptionProperties; |
| 32 | +import org.apache.parquet.crypto.FileEncryptionProperties; |
| 33 | +import org.apache.parquet.crypto.ParquetCipher; |
| 34 | +import org.apache.parquet.example.data.Group; |
| 35 | +import org.apache.parquet.example.data.simple.SimpleGroup; |
| 36 | +import org.apache.parquet.example.data.simple.SimpleGroupFactory; |
| 37 | +import org.apache.parquet.hadoop.ParquetWriter; |
| 38 | +import org.apache.parquet.hadoop.example.ExampleParquetWriter; |
| 39 | +import org.apache.parquet.hadoop.example.GroupWriteSupport; |
| 40 | +import org.apache.parquet.hadoop.metadata.ColumnPath; |
| 41 | +import org.apache.parquet.hadoop.metadata.CompressionCodecName; |
| 42 | +import org.apache.parquet.io.api.Binary; |
| 43 | +import org.apache.parquet.schema.MessageType; |
| 44 | +import org.apache.parquet.schema.Types; |
| 45 | + |
| 46 | +/** |
| 47 | + * Utility class for creating encrypted Parquet files for testing CLI commands. |
| 48 | + */ |
| 49 | +public final class EncryptedParquetFileTestHelper { |
| 50 | + |
| 51 | + // Standard test encryption keys (16 bytes for AES-128) |
| 52 | + public static final byte[] FOOTER_KEY = { |
| 53 | + 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10 |
| 54 | + }; |
| 55 | + |
| 56 | + public static final byte[] COLUMN_KEY_1 = { |
| 57 | + 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11 |
| 58 | + }; |
| 59 | + |
| 60 | + public static final byte[] COLUMN_KEY_2 = { |
| 61 | + 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10, 0x11, 0x12, 0x13 |
| 62 | + }; |
| 63 | + |
| 64 | + public static final String FOOTER_KEY_HEX = "0102030405060708090a0b0c0d0e0f10"; |
| 65 | + public static final String COLUMN_KEY_1_HEX = "02030405060708090a0b0c0d0e0f1011"; |
| 66 | + public static final String COLUMN_KEY_2_HEX = "0405060708090a0b0c0d0e0f10111213"; |
| 67 | + |
| 68 | + public static final String COLUMN_KEYS_CONFIG = COLUMN_KEY_1_HEX + ":name,email;" + COLUMN_KEY_2_HEX + ":phone"; |
| 69 | + |
| 70 | + private EncryptedParquetFileTestHelper() {} |
| 71 | + |
| 72 | + public static File createEncryptedParquetFile(File tempDir, String filename) throws IOException { |
| 73 | + return createEncryptedParquetFile(tempDir, filename, true, true); |
| 74 | + } |
| 75 | + |
| 76 | + public static File createEncryptedParquetFile( |
| 77 | + File tempDir, String filename, boolean enableBloomFilter, boolean encryptedFooter) throws IOException { |
| 78 | + |
| 79 | + MessageType schema = Types.buildMessage() |
| 80 | + .required(INT32) |
| 81 | + .named("id") |
| 82 | + .required(BINARY) |
| 83 | + .named("name") |
| 84 | + .required(BINARY) |
| 85 | + .named("email") |
| 86 | + .required(BINARY) |
| 87 | + .named("phone") |
| 88 | + .named("test_schema"); |
| 89 | + |
| 90 | + File file = new File(tempDir, filename); |
| 91 | + file.deleteOnExit(); |
| 92 | + |
| 93 | + Configuration conf = new Configuration(); |
| 94 | + GroupWriteSupport.setSchema(schema, conf); |
| 95 | + |
| 96 | + String[] encryptColumns = {"name", "email", "phone"}; |
| 97 | + FileEncryptionProperties encryptionProperties = |
| 98 | + createFileEncryptionProperties(encryptColumns, ParquetCipher.AES_GCM_CTR_V1, encryptedFooter); |
| 99 | + |
| 100 | + SimpleGroupFactory factory = new SimpleGroupFactory(schema); |
| 101 | + String[] nameValues = {"test_value_1", "test_value_2", "another_test", "bloom_filter_test", "final_value"}; |
| 102 | + String[] emailValues = { |
| 103 | + |
| 104 | + }; |
| 105 | + String[] phoneValues = {"555-0001", "555-0002", "555-0003", "555-0004", "555-0005"}; |
| 106 | + |
| 107 | + ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(new Path(file.toURI())) |
| 108 | + .withConf(conf) |
| 109 | + .withCompressionCodec(CompressionCodecName.UNCOMPRESSED) |
| 110 | + .withEncryption(encryptionProperties) |
| 111 | + .withPageSize(1024) |
| 112 | + .withRowGroupSize(4096); |
| 113 | + |
| 114 | + if (enableBloomFilter) { |
| 115 | + builder.withBloomFilterEnabled("name", true) |
| 116 | + .withBloomFilterEnabled("email", true) |
| 117 | + .withBloomFilterEnabled("phone", true); |
| 118 | + } |
| 119 | + |
| 120 | + try (ParquetWriter<Group> writer = builder.build()) { |
| 121 | + for (int i = 0; i < nameValues.length; i++) { |
| 122 | + SimpleGroup group = (SimpleGroup) factory.newGroup(); |
| 123 | + group.add("id", i + 1); |
| 124 | + group.add("name", Binary.fromString(nameValues[i])); |
| 125 | + group.add("email", Binary.fromString(emailValues[i])); |
| 126 | + group.add("phone", Binary.fromString(phoneValues[i])); |
| 127 | + writer.write(group); |
| 128 | + } |
| 129 | + } |
| 130 | + |
| 131 | + return file; |
| 132 | + } |
| 133 | + |
| 134 | + public static FileEncryptionProperties createFileEncryptionProperties( |
| 135 | + String[] encryptColumns, ParquetCipher cipher, boolean footerEncryption) { |
| 136 | + |
| 137 | + Map<String, byte[]> columnKeys = new HashMap<>(); |
| 138 | + columnKeys.put("name", COLUMN_KEY_1); |
| 139 | + columnKeys.put("email", COLUMN_KEY_1); |
| 140 | + columnKeys.put("phone", COLUMN_KEY_2); |
| 141 | + |
| 142 | + Map<ColumnPath, ColumnEncryptionProperties> columnPropertyMap = new HashMap<>(); |
| 143 | + for (String columnPath : encryptColumns) { |
| 144 | + ColumnPath column = ColumnPath.fromDotString(columnPath); |
| 145 | + byte[] columnKey = columnKeys.get(columnPath); |
| 146 | + |
| 147 | + ColumnEncryptionProperties columnProps = ColumnEncryptionProperties.builder(column) |
| 148 | + .withKey(columnKey) |
| 149 | + .withKeyMetaData(columnPath.getBytes(StandardCharsets.UTF_8)) |
| 150 | + .build(); |
| 151 | + columnPropertyMap.put(column, columnProps); |
| 152 | + } |
| 153 | + |
| 154 | + FileEncryptionProperties.Builder builder = FileEncryptionProperties.builder(FOOTER_KEY) |
| 155 | + .withFooterKeyMetadata("footkey".getBytes(StandardCharsets.UTF_8)) |
| 156 | + .withAlgorithm(cipher) |
| 157 | + .withEncryptedColumns(columnPropertyMap); |
| 158 | + |
| 159 | + if (!footerEncryption) { |
| 160 | + builder.withPlaintextFooter(); |
| 161 | + } |
| 162 | + |
| 163 | + return builder.build(); |
| 164 | + } |
| 165 | + |
| 166 | + public static Configuration createDecryptionConfiguration() { |
| 167 | + Configuration conf = new Configuration(); |
| 168 | + conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX); |
| 169 | + conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG); |
| 170 | + return conf; |
| 171 | + } |
| 172 | + |
| 173 | + public static void setDecryptionProperties(Configuration conf) { |
| 174 | + conf.set("parquet.encryption.footer.key", FOOTER_KEY_HEX); |
| 175 | + conf.set("parquet.encryption.column.keys", COLUMN_KEYS_CONFIG); |
| 176 | + } |
| 177 | +} |
0 commit comments