You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by an...@apache.org on 2017/10/27 13:07:22 UTC

oozie git commit: OZIE-2896 Ensure compatibility for existing LauncherMapper settings (andras.piros)

Repository: oozie
Updated Branches:
  refs/heads/master 01a7b893d -> 8b4f53c75


OZIE-2896 Ensure compatibility for existing LauncherMapper settings (andras.piros)


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

Branch: refs/heads/master
Commit: 8b4f53c75ade69d0997fd4d292d7753d1f5721ce
Parents: 01a7b89
Author: Andras Piros <an...@cloudera.com>
Authored: Fri Oct 27 15:05:21 2017 +0200
Committer: Andras Piros <an...@cloudera.com>
Committed: Fri Oct 27 15:05:21 2017 +0200

----------------------------------------------------------------------
 .../oozie/action/hadoop/JavaActionExecutor.java | 120 ++++---
 .../hadoop/LauncherConfigurationInjector.java   | 360 +++++++++++++++++++
 .../oozie/service/ConfigurationService.java     |   5 +
 core/src/main/resources/oozie-default.xml       | 112 +++++-
 .../action/hadoop/TestJavaActionExecutor.java   |  36 +-
 .../TestLauncherConfigurationInjector.java      | 240 +++++++++++++
 release-log.txt                                 |   1 +
 .../apache/oozie/action/hadoop/LauncherAM.java  |   2 +
 8 files changed, 819 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/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 76d0daa..6a25e5c 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
@@ -19,6 +19,7 @@
 package org.apache.oozie.action.hadoop;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.io.Closeables;
 import com.google.common.primitives.Ints;
@@ -107,33 +108,12 @@ import org.jdom.Element;
 import org.jdom.JDOMException;
 import org.jdom.Namespace;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.ConnectException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Objects;
-import java.util.Objects;
-import java.util.Properties;
 import java.util.Properties;
 import java.util.Set;
-import java.util.Set;
+
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.io.Closeables;
 
 
 public class JavaActionExecutor extends ActionExecutor {
@@ -150,6 +130,7 @@ public class JavaActionExecutor extends ActionExecutor {
     public static final String DEFAULT_LAUNCHER_MEMORY_MB = "oozie.launcher.default.memory.mb";
     public static final String DEFAULT_LAUNCHER_PRIORITY = "oozie.launcher.default.priority";
     public static final String DEFAULT_LAUNCHER_QUEUE = "oozie.launcher.default.queue";
+    public static final String DEFAULT_LAUNCHER_MAX_ATTEMPS = "oozie.launcher.default.max.attempts";
 
     public static final String MAX_EXTERNAL_STATS_SIZE = "oozie.external.stats.max.size";
     public static final String ACL_VIEW_JOB = "mapreduce.job.acl-view-job";
@@ -302,20 +283,6 @@ public class JavaActionExecutor extends ActionExecutor {
         return createBaseHadoopConf(context, actionXml);
     }
 
-    private static void injectLauncherProperties(Configuration srcConf, Configuration launcherConf) {
-        for (Map.Entry<String, String> entry : srcConf) {
-            if (entry.getKey().startsWith("oozie.launcher.")) {
-                String name = entry.getKey().substring("oozie.launcher.".length());
-                String value = entry.getValue();
-                // setting original KEY
-                launcherConf.set(entry.getKey(), value);
-                // setting un-prefixed key (to allow Hadoop job config
-                // for the launcher job
-                launcherConf.set(name, value);
-            }
-        }
-    }
-
     Configuration setupLauncherConf(Configuration conf, Element actionXml, Path appPath, Context context)
             throws ActionExecutorException {
         try {
@@ -324,7 +291,9 @@ public class JavaActionExecutor extends ActionExecutor {
             // Inject action defaults for launcher
             HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
             XConfiguration actionDefaultConf = has.createActionDefaultConf(conf.get(HADOOP_YARN_RM), getType());
-            injectLauncherProperties(actionDefaultConf, launcherConf);
+
+            new LauncherConfigurationInjector(actionDefaultConf).inject(launcherConf);
+
             // Inject <job-xml> and <configuration> for launcher
             try {
                 parseJobXmlAndConfiguration(context, actionXml, appPath, launcherConf, true);
@@ -405,7 +374,7 @@ public class JavaActionExecutor extends ActionExecutor {
             }
             checkForDisallowedProps(jobXmlConf, "job-xml");
             if (isLauncher) {
-                injectLauncherProperties(jobXmlConf, conf);
+                new LauncherConfigurationInjector(jobXmlConf).inject(conf);
             } else {
                 XConfiguration.copy(jobXmlConf, conf);
             }
@@ -416,7 +385,7 @@ public class JavaActionExecutor extends ActionExecutor {
             XConfiguration inlineConf = new XConfiguration(new StringReader(strConf));
             checkForDisallowedProps(inlineConf, "inline configuration");
             if (isLauncher) {
-                injectLauncherProperties(inlineConf, conf);
+                new LauncherConfigurationInjector(inlineConf).inject(conf);
             } else {
                 XConfiguration.copy(inlineConf, conf);
             }
@@ -1110,9 +1079,13 @@ public class JavaActionExecutor extends ActionExecutor {
         credentialsProperties.put(type, new CredentialsProperties(type, type));
     }
 
-    private ApplicationSubmissionContext createAppSubmissionContext(ApplicationId appId, Configuration launcherJobConf,
-                                                                    Context context, Configuration actionConf, String actionName,
-                                                                    Credentials credentials, Element actionXml)
+    private ApplicationSubmissionContext createAppSubmissionContext(final ApplicationId appId,
+                                                                    final Configuration launcherJobConf,
+                                                                    final Context actionContext,
+                                                                    final Configuration actionConf,
+                                                                    final String actionName,
+                                                                    final Credentials credentials,
+                                                                    final Element actionXml)
             throws IOException, HadoopAccessorException, URISyntaxException {
 
         ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
@@ -1121,12 +1094,13 @@ public class JavaActionExecutor extends ActionExecutor {
         setPriority(launcherJobConf, appContext);
         setQueue(launcherJobConf, appContext);
         appContext.setApplicationId(appId);
-        setApplicationName(context, actionName, appContext);
+        setApplicationName(actionContext, actionName, appContext);
         appContext.setApplicationType("Oozie Launcher");
+        setMaxAttempts(launcherJobConf, appContext);
 
         ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class);
 
-        final String user = context.getWorkflow().getUser();
+        final String user = actionContext.getWorkflow().getUser();
         // Set the resources to localize
         Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
         ClientDistributedCacheManager.determineTimestampsAndCacheVisibilities(launcherJobConf);
@@ -1135,16 +1109,16 @@ public class JavaActionExecutor extends ActionExecutor {
         HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
         launcherJobConf.set(LauncherAM.OOZIE_SUBMITTER_USER, user);
         LocalResource launcherJobConfLR = has.createLocalResourceForConfigurationFile(LauncherAM.LAUNCHER_JOB_CONF_XML, user,
-                launcherJobConf, context.getAppFileSystem().getUri(), context.getActionDir());
+                launcherJobConf, actionContext.getAppFileSystem().getUri(), actionContext.getActionDir());
         localResources.put(LauncherAM.LAUNCHER_JOB_CONF_XML, launcherJobConfLR);
         LocalResource actionConfLR = has.createLocalResourceForConfigurationFile(LauncherAM.ACTION_CONF_XML, user, actionConf,
-                context.getAppFileSystem().getUri(), context.getActionDir());
+                actionContext.getAppFileSystem().getUri(), actionContext.getActionDir());
         localResources.put(LauncherAM.ACTION_CONF_XML, actionConfLR);
         amContainer.setLocalResources(localResources);
 
         setEnvironmentVariables(launcherJobConf, amContainer);
 
-        List<String> vargs = createCommand(context);
+        List<String> vargs = createCommand(launcherJobConf, actionContext);
         setJavaOpts(launcherJobConf, actionXml, vargs);
         vargs.add(LauncherAM.class.getCanonicalName());
         vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR + ApplicationConstants.STDOUT);
@@ -1171,8 +1145,44 @@ public class JavaActionExecutor extends ActionExecutor {
         return appContext;
     }
 
-    private List<String> createCommand(Context context) {
-        List<String> vargs = new ArrayList<String>(6);
+    private void setMaxAttempts(Configuration launcherJobConf, ApplicationSubmissionContext appContext) {
+        int launcherMaxAttempts;
+        final int defaultLauncherMaxAttempts = ConfigurationService.getInt(DEFAULT_LAUNCHER_MAX_ATTEMPS);
+        if (launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS) != null) {
+            try {
+                launcherMaxAttempts = launcherJobConf.getInt(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS,
+                        defaultLauncherMaxAttempts);
+            } catch (final NumberFormatException ignored) {
+                launcherMaxAttempts = defaultLauncherMaxAttempts;
+            }
+        } else {
+            LOG.warn("Invalid configuration value [{0}] defined for launcher max attempts count, using default [{1}].",
+                    launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS),
+                    defaultLauncherMaxAttempts);
+            launcherMaxAttempts = defaultLauncherMaxAttempts;
+        }
+
+        LOG.trace("Reading from configuration max attempts count of the Launcher AM. [launcherMaxAttempts={0}]",
+                launcherMaxAttempts);
+
+        if (launcherMaxAttempts > 0) {
+            LOG.trace("Setting max attempts of the Launcher AM. [launcherMaxAttempts={0}]", launcherMaxAttempts);
+            appContext.setMaxAppAttempts(launcherMaxAttempts);
+        }
+        else {
+            LOG.warn("Not setting max attempts of the Launcher AM, value is invalid. [launcherMaxAttempts={0}]",
+                    launcherMaxAttempts);
+        }
+    }
+
+    private List<String> createCommand(final Configuration launcherJobConf, final Context context) {
+        final List<String> vargs = new ArrayList<String>(6);
+
+        String launcherLogLevel = launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY);
+        if (Strings.isNullOrEmpty(launcherLogLevel)) {
+            launcherLogLevel = "INFO";
+        }
+
         vargs.add(Apps.crossPlatformify(ApplicationConstants.Environment.JAVA_HOME.toString())
                 + "/bin/java");
 
@@ -1180,9 +1190,10 @@ public class JavaActionExecutor extends ActionExecutor {
         vargs.add("-Dlog4j.debug=true");
         vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR);
         vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_SIZE + "=" + 1024 * 1024);
-        vargs.add("-Dhadoop.root.logger=INFO,CLA");
+        vargs.add("-Dhadoop.root.logger=" + launcherLogLevel + ",CLA");
         vargs.add("-Dhadoop.root.logfile=" + TaskLog.LogName.SYSLOG);
         vargs.add("-Dsubmitter.user=" + context.getWorkflow().getUser());
+
         return vargs;
     }
 
@@ -1256,13 +1267,14 @@ public class JavaActionExecutor extends ActionExecutor {
     }
 
     private void setQueue(Configuration launcherJobConf, ApplicationSubmissionContext appContext) {
-        String queue;
+        String launcherQueueName;
         if (launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY) != null) {
-            queue = launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY);
+            launcherQueueName = launcherJobConf.get(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY);
         } else {
-            queue = Preconditions.checkNotNull(ConfigurationService.get(DEFAULT_LAUNCHER_QUEUE), "Default queue is undefined");
+            launcherQueueName = Preconditions.checkNotNull(
+                    ConfigurationService.get(DEFAULT_LAUNCHER_QUEUE), "Default launcherQueueName is undefined");
         }
-        appContext.setQueue(queue);
+        appContext.setQueue(launcherQueueName);
     }
 
     private void setPriority(Configuration launcherJobConf, ApplicationSubmissionContext appContext) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/core/src/main/java/org/apache/oozie/action/hadoop/LauncherConfigurationInjector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/LauncherConfigurationInjector.java b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherConfigurationInjector.java
new file mode 100644
index 0000000..c2c0563
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/LauncherConfigurationInjector.java
@@ -0,0 +1,360 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.action.hadoop;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.oozie.service.ConfigurationService;
+import org.apache.oozie.util.XLog;
+
+import java.util.Collection;
+import java.util.Map;
+
+/**
+ * Launcher AM's {@link Configuration} is injected using {@code <launcher>} element specific, YARN, MapReduce v2, and
+ * MapReduce v1 specific override and prepend properties employing following rules:
+ * <ul>
+ *     <li>if {@code oozie.launcher.override} config property is {@code false}, original behavior will run: each and every parameter
+ *     from {@link #sourceConfiguration} that begins with {@link #OOZIE_LAUNCHER_PREFIX} will be copied over to the target launcher
+ *     {@code Configuration} with key without the prefix, and original value. Example input:
+ *     {@code <launcher><vcores>1</vcores></launcher>}. Example output: {@code oozie.launcher.vcores=1} and {@code vcores=1}</li>
+ *
+ *     <li>or else, when properties are defined inside global or application specific {@code <launcher>} tags, those take
+ *     precedence over the override parameters defined in the application section with {@link #OVERRIDE_PREFIX}. An example: inside
+ *     {@code <launcher>} there was {@code oozie.launcher.vcores} defined, and also one of the parameters defined as values in
+ *     {@code oozie-default.xml#oozie.launcher.override.vcores}, e.g. {@code yarn.app.mapreduce.am.resource.cpu-vcores} is present,
+ *     the original value coming from &gt;launcher&lt; tag will be used and copied over as {@code oozie.launcher.vcores}. Example
+ *     input: {@code <launcher><vcores>1</vcores></launcher>}, and {@code yarn.app.mapreduce.am.resource.cpu-vcores=2} are both set.
+ *     Example output: {@code oozie.launcher.vcores=1}</li>
+ *
+ *     <li>or else, when no properties are defined inside global or application specific {@code <launcher>} tags, there is a
+ *     precedence ordering among the override parameters defined in the application section with {@link #OVERRIDE_PREFIX}. An
+ *     example: {@code oozie.launcher.vcores} is not defined inside {@code <launcher>}, but there are both
+ *     {@code oozie.launcher.override.vcores} properties {@code yarn.app.mapreduce.am.resource.cpu-vcores} and
+ *     {@code mapreduce.map.cpu.vcores} present: the value of {@code yarn.app.mapreduce.am.resource.cpu-vcores} will be used as
+ *     {@code oozie.launcher.vcores} inside target Launcher AM {@code Configuration}. Example input:
+ *     {@code yarn.app.mapreduce.am.resource.cpu-vcores=1} and  {@code mapreduce.map.cpu.vcores=2}. Example output:
+ *     {@code oozie.launcher.vcores=1}</li>
+ *
+ *     <li>or else, no properties are defined inside global or application specific {@code <launcher>} tags, nor there are any
+ *     override parameters. In that case, {@code oozie.launcher.vcores} inside target Launcher AM {@code Configuration} is not
+ *     filled. Example input: nothing. Example output: nothing</li>
+ *
+ *     <li>when prepending properties (the ones pointing to keys beginning with {@link #PREPEND_PREFIX}) are defined, these are
+ *     prepended to the launcher settings coming from other sources. If no launcher settings of the available
+ *     {@code #PREPEND_PREFIX} are defined, there is nothing prepended. Example input: {@code <launcher><env>B=BB</env></launcher>}
+ *     and {@code oozie.launcher.prepend.env="A=AA"}. Example output: {@code oozie.launcher.env="A=AA B=BB"}</li>
+ * </ul>
+ */
+class LauncherConfigurationInjector {
+    private static final XLog LOG = XLog.getLog(LauncherConfigurationInjector.class);
+
+    private static final String OOZIE_LAUNCHER_PREFIX = "oozie.launcher.";
+    private static final String OVERRIDE_PREFIX = OOZIE_LAUNCHER_PREFIX + "override.";
+    private static final String PREPEND_PREFIX = OOZIE_LAUNCHER_PREFIX + "prepend.";
+
+    /**
+     * All the Launcher AM override property keys as configured in {@code oozie-default.xml} beginning with
+     * {@link #OVERRIDE_PREFIX}. Important to use {@link LinkedHashMultimap}, as we want an unified set of values those insertion /
+     * configuration order is preserved.
+     * <p/>
+     * By the time it's initialized, it will have content like that:
+     * <pre>
+     * {@code
+     * {oozie.launcher.override.priority=
+     *          [mapreduce.job.priority, mapred.job.priority],
+     *  oozie.launcher.override.queue=
+     *          [mapreduce.job.queuename, mapred.job.queue.name],
+     *  oozie.launcher.override.javaopts=
+     *          [yarn.app.mapreduce.am.command-opts, mapreduce.map.java.opts, mapred.child.java.opts],
+     *  oozie.launcher.override.log.level=
+     *          [mapreduce.map.log.level, mapred.map.child.log.level],
+     *  oozie.launcher.override.max.attempts=
+     *          [mapreduce.map.maxattempts, mapred.map.max.attempts],
+     *  oozie.launcher.override.vcores=
+     *          [yarn.app.mapreduce.am.resource.cpu-vcores, mapreduce.map.cpu.vcores],
+     *  oozie.launcher.override.memory.mb=
+     *          [yarn.app.mapreduce.am.resource.mb, mapreduce.map.memory.mb, mapred.job.map.memory.mb],
+     *  oozie.launcher.override.env=
+     *          [yarn.app.mapreduce.am.env, mapreduce.map.env, mapred.child.env]}
+     * }
+     * </pre>
+     */
+    private static final Multimap<String, String> CONFIGURED_OVERRIDE_PROPERTIES = LinkedHashMultimap.create();
+
+    /**
+     * All the Launcher AM prepend property keys as configured in {@code oozie-default.xml} beginning with {@link #PREPEND_PREFIX}.
+     * Important to use {@link LinkedHashMultimap}, as we want an unified set of values those insertion / configuration order is
+     * preserved.
+     * <p/>
+     * By the time it's initialized, it will have content like that:
+     * <pre>
+     * {@code
+     * {oozie.launcher.prepend.javaopts=
+     *      [yarn.app.mapreduce.am.admin-command-opts],
+     *  oozie.launcher.prepend.env=
+     *      [yarn.app.mapreduce.am.admin.user.env]}
+     * }
+     * </pre>
+     */
+    private static final Multimap<String, String> CONFIGURED_PREPEND_PROPERTIES = LinkedHashMultimap.create();
+
+    /**
+     * Matches everything beginning with {@code oozie.launcher.override.}
+     */
+    private static final String OVERRIDE_REGEX = "^oozie\\.launcher\\.override\\.+";
+
+    /**
+     * Matches everything beginning with {@code oozie.launcher.prepend.}
+     */
+    private static final String PREPEND_REGEX = "^oozie\\.launcher\\.prepend\\.+";
+
+    static {
+        fillConfigPropertiesByRegex(OVERRIDE_REGEX, CONFIGURED_OVERRIDE_PROPERTIES);
+        fillConfigPropertiesByRegex(PREPEND_REGEX, CONFIGURED_PREPEND_PROPERTIES);
+    }
+
+    /**
+     * Fill in the target {@code Multimap} based like this: keys are extracted from {@link ConfigurationService} using {@code regex}
+     * as prefix, values are the ones under the same key inside {@code oozie-[default,site].xml}.
+     * <p/>
+     * Depending on what {@code Multimap} we use, the values might keep config / insert order, and filter duplicates. At the end of
+     * this call {@code target} will have all the key / value mappings enlisted in the Oozie configuration files
+     * {@code oozie-[default,site].xml} those keys match {@code regex}.
+     * <p/>
+     * An example:
+     * {@code oozie-default.xml} snippet:
+     * <pre>
+     * {@code
+     *         <property>
+     *             <name>oozie.launcher.override.max.attempts</name>
+     *             <value>mapreduce.map.maxattempts,mapred.map.max.attempts</value>
+     *         </property>
+     * }
+     * </pre>
+     * results in:
+     * <pre>
+     * {@code
+     *  {oozie.launcher.override.max.attempts=
+     *          [mapreduce.map.maxattempts, mapred.map.max.attempts]}
+     * }
+     * </pre>
+     * @param regex regular expression to match the keys, e.g. {@link #OVERRIDE_REGEX}
+     * @param target a {@code Multimap<String, String>} as target, e.g. {@link #CONFIGURED_OVERRIDE_PROPERTIES}
+     */
+    private static void fillConfigPropertiesByRegex(final String regex, final Multimap<String, String> target) {
+        for (final Map.Entry<String, String> overrideEntry : ConfigurationService.getValByRegex(regex).entrySet()) {
+            final String key = overrideEntry.getKey();
+            for (final String value : ConfigurationService.getStrings(key)) {
+                target.put(key, value);
+            }
+        }
+    }
+
+    /**
+     * The source {@code Configuration} provided by the caller
+     */
+    private final Configuration sourceConfiguration;
+
+    /**
+     * Override values that are actually present in {@link #sourceConfiguration}
+     */
+    private final Map<String, String> overridesPresentInSource = Maps.newLinkedHashMap();
+
+    /**
+     * Prepend values that are actually present in {@link #sourceConfiguration}
+     */
+    private final Map<String, String> prependsPresentInSource = Maps.newLinkedHashMap();
+
+    LauncherConfigurationInjector(final Configuration sourceConfiguration) {
+        this.sourceConfiguration = sourceConfiguration;
+    }
+
+    /**
+     * Inject the overridden and prepended values from {@link #sourceConfiguration} into {@code launcherConf} wherever applicable.
+     * @param launcherConf the target {@code Configuration} of the Launcher AM
+     */
+    void inject(final Configuration launcherConf) {
+        LOG.debug("Injecting configuration entries to launcher configuration.");
+
+        copyToLauncherConf(sourceConfiguration, launcherConf);
+
+        LOG.debug("Configuration entries copied to launcher configuration successfully.");
+
+        if (ConfigurationService.getBoolean("oozie.launcher.override")) {
+            LOG.debug("Overriding and prepending launcher configuration.");
+            LOG.trace("Filling override and prepend configuration values.");
+
+            fillOverridesOrPrepends(CONFIGURED_OVERRIDE_PROPERTIES, OVERRIDE_PREFIX, overridesPresentInSource);
+            fillOverridesOrPrepends(CONFIGURED_PREPEND_PROPERTIES, PREPEND_PREFIX, prependsPresentInSource);
+
+            LOG.trace("Override and prepend configuration values filled. " +
+                            "[overridesPresentInSource={0};prependsPresentInSource={1}]",
+                    overridesPresentInSource,
+                    prependsPresentInSource);
+
+            overrideAndPrependLauncherConf(launcherConf);
+
+            LOG.debug("Launcher configuration overridden and prepended.");
+            LOG.trace("Launcher configuration finalized. [launcherConf={0}]", launcherConf);
+        }
+
+        LOG.debug("Configuration entries injected to launcher configuration.");
+    }
+
+    /**
+     * Fill override / prepend state, based on preconfigured values.
+     * <p/>
+     * An example. Having inputs (no launcher tag is present, and only MapReduce v1 specific parameter is there):
+     * <pre>
+     *     {@code
+     *     this.sourceConfiguration={mapred.map.max.attempts=2}
+     *     overrideOrPrependProperties={oozie.launcher.override.max.attempts=[mapreduce.map.maxattempts, mapred.map.max.attempts]}
+     *     prefix="oozie.launcher.override."
+     *     target={}
+     *     }
+     * </pre>
+     * results in:
+     * <pre>
+     *     {@code
+     *     target={oozie.launcher.max.attempts=2}
+     *     }
+     * </pre>
+     * thus, the MapReduce v1 specific parameter is applied to the specific Launcher AM {@code Configuration}.
+     * <p/>
+     * Note that defined {@code oozie.launcher.override.*} properties will have an output substituting / adding an config entry
+     * {@code oozie.launcher.*}, while {@code oozie.launcher.prepend.*} entries will be prepended to existing
+     * {@code oozie.launcher.*} entries (or to new ones if {@code oozie.launcher.override.*} is present also).
+     * @param overrideOrPrependProperties either {@link #CONFIGURED_OVERRIDE_PROPERTIES} or {@link #CONFIGURED_PREPEND_PROPERTIES}
+     * @param prefix either {@link #OVERRIDE_PREFIX} or {@link #PREPEND_PREFIX}
+     * @param target a {@code Map<String, String>} consisting of configured override or prepend values as present in the source
+     */
+    private void fillOverridesOrPrepends(final Multimap<String, String> overrideOrPrependProperties,
+                                         final String prefix,
+                                         final Map<String, String> target) {
+        for (final Map.Entry<String, Collection<String>> overrideProperty : overrideOrPrependProperties.asMap().entrySet()) {
+            final String launcherKey = overrideProperty.getKey().replace(prefix, OOZIE_LAUNCHER_PREFIX);
+            final Collection<String> sourceKeys = overrideProperty.getValue();
+            LOG.trace("Filling launcher override / prepend value. [sourceKeys={0};launcherKey={1}]", sourceKeys, launcherKey);
+
+            fillOverrideOrPrependLauncherValue(sourceKeys, launcherKey, target);
+        }
+    }
+
+    /**
+     * Fill one piece of override / prepend state, based on preconfigured value.
+     * @param sourceKeys
+     * @param launcherKey
+     * @param target
+     */
+    private void fillOverrideOrPrependLauncherValue(final Collection<String> sourceKeys,
+                                                    final String launcherKey,
+                                                    final Map<String, String> target) {
+        final String sourceValue = getFirstSourceValue(sourceKeys);
+        if (!Strings.isNullOrEmpty(sourceValue)) {
+            target.put(launcherKey, sourceValue);
+        }
+    }
+
+    /**
+     * Get the first non-empty value present in {@link #sourceConfiguration} given one of {@code propertyKeys}, order-sensitively.
+     * @param propertyKeys
+     * @return
+     */
+    private String getFirstSourceValue(final Collection<String> propertyKeys) {
+        if (propertyKeys == null) {
+            return null;
+        }
+
+        for (final String propertyKey : propertyKeys) {
+            final String propertyValue = sourceConfiguration.get(propertyKey);
+            if (!Strings.isNullOrEmpty(propertyValue)) {
+                return propertyValue;
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Based on the override / prepend state, modify {@code launcherConfiguration}.
+     * @param launcherConfiguration the target {@code Configuration} of the Launcher AM
+     */
+    private void overrideAndPrependLauncherConf(final Configuration launcherConfiguration) {
+        for (final String overrideKey : overridesPresentInSource.keySet()) {
+            overrideSingleValue(launcherConfiguration, overrideKey);
+        }
+
+        for (final String prependKey : prependsPresentInSource.keySet()) {
+            prependSingleValue(launcherConfiguration, prependKey);
+        }
+    }
+
+    /**
+     * Based on one piece of override state, override a single entry in {@code launcherConfiguration}.
+     * @param launcherConfiguration the target {@code Configuration} of the Launcher AM
+     * @param key the override key
+     */
+    private void overrideSingleValue(final Configuration launcherConfiguration, final String key) {
+        final String originalLauncherValue = launcherConfiguration.get(key);
+        final String overrideValue = overridesPresentInSource.get(key);
+        if (Strings.isNullOrEmpty(originalLauncherValue) && overrideValue != null) {
+            launcherConfiguration.set(key, overrideValue);
+        }
+    }
+
+    /**
+     * Based on one piece of prepend state, prepend a single entry in {@code launcherConfiguration}.
+     * @param launcherConfiguration the target {@code Configuration} of the Launcher AM
+     * @param key the prepend key
+     */
+    private void prependSingleValue(final Configuration launcherConfiguration, final String key) {
+        final String launcherValue = launcherConfiguration.get(key);
+        final String prependValue = prependsPresentInSource.get(key);
+        if (!Strings.isNullOrEmpty(prependValue)) {
+            if (Strings.isNullOrEmpty(launcherValue)) {
+                launcherConfiguration.set(key, prependValue);
+            } else {
+                launcherConfiguration.set(key, prependValue + " " + launcherValue);
+            }
+        }
+    }
+
+    /**
+     * Copy each and every configuration entry between {@code source} and {@code target} beginning with
+     * {@link #OOZIE_LAUNCHER_PREFIX}. Note that both original and un-prefixed keys will be present with the original value.
+     * @param source
+     * @param target
+     */
+    private void copyToLauncherConf(final Configuration source, final Configuration target) {
+        for (final Map.Entry<String, String> entry : source) {
+            if (entry.getKey().startsWith(OOZIE_LAUNCHER_PREFIX)) {
+                final String name = entry.getKey().substring(OOZIE_LAUNCHER_PREFIX.length());
+                final String value = entry.getValue();
+                // setting original KEY
+                target.set(entry.getKey(), value);
+                // setting un-prefixed key to allow Hadoop job config for the launcher job
+                target.set(name, value);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
index a519330..4af022f 100644
--- a/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
+++ b/core/src/main/java/org/apache/oozie/service/ConfigurationService.java
@@ -564,6 +564,7 @@ public class ConfigurationService implements Service, Instrumentable {
     public static long getLong(Configuration conf, String name) {
         return getLong(conf, name, ConfigUtils.LONG_DEFAULT);
     }
+
     public static long getLong(Configuration conf, String name, long defultValue) {
         return conf.getLong(name, defultValue);
     }
@@ -600,4 +601,8 @@ public class ConfigurationService implements Service, Instrumentable {
         return getPassword(conf, name, defaultValue);
     }
 
+    public static Map<String, String> getValByRegex(final String regex) {
+        final Configuration conf = Services.get().getConf();
+        return conf.getValByRegex(regex);
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/core/src/main/resources/oozie-default.xml
----------------------------------------------------------------------
diff --git a/core/src/main/resources/oozie-default.xml b/core/src/main/resources/oozie-default.xml
index 9ba8fd4..b203699 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -3115,4 +3115,114 @@ will be the requeue interval for the actions which are waiting for a long time w
             The default YARN queue where the Launcher AM is placed
         </description>
     </property>
-</configuration>
+
+    <property>
+        <name>oozie.launcher.default.max.attempts</name>
+        <value>2</value>
+        <description>
+            The default YARN maximal attempt count of the Launcher AM
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override</name>
+        <value>true</value>
+        <description>
+            Whether oozie.launcher.override.* and oozie.launcher.prepend.* parameters have to be considered when submitting a YARN
+            LauncherAM. That is, existing MapReduce v1, MapReduce v2, or YARN parameters used in the action configuration should be
+            populated to the Application Master launcher configuration, or not. Generally, first &lt;launcher/&gt; tag specific user
+            settings, then YARN configuration settings, then MapReduce v2, and at last, MapReduce v1 properties are copied to
+            launcher configuration.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.max.attempts</name>
+        <value>mapreduce.map.maxattempts,mapred.map.max.attempts</value>
+        <description>
+            A comma separated list of MapReduce v1 and MapReduce v2 properties to override the max attempts of the MapReduce
+            Application Master. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.memory.mb</name>
+        <value>yarn.app.mapreduce.am.resource.mb,mapreduce.map.memory.mb,mapred.job.map.memory.mb</value>
+        <description>
+            A comma separated list of MapReduce v1, MapReduce v2, and YARN properties to override the memory amount in MB of the
+            MapReduce Application Master. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.vcores</name>
+        <value>yarn.app.mapreduce.am.resource.cpu-vcores,mapreduce.map.cpu.vcores</value>
+        <description>
+            A comma separated list of MapReduce v1, MapReduce v2, and YARN properties to override the CPU vcore count of the
+            MapReduce Application Master. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.log.level</name>
+        <value>mapreduce.map.log.level,mapred.map.child.log.level</value>
+        <description>
+            A comma separated list of MapReduce v1, MapReduce v2, and YARN properties to override the logging level of the MapReduce
+            Application Master. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.javaopts</name>
+        <value>yarn.app.mapreduce.am.command-opts,mapreduce.map.java.opts,mapred.child.java.opts</value>
+        <description>
+            A comma separated list of MapReduce v1, MapReduce v2, and YARN properties to override MapReduce Application Master JVM
+            options. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.prepend.javaopts</name>
+        <value>yarn.app.mapreduce.am.admin-command-opts</value>
+        <description>
+            A comma separated list of YARN properties to prepend to MapReduce Application Master JVM options. The first one that is
+            found will be prepended to the list of JVM options.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.env</name>
+        <value>yarn.app.mapreduce.am.env,mapreduce.map.env,mapred.child.env</value>
+        <description>
+            A comma separated list of MapReduce v1, MapReduce v2, and YARN properties to override MapReduce Application Master
+            environment variable settings. The first one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.prepend.env</name>
+        <value>yarn.app.mapreduce.am.admin.user.env</value>
+        <description>
+            A comma separated list of YARN properties to prepend to MapReduce Application Master environment settings. The first one
+            that is found will be prepended to the list of environment settings.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.priority</name>
+        <value>mapreduce.job.priority,mapred.job.priority</value>
+        <description>
+            A comma separated list of MapReduce v1 and MapReduce v2 to override MapReduce Application Master job priority. The first
+            one that is found will be used.
+        </description>
+    </property>
+
+    <property>
+        <name>oozie.launcher.override.queue</name>
+        <value>mapreduce.job.queuename,mapred.job.queue.name</value>
+        <description>
+            A comma separated list of MapReduce v1 and MapReduce v2 properties to override MapReduce Application Master job queue
+            name. The first one that is found will be used.
+        </description>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
index a7bd357..adb2168 100644
--- a/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
@@ -2180,7 +2180,8 @@ public class TestJavaActionExecutor extends ActionExecutorTestCase {
         assertEquals("AA", conf.get("a"));
         assertEquals("action.barbar", conf.get("oozie.launcher.action.foofoo"));
         assertEquals("action.barbar", conf.get("action.foofoo"));
-        assertEquals(5, conf.size());
+        assertEquals("max-attempts", "1", conf.get("oozie.launcher.max.attempts"));
+        assertEquals(6, conf.size());
 
         conf = new Configuration(false);
         Assert.assertEquals(0, conf.size());
@@ -2189,7 +2190,8 @@ public class TestJavaActionExecutor extends ActionExecutorTestCase {
         assertEquals(getJobTrackerUri(), conf.get(YARN_RESOURCEMANAGER_ADDRESS));
         assertEquals("action.barbar", conf.get("oozie.launcher.action.foofoo"));
         assertEquals("action.barbar", conf.get("action.foofoo"));
-        assertEquals(3, conf.size());
+        assertEquals("max-attempts", "1", conf.get("oozie.launcher.max.attempts"));
+        assertEquals(4, conf.size());
     }
 
     public void testDefaultConfigurationInActionConf() throws Exception {
@@ -2540,6 +2542,36 @@ public class TestJavaActionExecutor extends ActionExecutorTestCase {
         return conf.get(org.apache.hadoop.yarn.conf.YarnConfiguration.RM_SCHEDULER).contains(FairScheduler.class.getName());
     }
 
+    public void testSubmitLauncherConfigurationOverridesLauncherMapperProperties() throws Exception {
+        final String actionXml = "<java>" +
+                "<job-tracker>" + getJobTrackerUri() + "</job-tracker>" +
+                "<name-node>" + getNameNodeUri() + "</name-node>" +
+                "<configuration>" +
+                "  <property><name>oozie.launcher.queue</name><value>default1</value></property>" +
+                "  <property><name>mapreduce.job.queuename</name><value>default2</value></property>" +
+                "</configuration>" +
+                "<main-class>" + LauncherMainTester.class.getName() + "</main-class>" +
+                "</java>";
+        final Context context = createContext(actionXml, null);
+
+        submitAction(context);
+
+        final ApplicationId appId = ConverterUtils.toApplicationId(context.getAction().getExternalId());
+        final Configuration conf = getHadoopAccessorService().createConfiguration(getJobTrackerUri());
+
+        final String queue = getHadoopAccessorService().createYarnClient(getTestUser(), conf)
+                .getApplicationReport(appId).getQueue();
+        assertEquals("queue name", "default1", queue);
+
+        final ActionExecutor ae = new JavaActionExecutor();
+        ae.check(context, context.getAction());
+        assertEquals("FAILED/KILLED", context.getAction().getExternalStatus());
+        assertNull(context.getAction().getData());
+
+        ae.end(context, context.getAction());
+        assertEquals(WorkflowAction.Status.ERROR, context.getAction().getStatus());
+    }
+
     private HadoopAccessorService getHadoopAccessorService() {
         return Services.get().get(HadoopAccessorService.class);
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/core/src/test/java/org/apache/oozie/action/hadoop/TestLauncherConfigurationInjector.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/action/hadoop/TestLauncherConfigurationInjector.java b/core/src/test/java/org/apache/oozie/action/hadoop/TestLauncherConfigurationInjector.java
new file mode 100644
index 0000000..4c5b0f9
--- /dev/null
+++ b/core/src/test/java/org/apache/oozie/action/hadoop/TestLauncherConfigurationInjector.java
@@ -0,0 +1,240 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.action.hadoop;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.oozie.service.ConfigurationService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.test.XTestCase;
+
+public class TestLauncherConfigurationInjector extends XTestCase {
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        new Services().init();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        if (Services.get() != null) {
+            Services.get().destroy();
+        }
+        super.tearDown();
+    }
+
+    public void testOverrideSwitchedOffSourceCopiedToTargetWithTwoDifferentKeys() {
+        ConfigurationService.setBoolean("oozie.launcher.override", false);
+
+        final Configuration sourceConf = SourceConfigurationFactory.createOverridingAndLauncherEntries();
+        final Configuration launcherConf = newConfigurationWithoutDefaults();
+
+        new LauncherConfigurationInjector(sourceConf).inject(launcherConf);
+
+        assertLauncherAndDefaultEntries(launcherConf);
+    }
+
+    private static Configuration newConfigurationWithoutDefaults() {
+        return new Configuration(false);
+    }
+
+    private static Configuration newConfigurationWithDefaults() {
+        return new Configuration(true);
+    }
+
+    private void assertLauncherAndDefaultEntries(final Configuration launcherConf) {
+        assertEquals("launcher max attempts", 1, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS, -1));
+        assertEquals("max attempts", 1, launcherConf.getInt("max.attempts", -1));
+        assertEquals("launcher memory mb", 512, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_MEMORY_MB_PROPERTY, -1));
+        assertEquals("memory mb", 512, launcherConf.getInt("memory.mb", -1));
+        assertEquals("launcher vcores", 4, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_VCORES_PROPERTY, -1));
+        assertEquals("vcores", 4, launcherConf.getInt("vcores", -1));
+        assertEquals("launcher log level", "DEBUG", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY));
+        assertEquals("log level", "DEBUG", launcherConf.get("log.level"));
+        assertTrue("launcher java opts",
+                launcherConf.get(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY).contains("-XX:MaxPermSize=128m"));
+        assertTrue("java opts", launcherConf.get("javaopts").contains("-XX:MaxPermSize=128m"));
+        assertTrue("launcher env", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY).contains("PATH=/path2:$PATH"));
+        assertTrue("env", launcherConf.get("env").contains("PATH=/path2:$PATH"));
+        assertEquals("launcher priority", 2, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_PRIORITY_PROPERTY, -1));
+        assertEquals("priority", 2, launcherConf.getInt("priority", -1));
+        assertTrue("launcher queue", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY).contains("default2"));
+        assertTrue("queue", launcherConf.get("queue").contains("default2"));
+    }
+
+    public void testLauncherConfigSourceCopiedToTarget() {
+        final Configuration sourceConf = SourceConfigurationFactory.createLauncherEntries();
+        final Configuration launcherConf = newConfigurationWithoutDefaults();
+
+        new LauncherConfigurationInjector(sourceConf).inject(launcherConf);
+
+        assertLauncherAndDefaultEntries(launcherConf);
+    }
+
+    public void testOverridingConfigCopiedToTarget() {
+        final Configuration sourceConf = SourceConfigurationFactory.createOverridingEntries();
+        final Configuration launcherConf = newConfigurationWithoutDefaults();
+
+        new LauncherConfigurationInjector(sourceConf).inject(launcherConf);
+
+        assertHigherRankingOverridingAndNoDefaultEntries(launcherConf);
+    }
+
+    private void assertHigherRankingOverridingAndNoDefaultEntries(final Configuration launcherConf) {
+        assertEquals("launcher max-attempts", 4, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS, -1));
+        assertEquals("max-attempts", -1, launcherConf.getInt("max.attempts", -1));
+        assertEquals("launcher memory mb", 2048, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_MEMORY_MB_PROPERTY, -1));
+        assertEquals("memory mb", -1, launcherConf.getInt("memory.mb", -1));
+        assertEquals("launcher vcores", 16, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_VCORES_PROPERTY, -1));
+        assertEquals("vcores", -1, launcherConf.getInt("vcores", -1));
+        assertEquals("launcher log level", "TRACE", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY));
+        assertNull("log level", launcherConf.get("log.level"));
+        assertTrue("launcher java opts",
+                launcherConf.get(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY).contains("-XX:MaxPermSize=256m"));
+        assertNull("java opts", launcherConf.get("javaopts"));
+        assertTrue("launcher env", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY).contains("PATH=/path1:$PATH"));
+        assertNull("env", launcherConf.get("env"));
+        assertEquals("launcher priority", 1, launcherConf.getInt(LauncherAM.OOZIE_LAUNCHER_PRIORITY_PROPERTY, -1));
+        assertEquals("priority", -1, launcherConf.getInt("priority", -1));
+        assertTrue("launcher queue", launcherConf.get(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY).contains("default1"));
+        assertNull("queue", launcherConf.get("queue"));
+    }
+
+    public void testMultipleOverrideOrder() {
+        final Configuration sourceConf = SourceConfigurationFactory.createMultipleOverridingEntries();
+        final Configuration launcherConf = newConfigurationWithoutDefaults();
+
+        new LauncherConfigurationInjector(sourceConf).inject(launcherConf);
+
+        assertHigherRankingOverridingAndNoDefaultEntries(launcherConf);
+    }
+
+    public void testPrependLauncherConfigSourcePrependedToTarget() {
+        final Configuration sourceConf = SourceConfigurationFactory.createPrependingAndLauncherEntries();
+        final Configuration launcherConf = newConfigurationWithoutDefaults();
+
+        new LauncherConfigurationInjector(sourceConf).inject(launcherConf);
+
+        assertPrependedLauncherAndDefaultEntries(launcherConf);
+    }
+
+    private void assertPrependedLauncherAndDefaultEntries(final Configuration launcherConf) {
+        assertTrue("launcher java opts",
+                launcherConf.get(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY).contains("-XX:+UseParNewGC -XX:MaxPermSize=128m"));
+        assertTrue("java opts", launcherConf.get("javaopts").contains("-XX:MaxPermSize=128m"));
+        assertFalse("java opts", launcherConf.get("javaopts").contains("-XX:+UseParNewGC"));
+        assertTrue("launcher env",
+                launcherConf.get(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY).contains("ENV=env:$ENV PATH=/path2:$PATH"));
+        assertTrue("env", launcherConf.get("env").contains("PATH=/path2:$PATH"));
+        assertFalse("env", launcherConf.get("env").contains("ENV=env:$ENV"));
+    }
+
+    private static class SourceConfigurationFactory {
+
+        private static Configuration createOverridingAndLauncherEntries() {
+            final Configuration sourceConf = newConfigurationWithDefaults();
+
+            sourceConf.set("mapreduce.map.maxattempts", "4");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS, "1");
+            sourceConf.set("yarn.app.mapreduce.am.resource.mb", "2048");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_MEMORY_MB_PROPERTY, "512");
+            sourceConf.set("yarn.app.mapreduce.am.resource.cpu-vcores", "16");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_VCORES_PROPERTY, "4");
+            sourceConf.set("mapreduce.map.log.level", "TRACE");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY, "DEBUG");
+            sourceConf.set("yarn.app.mapreduce.am.command-opts", "-XX:MaxPermSize=256m");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY, "-XX:MaxPermSize=128m");
+            sourceConf.set("yarn.app.mapreduce.am.env", "PATH=/path1:$PATH");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY, "PATH=/path2:$PATH");
+            sourceConf.set("mapreduce.job.priority", "1");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_PRIORITY_PROPERTY, "2");
+            sourceConf.set("mapreduce.job.queuename", "default1");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY, "default2");
+
+            return sourceConf;
+        }
+
+        private static Configuration createLauncherEntries() {
+            final Configuration sourceConf = newConfigurationWithDefaults();
+
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_MAX_ATTEMPTS, "1");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_MEMORY_MB_PROPERTY, "512");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_VCORES_PROPERTY, "4");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY, "DEBUG");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY, "-XX:MaxPermSize=128m");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY, "PATH=/path2:$PATH");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_PRIORITY_PROPERTY, "2");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_QUEUE_PROPERTY, "default2");
+
+            return sourceConf;
+        }
+
+        private static Configuration createOverridingEntries() {
+            final Configuration sourceConf = newConfigurationWithDefaults();
+
+            sourceConf.set("mapreduce.map.maxattempts", "4");
+            sourceConf.set("yarn.app.mapreduce.am.resource.mb", "2048");
+            sourceConf.set("yarn.app.mapreduce.am.resource.cpu-vcores", "16");
+            sourceConf.set("mapreduce.map.log.level", "TRACE");
+            sourceConf.set("yarn.app.mapreduce.am.command-opts", "-XX:MaxPermSize=256m");
+            sourceConf.set("yarn.app.mapreduce.am.env", "PATH=/path1:$PATH");
+            sourceConf.set("mapreduce.job.priority", "1");
+            sourceConf.set("mapreduce.job.queuename", "default1");
+
+            return sourceConf;
+        }
+
+        private static Configuration createMultipleOverridingEntries() {
+            final Configuration sourceConf = newConfigurationWithDefaults();
+
+            sourceConf.set("mapred.map.max.attempts", "5");
+            sourceConf.set("mapreduce.map.maxattempts", "4");
+            sourceConf.set("mapred.job.map.memory.mb", "2050");
+            sourceConf.set("mapreduce.map.memory.mb", "2049");
+            sourceConf.set("yarn.app.mapreduce.am.resource.mb", "2048");
+            sourceConf.set("mapreduce.map.cpu.vcores", "17");
+            sourceConf.set("yarn.app.mapreduce.am.resource.cpu-vcores", "16");
+            sourceConf.set("mapred.map.child.log.level", "DEBUG");
+            sourceConf.set("mapreduce.map.log.level", "TRACE");
+            sourceConf.set("mapred.child.java.opts", "-XX:MaxPermSize=258m");
+            sourceConf.set("mapreduce.map.java.opts", "-XX:MaxPermSize=257m");
+            sourceConf.set("yarn.app.mapreduce.am.command-opts", "-XX:MaxPermSize=256m");
+            sourceConf.set("mapred.child.env", "PATH=/path3:$PATH");
+            sourceConf.set("mapreduce.map.env", "PATH=/path2:$PATH");
+            sourceConf.set("yarn.app.mapreduce.am.env", "PATH=/path1:$PATH");
+            sourceConf.set("mapred.job.priority", "2");
+            sourceConf.set("mapreduce.job.priority", "1");
+            sourceConf.set("mapred.job.queue.name", "default2");
+            sourceConf.set("mapreduce.job.queuename", "default1");
+
+            return sourceConf;
+        }
+
+        private static Configuration createPrependingAndLauncherEntries() {
+            final Configuration sourceConf = newConfigurationWithDefaults();
+
+            sourceConf.set("yarn.app.mapreduce.am.admin-command-opts", "-XX:+UseParNewGC");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_JAVAOPTS_PROPERTY, "-XX:MaxPermSize=128m");
+            sourceConf.set("yarn.app.mapreduce.am.admin.user.env", "ENV=env:$ENV");
+            sourceConf.set(LauncherAM.OOZIE_LAUNCHER_ENV_PROPERTY, "PATH=/path2:$PATH");
+
+            return sourceConf;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 8cc66ef..ba7cf86 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.0.0 release (trunk - unreleased)
 
+OOZIE-2896 Ensure compatibility for existing LauncherMapper settings (andras.piros)
 OOZIE-3095 Job submit command is not logged in audit log (puru)
 OOZIE-3096 Pre-commit job is timing out regularly (gezapeti)
 OOZIE-2406 Completely rewrite GraphGenerator code (andras.piros via gezapeti)

http://git-wip-us.apache.org/repos/asf/oozie/blob/8b4f53c7/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 ee1a32a..9cacd2e 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
@@ -64,6 +64,8 @@ public class LauncherAM {
     public static final String OOZIE_LAUNCHER_JAVAOPTS_PROPERTY = "oozie.launcher.javaopts";
     public static final String OOZIE_LAUNCHER_ENV_PROPERTY = "oozie.launcher.env";
     public static final String OOZIE_LAUNCHER_SHARELIB_PROPERTY = "oozie.launcher.sharelib";
+    public static final String OOZIE_LAUNCHER_LOG_LEVEL_PROPERTY = "oozie.launcher.log.level";
+    public static final String OOZIE_LAUNCHER_MAX_ATTEMPTS = "oozie.launcher.max.attempts";
 
     public static final String JAVA_CLASS_PATH = "java.class.path";
     public static final String OOZIE_ACTION_ID = "oozie.action.id";