You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by bz...@apache.org on 2015/04/13 23:11:07 UTC

oozie git commit: OOZIE-2051 Allows multiple default action configuration files per action (venkatnrangan via bzhang)

Repository: oozie
Updated Branches:
  refs/heads/master f0bceb0bc -> a3606f203


OOZIE-2051 Allows multiple default action configuration files per action (venkatnrangan via bzhang)


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

Branch: refs/heads/master
Commit: a3606f2035b11d8e396fcde8b771b22cd4e6dd1a
Parents: f0bceb0
Author: Bowen Zhang <bo...@yahoo.com>
Authored: Mon Apr 13 14:09:26 2015 -0700
Committer: Bowen Zhang <bo...@yahoo.com>
Committed: Mon Apr 13 14:10:38 2015 -0700

----------------------------------------------------------------------
 .../oozie/service/HadoopAccessorService.java    | 91 ++++++++++++++++----
 .../service/TestHadoopAccessorService.java      | 67 +++++++++++++-
 .../src/test/resources/test-action-config-1.xml | 31 +++++++
 .../src/test/resources/test-action-config-2.xml | 37 ++++++++
 .../src/test/resources/test-action-config-3.xml | 36 ++++++++
 core/src/test/resources/test-action-config.xml  |  4 +
 .../test/resources/test-default-config-1.xml    | 42 +++++++++
 core/src/test/resources/test-default-config.xml | 36 ++++++++
 .../src/site/twiki/AG_ActionConfiguration.twiki | 55 ++++++++++++
 docs/src/site/twiki/index.twiki                 |  1 +
 release-log.txt                                 |  1 +
 11 files changed, 383 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java b/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
index 47d44cd..102699b 100644
--- a/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
+++ b/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
@@ -39,12 +39,15 @@ import org.apache.oozie.util.JobUtils;
 
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
@@ -80,6 +83,8 @@ public class HadoopAccessorService implements Service {
     protected static final String HADOOP_YARN_RM = "yarn.resourcemanager.address";
     private static final Map<String, Text> mrTokenRenewers = new HashMap<String, Text>();
 
+    private static final String DEFAULT_ACTIONNAME = "default";
+
     private Set<String> jobTrackerWhitelist = new HashSet<String>();
     private Set<String> nameNodeWhitelist = new HashSet<String>();
     private Map<String, Configuration> hadoopConfigs = new HashMap<String, Configuration>();
@@ -109,7 +114,7 @@ public class HadoopAccessorService implements Service {
             }
             jobTrackerWhitelist.add(tmp);
         }
-        XLog.getLog(getClass()).info(
+        LOG.info(
                 "JOB_TRACKER_WHITELIST :" + jobTrackerWhitelist.toString()
                         + ", Total entries :" + jobTrackerWhitelist.size());
         for (String name : ConfigurationService.getStrings(conf, NAME_NODE_WHITELIST)) {
@@ -119,12 +124,12 @@ public class HadoopAccessorService implements Service {
             }
             nameNodeWhitelist.add(tmp);
         }
-        XLog.getLog(getClass()).info(
+        LOG.info(
                 "NAME_NODE_WHITELIST :" + nameNodeWhitelist.toString()
                         + ", Total entries :" + nameNodeWhitelist.size());
 
         boolean kerberosAuthOn = ConfigurationService.getBoolean(conf, KERBEROS_AUTH_ENABLED);
-        XLog.getLog(getClass()).info("Oozie Kerberos Authentication [{0}]", (kerberosAuthOn) ? "enabled" : "disabled");
+        LOG.info("Oozie Kerberos Authentication [{0}]", (kerberosAuthOn) ? "enabled" : "disabled");
         if (kerberosAuthOn) {
             kerberosInit(conf);
         }
@@ -175,8 +180,8 @@ public class HadoopAccessorService implements Service {
                 conf.set("hadoop.security.authentication", "kerberos");
                 UserGroupInformation.setConfiguration(conf);
                 UserGroupInformation.loginUserFromKeytab(principal, keytabFile);
-                XLog.getLog(getClass()).info("Got Kerberos ticket, keytab [{0}], Oozie principal principal [{1}]",
-                                             keytabFile, principal);
+                LOG.info("Got Kerberos ticket, keytab [{0}], Oozie principal principal [{1}]",
+                        keytabFile, principal);
             }
             catch (ServiceException ex) {
                 throw ex;
@@ -298,17 +303,63 @@ public class HadoopAccessorService implements Service {
         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);
+            // See if a dir with the action name exists.   If so, load all the xml files in the dir
+            File actionConfDir = new File(dir, action);
+
+            if (actionConfDir.exists() && actionConfDir.isDirectory()) {
+                LOG.info("Processing configuration files under [{0}]"
+                                + " for action [{1}] and hostPort [{2}]",
+                        actionConfDir.getAbsolutePath(), action, hostPort);
+                File[] xmlFiles = actionConfDir.listFiles(
+                        new FilenameFilter() {
+                            @Override
+                            public boolean accept(File dir, String name) {
+                                return name.endsWith(".xml");
+                            }});
+                Arrays.sort(xmlFiles, new Comparator<File>() {
+                    @Override
+                    public int compare(File o1, File o2) {
+                        return o1.getName().compareTo(o2.getName());
+                    }
+                });
+                for (File f : xmlFiles) {
+                    if (f.isFile() && f.canRead()) {
+                        LOG.info("Processing configuration file [{0}]", f.getName());
+                        FileInputStream fis = null;
+                        try {
+                            fis = new FileInputStream(f);
+                            XConfiguration conf = new XConfiguration(fis);
+                            XConfiguration.copy(conf, actionConf);
+                        }
+                        catch (IOException ex) {
+                            LOG
+                                .warn("Could not read file [{0}] for action [{1}] configuration and hostPort [{2}]",
+                                        f.getAbsolutePath(), action, hostPort);
+                        }
+                        finally {
+                            if (fis != null) {
+                                try { fis.close(); } catch(IOException ioe) { }
+                            }
+                        }
+                    }
                 }
             }
         }
+
+        // Now check for <action.xml>   This way <action.xml> has priority over <action-dir>/*.xml
+
+        File actionConfFile = new File(dir, action + ".xml");
+        if (actionConfFile.exists()) {
+            try {
+                XConfiguration conf = new XConfiguration(new FileInputStream(actionConfFile));
+                XConfiguration.copy(conf, actionConf);
+            }
+            catch (IOException ex) {
+                LOG.warn("Could not read file [{0}] for action [{1}] configuration for hostPort [{2}]",
+                        actionConfFile.getAbsolutePath(), action, hostPort);
+            }
+        }
+
         return actionConf;
     }
 
@@ -334,7 +385,17 @@ public class HadoopAccessorService implements Service {
             // 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);
+
+            // We first load a action of type default
+            // This allows for global configuration for all actions - for example
+            // all launchers in one queue and actions in another queue
+            // Are some configuration that applies to multiple actions - like
+            // config libraries path etc
+            actionConf = loadActionConf(hostPort, DEFAULT_ACTIONNAME);
+
+            // Action specific default configuration will override the default action config
+
+            XConfiguration.copy(loadActionConf(hostPort, action), actionConf);
             hostPortActionConfigs.put(action, actionConf);
         }
         return new XConfiguration(actionConf.toProperties());
@@ -535,7 +596,7 @@ public class HadoopAccessorService implements Service {
         String uriScheme = uri.getScheme();
         if (uriScheme != null) {    // skip the check if no scheme is given
             if(!supportedSchemes.isEmpty()) {
-                XLog.getLog(this.getClass()).debug("Checking if filesystem " + uriScheme + " is supported");
+                LOG.debug("Checking if filesystem " + uriScheme + " is supported");
                 if (!supportedSchemes.contains(uriScheme)) {
                     throw new HadoopAccessorException(ErrorCode.E0904, uriScheme, uri.toString());
                 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java b/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
index aa65065..bbe2594 100644
--- a/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
+++ b/core/src/test/java/org/apache/oozie/service/TestHadoopAccessorService.java
@@ -32,19 +32,45 @@ import java.io.OutputStream;
 import java.net.URI;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.ErrorCode;
+import org.apache.oozie.util.XConfiguration;
 
 public class TestHadoopAccessorService extends XTestCase {
 
     protected void setUp() throws Exception {
         super.setUp();
         new File(getTestCaseConfDir(), "hadoop-confx").mkdir();
-        new File(getTestCaseConfDir(), "action-confx").mkdir();
+        File actConfXDir = new File(getTestCaseConfDir(), "action-confx");
+        actConfXDir.mkdir();
+        new File(actConfXDir, "action").mkdir();
+        // Create the default action dir
+        new File(actConfXDir, "default").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"));
+        os = new FileOutputStream(new File(actConfXDir, "action.xml"));
         IOUtils.copyStream(is, os);
+
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-default-config.xml");
+        os = new FileOutputStream(new File(actConfXDir, "default.xml"));
+        IOUtils.copyStream(is, os);
+
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-action-config-1.xml");
+        os = new FileOutputStream(new File(actConfXDir + "/action", "a-conf.xml"));
+        IOUtils.copyStream(is, os);
+
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-action-config-2.xml");
+        os = new FileOutputStream(new File(actConfXDir + "/action", "b-conf.xml"));
+        IOUtils.copyStream(is, os);
+
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-action-config-3.xml");
+        os = new FileOutputStream(new File(actConfXDir + "/action", "c-conf-3.xml"));
+        IOUtils.copyStream(is, os);
+
+        is = Thread.currentThread().getContextClassLoader().getResourceAsStream("test-default-config-1.xml");
+        os = new FileOutputStream(new File(actConfXDir + "/default", "z-conf.xml"));
+        IOUtils.copyStream(is, os);
+
         setSystemProperty("oozie.service.HadoopAccessorService.hadoop.configurations",
                           "*=hadoop-conf,jt=hadoop-confx");
         setSystemProperty("oozie.service.HadoopAccessorService.action.configurations",
@@ -74,10 +100,45 @@ public class TestHadoopAccessorService extends XTestCase {
         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 testActionConfigurations() throws Exception {
+        Services services = Services.get();
+        HadoopAccessorService has = services.get(HadoopAccessorService.class);
+        assertNotNull(has);
+        XConfiguration conf = has.createActionDefaultConf("jt", "action");
+        assertNotNull(conf);
+
+        // Check that the param only in default.xml is still present
+        assertEquals("default.bar", conf.get("default.foo"));
+
+        // And a property that is present in one of the conf files in default dir
+        assertEquals("default.bus", conf.get("default.car"));
+
+        // Check that a default param is overridden by one of the action config files
+        assertEquals("action.bar", conf.get("action.foo"));
+
+        // Check that params from <action-dir>/action/conf files is still present
+        assertEquals("action.car", conf.get("action.boo"));
+        assertEquals("action.carcar", conf.get("oozie.launcher.action.booboo"));
+
+        /*
+            Check precedence - Order of precedence - 0 is the lowest.   Parameters in files of
+            lower precedence will be overridden by redefinitions in higher precedence files.
+
+            0 - All files in defaultdir/*.xml (sorted by lexical name)
+               Files with names lexically lower have lesser precedence than the following ones.
+            1 - default.xml
+            2 - All files in actiondir/*.xml (sort by lexical name)
+               Files with names lexically lower have lesser precedence than the following ones
+            3 - action.xml
+         */
+        assertEquals("100", conf.get("action.testprop"));
+        assertEquals("1", conf.get("default.testprop"));
+
+    }
+
     public void testAccessor() throws Exception {
         Services services = Services.get();
         HadoopAccessorService has = services.get(HadoopAccessorService.class);

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-action-config-1.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-action-config-1.xml b/core/src/test/resources/test-action-config-1.xml
new file mode 100644
index 0000000..028ad18
--- /dev/null
+++ b/core/src/test/resources/test-action-config-1.xml
@@ -0,0 +1,31 @@
+<?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>default.param</name>
+        <value>default.value</value>
+    </property>
+
+    <property>
+        <name>action.testprop</name>
+        <value>1</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-action-config-2.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-action-config-2.xml b/core/src/test/resources/test-action-config-2.xml
new file mode 100644
index 0000000..0376cdc
--- /dev/null
+++ b/core/src/test/resources/test-action-config-2.xml
@@ -0,0 +1,37 @@
+<?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.boo</name>
+        <value>action.car</value>
+    </property>
+
+    <property>
+        <name>oozie.launcher.action.booboo</name>
+        <value>action.carcar</value>
+    </property>
+
+    <property>
+        <name>action.testprop</name>
+        <value>2</value>
+    </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-action-config-3.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-action-config-3.xml b/core/src/test/resources/test-action-config-3.xml
new file mode 100644
index 0000000..d47796b
--- /dev/null
+++ b/core/src/test/resources/test-action-config-3.xml
@@ -0,0 +1,36 @@
+<?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>default.param</name>
+        <value>default.value</value>
+    </property>
+
+    <property>
+        <name>action.testprop</name>
+        <value>3</value>
+    </property>
+
+    <property>
+        <name>action.testprop</name>
+        <value>1</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-action-config.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-action-config.xml b/core/src/test/resources/test-action-config.xml
index 59db9b5..6817432 100644
--- a/core/src/test/resources/test-action-config.xml
+++ b/core/src/test/resources/test-action-config.xml
@@ -29,4 +29,8 @@
         <value>action.barbar</value>
     </property>
 
+    <property>
+        <name>action.testprop</name>
+        <value>100</value>
+    </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-default-config-1.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-default-config-1.xml b/core/src/test/resources/test-default-config-1.xml
new file mode 100644
index 0000000..367755a
--- /dev/null
+++ b/core/src/test/resources/test-default-config-1.xml
@@ -0,0 +1,42 @@
+<?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.boo</name>
+        <value>action.car</value>
+    </property>
+
+    <property>
+        <name>default.car</name>
+        <value>default.bus</value>
+    </property>
+
+    <property>
+        <name>oozie.launcher.action.booboo</name>
+        <value>action.carcar</value>
+    </property>
+
+    <property>
+        <name>default.testprop</name>
+        <value>0</value>
+    </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/core/src/test/resources/test-default-config.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/test-default-config.xml b/core/src/test/resources/test-default-config.xml
new file mode 100644
index 0000000..a9232bf
--- /dev/null
+++ b/core/src/test/resources/test-default-config.xml
@@ -0,0 +1,36 @@
+<?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>default.foo</name>
+        <value>default.bar</value>
+    </property>
+
+    <property>
+        <name>action.foo</name>
+        <value>action.default</value>
+    </property>
+
+    <property>
+        <name>default.testprop</name>
+        <value>1</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/docs/src/site/twiki/AG_ActionConfiguration.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/AG_ActionConfiguration.twiki b/docs/src/site/twiki/AG_ActionConfiguration.twiki
new file mode 100644
index 0000000..f8e6edc
--- /dev/null
+++ b/docs/src/site/twiki/AG_ActionConfiguration.twiki
@@ -0,0 +1,55 @@
+<noautolink>
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+---+!! Action Configuration
+
+Oozie supports providing default configuration for actions of a particular
+action type and default configuration for all actions
+
+---++ Hadoop Default Configuration Values
+
+Oozie supports action configuration equivalent to the component's =*-site.xml= files.
+
+The configuration property in the =oozie-site.xml= is =oozie.service.HadoopAccessorService.action.configurations=
+and its value must follow the pattern _[AUTHORITY=ACTION-CONF-DIR_,]*. Where _AUTHORITY_ is the _HOST:PORT_ of
+the Hadoop service (JobTracker/ResourceManager or HDFS). The _ACTION-CONF-DIR_ is the action configuration directory. If the specified directory is a relative path, it will be looked under the Oozie configuration directory. An absolute path can
+also be specified. Oozie will load and process the action configuration files in the following order.
+
+   1. All files in _default_/*.xml (sorted by lexical name, files with names lexically lower have lesser precedence than the following ones), if present.
+   1. _default_.xml, if present.
+   1. All files in _actionname_/*.xml (sorted by lexical name, files with names lexically lower have lesser precedence than the following ones), if present.
+   1. _actionname_.xml, if present.
+
+
+For example, for _Hive_ action (which has the _actionname_ defined as _hive_ ), the list of files (under relevant _ACTION-CONF-DIR_ ) processed would be,
+
+   1. All files in _default_/*.xml, if present
+   1. _default_.xml, if present.
+   1. All files in _hive_/*.xml, if present
+   1. _hive_.xml, if present.
+
+
+Files processed earlier for an action have the lowest precedence and can have the configuration parameters redefined.  All files and directories are relative to the _ACTION-CONF-DIR_ directory.
+
+In addition to explicit authorities, a '*' wildcard is supported. The configuration file associated with the wildcard
+will be used as default if there is no action configuration for the requested Hadoop service.
+
+For example, the configuration in the =oozie-site.xml= would look like:
+
+<verbatim>
+...
+    <property>
+        <name>oozie.service.HadoopAccessorService.action.configurations</name>
+        <value>*=hadoop-conf,jt-bar:8021=bar-cluster,nn-bar:8020=bar-cluster</value>
+    </property>
+...
+</verbatim>
+
+The action configuration files use the Hadoop configuration syntax.
+
+By default Oozie does not define any default action configurations.
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+</noautolink>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/docs/src/site/twiki/index.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/index.twiki b/docs/src/site/twiki/index.twiki
index 50c6213..8591530 100644
--- a/docs/src/site/twiki/index.twiki
+++ b/docs/src/site/twiki/index.twiki
@@ -73,6 +73,7 @@ Enough reading already? Follow the steps in [[DG_QuickStart][Oozie Quick Start]]
    * [[AG_Install][Oozie Install]]
    * [[AG_OozieLogging][Oozie Logging]]
    * [[AG_HadoopConfiguration][Hadoop Configuration]]
+   * [[AG_ActionConfiguration][Action Configuration]]
    * [[AG_Monitoring][Oozie Monitoring]]
    * [[DG_CommandLineTool][Command Line Tool]]
    * [[AG_OozieUpgrade][Oozie Upgrade]]

http://git-wip-us.apache.org/repos/asf/oozie/blob/a3606f20/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index bfa9fdd..74fbd57 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.2.0 release (trunk - unreleased)
 
+OOZIE-2051 Allows multiple default action configuration files per action (venkatnrangan via bzhang)
 OOZIE-2170 Oozie should automatically set configs to make Spark jobs show up in the Spark History Server (rkanter)
 OOZIE-2140 Audit Log should be shown in Oozie UI (puru)
 OOZIE-2139 Coord update doesn't work for job which is submitted by bundle (puru)