RE: AvroParquetOutputFormat - Unable to Write Arrays with Null Elements

2021-05-14 Thread Hailu, Andreas [Engineering]
I was able to get something working locally. I'll open a JIRA and have a PR 
once I have sufficient tests in place.

// ah

From: Hailu, Andreas [Engineering]
Sent: Friday, May 14, 2021 12:09 PM
To: dev@parquet.apache.org
Subject: AvroParquetOutputFormat - Unable to Write Arrays with Null Elements

Hi folks, I'm using v1.11.1 of the parquet-mr library as part of a Java 
application that takes Avro records and writes them into Parquet files using 
the AvroParquetOutputFormat. There are Avro records with array type fields that 
will have null elements, e.g. [ "Foo", "Bar", null, "Baz"]. Here's an example 
Avro schema:

{
  "type": "record",
  "name": "NullLists",
  "namespace": "com.test",
  "fields": [
{
  "name": "KeyID",
  "type": "string"
},
{
  "name": "NullableList",
  "type": [
"null",
{
"type": "array",
"items": [
"null",
"string"
]
}
  ],
  "default": null
}
  ]
}

I'm trying to write the following record:

{
  "KeyID": "0",
  "NullableList": [
"foo",
null,
"baz"
  ]
}

I thought I could use the 3-level list writer to support this, however, it 
results in the following exception:

Caused by: java.lang.ClassCastException: repeated binary array (STRING) is not 
a group
at org.apache.parquet.schema.Type.asGroupType(Type.java:250)
at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)
at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)
at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)
at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)
at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128)

Is this kind of record supported? I have also tried the 
"parquet.avro.add-list-element-records" option set to false as well, with no 
luck.



Andreas Hailu
Data Lake Engineering | Goldman Sachs & Co.




Your Personal Data: We may collect and process information about you that may 
be subject to data protection laws. For more information about how we use and 
disclose your personal data, how we protect your information, our legal basis 
to use your information, your rights and who you can contact, please refer to: 
www.gs.com/privacy-notices


[jira] [Commented] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)


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

Andreas Hailu commented on PARQUET-2051:


Hi folks, I am working on the change in my own repo [1] and will be submitting 
a PR when I have sufficient tests in place.

[1] [https://github.com/hailuand/parquet-mr/tree/avrowritesupport-config] 

> AvroWriteSupport does not pass Configuration to AvroSchemaConverter on 
> Creation
> ---
>
> Key: PARQUET-2051
> URL: https://issues.apache.org/jira/browse/PARQUET-2051
> Project: Parquet
>  Issue Type: Bug
>Reporter: Andreas Hailu
>Priority: Major
>
> Because of this, we're unable to fully leverage the ThreeLevelListWriter 
> functionality when trying to write Avro lists out using Parquet through the 
> AvroParquetOutputFormat.
> The following record is used for testing:
>  Schema:
> { "type": "record", "name": "NullLists", "namespace": "com.test", "fields": [ 
> \{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": [ 
> "null", { "type": "array", "items": [ "null", "string" ] } ], "default": null 
> } ] }
> Record (using basic JSON just for display purposes):
> { "KeyID": "0", "NullableList": [ "foo", null, "baz" ] }
> During testing, we see the following exception:
> {quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
> (STRING) is not a group}}
>  \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
>  \{{ at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
> {quote}
> Upon review, it was found that the configuration option that was set in 
> AvroWriteSupport for the ThreeLevelListWriter, 
> parquet.avro.write-old-list-structure being set to false, was never shared 
> with the AvroSchemaConverter.
> Once we made this change and tested locally, we observe the record with nulls 
> in the array being successfully written by AvroParquetOutputFormat. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Updated] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas Hailu updated PARQUET-2051:
---
Description: 
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 Schema:

{ "type": "record", "name": "NullLists", "namespace": "com.test", "fields": [ 
\{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": [ 
"null", { "type": "array", "items": [ "null", "string" ] } ], "default": null } 
] }

Record (using basic JSON just for display purposes):

{ "KeyID": "0", "NullableList": [ "foo", null, "baz" ] }

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
 \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
 \{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 

  was:
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 Schema:

{

  "type": "record", "name": "NullLists", "namespace": "com.test",

  "fields": [

     \{ "name": "KeyID", "type": "string" },

     \{ "name": "NullableList", "type": [ "null", { "type": "array", "items": [ 
"null", "string" ] } ], "default": null } ] }

  ]

}

Record (in  basic JSON format, not Avro-JSON spec per se)

{

  "KeyID": "0",

  "NullableList": [ "foo", null, "baz" ]

}

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
 \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
 \{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 


> AvroWriteSupport does not pass Configuration to AvroSchemaConverter on 
> Creation
> ---
>
> Key: PARQUET-2051
> URL: https://issues.apache.org/jira/browse/PARQUET-2051
> Project: Parquet
>  Issue Type: Bug
>Reporter: Andreas Hailu
>Priority: Major
>
> Because of this, we're unable to fully leverage the ThreeLevelListWriter 
> functionality when trying to write Avro lists out using Parquet through the 
> AvroParquetOutputFormat.
> The following record is used for testing:
>  Schema:
> { "type": "record", "name": "NullLists", "namespace": "com.test", "fields": [ 
> \{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": [ 
> "null", { "type": "array", "items": [ "null", "string" ] } ], "default": null 
> } ] }
> Record (using basic JSON just for display purposes):
> { "KeyID"

[jira] [Updated] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas Hailu updated PARQUET-2051:
---
Description: 
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 Schema:

{

  "type": "record", "name": "NullLists", "namespace": "com.test",

  "fields": [

     \{ "name": "KeyID", "type": "string" },

     \{ "name": "NullableList", "type": [ "null", { "type": "array", "items": [ 
"null", "string" ] } ], "default": null } ] }

  ]

}

Record (in  basic JSON format, not Avro-JSON spec per se)

{

  "KeyID": "0",

  "NullableList": [ "foo", null, "baz" ]

}

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
 \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
 \{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 

  was:
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 Schema:

{{{ "type": "record", "name": "NullLists", "namespace": "com.test", "fields": [ 
\{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": [ 
"null", { "type": "array", "items": [ "null", "string" ] } ], "default": null } 
] } }}

Record (in  basic JSON format, not Avro-JSON spec per se)

{{{ "KeyID": "0", "NullableList": [ "foo", null, "baz" ] }}}

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
 \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
 \{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 


> AvroWriteSupport does not pass Configuration to AvroSchemaConverter on 
> Creation
> ---
>
> Key: PARQUET-2051
> URL: https://issues.apache.org/jira/browse/PARQUET-2051
> Project: Parquet
>  Issue Type: Bug
>Reporter: Andreas Hailu
>Priority: Major
>
> Because of this, we're unable to fully leverage the ThreeLevelListWriter 
> functionality when trying to write Avro lists out using Parquet through the 
> AvroParquetOutputFormat.
> The following record is used for testing:
>  Schema:
> {
>   "type": "record", "name": "NullLists", "namespace": "com.test",
>   "fields": [
>      \{ "name": "KeyID", "type": "string" },
>      \{ "name": "NullableList", "type": [ "null", { "type": "array", "items": 
> [ "null", "string" ] } ], "default": null } ] }
>   ]
> }
> Record (in  basic JSON f

[jira] [Updated] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas Hailu updated PARQUET-2051:
---
Description: 
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 Schema:

{{{ "type": "record", "name": "NullLists", "namespace": "com.test", "fields": [ 
\{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": [ 
"null", { "type": "array", "items": [ "null", "string" ] } ], "default": null } 
] } }}

Record (in  basic JSON format, not Avro-JSON spec per se)

{{{ "KeyID": "0", "NullableList": [ "foo", null, "baz" ] }}}

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
 \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
 \{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
 \{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 

  was:
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 


> AvroWriteSupport does not pass Configuration to AvroSchemaConverter on 
> Creation
> ---
>
> Key: PARQUET-2051
> URL: https://issues.apache.org/jira/browse/PARQUET-2051
> Project: Parquet
>  Issue Type: Bug
>Reporter: Andreas Hailu
>Priority: Major
>
> Because of this, we're unable to fully leverage the ThreeLevelListWriter 
> functionality when trying to write Avro lists out using Parquet through the 
> AvroParquetOutputFormat.
> The following record is used for testing:
>  Schema:
> {{{ "type": "record", "name": "NullLists", "namespace": "com.test", "fields": 
> [ \{ "name": "KeyID", "type": "string" }, \{ "name": "NullableList", "type": 
> [ "null", { "type": "array", "items": [ "null", "string" ] } ], "default": 
> null } ] } }}
> Record (in  basic JSON format, not Avro-JSON spec per se)
> {{{ "KeyID": "0", "NullableList": [ "foo", null, "baz" ] }}}
> During testing, we see the following exception:
> {quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
> (STRING) is not a group}}
>  \{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
>  \{{ at 
> org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:61

[jira] [Updated] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2051?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Andreas Hailu updated PARQUET-2051:
---
Description: 
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet through the 
AvroParquetOutputFormat.

The following record is used for testing:

 

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group}}
{{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
{{ at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
{{ at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
{quote}
Upon review, it was found that the configuration option that was set in 
AvroWriteSupport for the ThreeLevelListWriter, 
parquet.avro.write-old-list-structure being set to false, was never shared with 
the AvroSchemaConverter.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by AvroParquetOutputFormat. 

  was:
Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet.

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group
at org.apache.parquet.schema.Type.asGroupType(Type.java:250)
at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)
at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)
at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)
at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)
at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128)}}
{quote}
Upon review, it was found that the configuration option that was set in 
`AvroWriteSupport` for the ThreeLevelListWriter, 
`parquet.avro.write-old-list-structure` being set to false, was never shared 
with the `AvroSchemaConverter`.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by `AvroParquetOutputFormat`. 


> AvroWriteSupport does not pass Configuration to AvroSchemaConverter on 
> Creation
> ---
>
> Key: PARQUET-2051
> URL: https://issues.apache.org/jira/browse/PARQUET-2051
> Project: Parquet
>  Issue Type: Bug
>Reporter: Andreas Hailu
>Priority: Major
>
> Because of this, we're unable to fully leverage the ThreeLevelListWriter 
> functionality when trying to write Avro lists out using Parquet through the 
> AvroParquetOutputFormat.
> The following record is used for testing:
>  
> During testing, we see the following exception:
> {quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
> (STRING) is not a group}}
> {{ at org.apache.parquet.schema.Type.asGroupType(Type.java:250)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)}}
> {{ at 
> org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)}}
> {{ at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128}}
> {quote}
> Upon review, it was found that the configuration option that was set in 
> AvroWriteSupport for the ThreeLevelListWriter, 
> parquet.avro.write-old-list-structure being set to false, was never shared 
> with

[jira] [Created] (PARQUET-2051) AvroWriteSupport does not pass Configuration to AvroSchemaConverter on Creation

2021-05-14 Thread Andreas Hailu (Jira)
Andreas Hailu created PARQUET-2051:
--

 Summary: AvroWriteSupport does not pass Configuration to 
AvroSchemaConverter on Creation
 Key: PARQUET-2051
 URL: https://issues.apache.org/jira/browse/PARQUET-2051
 Project: Parquet
  Issue Type: Bug
Reporter: Andreas Hailu


Because of this, we're unable to fully leverage the ThreeLevelListWriter 
functionality when trying to write Avro lists out using Parquet.

During testing, we see the following exception:
{quote}{{Caused by: java.lang.ClassCastException: repeated binary array 
(STRING) is not a group
at org.apache.parquet.schema.Type.asGroupType(Type.java:250)
at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)
at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)
at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)
at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)
at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128)}}
{quote}
Upon review, it was found that the configuration option that was set in 
`AvroWriteSupport` for the ThreeLevelListWriter, 
`parquet.avro.write-old-list-structure` being set to false, was never shared 
with the `AvroSchemaConverter`.

Once we made this change and tested locally, we observe the record with nulls 
in the array being successfully written by `AvroParquetOutputFormat`. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2050) Expose repetition & definition level from ColumnIO

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2050:
-

sunchao commented on pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908#issuecomment-841560801


   @shangxinli @ggershinsky @gszadovszky could you review this? Thanks!


-- 
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:
us...@infra.apache.org


> Expose repetition & definition level from ColumnIO
> --
>
> Key: PARQUET-2050
> URL: https://issues.apache.org/jira/browse/PARQUET-2050
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Assignee: Chao Sun
>Priority: Minor
>
> {{ColumnIO}} is pretty useful for obtaining repetition and definition level 
> info, for constructing nested records (the {{ColumnDescriptor}} only expose 
> the info for leave nodes). However, currently {{getDefinitionLevel}} and 
> {{getRepetitionLevel}} are both package-private and other applications depend 
> on Parquet have to find workaround for this (e.g., [ColumnIOUtil used by 
> Presto|https://github.com/prestodb/presto-hive-apache/blob/master/src/main/java/org/apache/parquet/io/ColumnIOUtil.java]).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] sunchao commented on pull request #908: PARQUET-2050: Expose repetition & definition level from ColumnIO

2021-05-14 Thread GitBox


sunchao commented on pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908#issuecomment-841560801


   @shangxinli @ggershinsky @gszadovszky could you review this? Thanks!


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-2050) Expose repetition & definition level from ColumnIO

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2050:
-

sunchao commented on pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908#issuecomment-841541021


   For rationale, please check the JIRA 
https://issues.apache.org/jira/browse/PARQUET-2050. Not sure where should I put 
those in the PR description.


-- 
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:
us...@infra.apache.org


> Expose repetition & definition level from ColumnIO
> --
>
> Key: PARQUET-2050
> URL: https://issues.apache.org/jira/browse/PARQUET-2050
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Assignee: Chao Sun
>Priority: Minor
>
> {{ColumnIO}} is pretty useful for obtaining repetition and definition level 
> info, for constructing nested records (the {{ColumnDescriptor}} only expose 
> the info for leave nodes). However, currently {{getDefinitionLevel}} and 
> {{getRepetitionLevel}} are both package-private and other applications depend 
> on Parquet have to find workaround for this (e.g., [ColumnIOUtil used by 
> Presto|https://github.com/prestodb/presto-hive-apache/blob/master/src/main/java/org/apache/parquet/io/ColumnIOUtil.java]).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] sunchao commented on pull request #908: PARQUET-2050: Expose repetition & definition level from ColumnIO

2021-05-14 Thread GitBox


sunchao commented on pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908#issuecomment-841541021


   For rationale, please check the JIRA 
https://issues.apache.org/jira/browse/PARQUET-2050. Not sure where should I put 
those in the PR description.


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-2050) Expose repetition & definition level from ColumnIO

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2050:
-

sunchao opened a new pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [X] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [X] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [X] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
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:
us...@infra.apache.org


> Expose repetition & definition level from ColumnIO
> --
>
> Key: PARQUET-2050
> URL: https://issues.apache.org/jira/browse/PARQUET-2050
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Assignee: Chao Sun
>Priority: Minor
>
> {{ColumnIO}} is pretty useful for obtaining repetition and definition level 
> info, for constructing nested records (the {{ColumnDescriptor}} only expose 
> the info for leave nodes). However, currently {{getDefinitionLevel}} and 
> {{getRepetitionLevel}} are both package-private and other applications depend 
> on Parquet have to find workaround for this (e.g., [ColumnIOUtil used by 
> Presto|https://github.com/prestodb/presto-hive-apache/blob/master/src/main/java/org/apache/parquet/io/ColumnIOUtil.java]).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] sunchao opened a new pull request #908: PARQUET-2050: Expose repetition & definition level from ColumnIO

2021-05-14 Thread GitBox


sunchao opened a new pull request #908:
URL: https://github.com/apache/parquet-mr/pull/908


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [X] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [X] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [X] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
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:
us...@infra.apache.org




[jira] [Created] (PARQUET-2050) Expose repetition & definition level from ColumnIO

2021-05-14 Thread Chao Sun (Jira)
Chao Sun created PARQUET-2050:
-

 Summary: Expose repetition & definition level from ColumnIO
 Key: PARQUET-2050
 URL: https://issues.apache.org/jira/browse/PARQUET-2050
 Project: Parquet
  Issue Type: Improvement
Reporter: Chao Sun
Assignee: Chao Sun


{{ColumnIO}} is pretty useful for obtaining repetition and definition level 
info, for constructing nested records (the {{ColumnDescriptor}} only expose the 
info for leave nodes). However, currently {{getDefinitionLevel}} and 
{{getRepetitionLevel}} are both package-private and other applications depend 
on Parquet have to find workaround for this (e.g., [ColumnIOUtil used by 
Presto|https://github.com/prestodb/presto-hive-apache/blob/master/src/main/java/org/apache/parquet/io/ColumnIOUtil.java]).



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2048) Deprecate BaseRecordReader

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2048:
-

belugabehr opened a new pull request #906:
URL: https://github.com/apache/parquet-mr/pull/906


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
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:
us...@infra.apache.org


> Deprecate BaseRecordReader
> --
>
> Key: PARQUET-2048
> URL: https://issues.apache.org/jira/browse/PARQUET-2048
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> No longer used.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2048) Deprecate BaseRecordReader

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2048:
-

belugabehr closed pull request #906:
URL: https://github.com/apache/parquet-mr/pull/906


   


-- 
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:
us...@infra.apache.org


> Deprecate BaseRecordReader
> --
>
> Key: PARQUET-2048
> URL: https://issues.apache.org/jira/browse/PARQUET-2048
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> No longer used.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] belugabehr closed pull request #906: PARQUET-2048: Deprecate BaseRecordReader

2021-05-14 Thread GitBox


belugabehr closed pull request #906:
URL: https://github.com/apache/parquet-mr/pull/906


   


-- 
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:
us...@infra.apache.org




AvroParquetOutputFormat - Unable to Write Arrays with Null Elements

2021-05-14 Thread Hailu, Andreas [Engineering]
Hi folks, I'm using v1.11.1 of the parquet-mr library as part of a Java 
application that takes Avro records and writes them into Parquet files using 
the AvroParquetOutputFormat. There are Avro records with array type fields that 
will have null elements, e.g. [ "Foo", "Bar", null, "Baz"]. Here's an example 
Avro schema:

{
  "type": "record",
  "name": "NullLists",
  "namespace": "com.test",
  "fields": [
{
  "name": "KeyID",
  "type": "string"
},
{
  "name": "NullableList",
  "type": [
"null",
{
"type": "array",
"items": [
"null",
"string"
]
}
  ],
  "default": null
}
  ]
}

I'm trying to write the following record:

{
  "KeyID": "0",
  "NullableList": [
"foo",
null,
"baz"
  ]
}

I thought I could use the 3-level list writer to support this, however, it 
results in the following exception:

Caused by: java.lang.ClassCastException: repeated binary array (STRING) is not 
a group
at org.apache.parquet.schema.Type.asGroupType(Type.java:250)
at 
org.apache.parquet.avro.AvroWriteSupport$ThreeLevelListWriter.writeCollection(AvroWriteSupport.java:612)
at 
org.apache.parquet.avro.AvroWriteSupport$ListWriter.writeList(AvroWriteSupport.java:397)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:355)
at 
org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:278)
at 
org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:191)
at 
org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:165)
at 
org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:128)

Is this kind of record supported? I have also tried the 
"parquet.avro.add-list-element-records" option set to false as well, with no 
luck.



Andreas Hailu
Data Lake Engineering | Goldman Sachs & Co.




Your Personal Data: We may collect and process information about you that may 
be subject to data protection laws. For more information about how we use and 
disclose your personal data, how we protect your information, our legal basis 
to use your information, your rights and who you can contact, please refer to: 
www.gs.com/privacy-notices


[jira] [Commented] (PARQUET-2049) Deprecate Exceptions Class

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2049:
-

belugabehr opened a new pull request #907:
URL: https://github.com/apache/parquet-mr/pull/907


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
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:
us...@infra.apache.org


> Deprecate Exceptions Class
> --
>
> Key: PARQUET-2049
> URL: https://issues.apache.org/jira/browse/PARQUET-2049
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] belugabehr opened a new pull request #907: PARQUET-2049: Deprecate Exceptions Class

2021-05-14 Thread GitBox


belugabehr opened a new pull request #907:
URL: https://github.com/apache/parquet-mr/pull/907


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
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:
us...@infra.apache.org




[jira] [Created] (PARQUET-2049) Deprecate Exceptions Class

2021-05-14 Thread David Mollitor (Jira)
David Mollitor created PARQUET-2049:
---

 Summary: Deprecate Exceptions Class
 Key: PARQUET-2049
 URL: https://issues.apache.org/jira/browse/PARQUET-2049
 Project: Parquet
  Issue Type: Improvement
Reporter: David Mollitor
Assignee: David Mollitor






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] belugabehr commented on pull request #745: PARQUET-1761: Lower Logging Level in ParquetOutputFormat

2021-05-14 Thread GitBox


belugabehr commented on pull request #745:
URL: https://github.com/apache/parquet-mr/pull/745#issuecomment-841301430


   @gszadovszky For me, since the properties are passed in, it is on the caller 
to log them in the client code before passing to Parquet.  The DEBUG logging 
allows the user to validate.
   
   Anyway, I have proposed simplifying the logging into a single statement (to 
reduce SPAM in the logs) and to debug only the large properties block that is 
unwieldy, and introduces new-lines which can make parsing the logs tricky 
(breaks the normal pattern of one log message per line.


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-1761) Lower Logging Level in ParquetOutputFormat

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1761:
-

belugabehr edited a comment on pull request #745:
URL: https://github.com/apache/parquet-mr/pull/745#issuecomment-841301430


   @gszadovszky For me, since the properties are passed in, it is on the caller 
to log them in the client code before passing to Parquet.  The DEBUG logging 
allows the user to validate.
   
   Anyway, I have proposed simplifying the logging into a single statement (to 
reduce SPAM in the logs) and to debug only the large properties block that is 
unwieldy, and introduces new-lines which can make parsing the logs tricky 
(breaks the normal pattern of one log message per line).


-- 
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:
us...@infra.apache.org


> Lower Logging Level in ParquetOutputFormat
> --
>
> Key: PARQUET-1761
> URL: https://issues.apache.org/jira/browse/PARQUET-1761
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>  Labels: pull-request-available
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1761) Lower Logging Level in ParquetOutputFormat

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1761:
-

belugabehr commented on pull request #745:
URL: https://github.com/apache/parquet-mr/pull/745#issuecomment-841301430


   @gszadovszky For me, since the properties are passed in, it is on the caller 
to log them in the client code before passing to Parquet.  The DEBUG logging 
allows the user to validate.
   
   Anyway, I have proposed simplifying the logging into a single statement (to 
reduce SPAM in the logs) and to debug only the large properties block that is 
unwieldy, and introduces new-lines which can make parsing the logs tricky 
(breaks the normal pattern of one log message per line.


-- 
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:
us...@infra.apache.org


> Lower Logging Level in ParquetOutputFormat
> --
>
> Key: PARQUET-1761
> URL: https://issues.apache.org/jira/browse/PARQUET-1761
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>  Labels: pull-request-available
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] belugabehr edited a comment on pull request #745: PARQUET-1761: Lower Logging Level in ParquetOutputFormat

2021-05-14 Thread GitBox


belugabehr edited a comment on pull request #745:
URL: https://github.com/apache/parquet-mr/pull/745#issuecomment-841301430


   @gszadovszky For me, since the properties are passed in, it is on the caller 
to log them in the client code before passing to Parquet.  The DEBUG logging 
allows the user to validate.
   
   Anyway, I have proposed simplifying the logging into a single statement (to 
reduce SPAM in the logs) and to debug only the large properties block that is 
unwieldy, and introduces new-lines which can make parsing the logs tricky 
(breaks the normal pattern of one log message per line).


-- 
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:
us...@infra.apache.org




[jira] [Updated] (PARQUET-2048) Deprecate BaseRecordReader

2021-05-14 Thread David Mollitor (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

David Mollitor updated PARQUET-2048:

Summary: Deprecate BaseRecordReader  (was: Remove BaseRecordReader)

> Deprecate BaseRecordReader
> --
>
> Key: PARQUET-2048
> URL: https://issues.apache.org/jira/browse/PARQUET-2048
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> No longer used.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-2046) Upgrade Apache POM to 23

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2046:
-

Fokko commented on a change in pull request #904:
URL: https://github.com/apache/parquet-mr/pull/904#discussion_r632551531



##
File path: pom.xml
##
@@ -4,7 +4,7 @@
   
 org.apache
 apache
-16
+23

Review comment:
   It is nice to keep everything in sync. The parent pom will also bump the 
plugins used by Maven.




-- 
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:
us...@infra.apache.org


> Upgrade Apache POM to 23
> 
>
> Key: PARQUET-2046
> URL: https://issues.apache.org/jira/browse/PARQUET-2046
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] Fokko commented on a change in pull request #904: PARQUET-2046: Upgrade Apache POM to 23

2021-05-14 Thread GitBox


Fokko commented on a change in pull request #904:
URL: https://github.com/apache/parquet-mr/pull/904#discussion_r632551531



##
File path: pom.xml
##
@@ -4,7 +4,7 @@
   
 org.apache
 apache
-16
+23

Review comment:
   It is nice to keep everything in sync. The parent pom will also bump the 
plugins used by Maven.




-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-2046) Upgrade Apache POM to 23

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2046:
-

belugabehr commented on pull request #904:
URL: https://github.com/apache/parquet-mr/pull/904#issuecomment-841250758


   This PR also has the affect of removing explicit plugin version numbers.  
This is often not required, just artifacts of previous work. Removing version 
numbers, and inheriting them from the parent POM, ensure all modules are 
building in the same way.


-- 
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:
us...@infra.apache.org


> Upgrade Apache POM to 23
> 
>
> Key: PARQUET-2046
> URL: https://issues.apache.org/jira/browse/PARQUET-2046
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] belugabehr commented on pull request #904: PARQUET-2046: Upgrade Apache POM to 23

2021-05-14 Thread GitBox


belugabehr commented on pull request #904:
URL: https://github.com/apache/parquet-mr/pull/904#issuecomment-841250758


   This PR also has the affect of removing explicit plugin version numbers.  
This is often not required, just artifacts of previous work. Removing version 
numbers, and inheriting them from the parent POM, ensure all modules are 
building in the same way.


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-1633) Integer overflow in ParquetFileReader.ConsecutiveChunkList

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1633:
-

gszadovszky commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-841139504


   @advancedxy, thanks for explaining.
   I think, the best option is 2. It is up to the user to provide enough 
resources for handling the large row groups or not writing them.
   Meanwhile, even though I've written I can accept lacking of unit tests in 
some situations my concern in this case is I am not sure that every aspect of a 
large row group is handled properly. So, we clearly need to validate this fix 
with such large row groups. This test can be even implemented in this source 
code but we must not include it in the unit tests or integration tests we run 
regularly. 


-- 
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:
us...@infra.apache.org


> Integer overflow in ParquetFileReader.ConsecutiveChunkList
> --
>
> Key: PARQUET-1633
> URL: https://issues.apache.org/jira/browse/PARQUET-1633
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.10.1
>Reporter: Ivan Sadikov
>Priority: Major
>
> When reading a large Parquet file (2.8GB), I encounter the following 
> exception:
> {code:java}
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> dbfs:/user/hive/warehouse/demo.db/test_table/part-00014-tid-1888470069989036737-593c82a4-528b-4975-8de0-5bcbc5e9827d-10856-1-c000.snappy.parquet
> at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
> at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
> at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:40)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1$$anon$2.getNext(FileScanRDD.scala:228)
> ... 14 more
> Caused by: java.lang.IllegalArgumentException: Illegal Capacity: -212
> at java.util.ArrayList.(ArrayList.java:157)
> at 
> org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1169){code}
>  
> The file metadata is:
>  * block 1 (3 columns)
>  ** rowCount: 110,100
>  ** totalByteSize: 348,492,072
>  ** compressedSize: 165,689,649
>  * block 2 (3 columns)
>  ** rowCount: 90,054
>  ** totalByteSize: 3,243,165,541
>  ** compressedSize: 2,509,579,966
>  * block 3 (3 columns)
>  ** rowCount: 105,119
>  ** totalByteSize: 350,901,693
>  ** compressedSize: 144,952,177
>  * block 4 (3 columns)
>  ** rowCount: 48,741
>  ** totalByteSize: 1,275,995
>  ** compressedSize: 914,205
> I don't have the code to reproduce the issue, unfortunately; however, I 
> looked at the code and it seems that integer {{length}} field in 
> ConsecutiveChunkList overflows, which results in negative capacity for array 
> list in {{readAll}} method:
> {code:java}
> int fullAllocations = length / options.getMaxAllocationSize();
> int lastAllocationSize = length % options.getMaxAllocationSize();
>   
> int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0);
> List buffers = new ArrayList<>(numAllocations);{code}
>  
> This is caused by cast to integer in {{readNextRowGroup}} method in 
> ParquetFileReader:
> {code:java}
> currentChunks.addChunk(new ChunkDescriptor(columnDescriptor, mc, startingPos, 
> (int)mc.getTotalSize()));
> {code}
> which overflows when total size of the column is larger than 
> Integer.MAX_VALUE.
> I would appreciate if you could help addressing the issue. Thanks!
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky commented on pull request #902: PARQUET-1633 Fix integer overflow

2021-05-14 Thread GitBox


gszadovszky commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-841139504


   @advancedxy, thanks for explaining.
   I think, the best option is 2. It is up to the user to provide enough 
resources for handling the large row groups or not writing them.
   Meanwhile, even though I've written I can accept lacking of unit tests in 
some situations my concern in this case is I am not sure that every aspect of a 
large row group is handled properly. So, we clearly need to validate this fix 
with such large row groups. This test can be even implemented in this source 
code but we must not include it in the unit tests or integration tests we run 
regularly. 


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-2047) Clean Up Code

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2047:
-

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



##
File path: 
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java
##
@@ -19,7 +19,6 @@
 package org.apache.parquet.hadoop.metadata;
 
 
-import org.apache.parquet.format.CompressionCodec;

Review comment:
   I am not sure why it got removed. This import is needed in this class 
and even the checks are failing because of it.




-- 
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:
us...@infra.apache.org


> Clean Up Code
> -
>
> Key: PARQUET-2047
> URL: https://issues.apache.org/jira/browse/PARQUET-2047
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>
> * Removed unused code
>  * Remove unused imports
>  * Add @Override annotations
> Mostly throwing away superfluous stuff. Less is more.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #905: PARQUET-2047: Clean Up Code

2021-05-14 Thread GitBox


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



##
File path: 
parquet-common/src/main/java/org/apache/parquet/hadoop/metadata/CompressionCodecName.java
##
@@ -19,7 +19,6 @@
 package org.apache.parquet.hadoop.metadata;
 
 
-import org.apache.parquet.format.CompressionCodec;

Review comment:
   I am not sure why it got removed. This import is needed in this class 
and even the checks are failing because of it.




-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-1633) Integer overflow in ParquetFileReader.ConsecutiveChunkList

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1633:
-

advancedxy commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-841124415


   > @eadwright,
   > 
   > I'll try to summarize the issue, please correct me if I'm wrong. 
Parquet-mr is not able to write such big row groups (>2GB) because of the `int` 
array size limitation. Meanwhile, both the format and some other 
implementations allow such big row groups. So, parquet-mr shall be prepared for 
this issue in some way.
   > One option is to "simply" read the large row groups. It would require 
significant efforts to use proper memory handling objects that would properly 
support reading the large row groups. (A similar effort would also make 
parquet-mr available to write larger row groups than 2GB.)
   > 
   > The other option is to handle the too large row groups with a proper error 
message in parquet-mr without allowing silent overflows. This second option 
would be this effort. It is great to handle to potential int overflows but the 
main point, I think, would be at the footer conversion 
(`ParquetMetadataConverter`) where we create our own object structure from the 
file footer. At this point we can throw the proper error messages if the row 
group is too large to be handled (for now) in parquet-mr.
   > BTW, it might not be enough to check the potential overflows to validate 
if we can read a row group size. (See e.g. the source code of 
[ArrayList](https://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/java/util/ArrayList.java#l229).)
   > 
   > About the lack of the unit tests. I can accept in some cases where unit 
tests are not practically feasible to be implemented. In these cases I usually 
ask to validate the code offline.
   
   Hi @gszadovszky parquet-mr is able to produce big row groups. We found some 
files wrote by Spark(which uses parquet-mr) have this problem. See 
https://issues.apache.org/jira/browse/PARQUET-2045 for details.
   
   There are two options to fix this problem:
   1. fail at writer side when creating such large row group/column chunk
   2. support at reader side, which is this approach. It would require a lot of 
resource, but it's feasible.
   
   Either option is fine for me, WDYT?


-- 
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:
us...@infra.apache.org


> Integer overflow in ParquetFileReader.ConsecutiveChunkList
> --
>
> Key: PARQUET-1633
> URL: https://issues.apache.org/jira/browse/PARQUET-1633
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.10.1
>Reporter: Ivan Sadikov
>Priority: Major
>
> When reading a large Parquet file (2.8GB), I encounter the following 
> exception:
> {code:java}
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> dbfs:/user/hive/warehouse/demo.db/test_table/part-00014-tid-1888470069989036737-593c82a4-528b-4975-8de0-5bcbc5e9827d-10856-1-c000.snappy.parquet
> at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
> at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
> at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:40)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1$$anon$2.getNext(FileScanRDD.scala:228)
> ... 14 more
> Caused by: java.lang.IllegalArgumentException: Illegal Capacity: -212
> at java.util.ArrayList.(ArrayList.java:157)
> at 
> org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1169){code}
>  
> The file metadata is:
>  * block 1 (3 columns)
>  ** rowCount: 110,100
>  ** totalByteSize: 348,492,072
>  ** compressedSize: 165,689,649
>  * block 2 (3 columns)
>  ** rowCount: 90,054
>  ** totalByteSize: 3,243,165,541
>  ** compressedSize: 2,509,579,966
>  * block 3 (3 columns)
>  ** rowCount: 105,119
>  ** totalByteSize: 350,901,693
>  ** compressedSize: 144,952,177
>  * block 4 (3 columns)
>  ** rowCount: 48,741
>  ** totalByteSize: 1,275,995
>  ** compressedSize: 914,205
> I don't have the code to reproduce the issue, unfortunately; however, I 
> looked at the code and it seems that integer {{length}} field in 
> ConsecutiveChunkList overflows, which results in negative capacity for array 
> list in {{readAll}} method:
> {code:java}
> int fullAllocations = length / options.getMaxAl

[GitHub] [parquet-mr] advancedxy commented on pull request #902: PARQUET-1633 Fix integer overflow

2021-05-14 Thread GitBox


advancedxy commented on pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#issuecomment-841124415


   > @eadwright,
   > 
   > I'll try to summarize the issue, please correct me if I'm wrong. 
Parquet-mr is not able to write such big row groups (>2GB) because of the `int` 
array size limitation. Meanwhile, both the format and some other 
implementations allow such big row groups. So, parquet-mr shall be prepared for 
this issue in some way.
   > One option is to "simply" read the large row groups. It would require 
significant efforts to use proper memory handling objects that would properly 
support reading the large row groups. (A similar effort would also make 
parquet-mr available to write larger row groups than 2GB.)
   > 
   > The other option is to handle the too large row groups with a proper error 
message in parquet-mr without allowing silent overflows. This second option 
would be this effort. It is great to handle to potential int overflows but the 
main point, I think, would be at the footer conversion 
(`ParquetMetadataConverter`) where we create our own object structure from the 
file footer. At this point we can throw the proper error messages if the row 
group is too large to be handled (for now) in parquet-mr.
   > BTW, it might not be enough to check the potential overflows to validate 
if we can read a row group size. (See e.g. the source code of 
[ArrayList](https://hg.openjdk.java.net/jdk8/jdk8/jdk/file/tip/src/share/classes/java/util/ArrayList.java#l229).)
   > 
   > About the lack of the unit tests. I can accept in some cases where unit 
tests are not practically feasible to be implemented. In these cases I usually 
ask to validate the code offline.
   
   Hi @gszadovszky parquet-mr is able to produce big row groups. We found some 
files wrote by Spark(which uses parquet-mr) have this problem. See 
https://issues.apache.org/jira/browse/PARQUET-2045 for details.
   
   There are two options to fix this problem:
   1. fail at writer side when creating such large row group/column chunk
   2. support at reader side, which is this approach. It would require a lot of 
resource, but it's feasible.
   
   Either option is fine for me, WDYT?


-- 
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:
us...@infra.apache.org




[jira] [Commented] (PARQUET-1633) Integer overflow in ParquetFileReader.ConsecutiveChunkList

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1633:
-

advancedxy commented on a change in pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#discussion_r632394126



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -1464,7 +1464,7 @@ protected PageHeader readPageHeader(BlockCipher.Decryptor 
blockDecryptor, byte[]
  */
 private void verifyCrc(int referenceCrc, byte[] bytes, String 
exceptionMsg) {
   crc.reset();
-  crc.update(bytes);
+  crc.update(bytes, 0, bytes.length);

Review comment:
   This is unrelated, I would prefer to update this in another PR.

##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -1763,8 +1763,8 @@ public void addChunk(ChunkDescriptor descriptor) {
 public void readAll(SeekableInputStream f, ChunkListBuilder builder) 
throws IOException {
   f.seek(offset);
 
-  int fullAllocations = length / options.getMaxAllocationSize();
-  int lastAllocationSize = length % options.getMaxAllocationSize();
+  int fullAllocations = (int)(length / options.getMaxAllocationSize());

Review comment:
   `(int)(length / options.getMaxAllocationSize())` -> ` 
Math.toIntExact(length / options.getMaxAllocationSize());`




-- 
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:
us...@infra.apache.org


> Integer overflow in ParquetFileReader.ConsecutiveChunkList
> --
>
> Key: PARQUET-1633
> URL: https://issues.apache.org/jira/browse/PARQUET-1633
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.10.1
>Reporter: Ivan Sadikov
>Priority: Major
>
> When reading a large Parquet file (2.8GB), I encounter the following 
> exception:
> {code:java}
> Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value 
> at 0 in block -1 in file 
> dbfs:/user/hive/warehouse/demo.db/test_table/part-00014-tid-1888470069989036737-593c82a4-528b-4975-8de0-5bcbc5e9827d-10856-1-c000.snappy.parquet
> at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
> at 
> org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
> at 
> org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:40)
> at 
> org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1$$anon$2.getNext(FileScanRDD.scala:228)
> ... 14 more
> Caused by: java.lang.IllegalArgumentException: Illegal Capacity: -212
> at java.util.ArrayList.(ArrayList.java:157)
> at 
> org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1169){code}
>  
> The file metadata is:
>  * block 1 (3 columns)
>  ** rowCount: 110,100
>  ** totalByteSize: 348,492,072
>  ** compressedSize: 165,689,649
>  * block 2 (3 columns)
>  ** rowCount: 90,054
>  ** totalByteSize: 3,243,165,541
>  ** compressedSize: 2,509,579,966
>  * block 3 (3 columns)
>  ** rowCount: 105,119
>  ** totalByteSize: 350,901,693
>  ** compressedSize: 144,952,177
>  * block 4 (3 columns)
>  ** rowCount: 48,741
>  ** totalByteSize: 1,275,995
>  ** compressedSize: 914,205
> I don't have the code to reproduce the issue, unfortunately; however, I 
> looked at the code and it seems that integer {{length}} field in 
> ConsecutiveChunkList overflows, which results in negative capacity for array 
> list in {{readAll}} method:
> {code:java}
> int fullAllocations = length / options.getMaxAllocationSize();
> int lastAllocationSize = length % options.getMaxAllocationSize();
>   
> int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0);
> List buffers = new ArrayList<>(numAllocations);{code}
>  
> This is caused by cast to integer in {{readNextRowGroup}} method in 
> ParquetFileReader:
> {code:java}
> currentChunks.addChunk(new ChunkDescriptor(columnDescriptor, mc, startingPos, 
> (int)mc.getTotalSize()));
> {code}
> which overflows when total size of the column is larger than 
> Integer.MAX_VALUE.
> I would appreciate if you could help addressing the issue. Thanks!
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] advancedxy commented on a change in pull request #902: PARQUET-1633 Fix integer overflow

2021-05-14 Thread GitBox


advancedxy commented on a change in pull request #902:
URL: https://github.com/apache/parquet-mr/pull/902#discussion_r632394126



##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -1464,7 +1464,7 @@ protected PageHeader readPageHeader(BlockCipher.Decryptor 
blockDecryptor, byte[]
  */
 private void verifyCrc(int referenceCrc, byte[] bytes, String 
exceptionMsg) {
   crc.reset();
-  crc.update(bytes);
+  crc.update(bytes, 0, bytes.length);

Review comment:
   This is unrelated, I would prefer to update this in another PR.

##
File path: 
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
##
@@ -1763,8 +1763,8 @@ public void addChunk(ChunkDescriptor descriptor) {
 public void readAll(SeekableInputStream f, ChunkListBuilder builder) 
throws IOException {
   f.seek(offset);
 
-  int fullAllocations = length / options.getMaxAllocationSize();
-  int lastAllocationSize = length % options.getMaxAllocationSize();
+  int fullAllocations = (int)(length / options.getMaxAllocationSize());

Review comment:
   `(int)(length / options.getMaxAllocationSize())` -> ` 
Math.toIntExact(length / options.getMaxAllocationSize());`




-- 
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:
us...@infra.apache.org




[jira] [Resolved] (PARQUET-1922) Deprecate IOExceptionUtils

2021-05-14 Thread Gabor Szadovszky (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-1922?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Gabor Szadovszky resolved PARQUET-1922.
---
Resolution: Fixed

> Deprecate IOExceptionUtils
> --
>
> Key: PARQUET-1922
> URL: https://issues.apache.org/jira/browse/PARQUET-1922
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1922) Deprecate IOExceptionUtils

2021-05-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1922:
-

gszadovszky merged pull request #825:
URL: https://github.com/apache/parquet-mr/pull/825


   


-- 
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:
us...@infra.apache.org


> Deprecate IOExceptionUtils
> --
>
> Key: PARQUET-1922
> URL: https://issues.apache.org/jira/browse/PARQUET-1922
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky merged pull request #825: PARQUET-1922: Deprecate IOExceptionUtils

2021-05-14 Thread GitBox


gszadovszky merged pull request #825:
URL: https://github.com/apache/parquet-mr/pull/825


   


-- 
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:
us...@infra.apache.org