@@ -236,3 +236,146 @@ pub fn parquet_column<'a>(
236
236
. find ( |x| parquet_schema. get_column_root_idx ( * x) == root_idx) ?;
237
237
Some ( ( parquet_idx, field) )
238
238
}
239
+
240
+ #[ cfg( test) ]
241
+ mod test {
242
+ use crate :: arrow:: ArrowWriter ;
243
+ use crate :: file:: metadata:: { ParquetMetaData , ParquetMetaDataReader , ParquetMetaDataWriter } ;
244
+ use crate :: file:: properties:: { EnabledStatistics , WriterProperties } ;
245
+ use arrow_array:: { ArrayRef , Int32Array , RecordBatch } ;
246
+ use bytes:: Bytes ;
247
+ use std:: sync:: Arc ;
248
+
249
+ #[ test]
250
+ fn test_metadata_read_write_roundtrip ( ) {
251
+ let parquet_bytes = create_parquet_file ( ) ;
252
+
253
+ // read the metadata from the file
254
+ let original_metadata = ParquetMetaDataReader :: new ( )
255
+ . parse_and_finish ( & parquet_bytes)
256
+ . unwrap ( ) ;
257
+
258
+ // read metadata back from the serialized bytes and ensure it is the same
259
+ let metadata_bytes = metadata_to_bytes ( & original_metadata) ;
260
+ assert_ne ! (
261
+ metadata_bytes. len( ) ,
262
+ parquet_bytes. len( ) ,
263
+ "metadata is subset of parquet"
264
+ ) ;
265
+
266
+ let roundtrip_metadata = ParquetMetaDataReader :: new ( )
267
+ . parse_and_finish ( & metadata_bytes)
268
+ . unwrap ( ) ;
269
+
270
+ assert_eq ! ( original_metadata, roundtrip_metadata) ;
271
+ }
272
+
273
+ #[ test]
274
+ fn test_metadata_read_write_roundtrip_page_index ( ) {
275
+ let parquet_bytes = create_parquet_file ( ) ;
276
+
277
+ // read the metadata from the file including the page index structures
278
+ // (which are stored elsewhere in the footer)
279
+ let original_metadata = ParquetMetaDataReader :: new ( )
280
+ . with_page_indexes ( true )
281
+ . parse_and_finish ( & parquet_bytes)
282
+ . unwrap ( ) ;
283
+
284
+ // read metadata back from the serialized bytes and ensure it is the same
285
+ let metadata_bytes = metadata_to_bytes ( & original_metadata) ;
286
+ let roundtrip_metadata = ParquetMetaDataReader :: new ( )
287
+ . with_page_indexes ( true )
288
+ . parse_and_finish ( & metadata_bytes)
289
+ . unwrap ( ) ;
290
+
291
+ // Need to normalize the metadata first to remove offsets in data
292
+ let original_metadata = normalize_locations ( original_metadata) ;
293
+ let roundtrip_metadata = normalize_locations ( roundtrip_metadata) ;
294
+ assert_eq ! (
295
+ format!( "{original_metadata:#?}" ) ,
296
+ format!( "{roundtrip_metadata:#?}" )
297
+ ) ;
298
+ assert_eq ! ( original_metadata, roundtrip_metadata) ;
299
+ }
300
+
301
+ #[ test]
302
+ // Reproducer for https://github.com/apache/arrow-rs/issues/6464 (this should eventually pass)
303
+ #[ should_panic( expected = "missing required field ColumnIndex.null_pages" ) ]
304
+ fn test_metadata_read_write_partial_offset ( ) {
305
+ let parquet_bytes = create_parquet_file ( ) ;
306
+
307
+ // read the metadata from the file WITHOUT the page index structures
308
+ let original_metadata = ParquetMetaDataReader :: new ( )
309
+ . parse_and_finish ( & parquet_bytes)
310
+ . unwrap ( ) ;
311
+
312
+ // read metadata back from the serialized bytes requesting to read the offsets
313
+ let metadata_bytes = metadata_to_bytes ( & original_metadata) ;
314
+ let roundtrip_metadata = ParquetMetaDataReader :: new ( )
315
+ . with_page_indexes ( true ) // there are no page indexes in the metadata
316
+ . parse_and_finish ( & metadata_bytes)
317
+ . unwrap ( ) ;
318
+
319
+ // Need to normalize the metadata first to remove offsets in data
320
+ let original_metadata = normalize_locations ( original_metadata) ;
321
+ let roundtrip_metadata = normalize_locations ( roundtrip_metadata) ;
322
+ assert_eq ! (
323
+ format!( "{original_metadata:#?}" ) ,
324
+ format!( "{roundtrip_metadata:#?}" )
325
+ ) ;
326
+ assert_eq ! ( original_metadata, roundtrip_metadata) ;
327
+ }
328
+
329
+ // TODO: test reading parquet bytes from serialized metadata
330
+
331
+ /// Write a parquet filed into an in memory buffer
332
+ fn create_parquet_file ( ) -> Bytes {
333
+ let mut buf = vec ! [ ] ;
334
+ let data = vec ! [ 100 , 200 , 201 , 300 , 102 , 33 ] ;
335
+ let array: ArrayRef = Arc :: new ( Int32Array :: from ( data) ) ;
336
+ let batch = RecordBatch :: try_from_iter ( vec ! [ ( "id" , array) ] ) . unwrap ( ) ;
337
+ let props = WriterProperties :: builder ( )
338
+ . set_statistics_enabled ( EnabledStatistics :: Page )
339
+ . build ( ) ;
340
+
341
+ let mut writer = ArrowWriter :: try_new ( & mut buf, batch. schema ( ) , Some ( props) ) . unwrap ( ) ;
342
+ writer. write ( & batch) . unwrap ( ) ;
343
+ writer. finish ( ) . unwrap ( ) ;
344
+ drop ( writer) ;
345
+
346
+ Bytes :: from ( buf)
347
+ }
348
+
349
+ /// Serializes `ParquetMetaData` into a memory buffer, using `ParquetMetadataWriter
350
+ fn metadata_to_bytes ( metadata : & ParquetMetaData ) -> Bytes {
351
+ let mut buf = vec ! [ ] ;
352
+ ParquetMetaDataWriter :: new ( & mut buf, metadata)
353
+ . finish ( )
354
+ . unwrap ( ) ;
355
+ Bytes :: from ( buf)
356
+ }
357
+
358
+ /// Sets the page index offset locations in the metadata None
359
+ ///
360
+ /// This is because the offsets are used to find the relative location of the index
361
+ /// structures, and thus differ depending on how the structures are stored.
362
+ fn normalize_locations ( metadata : ParquetMetaData ) -> ParquetMetaData {
363
+ let mut metadata_builder = metadata. into_builder ( ) ;
364
+ for rg in metadata_builder. take_row_groups ( ) {
365
+ let mut rg_builder = rg. into_builder ( ) ;
366
+ for col in rg_builder. take_columns ( ) {
367
+ rg_builder = rg_builder. add_column_metadata (
368
+ col. into_builder ( )
369
+ . set_offset_index_offset ( None )
370
+ . set_index_page_offset ( None )
371
+ . set_column_index_offset ( None )
372
+ . build ( )
373
+ . unwrap ( ) ,
374
+ ) ;
375
+ }
376
+ let rg = rg_builder. build ( ) . unwrap ( ) ;
377
+ metadata_builder = metadata_builder. add_row_group ( rg) ;
378
+ }
379
+ metadata_builder. build ( )
380
+ }
381
+ }
0 commit comments