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,137 @@ 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+ Configuration conf = new Configuration ();
72+ conf .set ("parquet.encryption.footer.key" , "0102030405060708090a0b0c0d0e0f10" );
73+ conf .set (
74+ "parquet.encryption.column.keys" ,
75+ "02030405060708090a0b0c0d0e0f1011:name,email;0405060708090a0b0c0d0e0f10111213:phone" );
76+ command .setConf (conf );
77+
78+ Assert .assertEquals (0 , command .run ());
79+
80+ ShowBloomFilterCommand emailCommand = new ShowBloomFilterCommand (createLogger ());
81+ emailCommand .file = encryptedFile .getAbsolutePath ();
82+ emailCommand .columnPath = "email" ;
83+ emailCommand .
testValues =
Arrays .
asList (
new String [] {
"[email protected] " ,
"[email protected] " });
84+ emailCommand .setConf (conf );
85+
86+ Assert .assertEquals (0 , emailCommand .run ());
87+
88+ ShowBloomFilterCommand phoneCommand = new ShowBloomFilterCommand (createLogger ());
89+ phoneCommand .file = encryptedFile .getAbsolutePath ();
90+ phoneCommand .columnPath = "phone" ;
91+ phoneCommand .testValues = Arrays .asList (new String [] {"555-0001" , "555-9999" });
92+ phoneCommand .setConf (conf );
93+
94+ Assert .assertEquals (0 , phoneCommand .run ());
95+
96+ encryptedFile .delete ();
97+ }
98+
99+ private File createEncryptedFileWithBloomFilter () throws IOException {
100+ MessageType schema = Types .buildMessage ()
101+ .required (INT32 )
102+ .named ("id" )
103+ .required (BINARY )
104+ .named ("name" )
105+ .required (BINARY )
106+ .named ("email" )
107+ .required (BINARY )
108+ .named ("phone" )
109+ .named ("test_schema" );
110+
111+ File tempFile = new File (getTempFolder (), "encrypted_bloom_test.parquet" );
112+ tempFile .deleteOnExit ();
113+
114+ Configuration conf = new Configuration ();
115+ GroupWriteSupport .setSchema (schema , conf );
116+
117+ String [] encryptColumns = {"name" , "email" , "phone" };
118+ FileEncryptionProperties encryptionProperties =
119+ createFileEncryptionProperties (encryptColumns , ParquetCipher .AES_GCM_CTR_V1 , true );
120+
121+ SimpleGroupFactory factory = new SimpleGroupFactory (schema );
122+ String [] nameValues = {"test_value_1" , "test_value_2" , "another_test" , "bloom_filter_test" , "final_value" };
123+ String [] emailValues = {
124+ 125+ };
126+ String [] phoneValues = {"555-0001" , "555-0002" , "555-0003" , "555-0004" , "555-0005" };
127+
128+ try (ParquetWriter <Group > writer = ExampleParquetWriter .builder (new Path (tempFile .toURI ()))
129+ .withConf (conf )
130+ .withCompressionCodec (CompressionCodecName .UNCOMPRESSED )
131+ .withEncryption (encryptionProperties )
132+ .withBloomFilterEnabled ("name" , true )
133+ .withBloomFilterEnabled ("email" , true )
134+ .withBloomFilterEnabled ("phone" , true )
135+ .withPageSize (1024 )
136+ .withRowGroupSize (4096 )
137+ .build ()) {
138+
139+ for (int i = 0 ; i < nameValues .length ; i ++) {
140+ SimpleGroup group = (SimpleGroup ) factory .newGroup ();
141+ group .add ("id" , i + 1 );
142+ group .add ("name" , Binary .fromString (nameValues [i ]));
143+ group .add ("email" , Binary .fromString (emailValues [i ]));
144+ group .add ("phone" , Binary .fromString (phoneValues [i ]));
145+ writer .write (group );
146+ }
147+ }
148+
149+ return tempFile ;
150+ }
151+
152+ private FileEncryptionProperties createFileEncryptionProperties (
153+ String [] encryptColumns , ParquetCipher cipher , boolean footerEncryption ) {
154+
155+ byte [] footerKey = {
156+ 0x01 , 0x02 , 0x03 , 0x04 , 0x05 , 0x06 , 0x07 , 0x08 , 0x09 , 0x0a , 0x0b , 0x0c , 0x0d , 0x0e , 0x0f , 0x10
157+ };
158+
159+ byte [] sharedKey = new byte [] {
160+ 0x02 , 0x03 , 0x04 , 0x05 , 0x06 , 0x07 , 0x08 , 0x09 , 0x0a , 0x0b , 0x0c , 0x0d , 0x0e , 0x0f , 0x10 , 0x11
161+ };
162+ byte [] phoneKey = new byte [] {
163+ 0x04 , 0x05 , 0x06 , 0x07 , 0x08 , 0x09 , 0x0a , 0x0b , 0x0c , 0x0d , 0x0e , 0x0f , 0x10 , 0x11 , 0x12 , 0x13
164+ };
165+
166+ Map <String , byte []> columnKeys = new HashMap <>();
167+ columnKeys .put ("name" , sharedKey );
168+ columnKeys .put ("email" , sharedKey );
169+ columnKeys .put ("phone" , phoneKey );
170+
171+ Map <ColumnPath , ColumnEncryptionProperties > columnPropertyMap = new HashMap <>();
172+ for (String columnPath : encryptColumns ) {
173+ ColumnPath column = ColumnPath .fromDotString (columnPath );
174+ byte [] columnKey = columnKeys .get (columnPath );
175+
176+ ColumnEncryptionProperties columnProps = ColumnEncryptionProperties .builder (column )
177+ .withKey (columnKey )
178+ .withKeyMetaData (columnPath .getBytes (StandardCharsets .UTF_8 ))
179+ .build ();
180+ columnPropertyMap .put (column , columnProps );
181+ }
182+
183+ FileEncryptionProperties .Builder builder = FileEncryptionProperties .builder (footerKey )
184+ .withFooterKeyMetadata ("footkey" .getBytes (StandardCharsets .UTF_8 ))
185+ .withAlgorithm (cipher )
186+ .withEncryptedColumns (columnPropertyMap );
187+
188+ if (!footerEncryption ) {
189+ builder .withPlaintextFooter ();
190+ }
191+
192+ return builder .build ();
193+ }
40194}
0 commit comments