[jira] [Commented] (MAPREDUCE-5956) MapReduce AM should not use maxAttempts to determine if this is the last retry

2014-07-09 Thread Wangda Tan (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056026#comment-14056026
 ] 

Wangda Tan commented on MAPREDUCE-5956:
---

Thanks thoughts provided by [~vinodkv], had a offline discussion with Vinod, 
post summary here,

Basically there're 3 cases need cleanup.
a. Job completed (failed or succeeded, no matter it's lastRetry or not)
b. Failure happened, and captured by MRAppMasterShutDownHook
c. Failure happened, and doesn't capture by MRAppMasterShutDownHook

And for thoughts provided by Vinod,
{code}
1. YARN informs AM that it is the last retry as part of AM start-up or the 
register API
2. YARN informs the AM that this is the last retry as part of AM unregister
3. YARN has a way to run a separate cleanup container after it knows for sure 
that the application finished exhausting all its attempts
{code}

(1) can solve a. and part of b.
Why only part of b? Because it is possible MRAppMasterShutdownHook triggered 
but other possible failure happened causing cleanup not completed.
(2) can only solve a.
Reason is, if we don't have isLastRetry (or mayBeTheLastAttempt) properly set 
at register, we don't know if should do cleanup or not.
(3) can solve a. b. c.
Refer to YARN-2261 for more details.

I tried to work on (1) first, however, I found moving isLastRetry setup from 
MRAppMaster.init to RMCommunicator cause a lots code changes and lots of unit 
test failures, etc. 
So my suggestion is quickly finish (2), make job completed case correct, which 
is the most usual case. And push (3) forward.

I'll upload a patch in method (2) for review soon.

Thanks,
Wangda

 MapReduce AM should not use maxAttempts to determine if this is the last retry
 --

 Key: MAPREDUCE-5956
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5956
 Project: Hadoop Map/Reduce
  Issue Type: Sub-task
  Components: applicationmaster, mrv2
Reporter: Vinod Kumar Vavilapalli
Assignee: Wangda Tan
Priority: Blocker

 Found this while reviewing YARN-2074. The problem is that after YARN-2074, we 
 don't count AM preemption towards AM failures on RM side, but MapReduce AM 
 itself checks the attempt id against the max-attempt count to determine if 
 this is the last attempt.
 {code}
 public void computeIsLastAMRetry() {
   isLastAMRetry = appAttemptID.getAttemptId() = maxAppAttempts;
 }
 {code}
 This causes issues w.r.t deletion of staging directory etc..



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5956) MapReduce AM should not use maxAttempts to determine if this is the last retry

2014-07-09 Thread Zhijie Shen (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056074#comment-14056074
 ] 

Zhijie Shen commented on MAPREDUCE-5956:


bq. b. Failure happened, and captured by MRAppMasterShutDownHook

How can (2) work for b? Since MR AM doesn't know the preemption, the only 
possibility is that MR AM thinks it's not last retry, but RM thinks it is (RM 
may also think it's not last retry but with one fewer attempt). In this case, 
MR AM want to get the right last retry flag from RM. However, RMCommunicator is 
not supposed to do unregistration if RM AM doesn't think it's the last retry 
now. Hence I'm afraid MR AM doesn't have the chance to communicate with RM to 
inquiry the right information, unless the logic to trigger unregistration is 
modified. Please correct me if i'm missing something.

 MapReduce AM should not use maxAttempts to determine if this is the last retry
 --

 Key: MAPREDUCE-5956
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5956
 Project: Hadoop Map/Reduce
  Issue Type: Sub-task
  Components: applicationmaster, mrv2
Reporter: Vinod Kumar Vavilapalli
Assignee: Wangda Tan
Priority: Blocker

 Found this while reviewing YARN-2074. The problem is that after YARN-2074, we 
 don't count AM preemption towards AM failures on RM side, but MapReduce AM 
 itself checks the attempt id against the max-attempt count to determine if 
 this is the last attempt.
 {code}
 public void computeIsLastAMRetry() {
   isLastAMRetry = appAttemptID.getAttemptId() = maxAppAttempts;
 }
 {code}
 This causes issues w.r.t deletion of staging directory etc..



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (MAPREDUCE-5964) Add an application timeout service in RM to kill jobs which are not getting resources

2014-07-09 Thread Ashutosh Jindal (JIRA)
Ashutosh Jindal created MAPREDUCE-5964:
--

 Summary: Add an application timeout service in RM to kill jobs 
which are not getting resources
 Key: MAPREDUCE-5964
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5964
 Project: Hadoop Map/Reduce
  Issue Type: Improvement
  Components: resourcemanager
Reporter: Ashutosh Jindal


Currently , If an application is submitted to RM, the app keeps waiting until 
the resources are allocated for AM. Such an application may be stuck till a 
resource is allocated for AM, and this may be due to over utilization of Queue 
or User limits etc. In a production cluster, some periodic running applications 
may have lesser cluster share. So after waiting for some time, if resources are 
not available, such applications can be made as failed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (MAPREDUCE-5963) ShuffleHandler DB schema should be versioned with compatible/incompatible changes

2014-07-09 Thread Junping Du (JIRA)

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

Junping Du updated MAPREDUCE-5963:
--

Attachment: MAPREDUCE-5963.patch

Upload a draft patch. It should need some code in YARN-2045.

 ShuffleHandler DB schema should be versioned with compatible/incompatible 
 changes
 -

 Key: MAPREDUCE-5963
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5963
 Project: Hadoop Map/Reduce
  Issue Type: Sub-task
Affects Versions: 2.4.1
Reporter: Junping Du
Assignee: Junping Du
 Attachments: MAPREDUCE-5963.patch


 ShuffleHandler persist job shuffle info into DB schema, which should be 
 versioned with compatible/incompatible changes to support rolling upgrade.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5890) Support for encrypting Intermediate data and spills in local filesystem

2014-07-09 Thread Arun Suresh (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056489#comment-14056489
 ] 

Arun Suresh commented on MAPREDUCE-5890:


[~chris.douglas],
So, if our objective is to have CryptoUtils contained, and since it is 
currently being used only by the mapped framework, I'd prefer it remains where 
it is : in 
{{hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/o/a/h/mapreduce/task/crypto/}}
else It looks like the only other place it fits in is 
{{./hadoop-common-project/hadoop-common/src/main/java/o/a/h/crypto/}} in which 
case, it will look like a more generic utility and we will invite more people 
using it before it becomes stable.

I'd rather move it over once it stabilizes

 Support for encrypting Intermediate data and spills in local filesystem
 ---

 Key: MAPREDUCE-5890
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5890
 Project: Hadoop Map/Reduce
  Issue Type: New Feature
  Components: security
Affects Versions: 2.4.0
Reporter: Alejandro Abdelnur
Assignee: Arun Suresh
  Labels: encryption
 Attachments: MAPREDUCE-5890.10.patch, MAPREDUCE-5890.11.patch, 
 MAPREDUCE-5890.12.patch, MAPREDUCE-5890.13.patch, MAPREDUCE-5890.3.patch, 
 MAPREDUCE-5890.4.patch, MAPREDUCE-5890.5.patch, MAPREDUCE-5890.6.patch, 
 MAPREDUCE-5890.7.patch, MAPREDUCE-5890.8.patch, MAPREDUCE-5890.9.patch, 
 org.apache.hadoop.mapred.TestMRIntermediateDataEncryption-output.txt, 
 syslog.tar.gz


 For some sensitive data, encryption while in flight (network) is not 
 sufficient, it is required that while at rest it should be encrypted. 
 HADOOP-10150  HDFS-6134 bring encryption at rest for data in filesystem 
 using Hadoop FileSystem API. MapReduce intermediate data and spills should 
 also be encrypted while at rest.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5790) Default map hprof profile options do not work

2014-07-09 Thread Karthik Kambatla (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056597#comment-14056597
 ] 

Karthik Kambatla commented on MAPREDUCE-5790:
-

[~jira.shegalov] - can we iterate on this faster to unblock the release? 

 Default map hprof profile options do not work
 -

 Key: MAPREDUCE-5790
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5790
 Project: Hadoop Map/Reduce
  Issue Type: Bug
Affects Versions: 2.3.0
 Environment: java version 1.6.0_31
 Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
Reporter: Andrew Wang
Assignee: Gera Shegalov
Priority: Blocker
 Attachments: MAPREDUCE-5790.v01.patch


 I have an MR job doing the following:
 {code}
 Job job = Job.getInstance(conf);
 // Enable profiling
 job.setProfileEnabled(true);
 job.setProfileTaskRange(true, 0);
 job.setProfileTaskRange(false, 0);
 {code}
 When I run this job, some of my map tasks fail with this error message:
 {noformat}
 org.apache.hadoop.util.Shell$ExitCodeException: 
 /data/5/yarn/nm/usercache/hdfs/appcache/application_1394482121761_0012/container_1394482121761_0012_01_41/launch_container.sh:
  line 32: $JAVA_HOME/bin/java -Djava.net.preferIPv4Stack=true 
 -Dhadoop.metrics.log.level=WARN   -Xmx825955249 -Djava.io.tmpdir=$PWD/tmp 
 -Dlog4j.configuration=container-log4j.properties 
 -Dyarn.app.container.log.dir=/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41
  -Dyarn.app.container.log.filesize=0 -Dhadoop.root.logger=INFO,CLA 
 ${mapreduce.task.profile.params} org.apache.hadoop.mapred.YarnChild 
 10.20.212.12 43135 attempt_1394482121761_0012_r_00_0 41 
 1/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41/stdout
  
 2/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41/stderr
  : bad substitution
 {noformat}
 It looks like ${mapreduce.task.profile.params} is not getting subbed in 
 correctly.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5952) LocalContainerLauncher#renameMapOutputForReduce incorrectly assumes a single dir for mapOutIndex

2014-07-09 Thread Karthik Kambatla (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5952?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056595#comment-14056595
 ] 

Karthik Kambatla commented on MAPREDUCE-5952:
-

[~jira.shegalov] - can we iterate on this faster to unblock the release?

 LocalContainerLauncher#renameMapOutputForReduce incorrectly assumes a single 
 dir for mapOutIndex
 

 Key: MAPREDUCE-5952
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5952
 Project: Hadoop Map/Reduce
  Issue Type: Bug
  Components: mr-am, mrv2
Affects Versions: 2.3.0
Reporter: Gera Shegalov
Assignee: Gera Shegalov
Priority: Blocker
 Attachments: MAPREDUCE-5952.v01.patch


 The javadoc comment for {{renameMapOutputForReduce}} incorrectly refers to a 
 single map output directory, whereas this depends on LOCAL_DIRS.
 mapOutIndex should be set to subMapOutputFile.getOutputIndexFile()
 {code}
 2014-06-30 14:48:35,574 WARN [uber-SubtaskRunner] 
 org.apache.hadoop.mapred.LocalContainerLauncher: Exception running local 
 (uberized) 'child' : java.io.FileNotFoundException: File 
 /Users/gshegalov/workspace/hadoop-common/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/target/org.apache.hadoop.mapreduce.v2.TestMRJobs/org.apache.hadoop.mapreduce.v2.
   
 TestMRJobs-localDir-nm-2_3/usercache/gshegalov/appcache/application_1404164272885_0001/output/file.out.index
  does not exist
   at 
 org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:517)
   at 
 org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:726)
   at 
 org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:507)
   at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:337)
   
   at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:289)
   
   at 
 org.apache.hadoop.fs.RawLocalFileSystem.rename(RawLocalFileSystem.java:334)   
  
   at 
 org.apache.hadoop.fs.ChecksumFileSystem.rename(ChecksumFileSystem.java:504)
   at 
 org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.renameMapOutputForReduce(LocalContainerLauncher.java:471)
   at 
 org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.runSubtask(LocalContainerLauncher.java:370)
   at 
 org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.runTask(LocalContainerLauncher.java:292)
   at 
 org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler.access$200(LocalContainerLauncher.java:178)
   at 
 org.apache.hadoop.mapred.LocalContainerLauncher$EventHandler$1.run(LocalContainerLauncher.java:221)
   at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)  
   
   at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)   
   
   at java.util.concurrent.FutureTask.run(FutureTask.java:138) 
   
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
   at java.lang.Thread.run(Thread.java:695) 
 {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5890) Support for encrypting Intermediate data and spills in local filesystem

2014-07-09 Thread Chris Douglas (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14056631#comment-14056631
 ] 

Chris Douglas commented on MAPREDUCE-5890:
--

I was thinking {{o.a.h.mapred}}, with other internal classes.

 Support for encrypting Intermediate data and spills in local filesystem
 ---

 Key: MAPREDUCE-5890
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5890
 Project: Hadoop Map/Reduce
  Issue Type: New Feature
  Components: security
Affects Versions: 2.4.0
Reporter: Alejandro Abdelnur
Assignee: Arun Suresh
  Labels: encryption
 Attachments: MAPREDUCE-5890.10.patch, MAPREDUCE-5890.11.patch, 
 MAPREDUCE-5890.12.patch, MAPREDUCE-5890.13.patch, MAPREDUCE-5890.3.patch, 
 MAPREDUCE-5890.4.patch, MAPREDUCE-5890.5.patch, MAPREDUCE-5890.6.patch, 
 MAPREDUCE-5890.7.patch, MAPREDUCE-5890.8.patch, MAPREDUCE-5890.9.patch, 
 org.apache.hadoop.mapred.TestMRIntermediateDataEncryption-output.txt, 
 syslog.tar.gz


 For some sensitive data, encryption while in flight (network) is not 
 sufficient, it is required that while at rest it should be encrypted. 
 HADOOP-10150  HDFS-6134 bring encryption at rest for data in filesystem 
 using Hadoop FileSystem API. MapReduce intermediate data and spills should 
 also be encrypted while at rest.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (MAPREDUCE-5890) Support for encrypting Intermediate data and spills in local filesystem

2014-07-09 Thread Arun Suresh (JIRA)

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

Arun Suresh updated MAPREDUCE-5890:
---

Attachment: MAPREDUCE-5890.14.patch

Ok.. Uploaded patch with {{CryptoUtils}} moved. I still can't make it package 
protected though, since some code that uses it is in {{o.a.h.mapreduce}} and 
some in {{o.a.h.mapred}}

 Support for encrypting Intermediate data and spills in local filesystem
 ---

 Key: MAPREDUCE-5890
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5890
 Project: Hadoop Map/Reduce
  Issue Type: New Feature
  Components: security
Affects Versions: 2.4.0
Reporter: Alejandro Abdelnur
Assignee: Arun Suresh
  Labels: encryption
 Attachments: MAPREDUCE-5890.10.patch, MAPREDUCE-5890.11.patch, 
 MAPREDUCE-5890.12.patch, MAPREDUCE-5890.13.patch, MAPREDUCE-5890.14.patch, 
 MAPREDUCE-5890.3.patch, MAPREDUCE-5890.4.patch, MAPREDUCE-5890.5.patch, 
 MAPREDUCE-5890.6.patch, MAPREDUCE-5890.7.patch, MAPREDUCE-5890.8.patch, 
 MAPREDUCE-5890.9.patch, 
 org.apache.hadoop.mapred.TestMRIntermediateDataEncryption-output.txt, 
 syslog.tar.gz


 For some sensitive data, encryption while in flight (network) is not 
 sufficient, it is required that while at rest it should be encrypted. 
 HADOOP-10150  HDFS-6134 bring encryption at rest for data in filesystem 
 using Hadoop FileSystem API. MapReduce intermediate data and spills should 
 also be encrypted while at rest.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (MAPREDUCE-5965) Hadoop streaming throws error if list of input files is high. Error is: error=7, Argument list too long at if number of input file is high

2014-07-09 Thread Arup Malakar (JIRA)
Arup Malakar created MAPREDUCE-5965:
---

 Summary: Hadoop streaming throws error if list of input files is 
high. Error is: error=7, Argument list too long at if number of input file is 
high
 Key: MAPREDUCE-5965
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5965
 Project: Hadoop Map/Reduce
  Issue Type: Bug
Reporter: Arup Malakar


Hadoop streaming exposes all the key values in job conf as environment 
variables when it forks a process for streaming code to run. Unfortunately the 
variable mapreduce_input_fileinputformat_inputdir contains the list of input 
files, and Linux has a limit on size of environment variables + arguments.
Based on how long the list of files and their full path is this could be pretty 
huge. And given all of these variables are not even used it stops user from 
running hadoop job with large number of files, even though it could be run.

Linux throws E2BIG if the size is greater than certain size which is error code 
7. And java translates that to error=7, Argument list too long. More: 
http://man7.org/linux/man-pages/man2/execve.2.html I suggest skipping variables 
if it is greater than certain length. That way if user code requires the 
environment variable it would fail. It should also introduce a config variable 
to skip long variables, and set it to false by default. That way user has to 
specifically set it to true to invoke this feature.

Here is the exception:
{code}
Error: java.lang.RuntimeException: Error in configuring object at 
org.apache.hadoop.util.ReflectionUtils.setJobConf(ReflectionUtils.java:109) at 
org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:75) at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133) at 
org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:426) at 
org.apache.hadoop.mapred.MapTask.run(MapTask.java:342) at 
org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168) 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:1548)
 at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163) Caused by: 
java.lang.reflect.InvocationTargetException at 
sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) 
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:606) at 
org.apache.hadoop.util.ReflectionUtils.setJobConf(ReflectionUtils.java:106) ... 
9 more Caused by: java.lang.RuntimeException: Error in configuring object at 
org.apache.hadoop.util.ReflectionUtils.setJobConf(ReflectionUtils.java:109) at 
org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:75) at 
org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133) at 
org.apache.hadoop.mapred.MapRunner.configure(MapRunner.java:38) ... 14 more 
Caused by: java.lang.reflect.InvocationTargetException at 
sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) 
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:606) at 
org.apache.hadoop.util.ReflectionUtils.setJobConf(ReflectionUtils.java:106) ... 
17 more Caused by: java.lang.RuntimeException: configuration exception at 
org.apache.hadoop.streaming.PipeMapRed.configure(PipeMapRed.java:222) at 
org.apache.hadoop.streaming.PipeMapper.configure(PipeMapper.java:66) ... 22 
more Caused by: java.io.IOException: Cannot run program 
/data/hadoop/hadoop-yarn/cache/yarn/nm-local-dir/usercache/oo-analytics/appcache/application_1403599726264_13177/container_1403599726264_13177_01_06/./rbenv_runner.sh:
 error=7, Argument list too long at 
java.lang.ProcessBuilder.start(ProcessBuilder.java:1041) at 
org.apache.hadoop.streaming.PipeMapRed.configure(PipeMapRed.java:209) ... 23 
more Caused by: java.io.IOException: error=7, Argument list too long at 
java.lang.UNIXProcess.forkAndExec(Native Method) at 
java.lang.UNIXProcess.init(UNIXProcess.java:135) at 
java.lang.ProcessImpl.start(ProcessImpl.java:130) at 
java.lang.ProcessBuilder.start(ProcessBuilder.java:1022) ... 24 more Container 
killed by the ApplicationMaster. Container killed on request. Exit code is 143 
Container exited with a non-zero exit code 143
{code}

Hive does a similar trick: HIVE-2372 I have a patch for this, will soon submit 
a patch.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (MAPREDUCE-5956) MapReduce AM should not use maxAttempts to determine if this is the last retry

2014-07-09 Thread Zhijie Shen (JIRA)

[ 
https://issues.apache.org/jira/browse/MAPREDUCE-5956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14057067#comment-14057067
 ] 

Zhijie Shen commented on MAPREDUCE-5956:


Talked to Wangda offline. Basically, the current solution is not to compute 
isLastRetry at the beginning of MR AM lifecycle, and keep it false (except some 
corner cases). In this way, no matter the scenario a, b or c. MR AM is going to 
have a retry upon failure, and RM will decide whether the MR job still has a 
chance with preemption considered. Therefore, MR AM will always not lose the 
retry chance it should have, but it trades off the problem that the staging dir 
is not going to be cleaned up at the real last retry, which is going to be 
taken care of by YARN-2261. It's now clear to me, thanks Wangda! And +1 for the 
plan.

 MapReduce AM should not use maxAttempts to determine if this is the last retry
 --

 Key: MAPREDUCE-5956
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5956
 Project: Hadoop Map/Reduce
  Issue Type: Sub-task
  Components: applicationmaster, mrv2
Reporter: Vinod Kumar Vavilapalli
Assignee: Wangda Tan
Priority: Blocker

 Found this while reviewing YARN-2074. The problem is that after YARN-2074, we 
 don't count AM preemption towards AM failures on RM side, but MapReduce AM 
 itself checks the attempt id against the max-attempt count to determine if 
 this is the last attempt.
 {code}
 public void computeIsLastAMRetry() {
   isLastAMRetry = appAttemptID.getAttemptId() = maxAppAttempts;
 }
 {code}
 This causes issues w.r.t deletion of staging directory etc..



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (MAPREDUCE-5790) Default map hprof profile options do not work

2014-07-09 Thread Gera Shegalov (JIRA)

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

Gera Shegalov updated MAPREDUCE-5790:
-

Attachment: MAPREDUCE-5790.v02.patch

[~kasha], yes let us iterate on my JIRA's faster :) 

v02 with test parametrization to address [~vinodkv]'s comment.

 Default map hprof profile options do not work
 -

 Key: MAPREDUCE-5790
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5790
 Project: Hadoop Map/Reduce
  Issue Type: Bug
Affects Versions: 2.3.0
 Environment: java version 1.6.0_31
 Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
 Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
Reporter: Andrew Wang
Assignee: Gera Shegalov
Priority: Blocker
 Attachments: MAPREDUCE-5790.v01.patch, MAPREDUCE-5790.v02.patch


 I have an MR job doing the following:
 {code}
 Job job = Job.getInstance(conf);
 // Enable profiling
 job.setProfileEnabled(true);
 job.setProfileTaskRange(true, 0);
 job.setProfileTaskRange(false, 0);
 {code}
 When I run this job, some of my map tasks fail with this error message:
 {noformat}
 org.apache.hadoop.util.Shell$ExitCodeException: 
 /data/5/yarn/nm/usercache/hdfs/appcache/application_1394482121761_0012/container_1394482121761_0012_01_41/launch_container.sh:
  line 32: $JAVA_HOME/bin/java -Djava.net.preferIPv4Stack=true 
 -Dhadoop.metrics.log.level=WARN   -Xmx825955249 -Djava.io.tmpdir=$PWD/tmp 
 -Dlog4j.configuration=container-log4j.properties 
 -Dyarn.app.container.log.dir=/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41
  -Dyarn.app.container.log.filesize=0 -Dhadoop.root.logger=INFO,CLA 
 ${mapreduce.task.profile.params} org.apache.hadoop.mapred.YarnChild 
 10.20.212.12 43135 attempt_1394482121761_0012_r_00_0 41 
 1/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41/stdout
  
 2/var/log/hadoop-yarn/container/application_1394482121761_0012/container_1394482121761_0012_01_41/stderr
  : bad substitution
 {noformat}
 It looks like ${mapreduce.task.profile.params} is not getting subbed in 
 correctly.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (MAPREDUCE-5956) MapReduce AM should not use maxAttempts to determine if this is the last retry

2014-07-09 Thread Wangda Tan (JIRA)

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

Wangda Tan updated MAPREDUCE-5956:
--

Attachment: MR-5956.patch
MR-5956.patch

Attached a fix for this, please kindly review.
[~vinodkv], [~zjshen].

 MapReduce AM should not use maxAttempts to determine if this is the last retry
 --

 Key: MAPREDUCE-5956
 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5956
 Project: Hadoop Map/Reduce
  Issue Type: Sub-task
  Components: applicationmaster, mrv2
Reporter: Vinod Kumar Vavilapalli
Assignee: Wangda Tan
Priority: Blocker
 Attachments: MR-5956.patch, MR-5956.patch


 Found this while reviewing YARN-2074. The problem is that after YARN-2074, we 
 don't count AM preemption towards AM failures on RM side, but MapReduce AM 
 itself checks the attempt id against the max-attempt count to determine if 
 this is the last attempt.
 {code}
 public void computeIsLastAMRetry() {
   isLastAMRetry = appAttemptID.getAttemptId() = maxAppAttempts;
 }
 {code}
 This causes issues w.r.t deletion of staging directory etc..



--
This message was sent by Atlassian JIRA
(v6.2#6252)