1919
2020package org .apache .parquet .cli .commands ;
2121
22+ import static org .apache .parquet .schema .PrimitiveType .PrimitiveTypeName .BINARY ;
23+ import static org .apache .parquet .schema .PrimitiveType .PrimitiveTypeName .INT32 ;
24+
2225import java .io .File ;
2326import java .io .IOException ;
27+ import java .nio .charset .StandardCharsets ;
2428import java .util .Arrays ;
29+ import java .util .HashMap ;
30+ import java .util .Map ;
2531import org .apache .hadoop .conf .Configuration ;
32+ import org .apache .hadoop .fs .Path ;
33+ import org .apache .parquet .crypto .ColumnEncryptionProperties ;
34+ import org .apache .parquet .crypto .FileEncryptionProperties ;
35+ import org .apache .parquet .crypto .ParquetCipher ;
36+ import org .apache .parquet .example .data .Group ;
37+ import org .apache .parquet .example .data .simple .SimpleGroup ;
38+ import org .apache .parquet .example .data .simple .SimpleGroupFactory ;
39+ import org .apache .parquet .hadoop .ParquetWriter ;
40+ import org .apache .parquet .hadoop .example .ExampleParquetWriter ;
41+ import org .apache .parquet .hadoop .example .GroupWriteSupport ;
42+ import org .apache .parquet .hadoop .metadata .ColumnPath ;
43+ import org .apache .parquet .hadoop .metadata .CompressionCodecName ;
44+ import org .apache .parquet .io .api .Binary ;
45+ import org .apache .parquet .schema .MessageType ;
46+ import org .apache .parquet .schema .Types ;
2647import org .junit .Assert ;
2748import org .junit .Test ;
2849
@@ -37,4 +58,98 @@ public void testShowBloomFilterCommand() throws IOException {
3758 command .setConf (new Configuration ());
3859 Assert .assertEquals (0 , command .run ());
3960 }
61+
62+ @ Test
63+ public void testEncryptedFileWithBloomFilter () throws IOException {
64+ File encryptedFile = createEncryptedFileWithBloomFilter ();
65+
66+ ShowBloomFilterCommand command = new ShowBloomFilterCommand (createLogger ());
67+ command .file = encryptedFile .getAbsolutePath ();
68+ command .columnPath = "name" ;
69+ command .testValues = Arrays .asList (new String [] {"test_value_1" , "non_existent_value" });
70+
71+ command .footerKeyHex = "0102030405060708090a0b0c0d0e0f10" ;
72+ command .columnKeyHex = "02030405060708090a0b0c0d0e0f1011" ;
73+ command .setConf (new Configuration ());
74+
75+ Assert .assertEquals (0 , command .run ());
76+
77+ encryptedFile .delete ();
78+ }
79+
80+ /**
81+ * Creates an encrypted Parquet file with bloom filters for testing
82+ */
83+ private File createEncryptedFileWithBloomFilter () throws IOException {
84+ MessageType schema = Types .buildMessage ()
85+ .required (INT32 )
86+ .named ("id" )
87+ .required (BINARY )
88+ .named ("name" )
89+ .named ("test_schema" );
90+
91+ File tempFile = new File (getTempFolder (), "encrypted_bloom_test.parquet" );
92+ tempFile .deleteOnExit ();
93+
94+ Configuration conf = new Configuration ();
95+ GroupWriteSupport .setSchema (schema , conf );
96+
97+ String [] encryptColumns = {"name" };
98+ FileEncryptionProperties encryptionProperties =
99+ createFileEncryptionProperties (encryptColumns , ParquetCipher .AES_GCM_CTR_V1 , true );
100+
101+ SimpleGroupFactory factory = new SimpleGroupFactory (schema );
102+ String [] testValues = {"test_value_1" , "test_value_2" , "another_test" , "bloom_filter_test" , "final_value" };
103+
104+ try (ParquetWriter <Group > writer = ExampleParquetWriter .builder (new Path (tempFile .toURI ()))
105+ .withConf (conf )
106+ .withCompressionCodec (CompressionCodecName .UNCOMPRESSED )
107+ .withEncryption (encryptionProperties )
108+ .withBloomFilterEnabled ("name" , true )
109+ .withPageSize (1024 )
110+ .withRowGroupSize (4096 )
111+ .build ()) {
112+
113+ for (int i = 0 ; i < testValues .length ; i ++) {
114+ SimpleGroup group = (SimpleGroup ) factory .newGroup ();
115+ group .add ("id" , i + 1 );
116+ group .add ("name" , Binary .fromString (testValues [i ]));
117+ writer .write (group );
118+ }
119+ }
120+
121+ return tempFile ;
122+ }
123+
124+ private FileEncryptionProperties createFileEncryptionProperties (
125+ String [] encryptColumns , ParquetCipher cipher , boolean footerEncryption ) {
126+
127+ byte [] footerKey = {
128+ 0x01 , 0x02 , 0x03 , 0x04 , 0x05 , 0x06 , 0x07 , 0x08 , 0x09 , 0x0a , 0x0b , 0x0c , 0x0d , 0x0e , 0x0f , 0x10
129+ };
130+ byte [] columnKey = {
131+ 0x02 , 0x03 , 0x04 , 0x05 , 0x06 , 0x07 , 0x08 , 0x09 , 0x0a , 0x0b , 0x0c , 0x0d , 0x0e , 0x0f , 0x10 , 0x11
132+ };
133+
134+ Map <ColumnPath , ColumnEncryptionProperties > columnPropertyMap = new HashMap <>();
135+ for (String columnPath : encryptColumns ) {
136+ ColumnPath column = ColumnPath .fromDotString (columnPath );
137+ ColumnEncryptionProperties columnProps = ColumnEncryptionProperties .builder (column )
138+ .withKey (columnKey )
139+ .withKeyMetaData ("col" .getBytes (StandardCharsets .UTF_8 ))
140+ .build ();
141+ columnPropertyMap .put (column , columnProps );
142+ }
143+
144+ FileEncryptionProperties .Builder builder = FileEncryptionProperties .builder (footerKey )
145+ .withFooterKeyMetadata ("footkey" .getBytes (StandardCharsets .UTF_8 ))
146+ .withAlgorithm (cipher )
147+ .withEncryptedColumns (columnPropertyMap );
148+
149+ if (!footerEncryption ) {
150+ builder .withPlaintextFooter ();
151+ }
152+
153+ return builder .build ();
154+ }
40155}
0 commit comments