[
https://issues.apache.org/jira/browse/PARQUET-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17163921#comment-17163921
]
Øyvind Strømmen commented on PARQUET-1887:
------------------------------------------
I played around with this a bit, and was able to cover both these cases by
doing a {{flush}} and an {{initStore}} inside {{InternalParquetRecordWriter}}
whenever an exception is thrown. Could that be a possible fix for this? Please
see code here:
[https://github.com/InsulaVentus/parquet-mr/commit/d7ffcbe55746e0a3069cdd0a7c2c0cbc09a60854]
> Exception thrown by AvroParquetWriter#write causes all subsequent calls to it
> to fail
> -------------------------------------------------------------------------------------
>
> Key: PARQUET-1887
> URL: https://issues.apache.org/jira/browse/PARQUET-1887
> Project: Parquet
> Issue Type: Bug
> Components: parquet-avro
> Affects Versions: 1.11.0, 1.8.3
> Reporter: Øyvind Strømmen
> Priority: Major
> Attachments: person1_11_0.parquet, person1_8_3.parquet
>
>
> Please see sample code below:
> {code:java}
> Schema schema = new Schema.Parser().parse("""
> {
> "type": "record",
> "name": "person",
> "fields": [
> {
> "name": "address",
> "type": [
> "null",
> {
> "type": "array",
> "items": "string"
> }
> ],
> "default": null
> }
> ]
> }
> """
> );
> ParquetWriter<GenericRecord> writer =
> AvroParquetWriter.<GenericRecord>builder(new
> org.apache.hadoop.fs.Path("/tmp/person.parquet"))
> .withSchema(schema)
> .build();
> try {
> // To trigger exception, add array with null element.
> writer.write(new GenericRecordBuilder(schema).set("address",
> Arrays.asList("first", null, "last")).build());
> } catch (Exception e) {
> e.printStackTrace(); // "java.lang.NullPointerException: Array contains a
> null element at 1"
> }
> try {
> // At this point all future calls to writer.write will fail
> writer.write(new GenericRecordBuilder(schema).set("address",
> Arrays.asList("foo", "bar")).build());
> } catch (Exception e) {
> e.printStackTrace(); // "org.apache.parquet.io.InvalidRecordException:
> 1(r) > 0 ( schema r)"
> }
> writer.close();
> {code}
> It seems to me this is caused by state not being reset between writes. Is
> this the indented behavior of the writer? And if so, does one have to create
> a new writer whenever a write fails?
> I'm able to reproduce this using both parquet 1.8.3 and 1.11.0, and have
> attached a sample parquet file for each version.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)