You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by pb...@apache.org on 2016/12/13 13:20:02 UTC

[31/48] oozie git commit: Fixing remaining issues, removing TODOs and FIXMEs

Fixing remaining issues, removing TODOs and FIXMEs

Change-Id: Ieb762713fee2281cde5a7826bc03eafe16de64b1


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/739d53a4
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/739d53a4
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/739d53a4

Branch: refs/heads/oya
Commit: 739d53a47c101d9d779382177eaa19a9725a0e46
Parents: 8d60f7f
Author: Peter Bacsko <pb...@cloudera.com>
Authored: Fri Nov 25 11:02:44 2016 +0100
Committer: Peter Bacsko <pb...@cloudera.com>
Committed: Fri Nov 25 11:02:44 2016 +0100

----------------------------------------------------------------------
 .../action/hadoop/DistcpActionExecutor.java     |  5 ++
 .../oozie/action/hadoop/HiveActionExecutor.java |  5 ++
 .../oozie/action/hadoop/JavaActionExecutor.java | 53 +++++------
 .../action/hadoop/MapReduceActionExecutor.java  |  5 ++
 .../oozie/action/hadoop/PigActionExecutor.java  |  5 ++
 .../action/hadoop/SparkActionExecutor.java      | 10 +++
 .../action/hadoop/SqoopActionExecutor.java      |  4 +
 .../action/hadoop/AMRMCallBackHandler.java      |  2 +-
 .../apache/oozie/action/hadoop/LauncherAM.java  | 93 +++++++++-----------
 .../oozie/action/hadoop/TestLauncherAM.java     | 33 ++++---
 10 files changed, 122 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
index e000a08..fb26399 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/DistcpActionExecutor.java
@@ -107,6 +107,11 @@ public class DistcpActionExecutor extends JavaActionExecutor{
     }
 
     @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
+
+    @Override
     protected String getLauncherMain(Configuration launcherConf, Element actionXml) {
         return launcherConf.get(LauncherMapper.CONF_OOZIE_ACTION_MAIN_CLASS, CONF_OOZIE_DISTCP_ACTION_MAIN_CLASS);
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/HiveActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/HiveActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/HiveActionExecutor.java
index 82a955c..a2d4ca0 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/HiveActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/HiveActionExecutor.java
@@ -133,6 +133,11 @@ public class HiveActionExecutor extends ScriptLanguageActionExecutor {
     }
 
     @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
+
+    @Override
     protected JobConf loadHadoopDefaultResources(Context context, Element actionXml) {
         boolean loadDefaultResources = ConfigurationService
                 .getBoolean(HadoopAccessorService.ACTION_CONFS_LOAD_DEFAULT_RESOURCES);

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
index 56226ee..b8c870c 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
@@ -29,6 +29,7 @@ import java.nio.ByteBuffer;
 import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -37,7 +38,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
-import java.util.regex.Pattern;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -144,7 +144,6 @@ public class JavaActionExecutor extends ActionExecutor {
     protected static final String HADOOP_USER = "user.name";
 
     protected XLog LOG = XLog.getLog(getClass());
-    private static final Pattern heapPattern = Pattern.compile("-Xmx(([0-9]+)[mMgG])");
     private static final String JAVA_TMP_DIR_SETTINGS = "-Djava.io.tmpdir=";
 
     public XConfiguration workflowConf = null;
@@ -914,7 +913,6 @@ public class JavaActionExecutor extends ActionExecutor {
     }
 
     public void submitLauncher(FileSystem actionFs, final Context context, WorkflowAction action) throws ActionExecutorException {
-        JobClient jobClient = null;
         boolean exception = false;
         YarnClient yarnClient = null;
         try {
@@ -1054,20 +1052,6 @@ public class JavaActionExecutor extends ActionExecutor {
             if (yarnClient != null) {
                 Closeables.closeQuietly(yarnClient);
             }
- 
-            if (jobClient != null) {
-                try {
-                    jobClient.close();
-                }
-                catch (Exception e) {
-                    if (exception) {
-                        LOG.error("JobClient error: ", e);
-                    }
-                    else {
-                        throw convertException(e);
-                    }
-                }
-            }
         }
     }
 
@@ -1107,23 +1091,17 @@ public class JavaActionExecutor extends ActionExecutor {
         ClasspathUtils.setupClasspath(env, launcherJobConf);
 
         // FIXME: move this to specific places where it's actually needed - keeping it here for now
-        ClasspathUtils.addMapReduceToClasspath(env, launcherJobConf);
-
-        // FIXME: Pyspark fix
-        // FIXME: Do we want to support mapred.child.env?
-        env.put("SPARK_HOME", ".");
+        if (needToAddMRJars()) {
+            ClasspathUtils.addMapReduceToClasspath(env, launcherJobConf);
+        }
 
-        amContainer.setEnvironment(env);
+        addActionSpecificEnvVars(env);
+        amContainer.setEnvironment(Collections.unmodifiableMap(env));
 
         // Set the command
         List<String> vargs = new ArrayList<String>(6);
         vargs.add(MRApps.crossPlatformifyMREnv(launcherJobConf, ApplicationConstants.Environment.JAVA_HOME)
                 + "/bin/java");
-        // TODO: OYA: remove attach debugger to AM; useful for debugging
-//                    vargs.add("-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5005");
-
-        // FIXME: decide what to do with this method call - signature keeps changing
-        // MRApps.addLog4jSystemProperties("INFO", 1024 * 1024, 0, vargs, null);
 
         vargs.add("-Dlog4j.configuration=container-log4j.properties");
         vargs.add("-Dlog4j.debug=true");
@@ -1143,8 +1121,7 @@ public class JavaActionExecutor extends ActionExecutor {
         }
 
         List<String> vargsFinal = ImmutableList.of(mergedCommand.toString());
-        LOG.debug("Command to launch container for ApplicationMaster is : "
-                + mergedCommand);
+        LOG.debug("Command to launch container for ApplicationMaster is: {0}", mergedCommand);
         amContainer.setCommands(vargsFinal);
         appContext.setAMContainerSpec(amContainer);
 
@@ -1381,6 +1358,22 @@ public class JavaActionExecutor extends ActionExecutor {
         return action.getExternalId();
     }
 
+    /**
+     * If returns true, it means that we have to add Hadoop MR jars to the classpath. Subclasses should override this method if necessary.
+     *
+     */
+    protected boolean needToAddMRJars() {
+        return false;
+    }
+
+    /**
+     * Adds action-specific environment variables. Default implementation is no-op. Subclasses should override this method if necessary.
+     *
+     */
+    protected void addActionSpecificEnvVars(Map<String, String> env) {
+        // nop
+    }
+
     @Override
     public void check(Context context, WorkflowAction action) throws ActionExecutorException {
         boolean fallback = false;

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/MapReduceActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/MapReduceActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/MapReduceActionExecutor.java
index 11d1787..51a9c15 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/MapReduceActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/MapReduceActionExecutor.java
@@ -339,6 +339,11 @@ public class MapReduceActionExecutor extends JavaActionExecutor {
     }
 
     @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
+
+    @Override
     public void check(Context context, WorkflowAction action) throws ActionExecutorException {
         Map<String, String> actionData = Collections.emptyMap();
         JobConf jobConf = null;

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/PigActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/PigActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/PigActionExecutor.java
index cbb7436..347b034 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/PigActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/PigActionExecutor.java
@@ -124,6 +124,11 @@ public class PigActionExecutor extends ScriptLanguageActionExecutor {
     }
 
     @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
+
+    @Override
     protected JobConf loadHadoopDefaultResources(Context context, Element actionXml) {
         boolean loadDefaultResources = ConfigurationService
                 .getBoolean(HadoopAccessorService.ACTION_CONFS_LOAD_DEFAULT_RESOURCES);

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
index 32dadf0..c373758 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/SparkActionExecutor.java
@@ -153,6 +153,16 @@ public class SparkActionExecutor extends JavaActionExecutor {
     }
 
     @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
+
+    @Override
+    protected void addActionSpecificEnvVars(Map<String, String> env) {
+        env.put("SPARK_HOME", ".");
+    }
+
+    @Override
     protected String getLauncherMain(Configuration launcherConf, Element actionXml) {
         return launcherConf.get(LauncherMapper.CONF_OOZIE_ACTION_MAIN_CLASS, SPARK_MAIN_CLASS_NAME);
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/core/src/main/java/org/apache/oozie/action/hadoop/SqoopActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/SqoopActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/SqoopActionExecutor.java
index c6266df..54a860e 100644
--- a/core/src/main/java/org/apache/oozie/action/hadoop/SqoopActionExecutor.java
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/SqoopActionExecutor.java
@@ -225,6 +225,10 @@ public class SqoopActionExecutor extends JavaActionExecutor {
         }
     }
 
+    @Override
+    protected boolean needToAddMRJars() {
+        return true;
+    }
 
     /**
      * Return the sharelib name for the action.

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/AMRMCallBackHandler.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/AMRMCallBackHandler.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/AMRMCallBackHandler.java
index 63213e6..e6c9d04 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/AMRMCallBackHandler.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/AMRMCallBackHandler.java
@@ -53,7 +53,7 @@ public class AMRMCallBackHandler implements AMRMClientAsync.CallbackHandler {
 
     @Override
     public float getProgress() {
-        return 0.5f;    //TODO: OYA: maybe some action types can report better progress?
+        return 0.5f;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
index 881fa72..a8d1e09 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
@@ -24,7 +24,7 @@ import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.security.Permission;
-import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 import java.text.MessageFormat;
 import java.util.HashMap;
 import java.util.Map;
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 
 public class LauncherAM {
@@ -116,6 +117,8 @@ public class LauncherAM {
         Preconditions.checkArgument(!submitterUser.isEmpty(), "Submitter user is undefined");
         System.out.println("Submitter user is: " + submitterUser);
 
+        // We don't need remote/proxy user if the current login user is the workflow submitter
+        // Otherwise we have to create a remote user
         if (UserGroupInformation.getLoginUser().getShortUserName().equals(submitterUser)) {
             System.out.println("Using login user for UGI");
             ugi = UserGroupInformation.getLoginUser();
@@ -160,31 +163,13 @@ public class LauncherAM {
                 errorHolder.setErrorCause(ex);
                 throw ex;
             }
-
-            registerWithRM();
-
             actionDir = new Path(launcherJobConf.get(OOZIE_ACTION_DIR_PATH));
 
-            try {
-                System.out.println("\nStarting the execution of prepare actions");
-                executePrepare(ugi);
-                System.out.println("Completed the execution of prepare actions successfully");
-            } catch (Exception ex) {
-                errorHolder.setErrorMessage("Prepare execution in the Launcher AM has failed");
-                errorHolder.setErrorCause(ex);
-                throw ex;
-            }
-
+            registerWithRM();
+            executePrepare(ugi, errorHolder);
             final String[] mainArgs = getMainArguments(launcherJobConf);
-
-            // TODO: OYA: should we allow turning this off?
-            // TODO: OYA: what should default be?
-            if (launcherJobConf.getBoolean("oozie.launcher.print.debug.info", true)) {
-                printDebugInfo();
-            }
-
+            printDebugInfo();
             setupMainConfiguration();
-
             launcerExecutedProperly = runActionMain(mainArgs, errorHolder, ugi);
 
             if (launcerExecutedProperly) {
@@ -323,11 +308,12 @@ public class LauncherAM {
     }
 
     // Method to execute the prepare actions
-    private void executePrepare(UserGroupInformation ugi) throws Exception {
-        Exception e = ugi.doAs(new PrivilegedAction<Exception>() {
-            @Override
-            public Exception run() {
-                try {
+    private void executePrepare(UserGroupInformation ugi, ErrorHolder errorHolder) throws Exception {
+        try {
+            System.out.println("\nStarting the execution of prepare actions");
+            ugi.doAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
                     String prepareXML = launcherJobConf.get(ACTION_PREPARE_XML);
                     if (prepareXML != null) {
                         if (prepareXML.length() != 0) {
@@ -338,16 +324,14 @@ public class LauncherAM {
                             System.out.println("There are no prepare actions to execute.");
                         }
                     }
-                } catch (Exception e) {
-                    e.printStackTrace();
-                    return e;
+                    return null;
                 }
-                return null;
-            }
-        });
-
-        if (e != null) {
-            throw e;
+            });
+            System.out.println("Completed the execution of prepare actions successfully");
+        } catch (Exception ex) {
+            errorHolder.setErrorMessage("Prepare execution in the Launcher AM has failed");
+            errorHolder.setErrorCause(ex);
+            throw ex;
         }
     }
 
@@ -366,20 +350,21 @@ public class LauncherAM {
         System.setProperty("oozie.job.launch.time", String.valueOf(System.currentTimeMillis()));
     }
 
-    private boolean runActionMain(final String[] mainArgs, final ErrorHolder eHolder, UserGroupInformation ugi) {
+    private boolean runActionMain(final String[] mainArgs, final ErrorHolder eHolder, UserGroupInformation ugi) throws Exception {
         // using AtomicBoolean because we want to modify it inside run()
         final AtomicBoolean actionMainExecutedProperly = new AtomicBoolean(false);
 
-        ugi.doAs(new PrivilegedAction<Void>() {
+        ugi.doAs(new PrivilegedExceptionAction<Void>() {
             @Override
-            public Void run() {
+            public Void run() throws Exception {
                 try {
                     setRecoveryId();
-                    Class<?> klass = launcherJobConf.getClass(CONF_OOZIE_ACTION_MAIN_CLASS, Object.class);
+                    Class<?> klass = launcherJobConf.getClass(CONF_OOZIE_ACTION_MAIN_CLASS, null);
+                    Preconditions.checkNotNull(klass, "Launcher class should not be null");
                     System.out.println("Launcher class: " + klass.toString());
                     Method mainMethod = klass.getMethod("main", String[].class);
                     // Enable LauncherSecurityManager to catch System.exit calls
-                    launcherSecurityManager.set();
+                    launcherSecurityManager.enable();
                     mainMethod.invoke(null, (Object) mainArgs);
 
                     System.out.println();
@@ -430,7 +415,7 @@ public class LauncherAM {
                     eHolder.setErrorCause(t);
                 } finally {
                     // Disable LauncherSecurityManager
-                    launcherSecurityManager.unset();
+                    launcherSecurityManager.disable();
                 }
 
                 return null;
@@ -497,7 +482,11 @@ public class LauncherAM {
 
     private void updateActionDataWithFailure(ErrorHolder eHolder, Map<String, String> actionData) {
         if (eHolder.getErrorCause() != null && eHolder.getErrorCause().getMessage() != null) {
-            eHolder.setErrorMessage(eHolder.getErrorMessage() + ", " + eHolder.getErrorCause().getMessage());
+            if (Objects.equal(eHolder.getErrorMessage(), eHolder.getErrorCause().getMessage())) {
+                eHolder.setErrorMessage(eHolder.getErrorMessage());
+            } else {
+                eHolder.setErrorMessage(eHolder.getErrorMessage() + ", " + eHolder.getErrorCause().getMessage());
+            }
         }
 
         Properties errorProps = new Properties();
@@ -553,27 +542,27 @@ public class LauncherAM {
     public static class LauncherSecurityManager extends SecurityManager {
         private boolean exitInvoked;
         private int exitCode;
-        private SecurityManager securityManager;
+        private SecurityManager originalSecurityManager;
 
         public LauncherSecurityManager() {
             exitInvoked = false;
             exitCode = 0;
-            securityManager = System.getSecurityManager();
+            originalSecurityManager = System.getSecurityManager();
         }
 
         @Override
         public void checkPermission(Permission perm, Object context) {
-            if (securityManager != null) {
+            if (originalSecurityManager != null) {
                 // check everything with the original SecurityManager
-                securityManager.checkPermission(perm, context);
+                originalSecurityManager.checkPermission(perm, context);
             }
         }
 
         @Override
         public void checkPermission(Permission perm) {
-            if (securityManager != null) {
+            if (originalSecurityManager != null) {
                 // check everything with the original SecurityManager
-                securityManager.checkPermission(perm);
+                originalSecurityManager.checkPermission(perm);
             }
         }
 
@@ -592,15 +581,15 @@ public class LauncherAM {
             return exitCode;
         }
 
-        public void set() {
+        public void enable() {
             if (System.getSecurityManager() != this) {
                 System.setSecurityManager(this);
             }
         }
 
-        public void unset() {
+        public void disable() {
             if (System.getSecurityManager() == this) {
-                System.setSecurityManager(securityManager);
+                System.setSecurityManager(originalSecurityManager);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/739d53a4/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherAM.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherAM.java b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherAM.java
index 052673d..777cfdd 100644
--- a/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherAM.java
+++ b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestLauncherAM.java
@@ -56,7 +56,7 @@ import static org.mockito.Mockito.verifyZeroInteractions;
 import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
-import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 import java.util.Properties;
 
@@ -137,7 +137,7 @@ public class TestLauncherAM {
     private ExpectedFailureDetails failureDetails = new ExpectedFailureDetails();
 
     @Before
-    public void setup() throws IOException {
+    public void setup() throws Exception {
         configureMocksForHappyPath();
         launcherJobConfig.set(LauncherMapper.OOZIE_ACTION_RECOVERY_ID, "1");
         instantiateLauncher();
@@ -185,6 +185,20 @@ public class TestLauncherAM {
     }
 
     @Test
+    public void testLauncherClassNotDefined() throws Exception {
+        launcherJobConfig.unset(LauncherAM.CONF_OOZIE_ACTION_MAIN_CLASS);
+
+        executeLauncher();
+
+        failureDetails.expectedExceptionMessage("Launcher class should not be null")
+            .expectedErrorCode(EXIT_CODE_0)
+            .expectedErrorReason("Launcher class should not be null")
+            .withStackTrace();
+
+        assertFailedExecution();
+    }
+
+    @Test
     public void testMainIsSuccessfullyInvokedAndAsyncErrorReceived() throws Exception {
         ErrorHolder errorHolder = new ErrorHolder();
         errorHolder.setErrorCode(6);
@@ -394,7 +408,7 @@ public class TestLauncherAM {
 
         failureDetails.expectedExceptionMessage("IO error")
             .expectedErrorCode(EXIT_CODE_0)
-            .expectedErrorReason("IO error, IO error")
+            .expectedErrorReason("IO error")
             .withStackTrace();
 
         assertFailedExecution();
@@ -419,7 +433,7 @@ public class TestLauncherAM {
 
         failureDetails.expectedExceptionMessage("IO error")
             .expectedErrorCode(EXIT_CODE_0)
-            .expectedErrorReason("IO error, IO error")
+            .expectedErrorReason("IO error")
             .withStackTrace();
 
         verify(hdfsOperationsMock).readFileContents(any(Path.class), eq(launcherJobConfig));
@@ -452,8 +466,8 @@ public class TestLauncherAM {
                 containerId);
     }
 
-    @SuppressWarnings("unchecked")
-    private void configureMocksForHappyPath() throws IOException {
+     @SuppressWarnings("unchecked")
+    private void configureMocksForHappyPath() throws Exception {
         launcherJobConfig.set(LauncherAM.OOZIE_ACTION_DIR_PATH, "dummy");
         launcherJobConfig.set(LauncherAM.OOZIE_JOB_ID, "dummy");
         launcherJobConfig.set(LauncherAM.OOZIE_ACTION_ID, "dummy");
@@ -461,12 +475,11 @@ public class TestLauncherAM {
 
         given(localFsOperationsMock.readLauncherConf()).willReturn(launcherJobConfig);
         given(localFsOperationsMock.fileExists(any(File.class))).willReturn(true);
-
         willReturn(amRmAsyncClientMock).given(amRMClientAsyncFactoryMock).createAMRMClientAsync(anyInt());
-        given(ugiMock.doAs(any(PrivilegedAction.class))).willAnswer(new Answer<Object>() {
+        given(ugiMock.doAs(any(PrivilegedExceptionAction.class))).willAnswer(new Answer<Object>() {
             @Override
             public Object answer(InvocationOnMock invocation) throws Throwable {
-                PrivilegedAction<?> action = (PrivilegedAction<?>) invocation.getArguments()[0];
+                PrivilegedExceptionAction<?> action = (PrivilegedExceptionAction<?>) invocation.getArguments()[0];
                 return action.run();
             }
         });
@@ -498,7 +511,7 @@ public class TestLauncherAM {
         verify(amRmAsyncClientMock).registerApplicationMaster(anyString(), anyInt(), anyString());
         verify(amRmAsyncClientMock).unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, EMPTY_STRING, EMPTY_STRING);
         verify(amRmAsyncClientMock).stop();
-        verify(ugiMock, times(2)).doAs(any(PrivilegedAction.class)); // prepare & action main
+        verify(ugiMock, times(2)).doAs(any(PrivilegedExceptionAction.class)); // prepare & action main
         verify(hdfsOperationsMock).uploadActionDataToHDFS(any(Configuration.class), any(Path.class), any(Map.class));
         verify(launcherCallbackNotifierFactoryMock).createCallbackNotifier(any(Configuration.class));
         verify(launcherCallbackNotifierMock).notifyURL(actionResult);