2121import java .io .File ;
2222import java .io .IOException ;
2323import java .util .Arrays ;
24+ import java .util .Collections ;
2425import java .util .Optional ;
2526import org .apache .hadoop .conf .Configuration ;
2627import org .apache .hadoop .fs .Path ;
@@ -82,12 +83,12 @@ public void testBinaryColumnSizeStatistics() throws IOException {
8283
8384 SizeStatistics sizeStatistics = column .getSizeStatistics ();
8485 Assert .assertEquals (Optional .of (4L ), sizeStatistics .getUnencodedByteArrayDataBytes ());
85- Assert .assertEquals (Arrays . asList ( 4L ), sizeStatistics .getRepetitionLevelHistogram ());
86- Assert .assertEquals (Arrays . asList ( 0L , 4L ), sizeStatistics .getDefinitionLevelHistogram ());
86+ Assert .assertEquals (Collections . emptyList ( ), sizeStatistics .getRepetitionLevelHistogram ());
87+ Assert .assertEquals (Collections . emptyList ( ), sizeStatistics .getDefinitionLevelHistogram ());
8788
8889 ColumnIndex columnIndex = reader .readColumnIndex (column );
89- Assert .assertEquals (Arrays . asList ( 2L , 2L ), columnIndex .getRepetitionLevelHistogram ());
90- Assert .assertEquals (Arrays . asList ( 0L , 2L , 0L , 2L ), columnIndex .getDefinitionLevelHistogram ());
90+ Assert .assertEquals (Collections . emptyList ( ), columnIndex .getRepetitionLevelHistogram ());
91+ Assert .assertEquals (Collections . emptyList ( ), columnIndex .getDefinitionLevelHistogram ());
9192
9293 OffsetIndex offsetIndex = reader .readOffsetIndex (column );
9394 Assert .assertEquals (2 , offsetIndex .getPageCount ());
@@ -96,6 +97,63 @@ public void testBinaryColumnSizeStatistics() throws IOException {
9697 }
9798 }
9899
100+ @ Test
101+ public void testNestedRepeatedOptionalColumnSizeStatistics () throws IOException {
102+ MessageType schema = Types .buildMessage ()
103+ .optionalGroup ()
104+ .repeatedGroup ()
105+ .optional (PrimitiveType .PrimitiveTypeName .BINARY )
106+ .as (LogicalTypeAnnotation .stringType ())
107+ .named ("value" )
108+ .named ("list" )
109+ .named ("outer" )
110+ .named ("msg" );
111+
112+ Configuration conf = new Configuration ();
113+ GroupWriteSupport .setSchema (schema , conf );
114+
115+ GroupFactory factory = new SimpleGroupFactory (schema );
116+ Path path = newTempPath ();
117+ try (ParquetWriter <Group > writer = ExampleParquetWriter .builder (path )
118+ .withPageRowCountLimit (2 )
119+ .withMinRowCountForPageSizeCheck (1 )
120+ .withConf (conf )
121+ .build ()) {
122+ // Create groups with different nesting patterns
123+ Group g1 = factory .newGroup ();
124+ Group outer1 = g1 .addGroup ("outer" );
125+ Group list1 = outer1 .addGroup ("list" );
126+ list1 .append ("value" , "a" );
127+ Group list2 = outer1 .addGroup ("list" );
128+ list2 .append ("value" , "b" );
129+ writer .write (g1 );
130+
131+ Group g2 = factory .newGroup ();
132+ Group outer2 = g2 .addGroup ("outer" );
133+ Group list3 = outer2 .addGroup ("list" );
134+ list3 .append ("value" , "c" );
135+ writer .write (g2 );
136+ }
137+
138+ try (ParquetFileReader reader = ParquetFileReader .open (HadoopInputFile .fromPath (path , new Configuration ()))) {
139+ ParquetMetadata footer = reader .getFooter ();
140+ ColumnChunkMetaData column = footer .getBlocks ().get (0 ).getColumns ().get (0 );
141+
142+ SizeStatistics sizeStatistics = column .getSizeStatistics ();
143+ Assert .assertEquals (Optional .of (3L ), sizeStatistics .getUnencodedByteArrayDataBytes ());
144+ Assert .assertEquals (Arrays .asList (2L , 1L ), sizeStatistics .getRepetitionLevelHistogram ());
145+ Assert .assertEquals (Arrays .asList (0L , 0L , 0L , 3L ), sizeStatistics .getDefinitionLevelHistogram ());
146+
147+ ColumnIndex columnIndex = reader .readColumnIndex (column );
148+ Assert .assertEquals (Arrays .asList (2L , 1L ), sizeStatistics .getRepetitionLevelHistogram ());
149+ Assert .assertEquals (Arrays .asList (0L , 0L , 0L , 3L ), sizeStatistics .getDefinitionLevelHistogram ());
150+
151+ OffsetIndex offsetIndex = reader .readOffsetIndex (column );
152+ Assert .assertEquals (1 , offsetIndex .getPageCount ());
153+ Assert .assertEquals (Optional .of (3L ), offsetIndex .getUnencodedByteArrayDataBytes (0 ));
154+ }
155+ }
156+
99157 private Path newTempPath () throws IOException {
100158 File file = temp .newFile ();
101159 Preconditions .checkArgument (file .delete (), "Could not remove temp file" );
0 commit comments