ccleva opened a new pull request, #111: URL: https://github.com/apache/parquet-testing/pull/111
Follow-up of the discussion in https://github.com/apache/parquet-testing/pull/31. The initial `fixed_length_byte_array.parquet` file contains null values in a required field. The issue was identified in https://github.com/apache/arrow/issues/47662. This PR replaces the initial file with a new one, generated with the same parquet-java version, where the field is optional. Page min/max values and null count are kept the same than in the initial file to facilitate compatibility with existing tests. Tests using this file: https://github.com/apache/arrow/blob/27513af4eaeb5fe22cfcb13206ff23a7ca6b52b1/cpp/src/parquet/reader_test.cc#L1203 https://github.com/apache/arrow/blob/27513af4eaeb5fe22cfcb13206ff23a7ca6b52b1/cpp/src/parquet/page_index_test.cc#L242 https://github.com/apache/arrow-go/blob/ccb6818822e7a86d11c9ec4a9f42505546838d2a/parquet/metadata/page_index_test.go#L213 The initial file is moved into the `bad_data` folder and renamed. Modified script used to generate the new file: ```java package examples; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroupFactory; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.GroupWriteSupport; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; import java.io.IOException; import java.nio.ByteBuffer; public class GenerateTestFile { public static void main(String[] args) { Path path = new Path("/tmp/fixed_length_byte_array.parquet"); Configuration conf = new Configuration(); MessageType schema = Types.buildMessage() .optional(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY).length(4) .named("flba_field") .named("schema"); SimpleGroupFactory fact = new SimpleGroupFactory(schema); GroupWriteSupport.setSchema(schema, conf); int[] nulls_per_page = {9, 9, 19, 10, 13, 11, 11, 8, 9, 6}; try ( @SuppressWarnings("deprecation") ParquetWriter<Group> writer = new ParquetWriter<>( path, new GroupWriteSupport(), CompressionCodecName.UNCOMPRESSED, /*blockSize=*/1024 * 1024, /*pageSize=*/128, /*dictionaryPageSize=*/128, /*enableDictionary=*/false, /*validating=*/true, ParquetProperties.WriterVersion.PARQUET_1_0, conf)) { for (int i = 1000; i > 0; --i) { if (i % 100 == 99) { // Add the required number of null values after the first page element int nb_null = nulls_per_page[9 - i / 100]; for (int j = 0; j < nb_null; j++) { writer.write(fact.newGroup()); } i = i - nb_null + 1; } else { ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); writer.write(fact.newGroup() .append("flba_field", Binary.fromConstantByteArray(buffer.putInt(0, i).array()))); } } } catch (IOException e) { throw new RuntimeException(e); } } } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
