You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by tu...@apache.org on 2012/03/29 21:32:06 UTC

svn commit: r1307071 - in /incubator/oozie/trunk: ./ core/src/main/java/org/apache/oozie/action/hadoop/ core/src/main/java/org/apache/oozie/service/ core/src/main/resources/ core/src/test/java/org/apache/oozie/action/email/ core/src/test/java/org/apach...

Author: tucu
Date: Thu Mar 29 19:32:05 2012
New Revision: 1307071

URL: http://svn.apache.org/viewvc?rev=1307071&view=rev
Log:
OOZIE-792 add default action configs per cluster (tucu)

Added:
    incubator/oozie/trunk/core/src/test/resources/test-action-config.xml
Modified:
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
    incubator/oozie/trunk/core/src/main/resources/oozie-default.xml
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestDistCpActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestFsActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestHiveActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionError.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestShellActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestSqoopActionExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
    incubator/oozie/trunk/release-log.txt

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -193,6 +193,20 @@ public class JavaActionExecutor extends 
         return conf;
     }
 
+    private 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 {
@@ -203,18 +217,11 @@ public class JavaActionExecutor extends 
                 XConfiguration inlineConf = new XConfiguration(new StringReader(strConf));
 
                 XConfiguration launcherConf = new XConfiguration();
-                for (Map.Entry<String, String> entry : inlineConf) {
-                    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);
-                    }
-                }
-                checkForDisallowedProps(launcherConf, "inline launcher configuration");
+                HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
+                XConfiguration actionDefaultConf = has.createActionDefaultConf(conf.get(HADOOP_JOB_TRACKER), getType());
+                injectLauncherProperties(actionDefaultConf, launcherConf);
+                injectLauncherProperties(inlineConf, launcherConf);
+                checkForDisallowedProps(launcherConf, "launcher configuration");
                 XConfiguration.copy(launcherConf, conf);
             }
             return conf;
@@ -246,6 +253,9 @@ public class JavaActionExecutor extends 
     Configuration setupActionConf(Configuration actionConf, Context context, Element actionXml, Path appPath)
             throws ActionExecutorException {
         try {
+            HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
+            XConfiguration actionDefaults = has.createActionDefaultConf(actionConf.get(HADOOP_JOB_TRACKER), getType());
+            XConfiguration.injectDefaults(actionDefaults, actionConf);
             Namespace ns = actionXml.getNamespace();
             Element e = actionXml.getChild("job-xml", ns);
             if (e != null) {

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java Thu Mar 29 19:32:05 2012
@@ -58,6 +58,7 @@ public class HadoopAccessorService imple
     public static final String JOB_TRACKER_WHITELIST = CONF_PREFIX + "jobTracker.whitelist";
     public static final String NAME_NODE_WHITELIST = CONF_PREFIX + "nameNode.whitelist";
     public static final String HADOOP_CONFS = CONF_PREFIX + "hadoop.configurations";
+    public static final String ACTION_CONFS = CONF_PREFIX + "action.configurations";
     public static final String KERBEROS_AUTH_ENABLED = CONF_PREFIX + "kerberos.enabled";
     public static final String KERBEROS_KEYTAB = CONF_PREFIX + "keytab.file";
     public static final String KERBEROS_PRINCIPAL = CONF_PREFIX + "kerberos.principal";
@@ -67,6 +68,8 @@ public class HadoopAccessorService imple
     private Set<String> jobTrackerWhitelist = new HashSet<String>();
     private Set<String> nameNodeWhitelist = new HashSet<String>();
     private Map<String, Configuration> hadoopConfigs = new HashMap<String, Configuration>();
+    private Map<String, File> actionConfigDirs = new HashMap<String, File>();
+    private Map<String, Map<String, XConfiguration>> actionConfigs = new HashMap<String, Map<String, XConfiguration>>();
 
     private ConcurrentMap<String, UserGroupInformation> userUgiMap;
 
@@ -111,6 +114,7 @@ public class HadoopAccessorService imple
         userUgiMap = new ConcurrentHashMap<String, UserGroupInformation>();
 
         loadHadoopConfigs(conf);
+        preLoadActionConfigs(conf);
     }
 
     private void kerberosInit(Configuration serviceConf) throws ServiceException {
@@ -157,35 +161,57 @@ public class HadoopAccessorService imple
         return hadoopConf;
     }
 
-    private void loadHadoopConfigs(Configuration serviceConf) throws ServiceException {
-        try {
-            File configDir = new File(ConfigurationService.getConfigurationDirectory());
-            String[] confDefs = serviceConf.getStrings(HADOOP_CONFS, "*=hadoop-conf");
-            for (String confDef : confDefs) {
-                if (confDef.trim().length() > 0) {
-                    String[] parts = confDef.split("=");
-                    if (parts.length == 2) {
-                        String hostPort = parts[0];
-                        String confDir = parts[1];
-                        File dir = new File(confDir);
-                        if (!dir.isAbsolute()) {
-                            dir = new File(configDir, confDir);
-                        }
-                        if (dir.exists()) {
-                            Configuration conf = loadHadoopConf(dir);
-                            hadoopConfigs.put(hostPort.toLowerCase(), conf);
-                        }
-                        else {
-                            throw new ServiceException(ErrorCode.E0100, getClass().getName(),
-                                                       "could not find hadoop configuration directory: " +
-                                                       dir.getAbsolutePath());
-                        }
+    private Map<String, File> parseConfigDirs(String[] confDefs, String type) throws ServiceException, IOException {
+        Map<String, File> map = new HashMap<String, File>();
+        File configDir = new File(ConfigurationService.getConfigurationDirectory());
+        for (String confDef : confDefs) {
+            if (confDef.trim().length() > 0) {
+                String[] parts = confDef.split("=");
+                if (parts.length == 2) {
+                    String hostPort = parts[0];
+                    String confDir = parts[1];
+                    File dir = new File(confDir);
+                    if (!dir.isAbsolute()) {
+                        dir = new File(configDir, confDir);
+                    }
+                    if (dir.exists()) {
+                        map.put(hostPort.toLowerCase(), dir);
                     }
                     else {
                         throw new ServiceException(ErrorCode.E0100, getClass().getName(),
-                                                   "Incorrect hadoop configuration definition: " + confDef);
+                                                   "could not find " + type + " configuration directory: " +
+                                                   dir.getAbsolutePath());
                     }
                 }
+                else {
+                    throw new ServiceException(ErrorCode.E0100, getClass().getName(),
+                                               "Incorrect " + type + " configuration definition: " + confDef);
+                }
+            }
+        }
+        return map;
+    }
+
+    private void loadHadoopConfigs(Configuration serviceConf) throws ServiceException {
+        try {
+            Map<String, File> map = parseConfigDirs(serviceConf.getStrings(HADOOP_CONFS, "*=hadoop-conf"), "hadoop");
+            for (Map.Entry<String, File> entry : map.entrySet()) {
+                hadoopConfigs.put(entry.getKey(), loadHadoopConf(entry.getValue()));
+            }
+        }
+        catch (ServiceException ex) {
+            throw ex;
+        }
+        catch (Exception ex) {
+            throw new ServiceException(ErrorCode.E0100, getClass().getName(), ex.getMessage(), ex);
+        }
+    }
+
+    private void preLoadActionConfigs(Configuration serviceConf) throws ServiceException {
+        try {
+            actionConfigDirs = parseConfigDirs(serviceConf.getStrings(ACTION_CONFS, "*=hadoop-conf"), "action");
+            for (String hostport : actionConfigDirs.keySet()) {
+                actionConfigs.put(hostport, new ConcurrentHashMap<String, XConfiguration>());
             }
         }
         catch (ServiceException ex) {
@@ -213,6 +239,16 @@ public class HadoopAccessorService imple
         return ugi;
     }
 
+    /**
+     * Creates a JobConf using the site configuration for the specified hostname:port.
+     * <p/>
+     * If the specified hostname:port is not defined it falls back to the '*' site
+     * configuration if available. If the '*' site configuration is not available,
+     * the JobConf has all Hadoop defaults.
+     *
+     * @param hostPort hostname:port to lookup Hadoop site configuration.
+     * @return a JobConf with the corresponding site configuration for hostPort.
+     */
     public JobConf createJobConf(String hostPort) {
         JobConf jobConf = new JobConf();
         XConfiguration.copy(getConfiguration(hostPort), jobConf);
@@ -220,6 +256,52 @@ public class HadoopAccessorService imple
         return jobConf;
     }
 
+    private XConfiguration loadActionConf(String hostPort, String action) {
+        File dir = actionConfigDirs.get(hostPort);
+        XConfiguration actionConf = new XConfiguration();
+        if (dir != null) {
+            File actionConfFile = new File(dir, action + ".xml");
+            if (actionConfFile.exists()) {
+                try {
+                    actionConf = new XConfiguration(new FileInputStream(actionConfFile));
+                }
+                catch (IOException ex) {
+                    XLog.getLog(getClass()).warn("Could not read file [{0}] for action [{1}] configuration for hostPort [{2}]",
+                                                 actionConfFile.getAbsolutePath(), action, hostPort);
+                }
+            }
+        }
+        return actionConf;
+    }
+
+    /**
+     * Returns a Configuration containing any defaults for an action for a particular cluster.
+     * <p/>
+     * This configuration is used as default for the action configuration and enables cluster
+     * level default values per action.
+     *
+     * @param hostPort hostname"port to lookup the action default confiugration.
+     * @param action action name.
+     * @return the default configuration for the action for the specified cluster.
+     */
+    public XConfiguration createActionDefaultConf(String hostPort, String action) {
+        hostPort = (hostPort != null) ? hostPort.toLowerCase() : null;
+        Map<String, XConfiguration> hostPortActionConfigs = actionConfigs.get(hostPort);
+        if (hostPortActionConfigs == null) {
+            hostPortActionConfigs = actionConfigs.get("*");
+            hostPort = "*";
+        }
+        XConfiguration actionConf = hostPortActionConfigs.get(action);
+        if (actionConf == null) {
+            // doing lazy loading as we don't know upfront all actions, no need to synchronize
+            // as it is a read operation an in case of a race condition loading and inserting
+            // into the Map is idempotent and the action-config Map is a ConcurrentHashMap
+            actionConf = loadActionConf(hostPort, action);
+            hostPortActionConfigs.put(action, actionConf);
+        }
+        return new XConfiguration(actionConf.toProperties());
+    }
+
     private Configuration getConfiguration(String hostPort) {
         hostPort = (hostPort != null) ? hostPort.toLowerCase() : null;
         Configuration conf = hadoopConfigs.get(hostPort);

Modified: incubator/oozie/trunk/core/src/main/resources/oozie-default.xml
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/resources/oozie-default.xml?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/resources/oozie-default.xml (original)
+++ incubator/oozie/trunk/core/src/main/resources/oozie-default.xml Thu Mar 29 19:32:05 2012
@@ -1368,7 +1368,7 @@
         <value>*=hadoop-conf</value>
         <description>
             Comma separated AUTHORITY=HADOOP_CONF_DIR, where AUTHORITY is the HOST:PORT of
-            the Hadoop service (JobTracker, HDFS). The wildcard '*' configuration is
+            the Hadoop service (JobTracker, YARN, HDFS). The wildcard '*' configuration is
             used when there is no exact match for an authority. The HADOOP_CONF_DIR contains
             the relevant Hadoop *-site.xml files. If the path is relative is looked within
             the Oozie configuration directory; though the path can be absolute (i.e. to point
@@ -1376,7 +1376,23 @@
         </description>
     </property>
 
-    <!-- Credentials -->    
+
+    <property>
+        <name>oozie.service.HadoopAccessorService.action.configurations</name>
+        <value>*=hadoop-conf</value>
+        <description>
+            Comma separated AUTHORITY=ACTION_CONF_DIR, where AUTHORITY is the HOST:PORT of
+            the Hadoop MapReduce service (JobTracker, YARN). The wildcard '*' configuration is
+            used when there is no exact match for an authority. The ACTION_CONF_DIR may contain
+            ACTION.xml files where ACTION is the action type ('java', 'map-reduce', 'pig',
+            'hive', 'sqoop', etc.). If the ACTION.xml file exists, its properties will be used
+            as defaults properties for the action. If the path is relative is looked within
+            the Oozie configuration directory; though the path can be absolute (i.e. to point
+            to Hadoop client conf/ directories in the local filesystem.
+        </description>
+    </property>
+
+    <!-- Credentials -->
     <property>
         <name>oozie.credentials.credentialclasses</name>
         <value> </value>

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/email/TestEmailActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -43,7 +43,7 @@ public class TestEmailActionExecutor ext
     }
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", EmailActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java Thu Mar 29 19:32:05 2012
@@ -63,7 +63,7 @@ public abstract class ActionExecutorTest
         new Services().init();
     }
 
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
     }
 
     @Override

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestDistCpActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestDistCpActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestDistCpActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestDistCpActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -39,7 +39,7 @@ import org.apache.oozie.util.XConfigurat
 public class TestDistCpActionExecutor extends ActionExecutorTestCase{
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", DistcpActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestFsActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestFsActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestFsActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestFsActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -33,7 +33,7 @@ import java.text.MessageFormat;
 
 public class TestFsActionExecutor extends ActionExecutorTestCase {
 
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", FsActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestHiveActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestHiveActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestHiveActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestHiveActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -61,7 +61,7 @@ public class TestHiveActionExecutor exte
     private static final String OUTPUT_DIRNAME = "output";
     private static final String DATA_FILENAME = "data.txt";
 
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes",
                 HiveActionExecutor.class.getName());

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -19,6 +19,7 @@ package org.apache.oozie.action.hadoop;
 
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.StringReader;
@@ -67,9 +68,15 @@ import org.jdom.Element;
 public class TestJavaActionExecutor extends ActionExecutorTestCase {
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", JavaActionExecutor.class.getName());
+        setSystemProperty("oozie.service.HadoopAccessorService.action.configurations",
+                          "*=hadoop-conf," + getJobTrackerUri() + "=action-conf");
+        new File(getTestCaseConfDir(), "action-conf").mkdir();
+        InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-action-config.xml");
+        OutputStream os = new FileOutputStream(new File(getTestCaseConfDir() + "/action-conf", "java.xml"));
+        IOUtils.copyStream(is, os);
     }
 
     public void testLauncherJar() throws Exception {
@@ -187,6 +194,8 @@ public class TestJavaActionExecutor exte
         assertNull(conf.get("b"));
         assertNull(conf.get("oozie.launcher.d"));
         assertNull(conf.get("d"));
+        assertNull(conf.get("action.foo"));
+        assertEquals("action.barbar", conf.get("action.foofoo"));
 
         conf = ae.createBaseHadoopConf(context, actionXml);
         ae.setupActionConf(conf, context, actionXml, getFsTestCaseDir());
@@ -195,6 +204,7 @@ public class TestJavaActionExecutor exte
         assertEquals("BB", conf.get("b"));
         assertEquals("C", conf.get("c"));
         assertEquals("D", conf.get("oozie.launcher.d"));
+        assertEquals("action.bar", conf.get("action.foo"));
 
         conf = ae.createBaseHadoopConf(context, actionXml);
         ae.setupLauncherConf(conf, actionXml, getFsTestCaseDir(), context);

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionError.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionError.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionError.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionError.java Thu Mar 29 19:32:05 2012
@@ -48,7 +48,7 @@ import java.util.List;
 public class TestMapReduceActionError extends ActionExecutorTestCase {
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", MapReduceActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestMapReduceActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -50,7 +50,7 @@ import java.util.List;
 public class TestMapReduceActionExecutor extends ActionExecutorTestCase {
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", MapReduceActionExecutor.class.getName());
         setSystemProperty("oozie.credentials.credentialclasses", "cred=org.apache.oozie.action.hadoop.CredentialForTest");

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestPigActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -67,7 +67,7 @@ public class TestPigActionExecutor exten
             "store B into '$OUT' USING PigStorage();\n";
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", PigActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestShellActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestShellActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestShellActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestShellActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -48,7 +48,7 @@ public class TestShellActionExecutor ext
     private static final String PERL_SCRIPT_CONTENT = "print \"MY_VAR=TESTING\";";
 
     @Override
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.classes", ShellActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestSqoopActionExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestSqoopActionExecutor.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestSqoopActionExecutor.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestSqoopActionExecutor.java Thu Mar 29 19:32:05 2012
@@ -95,7 +95,7 @@ public class TestSqoopActionExecutor ext
             "<arg>I</arg>" +
             "</sqoop>";
 
-    protected void setSystemProps() {
+    protected void setSystemProps() throws Exception {
         super.setSystemProps();
         setSystemProperty("oozie.service.ActionService.executor.ext.classes", SqoopActionExecutor.class.getName());
     }

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java Thu Mar 29 19:32:05 2012
@@ -34,11 +34,17 @@ public class TestHadoopAccessorService e
     protected void setUp() throws Exception {
         super.setUp();
         new File(getTestCaseConfDir(), "hadoop-confx").mkdir();
+        new File(getTestCaseConfDir(), "action-confx").mkdir();
         InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-hadoop-config.xml");
         OutputStream os = new FileOutputStream(new File(getTestCaseConfDir() + "/hadoop-confx", "core-site.xml"));
         IOUtils.copyStream(is, os);
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-action-config.xml");
+        os = new FileOutputStream(new File(getTestCaseConfDir() + "/action-confx", "action.xml"));
+        IOUtils.copyStream(is, os);
         setSystemProperty("oozie.service.HadoopAccessorService.hadoop.configurations",
                           "*=hadoop-conf,jt=hadoop-confx");
+        setSystemProperty("oozie.service.HadoopAccessorService.action.configurations",
+                          "*=hadoop-conf,jt=action-confx");
         if (System.getProperty("oozie.test.hadoop.security", "simple").equals("kerberos")) {
             setSystemProperty("oozie.service.HadoopAccessorService.kerberos.enabled", "true");
             setSystemProperty("oozie.service.HadoopAccessorService.keytab.file", getKeytabFile());
@@ -60,7 +66,14 @@ public class TestHadoopAccessorService e
         assertNotNull(has.createJobConf("*"));
         assertNotNull(has.createJobConf("jt"));
         assertEquals("bar", has.createJobConf("jt").get("foo"));
+        assertNotNull(has.createActionDefaultConf("*", "action"));
+        assertNotNull(has.createActionDefaultConf("jt", "action"));
+        assertNotNull(has.createActionDefaultConf("jt", "actionx"));
+        assertNotNull(has.createActionDefaultConf("jtx", "action"));
+        assertEquals("action.bar", has.createActionDefaultConf("jt", "action").get("action.foo"));
+        assertNull(has.createActionDefaultConf("*", "action").get("action.foo"));
     }
+
     public void testAccessor() throws Exception {
         Services services = Services.get();
         HadoopAccessorService has = services.get(HadoopAccessorService.class);

Added: incubator/oozie/trunk/core/src/test/resources/test-action-config.xml
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/resources/test-action-config.xml?rev=1307071&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/test/resources/test-action-config.xml (added)
+++ incubator/oozie/trunk/core/src/test/resources/test-action-config.xml Thu Mar 29 19:32:05 2012
@@ -0,0 +1,32 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  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.
+-->
+<configuration>
+
+    <property>
+        <name>action.foo</name>
+        <value>action.bar</value>
+    </property>
+
+    <property>
+        <name>oozie.launcher.action.foofoo</name>
+        <value>action.barbar</value>
+    </property>
+
+</configuration>

Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1307071&r1=1307070&r2=1307071&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Thu Mar 29 19:32:05 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.2.0 release
 
+OOZIE-792 add default action configs per cluster (tucu)
 OOZIE-624 client side improvement of authentication for user defined options
 OOZIE-789 a few testcases using waitFor are timing out with YARN MiniCluster (tucu)
 OOZIE-781 Xerces validator used by Java gets stuck during pattern matching (Virag via Mohammad)