[ 
https://issues.apache.org/jira/browse/PARQUET-1887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17165643#comment-17165643
 ] 

ASF GitHub Bot commented on PARQUET-1887:
-----------------------------------------

gszadovszky commented on a change in pull request #804:
URL: https://github.com/apache/parquet-mr/pull/804#discussion_r460821213



##########
File path: 
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayCompatibility.java
##########
@@ -56,6 +59,30 @@ public static void setupNewBehaviorConfiguration() {
         AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, false);
   }
 
+  @Test
+  public void testReadEmptyParquetFileWriteNull() throws IOException {
+    final Schema schema;
+    try (InputStream avroSchema = 
Resources.getResource("persons.json").openStream()) {
+      schema = new Schema.Parser().parse(avroSchema);
+    }
+
+    try (ParquetWriter<GenericRecord> writer =
+           AvroParquetWriter.<GenericRecord>builder(new 
org.apache.hadoop.fs.Path("/tmp/persons.parquet"))
+             .withSchema(schema)
+             .build()) {
+
+      // To trigger exception, add array with null element.
+      try {
+        writer.write(new GenericRecordBuilder(schema).set("address", 
Arrays.asList("first", null, "last")).build());
+      } catch (NullPointerException e) {

Review comment:
       I would add a `fail` in the `try` block after write so we fail if no 
exception occurred.

##########
File path: 
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayCompatibility.java
##########
@@ -56,6 +59,30 @@ public static void setupNewBehaviorConfiguration() {
         AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS, false);
   }
 
+  @Test
+  public void testReadEmptyParquetFileWriteNull() throws IOException {
+    final Schema schema;
+    try (InputStream avroSchema = 
Resources.getResource("persons.json").openStream()) {
+      schema = new Schema.Parser().parse(avroSchema);
+    }
+
+    try (ParquetWriter<GenericRecord> writer =
+           AvroParquetWriter.<GenericRecord>builder(new 
org.apache.hadoop.fs.Path("/tmp/persons.parquet"))
+             .withSchema(schema)
+             .build()) {
+
+      // To trigger exception, add array with null element.
+      try {
+        writer.write(new GenericRecordBuilder(schema).set("address", 
Arrays.asList("first", null, "last")).build());
+      } catch (NullPointerException e) {
+        // We expect this one to fail
+      }
+
+      // At this point all future calls to writer.write will fail

Review comment:
       Not sure about this comment. I've thought that's the point of this fix 
to not to fail after the previous exception.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


> 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)

Reply via email to