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

Jun Zhang commented on FLINK-16646:
-----------------------------------

ContinuousFileReaderOperator use OrcRowInputFormat to read orc files and use 
the same orc schema, but when there are multiple files in the path, the system 
reads a file(split) and will call the close method of OrcRowInputFormat to set 
the schema to null, and then read the second orc file,but the schema is null 
now, the system throws a java.lang.NullPointerException, so we move set the 
schema to null into the closeInputFormat method.

hi, [~lzljs3620320] I submit a PR to solve the bug 

> flink read orc file throw a NullPointerException
> ------------------------------------------------
>
>                 Key: FLINK-16646
>                 URL: https://issues.apache.org/jira/browse/FLINK-16646
>             Project: Flink
>          Issue Type: Bug
>          Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile)
>    Affects Versions: 1.10.0
>            Reporter: Jun Zhang
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.11.0
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> When I use OrcRowInputFormat to read multiple orc files, the system throws 
> one NullPointerException .
> the code like this
>  
> {code:java}
> StreamExecutionEnvironment environment = 
> StreamExecutionEnvironment.getExecutionEnvironment();
> environment.setParallelism(1);
> String path = "file://tmp/dir";
> String schema = ..... ;
> OrcRowInputFormat orcRowInputFormat = new OrcRowInputFormat(
>    path,
>    schema,
>    new org.apache.hadoop.conf.Configuration());
> DataStream dataStream  =environment.createInput(orcRowInputFormat);
> dataStream.writeAsText("file:///tmp/aaa", FileSystem.WriteMode.OVERWRITE);
> environment.execute();
> {code}
>  
> the exception is 
>  
> {code:java}
> Caused by: java.lang.NullPointerExceptionCaused by: 
> java.lang.NullPointerException at 
> org.apache.flink.orc.shim.OrcShimV200.computeProjectionMask(OrcShimV200.java:188)
>  at 
> org.apache.flink.orc.shim.OrcShimV200.createRecordReader(OrcShimV200.java:120)
>  at org.apache.flink.orc.OrcSplitReader.<init>(OrcSplitReader.java:73) at 
> org.apache.flink.orc.OrcRowSplitReader.<init>(OrcRowSplitReader.java:50) at 
> org.apache.flink.orc.OrcRowInputFormat.open(OrcRowInputFormat.java:102) at 
> org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator$SplitReader.run(ContinuousFileReaderOperator.java:315)
> {code}
>  
>  



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

Reply via email to