Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
1.9.1
-
None
Description
Trying to load parquet files in Pig, that have the following causes an exception and parsing to fail:
- INT96 fields, for example:
message spark_schema { optional int96 datetime; }
The Exception thrown is:
Failed to parse: can't convert optional int96 myInt96 at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:201) at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1791) at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1764) at org.apache.pig.PigServer.registerQuery(PigServer.java:707) at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:1075) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:505) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:231) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:206) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:66) at org.apache.pig.Main.run(Main.java:564) at org.apache.pig.Main.main(Main.java:176) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.util.RunJar.run(RunJar.java:234) at org.apache.hadoop.util.RunJar.main(RunJar.java:148) Caused by: org.apache.parquet.pig.SchemaConversionException: can't convert optional int96 myInt96 at org.apache.parquet.pig.PigSchemaConverter.convertFields(PigSchemaConverter.java:202) at org.apache.parquet.pig.PigSchemaConverter.convert(PigSchemaConverter.java:178) at org.apache.parquet.pig.TupleReadSupport.getPigSchemaFromMultipleFiles(TupleReadSupport.java:95) at org.apache.parquet.pig.ParquetLoader.initSchema(ParquetLoader.java:300) at org.apache.parquet.pig.ParquetLoader.setInput(ParquetLoader.java:183) at org.apache.parquet.pig.ParquetLoader.getSchema(ParquetLoader.java:285) at org.apache.pig.newplan.logical.relational.LOLoad.getSchemaFromMetaData(LOLoad.java:175) at org.apache.pig.newplan.logical.relational.LOLoad.<init>(LOLoad.java:89) at org.apache.pig.parser.LogicalPlanBuilder.buildLoadOp(LogicalPlanBuilder.java:901) at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3568) at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1625) at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:1102) at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:560) at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:421) at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:191) ... 16 more Caused by: org.apache.pig.impl.logicalLayer.FrontendException: ERROR 0: NYI at org.apache.parquet.pig.PigSchemaConverter$1.convertINT96(PigSchemaConverter.java:242) at org.apache.parquet.pig.PigSchemaConverter$1.convertINT96(PigSchemaConverter.java:214) at org.apache.parquet.schema.PrimitiveType$PrimitiveTypeName$7.convert(PrimitiveType.java:223) at org.apache.parquet.pig.PigSchemaConverter.getSimpleFieldSchema(PigSchemaConverter.java:213) at org.apache.parquet.pig.PigSchemaConverter.getFieldSchema(PigSchemaConverter.java:320) at org.apache.parquet.pig.PigSchemaConverter.convertFields(PigSchemaConverter.java:193) ... 30 more
- Map Types without OriginalType, for example:
message spark_schema { optional binary a; optional group b (MAP) { repeated group map { required binary key; optional group value { optional fixed_len_byte_array(5) c; optional fixed_len_byte_array(7) d; } } } }
The Exception thrown is:
ERROR 1200: null Failed to parse: null at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:201) at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1791) at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1764) at org.apache.pig.PigServer.registerQuery(PigServer.java:707) at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:1075) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:505) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:231) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:206) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:66) at org.apache.pig.Main.run(Main.java:564) at org.apache.pig.Main.main(Main.java:176) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.util.RunJar.run(RunJar.java:234) at org.apache.hadoop.util.RunJar.main(RunJar.java:148) Caused by: java.lang.NullPointerException at org.apache.parquet.pig.PigSchemaConverter.getComplexFieldSchema(PigSchemaConverter.java:281) at org.apache.parquet.pig.PigSchemaConverter.getFieldSchema(PigSchemaConverter.java:322) at org.apache.parquet.pig.PigSchemaConverter.convertFields(PigSchemaConverter.java:193) at org.apache.parquet.pig.PigSchemaConverter.convert(PigSchemaConverter.java:178) at org.apache.parquet.pig.TupleReadSupport.getPigSchemaFromMultipleFiles(TupleReadSupport.java:95) at org.apache.parquet.pig.ParquetLoader.initSchema(ParquetLoader.java:300) at org.apache.parquet.pig.ParquetLoader.setInput(ParquetLoader.java:183) at org.apache.parquet.pig.ParquetLoader.getSchema(ParquetLoader.java:285) at org.apache.pig.newplan.logical.relational.LOLoad.getSchemaFromMetaData(LOLoad.java:175) at org.apache.pig.newplan.logical.relational.LOLoad.<init>(LOLoad.java:89) at org.apache.pig.parser.LogicalPlanBuilder.buildLoadOp(LogicalPlanBuilder.java:901) at org.apache.pig.parser.LogicalPlanGenerator.load_clause(LogicalPlanGenerator.java:3568) at org.apache.pig.parser.LogicalPlanGenerator.op_clause(LogicalPlanGenerator.java:1625) at org.apache.pig.parser.LogicalPlanGenerator.general_statement(LogicalPlanGenerator.java:1102) at org.apache.pig.parser.LogicalPlanGenerator.statement(LogicalPlanGenerator.java:560) at org.apache.pig.parser.LogicalPlanGenerator.query(LogicalPlanGenerator.java:421) at org.apache.pig.parser.QueryParserDriver.parse(QueryParserDriver.java:191) ... 16 more