[
https://issues.apache.org/jira/browse/PIG-4628?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14619249#comment-14619249
]
Koji Noguchi commented on PIG-4628:
-----------------------------------
Full stack trace
{noformat}
2015-06-08 22:24:39,246 WARN [main] org.apache.hadoop.mapred.YarnChild:
Exception running child : java.lang.RuntimeException:
java.lang.RuntimeException: java.io.IOException: Exception reading
file:/tmp/2/yarn-local/usercache/userA/appcache/application_1432697993142_199266/container_e06_1432697993142_199266_01_000003/container_tokens
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.init(WeightedRangePartitioner.java:155)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:75)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:58)
at org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:712)
at
org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
at
org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Map.collect(PigGenericMapReduce.java:135)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:281)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:274)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:784)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1694)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
Caused by: java.lang.RuntimeException: java.io.IOException: Exception reading
file:/tmp/2/yarn-local/usercache/userA/appcache/application_1432697993142_199266/container_e06_1432697993142_199266_01_000003/container_tokens
at
org.apache.hadoop.mapreduce.security.TokenCache.mergeBinaryTokens(TokenCache.java:141)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:119)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:100)
at
org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes(TokenCache.java:80)
at
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:242)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigFileInputFormat.listStatus(PigFileInputFormat.java:37)
at
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:385)
at org.apache.pig.impl.io.ReadToEndLoader.init(ReadToEndLoader.java:190)
at org.apache.pig.impl.io.ReadToEndLoader.<init>(ReadToEndLoader.java:126)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.init(WeightedRangePartitioner.java:127)
... 17 more
Caused by: java.io.IOException: Exception reading
file:/tmp/2/yarn-local/usercache/userA/appcache/application_1432697993142_199266/container_e06_1432697993142_199266_01_000003/container_tokens
at
org.apache.hadoop.security.Credentials.readTokenStorageFile(Credentials.java:174)
at
org.apache.hadoop.mapreduce.security.TokenCache.mergeBinaryTokens(TokenCache.java:136)
... 26 more
Caused by: java.io.FileNotFoundException: File
file:/tmp/2/yarn-local/usercache/userA/appcache/application_1432697993142_199266/container_e06_1432697993142_199266_01_000003/container_tokens
does not exist
at
org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:534)
at
org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:747)
at
org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:524)
at
org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:416)
at
org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>(ChecksumFileSystem.java:140)
at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:341)
at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:766)
at
org.apache.hadoop.security.Credentials.readTokenStorageFile(Credentials.java:169)
... 27 more
{noformat}
On IM, [~rohini]] taught me,
bq. [MAPREDUCE-3727|https://issues.apache.org/jira/browse/MAPREDUCE-3727] -
Hadoop ensures that for a new job submitted the
mapreduce.credentials.job.binary is removed. So this is creeping in from
somewhere else. If Pig 0.11 works then something in Pig 0.14 is the culprit.
Based on this feedback, ran a job with custom hadoop.jar that dumped thread
when mapreduce.job.credentials.binary was set.
{code:title=Configuration.java}
990 public void set(String name, String value) {
991 if(name.equals("mapreduce.job.credentials.binary")) {
992 System.err.println("Koji");
993 Thread.dumpStack();
994 }
995 set(name, value, null);
996 }
{code}
This job produced the stack trace of
{noformat}
Log Type: stderr
Log Upload Time: 7-Jul-2015 21:28:18
Log Length: 2336
Koji
java.lang.Exception: Stack trace
at java.lang.Thread.dumpStack(Thread.java:1342)
at org.apache.hadoop.conf.Configuration.set(Configuration.java:993)
at
org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil.toConfiguration(ConfigurationUtil.java:48)
at
org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil.toConfiguration(ConfigurationUtil.java:38)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.init(WeightedRangePartitioner.java:113)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:75)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:58)
at
org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:712)
at
org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
at
org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Map.collect(PigGenericMapReduce.java:135)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:281)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:274)
at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:784)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:415)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1694)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
{noformat}
Looking at the stack trace, we can see that it is being set from
{code:title=WeightedRangePartitioner.java}
112 if (!pigContext.getExecType().isLocal()) {
113 conf =
ConfigurationUtil.toConfiguration(pigContext.getProperties());
114 }
{code}
So the mapreduce.credentials.job.binary setting from the launcher was coming
from the serialized pigContext inside the jobconf.
This part of the code was changed in
[PIG-4257|https://issues.apache.org/jira/browse/PIG-4257] where the patch had
{code:title=PIG-4257-3.patch}
if (!pigContext.getExecType().isLocal()) {
- conf = new Configuration(true);
+ conf =
ConfigurationUtil.toConfiguration(pigContext.getProperties());
}
{code}
> Pig 0.14 job with order by fails in mapreduce mode with Oozie
> -------------------------------------------------------------
>
> Key: PIG-4628
> URL: https://issues.apache.org/jira/browse/PIG-4628
> Project: Pig
> Issue Type: Bug
> Components: impl
> Affects Versions: 0.14.0, 0.15.0
> Reporter: Koji Noguchi
> Assignee: Koji Noguchi
>
> A simple pig script with order-by submitted through oozie and running with
> mapreduce-mode
> {code}
> A = LOAD '$input' AS (a1:CHARARRAY,a2:CHARARRAY, );
> A_sorted = ORDER A BY url DESC PARALLEL 2;
> STORE A_sorted INTO '$output';
> {code}
> failed on our hadoop cluster which had security turned on. Part of the stack
> trace had
> {noformat}
> 2015-06-08 22:24:39,246 WARN [main] org.apache.hadoop.mapred.YarnChild:
> Exception running child : java.lang.RuntimeException:
> java.lang.RuntimeException: java.io.IOException: Exception reading
> file:/tmp/2/yarn-local/usercache/userA/appcache/application_1432697993142_199266/container_e06_1432697993142_199266_01_000003/container_tokens
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.init(WeightedRangePartitioner.java:155)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:75)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.partitioners.WeightedRangePartitioner.getPartition(WeightedRangePartitioner.java:58)
> at
> org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:712)
> at
> org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
> at
> org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapReduce$Map.collect(PigGenericMapReduce.java:135)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.runPipeline(PigGenericMapBase.java:281)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:274)
> at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigGenericMapBase.map(PigGenericMapBase.java:64)
> at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:784)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1694)
> at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
> {noformat}
> This failing job was from application_1432697993142_199305 and the error path
> was from application_1432697993142_199266 which was a oozie pig-launcher job.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)