[ https://issues.apache.org/jira/browse/OOZIE-2536?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15299031#comment-15299031 ]
Satish Subhashrao Saley commented on OOZIE-2536: ------------------------------------------------ My initial guess is that sometimes {{propagation-conf.xml}} is getting deleted and AsyncDispatcher event handler is unable to find it. Following is analysis so far - 1. [OOZIE-2129|https://issues.apache.org/jira/browse/OOZIE-2129] added {{propagation-conf.xml}} to configuration in Mapper phase of LauncherMapper. {code} Configuration.addDefaultResource(PROPAGATION_CONF_XML); {code} 2. An Event handler tries to relocalize (delete unnecessory) files in current directory {code} private void relocalize() { File[] curLocalFiles = curDir.listFiles(); for (int j = 0; j < curLocalFiles.length; ++j) { if (!localizedFiles.contains(curLocalFiles[j])) { // found one that wasn't there before: delete it boolean deleted = false; try { if (curFC != null) { // this is recursive, unlike File delete(): deleted = curFC.delete(new Path(curLocalFiles[j].getName()),true); } } catch (IOException e) { deleted = false; } if (!deleted) { LOG.warn("Unable to delete unexpected local file/dir " + curLocalFiles[j].getName() + ": insufficient permissions?"); } } } {code} If we follow the code from [here | https://github.com/apache/hadoop/blob/branch-2.7.2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java#L230], it would be as -- {code} runTask(launchEv, localMapFiles); -> runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,(numReduceTasks > 0), localMapFiles); -> relocalize(); {code} I suspect that sometimes the hash set named {{localizedFiles}} does not contain the {{propagation-conf.xml}}. Reason for that would be - 3. {{localizedFiles}} gets populated in [constructor of LocalContainerLauncher|https://github.com/apache/hadoop/blob/branch-2.7.2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java#L89-L115] {code} // Save list of files/dirs that are supposed to be present so can delete // any extras created by one task before starting subsequent task. Note // that there's no protection against deleted or renamed localization; // users who do that get what they deserve (and will have to disable // uberization in order to run correctly). File[] curLocalFiles = curDir.listFiles(); localizedFiles = new HashSet<File>(curLocalFiles.length); for (int j = 0; j < curLocalFiles.length; ++j) { localizedFiles.add(curLocalFiles[j]); } {code} In ApplicationMaster, [serviceInit() method | https://github.com/apache/hadoop/blob/branch-2.7.2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java#L438], we are instantiating {{ContainerLauncherRouter}} which contains {{LocalContainerLauncher}}. In the comment, it has been mentioned that {code} /** * By the time life-cycle of this router starts, job-init would have already * happened. */ private final class ContainerLauncherRouter extends AbstractService {code} It makes me think that sometimes, {{propagation-conf.xml}} gets added to the current working after the {{localizedFiles}} gets populated. If this is true, then in {{relocalize()}} method, {{propagation-conf.xml}} would get deleted. And when AsyncDispatcher Event Handler, is in [process of committing the job|https://github.com/apache/hadoop/blob/branch-2.7.2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java#L1700-L1718], it fails because it does not find {{propagation-conf.xml}} which was part of conf. > Shell action got stuck for 6 hours even after Exit status is 0 > -------------------------------------------------------------- > > Key: OOZIE-2536 > URL: https://issues.apache.org/jira/browse/OOZIE-2536 > Project: Oozie > Issue Type: Bug > Reporter: Satish Subhashrao Saley > Assignee: Satish Subhashrao Saley > > In out environment, we faced an issue where uberized Shell action was getting > stuck even though the shell action got completed with status 0. Please refer > the attached syslog and stdout if launcher job, here I point out partially > stdout : > {quote} > >>> Invoking Shell command line now >> > Stdoutput myshellType=qmyshellUpdate > Exit code of the Shell command 0 > <<< Invocation of Shell command completed <<< > <<< Invocation of Main class completed <<< > {quote} > syslog > {quote} > 2016-05-23 11:15:52,587 WARN [uber-SubtaskRunner] > org.apache.hadoop.mapred.LocalContainerLauncher: Unable to delete unexpected > local file/dir .action.xml.crc: insufficient permissions? > 2016-05-23 11:15:52,588 FATAL [AsyncDispatcher event handler] > org.apache.hadoop.conf.Configuration: error parsing conf propagation-conf.xml > java.io.FileNotFoundException: > /tmp/yarn-local/usercache/saley/appcache/application_1234_123/container_e01_1234_123_01_000001/propagation-conf.xml > (No such file or directory) > at java.io.FileInputStream.open0(Native Method) > at java.io.FileInputStream.open(FileInputStream.java:195) > at java.io.FileInputStream.<init>(FileInputStream.java:138) > at java.io.FileInputStream.<init>(FileInputStream.java:93) > at > sun.net.www.protocol.file.FileURLConnection.connect(FileURLConnection.java:90) > at > sun.net.www.protocol.file.FileURLConnection.getInputStream(FileURLConnection.java:188) > at java.net.URL.openStream(URL.java:1038) > at org.apache.hadoop.conf.Configuration.parse(Configuration.java:2468) > at > org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2539) > at > org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2492) > at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2405) > at org.apache.hadoop.conf.Configuration.get(Configuration.java:981) > at > org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1031) > at org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1251) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.getMemoryRequired(TaskAttemptImpl.java:568) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.updateMillisCounters(TaskAttemptImpl.java:1295) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.createJobCounterUpdateEventTASucceeded(TaskAttemptImpl.java:1323) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.access$3500(TaskAttemptImpl.java:147) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$SucceededTransition.transition(TaskAttemptImpl.java:1710) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$SucceededTransition.transition(TaskAttemptImpl.java:1701) > at > org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362) > at > org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302) > at > org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46) > at > org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:1085) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:146) > at > org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:1394) > at > org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:1386) > at > org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:184) > at > org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:110) > at java.lang.Thread.run(Thread.java:745) > 2016-05-23 11:15:52,590 FATAL [AsyncDispatcher event handler] > org.apache.hadoop.yarn.event.AsyncDispatcher: Error in dispatcher thread > java.lang.RuntimeException: java.io.FileNotFoundException: > /grid/5/tmp/yarn-local/usercache/saley/appcache/application_1234_123/container_e01_1234_123_01_000001/propagation-conf.xml > (No such file or directory) > at > org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2639) > at > org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2492) > at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2405) > at org.apache.hadoop.conf.Configuration.get(Configuration.java:981) > at > org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1031) > at org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1251) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.getMemoryRequired(TaskAttemptImpl.java:568) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.updateMillisCounters(TaskAttemptImpl.java:1295) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.createJobCounterUpdateEventTASucceeded(TaskAttemptImpl.java:1323) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.access$3500(TaskAttemptImpl.java:147) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$SucceededTransition.transition(TaskAttemptImpl.java:1710) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl$SucceededTransition.transition(TaskAttemptImpl.java:1701) > at > org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362) > at > org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302) > at > org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46) > at > org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:1085) > at > org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl.handle(TaskAttemptImpl.java:146) > at > org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:1394) > at > org.apache.hadoop.mapreduce.v2.app.MRAppMaster$TaskAttemptEventDispatcher.handle(MRAppMaster.java:1386) > at > org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:184) > at > org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:110) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.io.FileNotFoundException: > /tmp/yarn-local/usercache/saley/appcache/application_1234_123/container_e01_1234_123_01_000001/propagation-conf.xml > (No such file or directory) > at java.io.FileInputStream.open0(Native Method) > at java.io.FileInputStream.open(FileInputStream.java:195) > at java.io.FileInputStream.<init>(FileInputStream.java:138) > at java.io.FileInputStream.<init>(FileInputStream.java:93) > at > sun.net.www.protocol.file.FileURLConnection.connect(FileURLConnection.java:90) > at > sun.net.www.protocol.file.FileURLConnection.getInputStream(FileURLConnection.java:188) > at java.net.URL.openStream(URL.java:1038) > at org.apache.hadoop.conf.Configuration.parse(Configuration.java:2468) > at > org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2539) > ... 22 more > 2016-05-23 11:15:52,591 INFO [AsyncDispatcher ShutDown handler] > org.apache.hadoop.yarn.event.AsyncDispatcher: Exiting, bbye.. > 2016-05-23 11:15:52,591 ERROR [AsyncDispatcher ShutDown handler] > org.apache.hadoop.yarn.YarnUncaughtExceptionHandler: Thread > Thread[AsyncDispatcher ShutDown handler,5,main] threw an Exception. > java.lang.SecurityException: Intercepted System.exit(-1) > at > org.apache.oozie.action.hadoop.LauncherSecurityManager.checkExit(LauncherMapper.java:637) > at java.lang.Runtime.exit(Runtime.java:107) > at java.lang.System.exit(System.java:971) > at > org.apache.hadoop.yarn.event.AsyncDispatcher$2.run(AsyncDispatcher.java:294) > at java.lang.Thread.run(Thread.java:745) > 2016-05-23 11:16:44,589 WARN [LeaseRenewer:sa...@namenode.com:8020] > org.apache.hadoop.conf.Configuration: job.xml:an attempt to override final > parameter: hadoop.tmp.dir; Ignoring. > 2016-05-23 11:20:53,677 INFO [Socket Reader #2 for port 50500] > SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for saley > (auth:SIMPLE) > {quote} -- This message was sent by Atlassian JIRA (v6.3.4#6332)