[
https://issues.apache.org/jira/browse/PIG-5065?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15677183#comment-15677183
]
Roman Kotelnikov commented on PIG-5065:
---------------------------------------
So I've been checking code briefly, and found fix for similar issue on github
for HIVE (https://github.com/apache/hive/pull/96). AFAIK Pig's OrcStorage uses
hive implementation (OrcNewInputFormat class). The issue has been fixed for
versions 1.3.0, 2.2.0, 2.1.1, 2.0.2, but our Pig distribution builds against
0.14, so that seems like plausible cause.
I can't re-run it with tez right now, but will do it next week & post update
here just to make sure. Thanks again for checking.
> Unable to read records from multiple files written with OrcStorage
> ------------------------------------------------------------------
>
> Key: PIG-5065
> URL: https://issues.apache.org/jira/browse/PIG-5065
> Project: Pig
> Issue Type: Bug
> Affects Versions: 0.16.0
> Environment: Pig Version: 0.16.0
> Tez: 0.8.4
> OS: Amazon Linux AMI 2016.09 (rhel fedora) / EMR 5.1.0
> Reporter: Roman Kotelnikov
>
> Scenario:
> 1. Read data from .TSV file & store using ORC
> {code}
> x = load '$INPUT' USING PigStorage('\t', '-schema');
> y = FOREACH (GROUP x BY fields PARALLEL 10) { GENERATE *; };
> store y into '$OUTPUT' USING OrcStorage();
> {code}
> {code}
> [hadoop@ip-10-0-0-28 ~]$ hdfs dfs -ls /out
> Found 10 items
> -rw-r--r-- 1 hadoop hadoop 101340 2016-11-18 13:18
> /out/part-v001-o000-r-00000
> -rw-r--r-- 1 hadoop hadoop 94853 2016-11-18 13:18
> /out/part-v001-o000-r-00001
> -rw-r--r-- 1 hadoop hadoop 90494 2016-11-18 13:18
> /out/part-v001-o000-r-00002
> -rw-r--r-- 1 hadoop hadoop 109485 2016-11-18 13:18
> /out/part-v001-o000-r-00003
> -rw-r--r-- 1 hadoop hadoop 75547 2016-11-18 13:18
> /out/part-v001-o000-r-00004
> -rw-r--r-- 1 hadoop hadoop 115291 2016-11-18 13:18
> /out/part-v001-o000-r-00005
> -rw-r--r-- 1 hadoop hadoop 106647 2016-11-18 13:18
> /out/part-v001-o000-r-00006
> -rw-r--r-- 1 hadoop hadoop 80439 2016-11-18 13:18
> /out/part-v001-o000-r-00007
> -rw-r--r-- 1 hadoop hadoop 107185 2016-11-18 13:18
> /out/part-v001-o000-r-00008
> -rw-r--r-- 1 hadoop hadoop 122105 2016-11-18 13:18
> /out/part-v001-o000-r-00009
> {code}
> 2. Try to read data with
> {code}
> x = load '$INPUT' using OrcStorage();
> store x into '$OUTPUT' using PigStorage('\t', '-schema');
> {code}
> Getting following exception
> {code}
> 31768 [main] ERROR org.apache.pig.tools.grunt.GruntParser - ERROR 2998:
> Unhandled internal error. Vertex failed, vertexName=scope-2,
> vertexId=vertex_1479474344597_0006_1_00, diagnostics=[Task failed,
> taskId=task_1479474344597_0006_1_00_000000, diagnostics=[TaskAttempt 0
> failed, info=[Error: Error while running task ( failure ) :
> attempt_1479474344597_0006_1_00_000000_0:java.lang.ArrayIndexOutOfBoundsException:
> 1024
> at
> org.apache.orc.impl.RunLengthIntegerReaderV2.nextVector(RunLengthIntegerReaderV2.java:369)
> at
> org.apache.orc.impl.TreeReaderFactory$BytesColumnVectorUtil.commonReadByteArrays(TreeReaderFactory.java:1231)
> at
> org.apache.orc.impl.TreeReaderFactory$BytesColumnVectorUtil.readOrcByteArrays(TreeReaderFactory.java:1268)
> at
> org.apache.orc.impl.TreeReaderFactory$StringDirectTreeReader.nextVector(TreeReaderFactory.java:1368)
> at
> org.apache.orc.impl.TreeReaderFactory$StringTreeReader.nextVector(TreeReaderFactory.java:1212)
> at
> org.apache.orc.impl.TreeReaderFactory$StructTreeReader.nextVector(TreeReaderFactory.java:1754)
> at
> org.apache.orc.impl.TreeReaderFactory$ListTreeReader.nextVector(TreeReaderFactory.java:1902)
> at
> org.apache.orc.impl.TreeReaderFactory$StructTreeReader.nextBatch(TreeReaderFactory.java:1737)
> at
> org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1045)
> at
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.ensureBatch(RecordReaderImpl.java:77)
> at
> org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.hasNext(RecordReaderImpl.java:89)
> at
> org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat$OrcRecordReader.nextKeyValue(OrcNewInputFormat.java:110)
> at org.apache.pig.builtin.OrcStorage.getNext(OrcStorage.java:375)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigRecordReader.nextKeyValue(PigRecordReader.java:204)
> at
> org.apache.tez.mapreduce.lib.MRReaderMapReduce.next(MRReaderMapReduce.java:119)
> at
> org.apache.pig.backend.hadoop.executionengine.tez.plan.operator.POSimpleTezLoad.getNextTuple(POSimpleTezLoad.java:140)
> at
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:305)
> at
> org.apache.pig.backend.hadoop.executionengine.tez.plan.operator.POStoreTez.getNextTuple(POStoreTez.java:123)
> at
> org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.runPipeline(PigProcessor.java:376)
> at
> org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.run(PigProcessor.java:241)
> at
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:370)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
> at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)