Re: Parquet Verbose Logging

2020-01-24 Thread Ryan Blue
Let's see how it goes moving more things to debug. I don't think it should
be a policy to move everything to debug, but we can certainly put more
information at the debug level that is currently not needed at info.

On Fri, Jan 24, 2020 at 4:08 PM David Mollitor  wrote:

> Hey Ryan,
>
> I think you understand my position correctly and articulated it well.  My
> background is from higher up the stack; a consumer of these libraries.
>
> We may need to agree to disagree on this one.  Projects these days include
> 100+ libraries and I don't want to have to set a custom log level for each
> one.  Much easier for consumer of libraries to keep everything as quiet as
> possible and then only have to worry about a custom logging level when
> something goes wrong.  Parquet in particular logs a lot of stuff at INFO
> level that is very specific to Parquet and would only be useful (if at all)
> to someone that really knows the library, not something that would be
> helpful to the higher level application developer.
>
> Thanks.
>
>
>
> On Fri, Jan 24, 2020 at 6:48 PM Ryan Blue  wrote:
>
>> It sounds like we see logging differently. My approach is that for any
>> library, the type of information should be categorized using the same
>> criteria into log levels. For example, if it is a normal event you might
>> want to know about, use info. It looks like your approach is that the
>> levels should be set for information from the perspective of the end
>> application: is this behavior relevant to the end user?
>>
>> The problem is that you don't always know whether something is relevant
>> to the end user because that context depends on the application. For the
>> Parquet CLI, much more Parquet information is relevant than for Presto that
>> is scanning Parquet files. That's why I think it's best to categorize the
>> log information using a standard definition, and rely on the end
>> application to configure log levels for its users expectations.
>>
>> On Fri, Jan 24, 2020 at 10:29 AM David Mollitor 
>> wrote:
>>
>>> Hello Ryan,
>>>
>>> I appreciate you taking the time to share your thoughts.
>>>
>>> I'd just like to point out that there is also TRACE level logging if
>>> Parquet requires greater granularity.
>>>
>>> Furthermore, I'm not suggesting that there be an unbreakable rule that
>>> all logging must be DEBUG, but it should be the exception, not the rule.
>>> It is more likely the situation the the wrapping application would be
>>> responsible for logging at the INFO and WARN/ERROR level.  Something
>>> like
>>>
>>> try {
>>>LOG.info("Using Parquet to read file {}", path);
>>>avroParquetReader.read();
>>> } catch (Exception e) {
>>>   LOG.error("Failed to read Parquet file", e);
>>> }
>>>
>>> This is a very normal setup and doesn't require any additional logging
>>> from the Parquet library itself.  Once I see an error with "Failed to re
>>> Parquet file", then I'm going to turn on DEBUG logging and try to reproduce
>>> the error.
>>>
>>> Thanks,
>>> David
>>>
>>> On Fri, Jan 24, 2020 at 12:01 PM Ryan Blue 
>>> wrote:
>>>
 I don't agree with the idea to convert all of Parquet's logs to DEBUG
 level, but I do think that we can improve the levels of individual
 messages.

 If we convert all logs to debug, then turning on logs to see what
 Parquet
 is doing would show everything from opening an input file to position
 tracking in output files. That's way too much information, which is why
 we
 use different log levels to begin with.

 I think we should continue using log levels to distinguish between
 types of
 information: error for errors, warn for recoverable errors that may or
 may
 not indicate a problem, info for regular operations, and debug for extra
 information if you're debugging the Parquet library. Following the
 common
 convention enables people to choose what information they want instead
 of
 mixing it all together.

 If you want to only see error and warning logs from Parquet, then the
 right
 way to do that is to configure your logger so that the level for
 org.apache.parquet classes is warn. That's not to say I don't agree
 that we
 can cut down on what is logged at info and clean it up; I just don't
 think
 it's a good idea to abandon the idea of log levels to distinguish
 between
 different information the user of a library will need.

 On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec 
 wrote:

 > Hi,
 > I can help too.
 > Lukas
 >
 > Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
 > napsal:
 >
 > > Hello Team,
 > >
 > > I am happy to do the work of reviewing all Parquet logging, but I
 need
 > help
 > > getting the work committed.
 > >
 > > Fokko Driesprong has been a wonderfully ally in helping me get
 > incremental
 > > improvements into Parquet, but I wonder if there's 

Re: Parquet Verbose Logging

2020-01-24 Thread David Mollitor
Hey Ryan,

I think you understand my position correctly and articulated it well.  My
background is from higher up the stack; a consumer of these libraries.

We may need to agree to disagree on this one.  Projects these days include
100+ libraries and I don't want to have to set a custom log level for each
one.  Much easier for consumer of libraries to keep everything as quiet as
possible and then only have to worry about a custom logging level when
something goes wrong.  Parquet in particular logs a lot of stuff at INFO
level that is very specific to Parquet and would only be useful (if at all)
to someone that really knows the library, not something that would be
helpful to the higher level application developer.

Thanks.



On Fri, Jan 24, 2020 at 6:48 PM Ryan Blue  wrote:

> It sounds like we see logging differently. My approach is that for any
> library, the type of information should be categorized using the same
> criteria into log levels. For example, if it is a normal event you might
> want to know about, use info. It looks like your approach is that the
> levels should be set for information from the perspective of the end
> application: is this behavior relevant to the end user?
>
> The problem is that you don't always know whether something is relevant to
> the end user because that context depends on the application. For the
> Parquet CLI, much more Parquet information is relevant than for Presto that
> is scanning Parquet files. That's why I think it's best to categorize the
> log information using a standard definition, and rely on the end
> application to configure log levels for its users expectations.
>
> On Fri, Jan 24, 2020 at 10:29 AM David Mollitor  wrote:
>
>> Hello Ryan,
>>
>> I appreciate you taking the time to share your thoughts.
>>
>> I'd just like to point out that there is also TRACE level logging if
>> Parquet requires greater granularity.
>>
>> Furthermore, I'm not suggesting that there be an unbreakable rule that
>> all logging must be DEBUG, but it should be the exception, not the rule.
>> It is more likely the situation the the wrapping application would be
>> responsible for logging at the INFO and WARN/ERROR level.  Something
>> like
>>
>> try {
>>LOG.info("Using Parquet to read file {}", path);
>>avroParquetReader.read();
>> } catch (Exception e) {
>>   LOG.error("Failed to read Parquet file", e);
>> }
>>
>> This is a very normal setup and doesn't require any additional logging
>> from the Parquet library itself.  Once I see an error with "Failed to re
>> Parquet file", then I'm going to turn on DEBUG logging and try to reproduce
>> the error.
>>
>> Thanks,
>> David
>>
>> On Fri, Jan 24, 2020 at 12:01 PM Ryan Blue 
>> wrote:
>>
>>> I don't agree with the idea to convert all of Parquet's logs to DEBUG
>>> level, but I do think that we can improve the levels of individual
>>> messages.
>>>
>>> If we convert all logs to debug, then turning on logs to see what Parquet
>>> is doing would show everything from opening an input file to position
>>> tracking in output files. That's way too much information, which is why
>>> we
>>> use different log levels to begin with.
>>>
>>> I think we should continue using log levels to distinguish between types
>>> of
>>> information: error for errors, warn for recoverable errors that may or
>>> may
>>> not indicate a problem, info for regular operations, and debug for extra
>>> information if you're debugging the Parquet library. Following the common
>>> convention enables people to choose what information they want instead of
>>> mixing it all together.
>>>
>>> If you want to only see error and warning logs from Parquet, then the
>>> right
>>> way to do that is to configure your logger so that the level for
>>> org.apache.parquet classes is warn. That's not to say I don't agree that
>>> we
>>> can cut down on what is logged at info and clean it up; I just don't
>>> think
>>> it's a good idea to abandon the idea of log levels to distinguish between
>>> different information the user of a library will need.
>>>
>>> On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec 
>>> wrote:
>>>
>>> > Hi,
>>> > I can help too.
>>> > Lukas
>>> >
>>> > Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
>>> > napsal:
>>> >
>>> > > Hello Team,
>>> > >
>>> > > I am happy to do the work of reviewing all Parquet logging, but I
>>> need
>>> > help
>>> > > getting the work committed.
>>> > >
>>> > > Fokko Driesprong has been a wonderfully ally in helping me get
>>> > incremental
>>> > > improvements into Parquet, but I wonder if there's anyone else that
>>> can
>>> > > share in the load.
>>> > >
>>> > > Thanks,
>>> > > David
>>> > >
>>> > > On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer 
>>> > wrote:
>>> > >
>>> > > > Hello David,
>>> > > >
>>> > > > As I mentioned on PARQUET-1758, we have been frustrated by overly
>>> > verbose
>>> > > > logging in Parquet for a long time.  Various workarounds have been
>>> more
>>> > > or
>>> > > > less successful, e.g.

Re: Parquet Verbose Logging

2020-01-24 Thread Ryan Blue
It sounds like we see logging differently. My approach is that for any
library, the type of information should be categorized using the same
criteria into log levels. For example, if it is a normal event you might
want to know about, use info. It looks like your approach is that the
levels should be set for information from the perspective of the end
application: is this behavior relevant to the end user?

The problem is that you don't always know whether something is relevant to
the end user because that context depends on the application. For the
Parquet CLI, much more Parquet information is relevant than for Presto that
is scanning Parquet files. That's why I think it's best to categorize the
log information using a standard definition, and rely on the end
application to configure log levels for its users expectations.

On Fri, Jan 24, 2020 at 10:29 AM David Mollitor  wrote:

> Hello Ryan,
>
> I appreciate you taking the time to share your thoughts.
>
> I'd just like to point out that there is also TRACE level logging if
> Parquet requires greater granularity.
>
> Furthermore, I'm not suggesting that there be an unbreakable rule that all
> logging must be DEBUG, but it should be the exception, not the rule.  It is
> more likely the situation the the wrapping application would be responsible
> for logging at the INFO and WARN/ERROR level.  Something like
>
> try {
>LOG.info("Using Parquet to read file {}", path);
>avroParquetReader.read();
> } catch (Exception e) {
>   LOG.error("Failed to read Parquet file", e);
> }
>
> This is a very normal setup and doesn't require any additional logging
> from the Parquet library itself.  Once I see an error with "Failed to re
> Parquet file", then I'm going to turn on DEBUG logging and try to reproduce
> the error.
>
> Thanks,
> David
>
> On Fri, Jan 24, 2020 at 12:01 PM Ryan Blue 
> wrote:
>
>> I don't agree with the idea to convert all of Parquet's logs to DEBUG
>> level, but I do think that we can improve the levels of individual
>> messages.
>>
>> If we convert all logs to debug, then turning on logs to see what Parquet
>> is doing would show everything from opening an input file to position
>> tracking in output files. That's way too much information, which is why we
>> use different log levels to begin with.
>>
>> I think we should continue using log levels to distinguish between types
>> of
>> information: error for errors, warn for recoverable errors that may or may
>> not indicate a problem, info for regular operations, and debug for extra
>> information if you're debugging the Parquet library. Following the common
>> convention enables people to choose what information they want instead of
>> mixing it all together.
>>
>> If you want to only see error and warning logs from Parquet, then the
>> right
>> way to do that is to configure your logger so that the level for
>> org.apache.parquet classes is warn. That's not to say I don't agree that
>> we
>> can cut down on what is logged at info and clean it up; I just don't think
>> it's a good idea to abandon the idea of log levels to distinguish between
>> different information the user of a library will need.
>>
>> On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec  wrote:
>>
>> > Hi,
>> > I can help too.
>> > Lukas
>> >
>> > Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
>> > napsal:
>> >
>> > > Hello Team,
>> > >
>> > > I am happy to do the work of reviewing all Parquet logging, but I need
>> > help
>> > > getting the work committed.
>> > >
>> > > Fokko Driesprong has been a wonderfully ally in helping me get
>> > incremental
>> > > improvements into Parquet, but I wonder if there's anyone else that
>> can
>> > > share in the load.
>> > >
>> > > Thanks,
>> > > David
>> > >
>> > > On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer 
>> > wrote:
>> > >
>> > > > Hello David,
>> > > >
>> > > > As I mentioned on PARQUET-1758, we have been frustrated by overly
>> > verbose
>> > > > logging in Parquet for a long time.  Various workarounds have been
>> more
>> > > or
>> > > > less successful, e.g.
>> > > >
>> > > > https://github.com/bigdatagenomics/adam/issues/851 <
>> > > > https://github.com/bigdatagenomics/adam/issues/851>
>> > > >
>> > > > I would support a move making Parquet a silent partner.  :)
>> > > >
>> > > >michael
>> > > >
>> > > >
>> > > > > On Jan 23, 2020, at 10:25 AM, David Mollitor 
>> > > wrote:
>> > > > >
>> > > > > Hello Team,
>> > > > >
>> > > > > I have been a consumer of Apache Parquet through Apache Hive for
>> > > several
>> > > > > years now.  For a long time, logging in Parquet has been pretty
>> > > painful.
>> > > > > Some of the logging was going to STDOUT and some was going to
>> Log4J.
>> > > > > Overall, though the framework has been too verbose, spewing many
>> log
>> > > > lines
>> > > > > about internal details of Parquet I don't understand.
>> > > > >
>> > > > > The logging has gotten a lot better with recent releases moving
>> > solidly
>> > > > > into SLF4J.  That 

[jira] [Updated] (PARQUET-1778) Do Not Consider Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1778:

Description: 
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData.newInstance(SpecificData.java:470) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData.newRecord(SpecificData.java:491) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.parquet.avro.AvroRecordConverter.start(AvroRecordConverter.java:404) 
~[parquet-avro-1.11.0.jar:1.11.0]
at 
org.apache.parquet.io.RecordReaderImplementation.read(RecordReaderImplementation.java:392)
 ~[parquet-column-1.11.0.jar:1.11.0]
at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:226)
 ~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class and this constructor does does not exist.  
Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
namespace information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.

If I hack the information in a Schema and change the {{namespace}} or {{name}} 
fields to something bogus, it works as I would expect it to.  It successfully 
reads and returns a {{GenericRecord}}.

  was:
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData.newInstance(SpecificData.java:470) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData.newRecord(SpecificData.java:491) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.parquet.avro.AvroRecordConverter.start(AvroRecordConverter.java:404) 
~[parquet-avro-1.11.0.jar:1.11.0]
at 
org.apache.parquet.io.RecordReaderImplementation.read(RecordReaderImplementation.java:392)
 ~[parquet-column-1.11.0.jar:1.11.0]
at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:226)
 ~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class and this constructor does does not exist.  
Regardless, the {{GenericRecordReader}} should just 

[jira] [Updated] (PARQUET-1778) Do Not Consider Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1778:

Description: 
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData.newInstance(SpecificData.java:470) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData.newRecord(SpecificData.java:491) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.parquet.avro.AvroRecordConverter.start(AvroRecordConverter.java:404) 
~[parquet-avro-1.11.0.jar:1.11.0]
at 
org.apache.parquet.io.RecordReaderImplementation.read(RecordReaderImplementation.java:392)
 ~[parquet-column-1.11.0.jar:1.11.0]
at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:226)
 ~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) 
~[parquet-hadoop-1.11.0.jar:1.11.0]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class and this constructor does does not exist.  
Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
namespace information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.

  was:
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class and this constructor does does not exist.  
Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
namespace information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.


> Do Not Consider Class for Avro Generic Record Reader
> 
>
> Key: PARQUET-1778
> URL: https://issues.apache.org/jira/browse/PARQUET-1778
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Major
>
>  
> {code:java|title=Example Code}
> final ParquetReader reader = 
> AvroParquetReader.builder(path).build();
> final GenericRecord genericRecord = reader.read();
> {code}
> It fails with...
> {code:none}
> java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
>   at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
>   at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
> ~[na:1.8.0_232]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
> 

[jira] [Updated] (PARQUET-1778) Do Not Consider Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1778:

Summary: Do Not Consider Class for Avro Generic Record Reader  (was: Do Not 
Record Class for Avro Generic Record Reader)

> Do Not Consider Class for Avro Generic Record Reader
> 
>
> Key: PARQUET-1778
> URL: https://issues.apache.org/jira/browse/PARQUET-1778
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Major
>
>  
> {code:java|title=Example Code}
> final ParquetReader reader = 
> AvroParquetReader.builder(path).build();
> final GenericRecord genericRecord = reader.read();
> {code}
> It fails with...
> {code:none}
> java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
>   at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
>   at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
> ~[na:1.8.0_232]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
> ~[avro-1.9.1.jar:1.9.1]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
> ~[avro-1.9.1.jar:1.9.1]
>   at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
> {code}
> I was surprised because it should just load a {{GenericRecord}} view of the 
> data. But alas, I have the Avro Schema defined with the {{namespace}} and 
> {{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
> happens to be a real class on the class path, so it is trying to call the 
> public constructor on the class and this constructor does does not exist.  
> Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
> namespace information.
> I am putting {{GenericRecords}} into the Parquet file, I expect to get 
> {{GenericRecords}} back out when I read it.



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


[jira] [Updated] (PARQUET-1778) Do Not Record Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1778:

Description: 
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class and this constructor does does not exist.  
Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
namespace information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.

  was:
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class which does not exist.

There {{GenericRecordReader}} should always ignore this Avro Schema namespace 
information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.


> Do Not Record Class for Avro Generic Record Reader
> --
>
> Key: PARQUET-1778
> URL: https://issues.apache.org/jira/browse/PARQUET-1778
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Major
>
>  
> {code:java|title=Example Code}
> final ParquetReader reader = 
> AvroParquetReader.builder(path).build();
> final GenericRecord genericRecord = reader.read();
> {code}
> It fails with...
> {code:none}
> java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
>   at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
>   at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
> ~[na:1.8.0_232]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
> ~[avro-1.9.1.jar:1.9.1]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
> ~[avro-1.9.1.jar:1.9.1]
>   at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
> {code}
> I was surprised because it should just load a {{GenericRecord}} view of the 
> data. But alas, I have the Avro Schema defined with the {{namespace}} and 
> {{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
> happens to be a real class on the class path, so it is trying to call the 
> public constructor on the class and this constructor does does not exist.  
> Regardless, the {{GenericRecordReader}} should just ignore this Avro Schema 
> namespace information.
> I am putting {{GenericRecords}} 

[jira] [Updated] (PARQUET-1778) Do Not Record Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1778:

Description: 
 
{code:java|title=Example Code}
final ParquetReader reader = 
AvroParquetReader.builder(path).build();
final GenericRecord genericRecord = reader.read();
{code}
It fails with...
{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}
I was surprised because it should just load a {{GenericRecord}} view of the 
data. But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class which does not exist.

There {{GenericRecordReader}} should always ignore this Avro Schema namespace 
information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.

  was:
{code:java}

final ParquetReader reader = 
AvroParquetReader.builder(path).build();final 
ParquetReader reader = 
AvroParquetReader.builder(path).build(); final GenericRecord 
genericRecord = reader.read();
{code}

It fails with...

{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}

I was surprised because it should just load a {{GenericRecord}} view of the 
data.  But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class which does not exist.

There {{GenericRecordReader}} should always ignore this Avro Schema namespace 
information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.


> Do Not Record Class for Avro Generic Record Reader
> --
>
> Key: PARQUET-1778
> URL: https://issues.apache.org/jira/browse/PARQUET-1778
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Major
>
>  
> {code:java|title=Example Code}
> final ParquetReader reader = 
> AvroParquetReader.builder(path).build();
> final GenericRecord genericRecord = reader.read();
> {code}
> It fails with...
> {code:none}
> java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
>   at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
>   at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
> ~[na:1.8.0_232]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
> ~[avro-1.9.1.jar:1.9.1]
>   at 
> org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
> ~[avro-1.9.1.jar:1.9.1]
>   at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
> ~[na:1.8.0_232]
>   at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
> {code}
> I was surprised because it should just load a {{GenericRecord}} view of the 
> data. But alas, I have the Avro Schema defined with the {{namespace}} and 
> {{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
> happens to be a real class on the class path, so it is trying to call the 
> public constructor on the class which does not exist.
> There {{GenericRecordReader}} should always ignore this Avro Schema namespace 
> information.
> I am putting {{GenericRecords}} into 

[jira] [Created] (PARQUET-1778) Do Not Record Class for Avro Generic Record Reader

2020-01-24 Thread David Mollitor (Jira)
David Mollitor created PARQUET-1778:
---

 Summary: Do Not Record Class for Avro Generic Record Reader
 Key: PARQUET-1778
 URL: https://issues.apache.org/jira/browse/PARQUET-1778
 Project: Parquet
  Issue Type: Improvement
Reporter: David Mollitor


{code:java}

final ParquetReader reader = 
AvroParquetReader.builder(path).build();final 
ParquetReader reader = 
AvroParquetReader.builder(path).build(); final GenericRecord 
genericRecord = reader.read();
{code}

It fails with...

{code:none}
java.lang.NoSuchMethodException: io.github.belugabehr.app.Record.()
at java.lang.Class.getConstructor0(Class.java:3082) ~[na:1.8.0_232]
at java.lang.Class.getDeclaredConstructor(Class.java:2178) 
~[na:1.8.0_232]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:63) 
~[avro-1.9.1.jar:1.9.1]
at 
org.apache.avro.specific.SpecificData$1.computeValue(SpecificData.java:58) 
~[avro-1.9.1.jar:1.9.1]
at java.lang.ClassValue.getFromHashMap(ClassValue.java:227) 
~[na:1.8.0_232]
at java.lang.ClassValue.getFromBackup(ClassValue.java:209) 
~[na:1.8.0_232]
at java.lang.ClassValue.get(ClassValue.java:115) ~[na:1.8.0_232]
{code}

I was surprised because it should just load a {{GenericRecord}} view of the 
data.  But alas, I have the Avro Schema defined with the {{namespace}} and 
{{name}} fields pointing to {{io.github.belugabehr.app.Record}} which just so 
happens to be a real class on the class path, so it is trying to call the 
public constructor on the class which does not exist.

There {{GenericRecordReader}} should always ignore this Avro Schema namespace 
information.

I am putting {{GenericRecords}} into the Parquet file, I expect to get 
{{GenericRecords}} back out when I read it.



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


Re: Parquet Verbose Logging

2020-01-24 Thread Maarten Ballintijn
Hello,

For larger projects, the logging is sometimes broken up into namespaces
that have their own (default) logging level. That way lower level sub-systems 
are quiet in normal operation but can be “activated” when debugging issues etc.

This might make sense for Arrow as well? What does the Logger provide?

Cheers,
Maarten.



> On Jan 24, 2020, at 1:29 PM, David Mollitor  wrote:
> 
> Hello Ryan,
> 
> I appreciate you taking the time to share your thoughts.
> 
> I'd just like to point out that there is also TRACE level logging if
> Parquet requires greater granularity.
> 
> Furthermore, I'm not suggesting that there be an unbreakable rule that all
> logging must be DEBUG, but it should be the exception, not the rule.  It is
> more likely the situation the the wrapping application would be responsible
> for logging at the INFO and WARN/ERROR level.  Something like
> 
> try {
>   LOG.info("Using Parquet to read file {}", path);
>   avroParquetReader.read();
> } catch (Exception e) {
>  LOG.error("Failed to read Parquet file", e);
> }
> 
> This is a very normal setup and doesn't require any additional logging from
> the Parquet library itself.  Once I see an error with "Failed to re Parquet
> file", then I'm going to turn on DEBUG logging and try to reproduce the
> error.
> 
> Thanks,
> David
> 
> On Fri, Jan 24, 2020 at 12:01 PM Ryan Blue 
> wrote:
> 
>> I don't agree with the idea to convert all of Parquet's logs to DEBUG
>> level, but I do think that we can improve the levels of individual
>> messages.
>> 
>> If we convert all logs to debug, then turning on logs to see what Parquet
>> is doing would show everything from opening an input file to position
>> tracking in output files. That's way too much information, which is why we
>> use different log levels to begin with.
>> 
>> I think we should continue using log levels to distinguish between types of
>> information: error for errors, warn for recoverable errors that may or may
>> not indicate a problem, info for regular operations, and debug for extra
>> information if you're debugging the Parquet library. Following the common
>> convention enables people to choose what information they want instead of
>> mixing it all together.
>> 
>> If you want to only see error and warning logs from Parquet, then the right
>> way to do that is to configure your logger so that the level for
>> org.apache.parquet classes is warn. That's not to say I don't agree that we
>> can cut down on what is logged at info and clean it up; I just don't think
>> it's a good idea to abandon the idea of log levels to distinguish between
>> different information the user of a library will need.
>> 
>> On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec  wrote:
>> 
>>> Hi,
>>> I can help too.
>>> Lukas
>>> 
>>> Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
>>> napsal:
>>> 
 Hello Team,
 
 I am happy to do the work of reviewing all Parquet logging, but I need
>>> help
 getting the work committed.
 
 Fokko Driesprong has been a wonderfully ally in helping me get
>>> incremental
 improvements into Parquet, but I wonder if there's anyone else that can
 share in the load.
 
 Thanks,
 David
 
 On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer 
>>> wrote:
 
> Hello David,
> 
> As I mentioned on PARQUET-1758, we have been frustrated by overly
>>> verbose
> logging in Parquet for a long time.  Various workarounds have been
>> more
 or
> less successful, e.g.
> 
> https://github.com/bigdatagenomics/adam/issues/851 <
> https://github.com/bigdatagenomics/adam/issues/851>
> 
> I would support a move making Parquet a silent partner.  :)
> 
>   michael
> 
> 
>> On Jan 23, 2020, at 10:25 AM, David Mollitor 
 wrote:
>> 
>> Hello Team,
>> 
>> I have been a consumer of Apache Parquet through Apache Hive for
 several
>> years now.  For a long time, logging in Parquet has been pretty
 painful.
>> Some of the logging was going to STDOUT and some was going to
>> Log4J.
>> Overall, though the framework has been too verbose, spewing many
>> log
> lines
>> about internal details of Parquet I don't understand.
>> 
>> The logging has gotten a lot better with recent releases moving
>>> solidly
>> into SLF4J.  That is awesome and very welcomed.  However, (opinion
> alert) I
>> think the logging is still too verbose.  I think Parquet should be
>> a
> silent
>> partner in data processing.  If everything is going well, it should
>>> be
>> silent (DEBUG level logging).  If things are going wrong, it should
 throw
>> an Exception.
>> 
>> If an operator suspects Parquet is the issue (and that's rarely the
 first
>> thing to check), they can set the logging for all of the Loggers in
>>> the
>> entire Parquet package (org.apache.parquet) to DEBUG to get the
 required
>> information.  

Re: Writing to Local File

2020-01-24 Thread Ryan Blue
Sounds good! Thanks for filing that issue.

If you'd like to work on separating out the Hadoop code, I'd be happy to
help review. It's something I've been meaning to do for a while.

On Fri, Jan 24, 2020 at 10:13 AM David Mollitor  wrote:

> Thanks Ryan for the confirmation of my suspicions.
>
> That would certainly make a quick sample application easier to achieve
> from an adoption perspective.
>
> I had just put this JIRA in.  I'll leave it open for anyone to jump in on.
> https://issues.apache.org/jira/browse/PARQUET-1776
>
> Thanks,
> David
>
>
> On Fri, Jan 24, 2020 at 12:08 PM Ryan Blue 
> wrote:
>
>> There's not currently a way to do this without Hadoop. We've been working
>> on moving to the `InputFile` and `OutputFile` abstractions so that we can
>> get rid of it, but Parquet still depends on Hadoop libraries for
>> compression and we haven't pulled out the parts of Parquet that use the
>> new
>> abstraction from the older ones that accept Hadoop Paths, so you need to
>> have Hadoop in your classpath either way.
>>
>> To get to where you can write a file without Hadoop dependencies, I think
>> we need to create a new module that parquet-hadoop will depend on with the
>> `InputFile`/`OutputFile` implementation. Then we would refactor the Hadoop
>> classes to extend those implementations to avoid breaking the Hadoop
>> classes. We'd also need to implement the compression API directly on top
>> of
>> aircompressor in this module.
>>
>> On Thu, Jan 23, 2020 at 4:40 PM David Mollitor  wrote:
>>
>> > I am usually a user of Parquet through Hive or Spark, but I wanted to
>> sit
>> > down and write my own small example application of using the library
>> > directly.
>> >
>> > Is there some quick way that I can write a Parquet file to the local
>> file
>> > system using java.nio.Path (i.e., with no Hadoop dependencies?)
>> >
>> > Thanks!
>> >
>>
>>
>> --
>> Ryan Blue
>> Software Engineer
>> Netflix
>>
>

-- 
Ryan Blue
Software Engineer
Netflix


Re: Parquet Verbose Logging

2020-01-24 Thread David Mollitor
Hello Ryan,

I appreciate you taking the time to share your thoughts.

I'd just like to point out that there is also TRACE level logging if
Parquet requires greater granularity.

Furthermore, I'm not suggesting that there be an unbreakable rule that all
logging must be DEBUG, but it should be the exception, not the rule.  It is
more likely the situation the the wrapping application would be responsible
for logging at the INFO and WARN/ERROR level.  Something like

try {
   LOG.info("Using Parquet to read file {}", path);
   avroParquetReader.read();
} catch (Exception e) {
  LOG.error("Failed to read Parquet file", e);
}

This is a very normal setup and doesn't require any additional logging from
the Parquet library itself.  Once I see an error with "Failed to re Parquet
file", then I'm going to turn on DEBUG logging and try to reproduce the
error.

Thanks,
David

On Fri, Jan 24, 2020 at 12:01 PM Ryan Blue 
wrote:

> I don't agree with the idea to convert all of Parquet's logs to DEBUG
> level, but I do think that we can improve the levels of individual
> messages.
>
> If we convert all logs to debug, then turning on logs to see what Parquet
> is doing would show everything from opening an input file to position
> tracking in output files. That's way too much information, which is why we
> use different log levels to begin with.
>
> I think we should continue using log levels to distinguish between types of
> information: error for errors, warn for recoverable errors that may or may
> not indicate a problem, info for regular operations, and debug for extra
> information if you're debugging the Parquet library. Following the common
> convention enables people to choose what information they want instead of
> mixing it all together.
>
> If you want to only see error and warning logs from Parquet, then the right
> way to do that is to configure your logger so that the level for
> org.apache.parquet classes is warn. That's not to say I don't agree that we
> can cut down on what is logged at info and clean it up; I just don't think
> it's a good idea to abandon the idea of log levels to distinguish between
> different information the user of a library will need.
>
> On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec  wrote:
>
> > Hi,
> > I can help too.
> > Lukas
> >
> > Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
> > napsal:
> >
> > > Hello Team,
> > >
> > > I am happy to do the work of reviewing all Parquet logging, but I need
> > help
> > > getting the work committed.
> > >
> > > Fokko Driesprong has been a wonderfully ally in helping me get
> > incremental
> > > improvements into Parquet, but I wonder if there's anyone else that can
> > > share in the load.
> > >
> > > Thanks,
> > > David
> > >
> > > On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer 
> > wrote:
> > >
> > > > Hello David,
> > > >
> > > > As I mentioned on PARQUET-1758, we have been frustrated by overly
> > verbose
> > > > logging in Parquet for a long time.  Various workarounds have been
> more
> > > or
> > > > less successful, e.g.
> > > >
> > > > https://github.com/bigdatagenomics/adam/issues/851 <
> > > > https://github.com/bigdatagenomics/adam/issues/851>
> > > >
> > > > I would support a move making Parquet a silent partner.  :)
> > > >
> > > >michael
> > > >
> > > >
> > > > > On Jan 23, 2020, at 10:25 AM, David Mollitor 
> > > wrote:
> > > > >
> > > > > Hello Team,
> > > > >
> > > > > I have been a consumer of Apache Parquet through Apache Hive for
> > > several
> > > > > years now.  For a long time, logging in Parquet has been pretty
> > > painful.
> > > > > Some of the logging was going to STDOUT and some was going to
> Log4J.
> > > > > Overall, though the framework has been too verbose, spewing many
> log
> > > > lines
> > > > > about internal details of Parquet I don't understand.
> > > > >
> > > > > The logging has gotten a lot better with recent releases moving
> > solidly
> > > > > into SLF4J.  That is awesome and very welcomed.  However, (opinion
> > > > alert) I
> > > > > think the logging is still too verbose.  I think Parquet should be
> a
> > > > silent
> > > > > partner in data processing.  If everything is going well, it should
> > be
> > > > > silent (DEBUG level logging).  If things are going wrong, it should
> > > throw
> > > > > an Exception.
> > > > >
> > > > > If an operator suspects Parquet is the issue (and that's rarely the
> > > first
> > > > > thing to check), they can set the logging for all of the Loggers in
> > the
> > > > > entire Parquet package (org.apache.parquet) to DEBUG to get the
> > > required
> > > > > information.  Not to mention, the less logging it does, the faster
> it
> > > > will
> > > > > be.
> > > > >
> > > > > I've opened this discussion because I've got two PRs related to
> this
> > > > topic
> > > > > ready to go:
> > > > >
> > > > > PARQUET-1758
> > > > > PARQUET-1761
> > > > >
> > > > > Thanks,
> > > > > David
> > > >
> > > >
> > >
> >
>
>
> --
> Ryan Blue
> Software Engineer

Re: Writing to Local File

2020-01-24 Thread David Mollitor
Thanks Ryan for the confirmation of my suspicions.

That would certainly make a quick sample application easier to achieve from
an adoption perspective.

I had just put this JIRA in.  I'll leave it open for anyone to jump in on.
https://issues.apache.org/jira/browse/PARQUET-1776

Thanks,
David


On Fri, Jan 24, 2020 at 12:08 PM Ryan Blue 
wrote:

> There's not currently a way to do this without Hadoop. We've been working
> on moving to the `InputFile` and `OutputFile` abstractions so that we can
> get rid of it, but Parquet still depends on Hadoop libraries for
> compression and we haven't pulled out the parts of Parquet that use the new
> abstraction from the older ones that accept Hadoop Paths, so you need to
> have Hadoop in your classpath either way.
>
> To get to where you can write a file without Hadoop dependencies, I think
> we need to create a new module that parquet-hadoop will depend on with the
> `InputFile`/`OutputFile` implementation. Then we would refactor the Hadoop
> classes to extend those implementations to avoid breaking the Hadoop
> classes. We'd also need to implement the compression API directly on top of
> aircompressor in this module.
>
> On Thu, Jan 23, 2020 at 4:40 PM David Mollitor  wrote:
>
> > I am usually a user of Parquet through Hive or Spark, but I wanted to sit
> > down and write my own small example application of using the library
> > directly.
> >
> > Is there some quick way that I can write a Parquet file to the local file
> > system using java.nio.Path (i.e., with no Hadoop dependencies?)
> >
> > Thanks!
> >
>
>
> --
> Ryan Blue
> Software Engineer
> Netflix
>


[jira] [Assigned] (PARQUET-1777) add Parquet logo vector files to repo

2020-01-24 Thread Julien Le Dem (Jira)


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

Julien Le Dem reassigned PARQUET-1777:
--

Assignee: Julien Le Dem

> add Parquet logo vector files to repo
> -
>
> Key: PARQUET-1777
> URL: https://issues.apache.org/jira/browse/PARQUET-1777
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-format
>Reporter: Julien Le Dem
>Assignee: Julien Le Dem
>Priority: Major
>  Labels: pull-request-available
>




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


[jira] [Updated] (PARQUET-1777) add Parquet logo vector files to repo

2020-01-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated PARQUET-1777:

Labels: pull-request-available  (was: )

> add Parquet logo vector files to repo
> -
>
> Key: PARQUET-1777
> URL: https://issues.apache.org/jira/browse/PARQUET-1777
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Major
>  Labels: pull-request-available
>




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


[jira] [Commented] (PARQUET-1777) add Parquet logo vector files to repo

2020-01-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1777:
-

julienledem commented on pull request #157: PARQUET-1777: add Parquet logos
URL: https://github.com/apache/parquet-format/pull/157
 
 
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses [Parquet-1777 
Jira](https://issues.apache.org/jira/browse/PARQUET-1777)
 - https://issues.apache.org/jira/browse/PARQUET-1777
 - no extra dependencies
   
   ### 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
   
   - [ ] No new functionality
 

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


> add Parquet logo vector files to repo
> -
>
> Key: PARQUET-1777
> URL: https://issues.apache.org/jira/browse/PARQUET-1777
> Project: Parquet
>  Issue Type: Task
>  Components: parquet-format
>Reporter: Julien Le Dem
>Priority: Major
>  Labels: pull-request-available
>




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


[jira] [Created] (PARQUET-1777) add Parquet logo vector files to repo

2020-01-24 Thread Julien Le Dem (Jira)
Julien Le Dem created PARQUET-1777:
--

 Summary: add Parquet logo vector files to repo
 Key: PARQUET-1777
 URL: https://issues.apache.org/jira/browse/PARQUET-1777
 Project: Parquet
  Issue Type: Task
  Components: parquet-format
Reporter: Julien Le Dem






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


Re: Writing to Local File

2020-01-24 Thread Ryan Blue
There's not currently a way to do this without Hadoop. We've been working
on moving to the `InputFile` and `OutputFile` abstractions so that we can
get rid of it, but Parquet still depends on Hadoop libraries for
compression and we haven't pulled out the parts of Parquet that use the new
abstraction from the older ones that accept Hadoop Paths, so you need to
have Hadoop in your classpath either way.

To get to where you can write a file without Hadoop dependencies, I think
we need to create a new module that parquet-hadoop will depend on with the
`InputFile`/`OutputFile` implementation. Then we would refactor the Hadoop
classes to extend those implementations to avoid breaking the Hadoop
classes. We'd also need to implement the compression API directly on top of
aircompressor in this module.

On Thu, Jan 23, 2020 at 4:40 PM David Mollitor  wrote:

> I am usually a user of Parquet through Hive or Spark, but I wanted to sit
> down and write my own small example application of using the library
> directly.
>
> Is there some quick way that I can write a Parquet file to the local file
> system using java.nio.Path (i.e., with no Hadoop dependencies?)
>
> Thanks!
>


-- 
Ryan Blue
Software Engineer
Netflix


Re: Parquet Verbose Logging

2020-01-24 Thread Ryan Blue
I don't agree with the idea to convert all of Parquet's logs to DEBUG
level, but I do think that we can improve the levels of individual messages.

If we convert all logs to debug, then turning on logs to see what Parquet
is doing would show everything from opening an input file to position
tracking in output files. That's way too much information, which is why we
use different log levels to begin with.

I think we should continue using log levels to distinguish between types of
information: error for errors, warn for recoverable errors that may or may
not indicate a problem, info for regular operations, and debug for extra
information if you're debugging the Parquet library. Following the common
convention enables people to choose what information they want instead of
mixing it all together.

If you want to only see error and warning logs from Parquet, then the right
way to do that is to configure your logger so that the level for
org.apache.parquet classes is warn. That's not to say I don't agree that we
can cut down on what is logged at info and clean it up; I just don't think
it's a good idea to abandon the idea of log levels to distinguish between
different information the user of a library will need.

On Fri, Jan 24, 2020 at 6:30 AM lukas nalezenec  wrote:

> Hi,
> I can help too.
> Lukas
>
> Dne pá 24. 1. 2020 15:29 uživatel David Mollitor 
> napsal:
>
> > Hello Team,
> >
> > I am happy to do the work of reviewing all Parquet logging, but I need
> help
> > getting the work committed.
> >
> > Fokko Driesprong has been a wonderfully ally in helping me get
> incremental
> > improvements into Parquet, but I wonder if there's anyone else that can
> > share in the load.
> >
> > Thanks,
> > David
> >
> > On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer 
> wrote:
> >
> > > Hello David,
> > >
> > > As I mentioned on PARQUET-1758, we have been frustrated by overly
> verbose
> > > logging in Parquet for a long time.  Various workarounds have been more
> > or
> > > less successful, e.g.
> > >
> > > https://github.com/bigdatagenomics/adam/issues/851 <
> > > https://github.com/bigdatagenomics/adam/issues/851>
> > >
> > > I would support a move making Parquet a silent partner.  :)
> > >
> > >michael
> > >
> > >
> > > > On Jan 23, 2020, at 10:25 AM, David Mollitor 
> > wrote:
> > > >
> > > > Hello Team,
> > > >
> > > > I have been a consumer of Apache Parquet through Apache Hive for
> > several
> > > > years now.  For a long time, logging in Parquet has been pretty
> > painful.
> > > > Some of the logging was going to STDOUT and some was going to Log4J.
> > > > Overall, though the framework has been too verbose, spewing many log
> > > lines
> > > > about internal details of Parquet I don't understand.
> > > >
> > > > The logging has gotten a lot better with recent releases moving
> solidly
> > > > into SLF4J.  That is awesome and very welcomed.  However, (opinion
> > > alert) I
> > > > think the logging is still too verbose.  I think Parquet should be a
> > > silent
> > > > partner in data processing.  If everything is going well, it should
> be
> > > > silent (DEBUG level logging).  If things are going wrong, it should
> > throw
> > > > an Exception.
> > > >
> > > > If an operator suspects Parquet is the issue (and that's rarely the
> > first
> > > > thing to check), they can set the logging for all of the Loggers in
> the
> > > > entire Parquet package (org.apache.parquet) to DEBUG to get the
> > required
> > > > information.  Not to mention, the less logging it does, the faster it
> > > will
> > > > be.
> > > >
> > > > I've opened this discussion because I've got two PRs related to this
> > > topic
> > > > ready to go:
> > > >
> > > > PARQUET-1758
> > > > PARQUET-1761
> > > >
> > > > Thanks,
> > > > David
> > >
> > >
> >
>


-- 
Ryan Blue
Software Engineer
Netflix


Re: Preparing release 1.11.1

2020-01-24 Thread Michael Heuer
Hello Gabor,

When I brought similar concerns up on the dev@spark mailing list, I was told to 
bring that discussion to the Parquet community first.

I've been on the Avro and Parquet mailing lists since, hoping I might help 
coordinate between the three communities.  If Avro and Parquet are not upgraded 
in lock step, many compatibility issues are pushed to downstream projects to 
work around.  E.g.

https://beam.apache.org/documentation/io/built-in/parquet/ 


Our current workaround is so embarrassing I'd rather not mention it here.

   michael


> On Jan 24, 2020, at 5:18 AM, Gabor Szadovszky  wrote:
> 
> Thanks a lot, Michael for highlighting this. However, it is more a spark
> issue than a parquet one.
> Could you add your concerns to the spark PR/jira?
> 
> Thanks a lot,
> Gabor
> 
> On Thu, Jan 23, 2020 at 7:08 PM Michael Heuer  > wrote:
> 
>> For example,
>> 
>> https://github.com/bigdatagenomics/adam/pull/2245 
>>  <
>> https://github.com/bigdatagenomics/adam/pull/2245 
>> >
>> 
>> ...
>> Caused by: java.lang.NoSuchMethodError:
>> org.apache.parquet.schema.Types$PrimitiveBuilder.as(Lorg/apache/parquet/schema/LogicalTypeAnnotation;)Lorg/apache/parquet/schema/Types$Builder;
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:161)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertUnion(AvroSchemaConverter.java:226)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:182)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:141)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:244)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convertFields(AvroSchemaConverter.java:135)
>>at
>> org.apache.parquet.avro.AvroSchemaConverter.convert(AvroSchemaConverter.java:126)
>>at
>> org.apache.parquet.avro.AvroWriteSupport.init(AvroWriteSupport.java:121)
>>at
>> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
>>at
>> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
>>at
>> org.apache.spark.rdd.InstrumentedOutputFormat.getRecordWriter(InstrumentedOutputFormat.scala:35)
>>at org.apache.spark.internal.io
>> .HadoopMapReduceWriteConfigUtil.initWriter(SparkHadoopWriter.scala:350)
>>at org.apache.spark.internal.io
>> .SparkHadoopWriter$.org$apache$spark$internal$io$SparkHadoopWriter$$executeTask(SparkHadoopWriter.scala:120)
>>at org.apache.spark.internal.io
>> .SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:83)
>>at org.apache.spark.internal.io
>> .SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:78)
>>at
>> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
>>at org.apache.spark.scheduler.Task.run(Task.scala:123)
>>at
>> org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
>>at
>> org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
>>at
>> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
>>at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>>at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>>at java.lang.Thread.run(Thread.java:748)
>> 
>> 
>>> On Jan 23, 2020, at 10:52 AM, Michael Heuer  wrote:
>>> 
>>> Hello Gabor,
>>> 
>>> This Spark PR upgrades Parquet but does not upgrade Avro, note the
>> exclusion for parquet-avro
>>> 
>>> 
>> https://github.com/apache/spark/pull/26804/files#diff-600376dffeb79835ede4a0b285078036R2104
>> <
>> https://github.com/apache/spark/pull/26804/files#diff-600376dffeb79835ede4a0b285078036R2104
>>  
>> 
>>> 
>>> 
>>> Parquet 1.11.0/1.11.1 depends on Avro 1.9.1 and Spark depends on Avro
>> 1.8.2, how will this Spark PR be compatible?
>>> 
>>>   michael
>>> 
>>> 
 On Jan 23, 2020, at 3:38 AM, Gabor Szadovszky >>> 
>> >> wrote:
 
 Thanks, Fokko. I've linked the related issues to the release jira as
 blockers.
 Currently, every issue is resolved. Waiting for feedback if the
 fixes/descriptions are correct and if we need to fix anything else for
 Spark.
 
 On Wed, Jan 22, 2020 at 5:18 PM Driesprong, Fokko >>> 
>> >>
 wrote:
 
> Thank you Gabor,
> 
> What kind of issues are found? Let me know if I can help 

[jira] [Updated] (PARQUET-1776) Add Java NIO Avro OutputFile InputFile

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1776:

Description: Add a wrapper around Java NIO Path for 
{{org.apache.parquet.io.OutputFile}} and {{org.apache.parquet.io.InputFile}}  
(was: Add a wrapper around Java NIO for {{org.apache.parquet.io.OutputFile}} 
and {{org.apache.parquet.io.InputFile}})

> Add Java NIO Avro OutputFile InputFile
> --
>
> Key: PARQUET-1776
> URL: https://issues.apache.org/jira/browse/PARQUET-1776
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-avro
>Reporter: David Mollitor
>Priority: Minor
>
> Add a wrapper around Java NIO Path for {{org.apache.parquet.io.OutputFile}} 
> and {{org.apache.parquet.io.InputFile}}



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


[jira] [Updated] (PARQUET-1776) Add Java NIO Avro OutputFile InputFile

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1776:

Labels:   (was: avro)

> Add Java NIO Avro OutputFile InputFile
> --
>
> Key: PARQUET-1776
> URL: https://issues.apache.org/jira/browse/PARQUET-1776
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-avro
>Reporter: David Mollitor
>Priority: Minor
>
> Add a wrapper around Java NIO for {{org.apache.parquet.io.OutputFile}} and 
> {{org.apache.parquet.io.InputFile}}



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


[jira] [Created] (PARQUET-1776) Add Java NIO Avro OutputFile InputFile

2020-01-24 Thread David Mollitor (Jira)
David Mollitor created PARQUET-1776:
---

 Summary: Add Java NIO Avro OutputFile InputFile
 Key: PARQUET-1776
 URL: https://issues.apache.org/jira/browse/PARQUET-1776
 Project: Parquet
  Issue Type: Improvement
Reporter: David Mollitor


Add a wrapper around Java NIO for {{org.apache.parquet.io.OutputFile}} and 
{{org.apache.parquet.io.InputFile}}



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


[jira] [Updated] (PARQUET-1776) Add Java NIO Avro OutputFile InputFile

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1776:

Labels: avro  (was: )

> Add Java NIO Avro OutputFile InputFile
> --
>
> Key: PARQUET-1776
> URL: https://issues.apache.org/jira/browse/PARQUET-1776
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Priority: Minor
>  Labels: avro
>
> Add a wrapper around Java NIO for {{org.apache.parquet.io.OutputFile}} and 
> {{org.apache.parquet.io.InputFile}}



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


[jira] [Updated] (PARQUET-1776) Add Java NIO Avro OutputFile InputFile

2020-01-24 Thread David Mollitor (Jira)


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

David Mollitor updated PARQUET-1776:

Component/s: parquet-avro

> Add Java NIO Avro OutputFile InputFile
> --
>
> Key: PARQUET-1776
> URL: https://issues.apache.org/jira/browse/PARQUET-1776
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-avro
>Reporter: David Mollitor
>Priority: Minor
>  Labels: avro
>
> Add a wrapper around Java NIO for {{org.apache.parquet.io.OutputFile}} and 
> {{org.apache.parquet.io.InputFile}}



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


[jira] [Commented] (PARQUET-1775) Deprecate AvroParquetWriter Builder Hadoop Path

2020-01-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1775:
-

belugabehr commented on pull request #750: PARQUET-1775: Deprecate 
AvroParquetWriter Builder Hadoop Path
URL: https://github.com/apache/parquet-mr/pull/750
 
 
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [X] My PR addresses the following 
[PARQUET-1775](https://issues.apache.org/jira/browse/PARQUET/PARQUET-1775) 
issues and references them in the PR title. For example, "PARQUET-1234: My 
Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-1775
 - 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
   
   - [X] 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 AvroParquetWriter Builder Hadoop Path
> ---
>
> Key: PARQUET-1775
> URL: https://issues.apache.org/jira/browse/PARQUET-1775
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>  Labels: pull-request-available
>
> Trying to write a sample program with Parquet and came across the following 
> quark:
>  
> The {{AvroParquetWriter}} has no qualms about building one using 
> {{org.apache.hadoop.fs.Path}}.  However, doing so in {{AvroParquetReader}} is 
> deprecated.  I think it's appropriate to remove all dependencies of Hadoop 
> from this simple reader/writer API.
>  
> To make it consistent, also deprecate the use of 
> {{org.apache.hadoop.fs.Path}} in the {{AvroParquetWriter.}}
>  
> [https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L38]
>  
> https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java#L47



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


[jira] [Updated] (PARQUET-1775) Deprecate AvroParquetWriter Builder Hadoop Path

2020-01-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot updated PARQUET-1775:

Labels: pull-request-available  (was: )

> Deprecate AvroParquetWriter Builder Hadoop Path
> ---
>
> Key: PARQUET-1775
> URL: https://issues.apache.org/jira/browse/PARQUET-1775
> Project: Parquet
>  Issue Type: Improvement
>Reporter: David Mollitor
>Assignee: David Mollitor
>Priority: Minor
>  Labels: pull-request-available
>
> Trying to write a sample program with Parquet and came across the following 
> quark:
>  
> The {{AvroParquetWriter}} has no qualms about building one using 
> {{org.apache.hadoop.fs.Path}}.  However, doing so in {{AvroParquetReader}} is 
> deprecated.  I think it's appropriate to remove all dependencies of Hadoop 
> from this simple reader/writer API.
>  
> To make it consistent, also deprecate the use of 
> {{org.apache.hadoop.fs.Path}} in the {{AvroParquetWriter.}}
>  
> [https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L38]
>  
> https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java#L47



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


[jira] [Created] (PARQUET-1775) Deprecate AvroParquetWriter Builder Hadoop Path

2020-01-24 Thread David Mollitor (Jira)
David Mollitor created PARQUET-1775:
---

 Summary: Deprecate AvroParquetWriter Builder Hadoop Path
 Key: PARQUET-1775
 URL: https://issues.apache.org/jira/browse/PARQUET-1775
 Project: Parquet
  Issue Type: Improvement
Reporter: David Mollitor
Assignee: David Mollitor


Trying to write a sample program with Parquet and came across the following 
quark:

 

The {{AvroParquetWriter}} has no qualms about building one using 
{{org.apache.hadoop.fs.Path}}.  However, doing so in {{AvroParquetReader}} is 
deprecated.  I think it's appropriate to remove all dependencies of Hadoop from 
this simple reader/writer API.

 

To make it consistent, also deprecate the use of {{org.apache.hadoop.fs.Path}} 
in the {{AvroParquetWriter.}}

 

[https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L38]

 

https://github.com/apache/parquet-mr/blob/8c1bc9bcdeeac8178fecf61d18dc56913907fd46/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java#L47



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


Re: Parquet Verbose Logging

2020-01-24 Thread lukas nalezenec
Hi,
I can help too.
Lukas

Dne pá 24. 1. 2020 15:29 uživatel David Mollitor  napsal:

> Hello Team,
>
> I am happy to do the work of reviewing all Parquet logging, but I need help
> getting the work committed.
>
> Fokko Driesprong has been a wonderfully ally in helping me get incremental
> improvements into Parquet, but I wonder if there's anyone else that can
> share in the load.
>
> Thanks,
> David
>
> On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer  wrote:
>
> > Hello David,
> >
> > As I mentioned on PARQUET-1758, we have been frustrated by overly verbose
> > logging in Parquet for a long time.  Various workarounds have been more
> or
> > less successful, e.g.
> >
> > https://github.com/bigdatagenomics/adam/issues/851 <
> > https://github.com/bigdatagenomics/adam/issues/851>
> >
> > I would support a move making Parquet a silent partner.  :)
> >
> >michael
> >
> >
> > > On Jan 23, 2020, at 10:25 AM, David Mollitor 
> wrote:
> > >
> > > Hello Team,
> > >
> > > I have been a consumer of Apache Parquet through Apache Hive for
> several
> > > years now.  For a long time, logging in Parquet has been pretty
> painful.
> > > Some of the logging was going to STDOUT and some was going to Log4J.
> > > Overall, though the framework has been too verbose, spewing many log
> > lines
> > > about internal details of Parquet I don't understand.
> > >
> > > The logging has gotten a lot better with recent releases moving solidly
> > > into SLF4J.  That is awesome and very welcomed.  However, (opinion
> > alert) I
> > > think the logging is still too verbose.  I think Parquet should be a
> > silent
> > > partner in data processing.  If everything is going well, it should be
> > > silent (DEBUG level logging).  If things are going wrong, it should
> throw
> > > an Exception.
> > >
> > > If an operator suspects Parquet is the issue (and that's rarely the
> first
> > > thing to check), they can set the logging for all of the Loggers in the
> > > entire Parquet package (org.apache.parquet) to DEBUG to get the
> required
> > > information.  Not to mention, the less logging it does, the faster it
> > will
> > > be.
> > >
> > > I've opened this discussion because I've got two PRs related to this
> > topic
> > > ready to go:
> > >
> > > PARQUET-1758
> > > PARQUET-1761
> > >
> > > Thanks,
> > > David
> >
> >
>


Re: Parquet Verbose Logging

2020-01-24 Thread David Mollitor
Hello Team,

I am happy to do the work of reviewing all Parquet logging, but I need help
getting the work committed.

Fokko Driesprong has been a wonderfully ally in helping me get incremental
improvements into Parquet, but I wonder if there's anyone else that can
share in the load.

Thanks,
David

On Thu, Jan 23, 2020 at 11:55 AM Michael Heuer  wrote:

> Hello David,
>
> As I mentioned on PARQUET-1758, we have been frustrated by overly verbose
> logging in Parquet for a long time.  Various workarounds have been more or
> less successful, e.g.
>
> https://github.com/bigdatagenomics/adam/issues/851 <
> https://github.com/bigdatagenomics/adam/issues/851>
>
> I would support a move making Parquet a silent partner.  :)
>
>michael
>
>
> > On Jan 23, 2020, at 10:25 AM, David Mollitor  wrote:
> >
> > Hello Team,
> >
> > I have been a consumer of Apache Parquet through Apache Hive for several
> > years now.  For a long time, logging in Parquet has been pretty painful.
> > Some of the logging was going to STDOUT and some was going to Log4J.
> > Overall, though the framework has been too verbose, spewing many log
> lines
> > about internal details of Parquet I don't understand.
> >
> > The logging has gotten a lot better with recent releases moving solidly
> > into SLF4J.  That is awesome and very welcomed.  However, (opinion
> alert) I
> > think the logging is still too verbose.  I think Parquet should be a
> silent
> > partner in data processing.  If everything is going well, it should be
> > silent (DEBUG level logging).  If things are going wrong, it should throw
> > an Exception.
> >
> > If an operator suspects Parquet is the issue (and that's rarely the first
> > thing to check), they can set the logging for all of the Loggers in the
> > entire Parquet package (org.apache.parquet) to DEBUG to get the required
> > information.  Not to mention, the less logging it does, the faster it
> will
> > be.
> >
> > I've opened this discussion because I've got two PRs related to this
> topic
> > ready to go:
> >
> > PARQUET-1758
> > PARQUET-1761
> >
> > Thanks,
> > David
>
>


Re: Preparing release 1.11.1

2020-01-24 Thread Gabor Szadovszky
Thanks a lot, Michael for highlighting this. However, it is more a spark
issue than a parquet one.
Could you add your concerns to the spark PR/jira?

Thanks a lot,
Gabor

On Thu, Jan 23, 2020 at 7:08 PM Michael Heuer  wrote:

> For example,
>
> https://github.com/bigdatagenomics/adam/pull/2245 <
> https://github.com/bigdatagenomics/adam/pull/2245>
>
> ...
> Caused by: java.lang.NoSuchMethodError:
> org.apache.parquet.schema.Types$PrimitiveBuilder.as(Lorg/apache/parquet/schema/LogicalTypeAnnotation;)Lorg/apache/parquet/schema/Types$Builder;
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:161)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertUnion(AvroSchemaConverter.java:226)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:182)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:141)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertField(AvroSchemaConverter.java:244)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convertFields(AvroSchemaConverter.java:135)
> at
> org.apache.parquet.avro.AvroSchemaConverter.convert(AvroSchemaConverter.java:126)
> at
> org.apache.parquet.avro.AvroWriteSupport.init(AvroWriteSupport.java:121)
> at
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
> at
> org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
> at
> org.apache.spark.rdd.InstrumentedOutputFormat.getRecordWriter(InstrumentedOutputFormat.scala:35)
> at org.apache.spark.internal.io
> .HadoopMapReduceWriteConfigUtil.initWriter(SparkHadoopWriter.scala:350)
> at org.apache.spark.internal.io
> .SparkHadoopWriter$.org$apache$spark$internal$io$SparkHadoopWriter$$executeTask(SparkHadoopWriter.scala:120)
> at org.apache.spark.internal.io
> .SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:83)
> at org.apache.spark.internal.io
> .SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:78)
> at
> org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
> at org.apache.spark.scheduler.Task.run(Task.scala:123)
> at
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
> at
> org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
> at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
>
>
> > On Jan 23, 2020, at 10:52 AM, Michael Heuer  wrote:
> >
> > Hello Gabor,
> >
> > This Spark PR upgrades Parquet but does not upgrade Avro, note the
> exclusion for parquet-avro
> >
> >
> https://github.com/apache/spark/pull/26804/files#diff-600376dffeb79835ede4a0b285078036R2104
> <
> https://github.com/apache/spark/pull/26804/files#diff-600376dffeb79835ede4a0b285078036R2104
> >
> >
> > Parquet 1.11.0/1.11.1 depends on Avro 1.9.1 and Spark depends on Avro
> 1.8.2, how will this Spark PR be compatible?
> >
> >michael
> >
> >
> >> On Jan 23, 2020, at 3:38 AM, Gabor Szadovszky  > wrote:
> >>
> >> Thanks, Fokko. I've linked the related issues to the release jira as
> >> blockers.
> >> Currently, every issue is resolved. Waiting for feedback if the
> >> fixes/descriptions are correct and if we need to fix anything else for
> >> Spark.
> >>
> >> On Wed, Jan 22, 2020 at 5:18 PM Driesprong, Fokko  >
> >> wrote:
> >>
> >>> Thank you Gabor,
> >>>
> >>> What kind of issues are found? Let me know if I can help in any way.
> >>>
> >>> Cheers, Fokko
> >>>
> >>> Op wo 22 jan. 2020 om 11:10 schreef Gabor Szadovszky  >:
> >>>
>  Dear All,
> 
>  During the migration to 1.11.0 in Spark we discovered some issues in
> the
>  parquet release. I am preparing the minor release 1.11.1 to fix these
>  issues. Created the jira
>  https://issues.apache.org/jira/browse/PARQUET-1774 <
> https://issues.apache.org/jira/browse/PARQUET-1774> to
>  track this effort. Feel free to link any bug jiras if they are
> >>> regressions
>  in 1.11.0.
>  The release will be prepared in the separate branch "parquet-1.11.x".
> >>> I'll
>  do the backports as required.
> 
>  Regards,
>  Gabor
> 
> >>>
> >
>
>