Update falcon branch 0.10-refactored-ui to be up to date with branch 0.10

Author: bvellanki <bvellanki@hortonworks.com>
Author: yzheng-hortonworks <yzheng@hortonworks.com>
Author: Praveen Adlakha <adlakha.praveen@gmail.com>
Author: peeyush b <pbishnoi@hortonworks.com>
Author: Venkatesan Ramachandran <vramachandran@hortonworks.com>
Author: Pallavi Rao <pallavi.rao@inmobi.com>
Author: Sowmya Ramesh <sramesh@hortonworks.com>
Author: Peeyush <peeyushb@apache.org>
Author: Venkat Ranganathan <venkat@hortonworks.com>
Author: Sowmya Ramesh <sowmya_kr@apache.org>
Author: Srikanth Sundarrajan <sriksun@hotmail.com>
Author: Ajay Yadava <ajayyadava@apache.org>
Author: Ajay Yadava <ajaynsit@gmail.com>
Author: Murali Ramasami <murali dot msse at gmail dot com>
Author: sandeep <sandysmdl@gmail.com>

Reviewers: "Sowmya Ramesh <sowmya_kr@apache.org>"

Closes #267 from bvellanki/0.10-refactored-ui
diff --git a/CHANGES.txt b/CHANGES.txt
index 8fb8186..767c5a2 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,60 +1,14 @@
 Apache Falcon Change log
-Trunk
-  TASKS:
-  INCOMPATIBLE CHANGES
-  NEW FEATURES
-    FALCON-1627 Provider integration with Azure Data Factory pipelines (Ying Zheng, Venkat Ranganathan, Sowmya Ramesh)
-   
-    FALCON-1664 Add Postgres support for native scheduler(Deepak Barr via Pragya Mittal)
 
-    FALCON-1495 In instance status list, show all runs for instances when requested by user(Narayan Periwal via Ajay Yadava)
+Future Releases:
+For changes in releases after 0.10, please refer to "release-docs" or
+https://github.com/apache/falcon/tree/master/release-docs/
 
-    FALCON-1230 Data based notification Service to notify execution instances when data becomes available(Pavan Kumar Kolamuri via Ajay Yadava)
+Release Version: 0.10
+For changes in release 0.10, please refer to "release-docs" or
+https://github.com/apache/falcon/blob/0.10/release-docs/0.10/CHANGES.0.10.md
 
-  IMPROVEMENTS
-    FALCON-1584 Falcon allows invalid hadoop queue name for schedulable feed entities (Venkatesan Ramachandran via Balu Vellanki)
-
-    FALCON-1774 Falcon to honour PRISM_URL env var (Praveen Adlakha) 
-
-    FALCON-1721 Checkstyle doesn't extend parent.
-    
-    FALCON-1818 Minor doc update for tar package locations after FALCON-1751 (Deepak Barr)
-
-    FALCON-1729 Database ingest to support password alias via keystore file (Venkatesan Ramachandran via Balu Vellanki)
-
-    FALCON-1751 Support assembly:single mojo(Ruoyu Wang via Ajay Yadava)
-
-    FALCON-763 Support feed listing for CatalogStorage (Balu Vellanki)
-
-    FALCON-1764 Remove temporary folder "localhost" created during tests(Praveen Adlakha via Ajay Yadava)
-
-    FALCON-1756 Remove PID files on service stop(Deepak Barr via Ajay Yadava)
-
-    FALCON-1771 Tool to merge pull requests (Ajay Yadava)
-
-    FALCON-1770 Update README file (Ajay Yadava)
-
-  BUG FIXES
-    FALCON-1842 Falcon build failed in Jenkins at OozieFeedWorkflowBuilderTest (Balu Vellanki)
-
-    FALCON-887 Support for multiple lib paths in falcon process (Sowmya Ramesh)
-
-    FALCON-1795 Kill api not killing waiting/ready instances
-   
-    FALCON-1804 Non-SLA feed throws NullPointerException.
-    
-    FALCON-1806 Update documentation for Import and Export. (Venkatesan Ramachandran via Balu Vellanki)
-
-    FALCON-1787 Ooozie pig-action.xml requires hive sharelib for HCatalog use(Sowmya Ramesh via Ajay Yadava)
-
-    FALCON-1792 Upgrade hadoop.version to 2.6.2 (Venkatesan Ramachandran via Peeyush Bishnoi)
-
-    FALCON-1796 [HOTFIX] Incorrect parent pom in distro module(Ajay Yadava)
-
-    FALCON-1779 Fix rat-check failure in trunk (Ajay Yadava)
-
-
-Proposed Release Version: 0.9
+Release Version: 0.9
   TASKS:
     FALCON-1778 Check licenses and copyright information (Pallavi Rao)
 
diff --git a/Installation-steps.txt b/Installation-steps.txt
index b86d6a1..84f0c99 100644
--- a/Installation-steps.txt
+++ b/Installation-steps.txt
@@ -41,10 +41,12 @@
 * export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m -noverify" && mvn clean install
 
 [optionally -Dhadoop.version=<<hadoop.version>> can be appended to build for a specific version of hadoop]
-*Note:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
-        Falcon build with JDK 1.7 using -noverify option
-        To compile Falcon with Hive Replication, optionally "-P hadoop-2,hivedr" can be appended. For this
-        Hive >= 1.2.0 and Oozie >= 4.2.0 should be available.
+*Note 1:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
+          Falcon build with JDK 1.7 using -noverify option
+*Note 2:* To compile Falcon with addon extensions, append additional profiles to build command using syntax -P<<profile1,profile2>>
+          For Hive Mirroring extension, use profile"hivedr". Hive >= 1.2.0 and Oozie >= 4.2.0 is required
+          For HDFS Snapshot mirroring extension, use profile "hdfs-snapshot-mirroring". Hadoop >= 2.7.0 is required
+          For ADF integration, use profile "adf"
 
 
 b. Building falcon from the source repository
@@ -55,10 +57,12 @@
 * export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m -noverify" && mvn clean install
 
 [optionally -Dhadoop.version=<<hadoop.version>> can be appended to build for a specific version of hadoop]
-*Note:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
-        Falcon build with JDK 1.7 using -noverify option
-        To compile Falcon with Hive Replication, optionally "-P hadoop-2,hivedr" can be appended. For this
-        Hive >= 1.2.0 and Oozie >= 4.2.0 should be available.
+*Note 1:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
+          Falcon build with JDK 1.7 using -noverify option
+*Note 2:* To compile Falcon with addon extensions, append additional profiles to build command using syntax -P<<profile1,profile2>>
+          For Hive Mirroring extension, use profile"hivedr". Hive >= 1.2.0 and Oozie >= 4.2.0 is required
+          For HDFS Snapshot mirroring extension, use profile "hdfs-snapshot-mirroring". Hadoop >= 2.7.0 is required
+          For ADF integration, use profile "adf"
 
 
 2. Deploying Falcon
@@ -118,8 +122,8 @@
 ~~~~~~~~~~~~~~~
 
 * bin/falcon admin -version
-  Falcon server build version: {Version:"0.3-SNAPSHOT-rd7e2be9afa2a5dc96acd1ec9e325f39c6b2f17f7",Mode:"embedded"}
-
+Falcon server build version: {"properties":[{"key":"Version","value":"0.10-rbe02edf0d5b10af27bbac694e536bef30885c00e"},
+{"key":"Mode","value":"embedded"},{"key":"authentication","value":"simple"},{"key":"safemode","value":"false"}]}
 * bin/falcon help
   (for more details about falcon cli usage)
 
diff --git a/LICENSE.txt b/LICENSE.txt
index 80f8aa2..f61385f 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -238,8 +238,24 @@
 This product bundles jasmine 2.0.2 which is available under a
 MIT license. For details, see docs/license/jasmine-LICENSE.txt
 
-
 This product bundles entypo icons which is available under a
 CC BY-SA license and Font is available under SIL license.
 For details, see docs/license/entypo-icons-LICENSE.txt and
-docs/license/entypo-font-LICENSE.txt
+docs/license/entypo-font-LICENSE.txt. (Entypo pictograms by Daniel 
+Bruce - www.entypo.com)
+
+This product bundles Cabin webfont [regular, italic & bold] under SIL 
+license. For details see docs/license/cabin-font-LICENSE.txt.
+(www.impallari.com & www.ikern.com)
+
+This product bundles normalize v3.0.1 which is available under 
+MIT license.  For details see docs/license/normalize-LICENSE.txt
+
+This product bundles animate v3.2.5 which is available under 
+MIT license For details see docs/license/animate-LICENSE.txt
+
+This product bundles ngTagsInput v2.3.0 which is available under
+MIT license. For details see docs/license/ngTagsInput-LICENSE.txt
+
+This product bundles ngMask v3.1.1 which is available under
+MIT license. For details see docs/license/ngMask-LICENSE.txt
diff --git a/NOTICE.txt b/NOTICE.txt
index c9259e8..7521153 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -1,6 +1,13 @@
 Apache Falcon
 
-Copyright 2011-2015 The Apache Software Foundation
+Copyright 2011-2016 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
+
+*****************
+CC BY-SA 3.0
+*****************
+
+The following binary components are provided under CC BY-SA 3.0
+  entypo-icon (Entypo pictograms by Daniel Bruce - www.entypo.com)
diff --git a/acquisition/pom.xml b/acquisition/pom.xml
index 38ea259..78bbc5f 100644
--- a/acquisition/pom.xml
+++ b/acquisition/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-acquisition</artifactId>
     <description>Apache Falcon Acquisition Module</description>
diff --git a/addons/adf/pom.xml b/addons/adf/pom.xml
index 0042f5c..dc0988a 100644
--- a/addons/adf/pom.xml
+++ b/addons/adf/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <artifactId>falcon-adf</artifactId>
diff --git a/addons/designer/pom.xml b/addons/designer/pom.xml
index 4be24c3..a6922df 100644
--- a/addons/designer/pom.xml
+++ b/addons/designer/pom.xml
@@ -21,7 +21,7 @@
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon.designer</groupId>
     <artifactId>designer-main</artifactId>
-    <version>0.6-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Apache Falcon Pipeline Designer</description>
     <name>Apache Falcon Pipeline Designer</name>
     <packaging>pom</packaging>
diff --git a/addons/extensions/hdfs-mirroring/pom.xml b/addons/extensions/hdfs-mirroring/pom.xml
index cb9304e..bfaf425 100644
--- a/addons/extensions/hdfs-mirroring/pom.xml
+++ b/addons/extensions/hdfs-mirroring/pom.xml
@@ -25,7 +25,7 @@
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon.extensions</groupId>
     <artifactId>falcon-hdfs-mirroring-extension</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Apache Falcon sample Hdfs mirroring extension</description>
     <name>Apache Falcon sample Hdfs mirroring extension</name>
     <packaging>jar</packaging>
diff --git a/addons/extensions/hdfs-mirroring/src/main/resources/runtime/hdfs-mirroring-workflow.xml b/addons/extensions/hdfs-mirroring/src/main/resources/runtime/hdfs-mirroring-workflow.xml
index 1e2282c..c0504fb 100644
--- a/addons/extensions/hdfs-mirroring/src/main/resources/runtime/hdfs-mirroring-workflow.xml
+++ b/addons/extensions/hdfs-mirroring/src/main/resources/runtime/hdfs-mirroring-workflow.xml
@@ -51,6 +51,10 @@
                     <name>oozie.launcher.mapreduce.job.hdfs-servers</name>
                     <value>${sourceClusterFS},${targetClusterFS}</value>
                 </property>
+                <property>
+                    <name>mapreduce.job.hdfs-servers</name>
+                    <value>${sourceClusterFS},${targetClusterFS}</value>
+                </property>
             </configuration>
             <main-class>org.apache.falcon.replication.FeedReplicator</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
diff --git a/addons/extensions/hdfs-snapshot-mirroring/pom.xml b/addons/extensions/hdfs-snapshot-mirroring/pom.xml
index b0b4819..7aaee3d 100644
--- a/addons/extensions/hdfs-snapshot-mirroring/pom.xml
+++ b/addons/extensions/hdfs-snapshot-mirroring/pom.xml
@@ -25,7 +25,7 @@
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon.extensions</groupId>
     <artifactId>falcon-hdfs-snapshot-mirroring-extension</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Apache Falcon HDFS Snapshot Mirroring Extension</description>
     <name>Apache Falcon Sample HDFS Snapshot Mirroring Extension</name>
     <packaging>jar</packaging>
diff --git a/addons/extensions/hive-mirroring/pom.xml b/addons/extensions/hive-mirroring/pom.xml
index adfb0be..9d08835 100644
--- a/addons/extensions/hive-mirroring/pom.xml
+++ b/addons/extensions/hive-mirroring/pom.xml
@@ -25,7 +25,7 @@
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon.extensions</groupId>
     <artifactId>falcon-hive-mirroring-extension</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Apache Falcon sample Hive mirroring extension</description>
     <name>Apache Falcon sample Hive mirroring extension</name>
     <packaging>jar</packaging>
diff --git a/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-secure-workflow.xml b/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-secure-workflow.xml
index 4bf048f..63e9a67 100644
--- a/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-secure-workflow.xml
+++ b/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-secure-workflow.xml
@@ -96,18 +96,16 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-sourceCluster</arg>
             <arg>${sourceCluster}</arg>
             <arg>-sourceMetastoreUri</arg>
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -144,8 +142,10 @@
             <arg>${clusterForJobNNKerberosPrincipal}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>lastevents</arg>
         </java>
@@ -190,8 +190,6 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-replicationMaxMaps</arg>
             <arg>${replicationMaxMaps}</arg>
             <arg>-distcpMaxMaps</arg>
@@ -202,10 +200,10 @@
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -244,8 +242,10 @@
             <arg>${clusterForJobNNKerberosPrincipal}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>export</arg>
             <arg>-counterLogDir</arg>
@@ -292,8 +292,6 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-replicationMaxMaps</arg>
             <arg>${replicationMaxMaps}</arg>
             <arg>-distcpMaxMaps</arg>
@@ -304,10 +302,10 @@
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -346,8 +344,10 @@
             <arg>${clusterForJobNNKerberosPrincipal}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>import</arg>
         </java>
diff --git a/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-workflow.xml b/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-workflow.xml
index 9f9bf92..4f6eec5 100644
--- a/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-workflow.xml
+++ b/addons/extensions/hive-mirroring/src/main/resources/runtime/hive-mirroring-workflow.xml
@@ -46,18 +46,16 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-sourceCluster</arg>
             <arg>${sourceCluster}</arg>
             <arg>-sourceMetastoreUri</arg>
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -80,8 +78,10 @@
             <arg>${clusterForJobRunWriteEP}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>lastevents</arg>
         </java>
@@ -118,8 +118,6 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-replicationMaxMaps</arg>
             <arg>${replicationMaxMaps}</arg>
             <arg>-distcpMaxMaps</arg>
@@ -130,10 +128,10 @@
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -158,8 +156,10 @@
             <arg>${clusterForJobRunWriteEP}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>export</arg>
             <arg>-counterLogDir</arg>
@@ -198,8 +198,6 @@
             <main-class>org.apache.falcon.hive.HiveDRTool</main-class>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-            <arg>-falconLibPath</arg>
-            <arg>${wf:conf("falcon.libpath")}</arg>
             <arg>-replicationMaxMaps</arg>
             <arg>${replicationMaxMaps}</arg>
             <arg>-distcpMaxMaps</arg>
@@ -210,10 +208,10 @@
             <arg>${sourceMetastoreUri}</arg>
             <arg>-sourceHiveServer2Uri</arg>
             <arg>${sourceHiveServer2Uri}</arg>
-            <arg>-sourceDatabase</arg>
-            <arg>${sourceDatabase}</arg>
-            <arg>-sourceTable</arg>
-            <arg>${sourceTable}</arg>
+            <arg>-sourceDatabases</arg>
+            <arg>${sourceDatabases}</arg>
+            <arg>-sourceTables</arg>
+            <arg>${sourceTables}</arg>
             <arg>-sourceStagingPath</arg>
             <arg>${sourceStagingPath}</arg>
             <arg>-sourceNN</arg>
@@ -238,8 +236,10 @@
             <arg>${clusterForJobRunWriteEP}</arg>
             <arg>-tdeEncryptionEnabled</arg>
             <arg>${tdeEncryptionEnabled}</arg>
-            <arg>-jobName</arg>
-            <arg>${jobName}-${nominalTime}</arg>
+            <arg>-hiveJobName</arg>
+            <arg>${hiveJobName}</arg>
+            <arg>-sourceDatabase</arg>
+            <arg>${sourceDatabase}</arg>
             <arg>-executionStage</arg>
             <arg>import</arg>
         </java>
diff --git a/addons/hdfs-snapshot-mirroring/pom.xml b/addons/hdfs-snapshot-mirroring/pom.xml
index d37185f..6d1ef87 100644
--- a/addons/hdfs-snapshot-mirroring/pom.xml
+++ b/addons/hdfs-snapshot-mirroring/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <artifactId>falcon-hdfs-snapshot-mirroring</artifactId>
@@ -85,56 +85,47 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-hdfs</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-core</artifactId>
-            <scope>compile</scope>
             <version>${hadoop.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-common</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-auth</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-distcp</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
diff --git a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicator.java b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicator.java
index 2e41cc0..6f5defe 100644
--- a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicator.java
+++ b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicator.java
@@ -73,8 +73,12 @@
         String sourceStorageUrl = cmd.getOptionValue(HdfsSnapshotMirrorProperties.SOURCE_NN.getName());
         String targetStorageUrl = cmd.getOptionValue(HdfsSnapshotMirrorProperties.TARGET_NN.getName());
 
-        DistributedFileSystem sourceFs = HdfsSnapshotUtil.getSourceFileSystem(cmd);
-        DistributedFileSystem targetFs = HdfsSnapshotUtil.getTargetFileSystem(cmd);
+        // Always add to getConf() so that configuration set by oozie action is
+        // available when creating DistributedFileSystem.
+        DistributedFileSystem sourceFs = HdfsSnapshotUtil.getSourceFileSystem(cmd,
+                new Configuration(getConf()));
+        DistributedFileSystem targetFs = HdfsSnapshotUtil.getTargetFileSystem(cmd,
+                new Configuration(getConf()));
 
         String currentSnapshotName = HdfsSnapshotUtil.SNAPSHOT_PREFIX
                 + cmd.getOptionValue(HdfsSnapshotMirrorProperties.SNAPSHOT_JOB_NAME.getName())
diff --git a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/retention/HdfsSnapshotEvictor.java b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/retention/HdfsSnapshotEvictor.java
index 22e3377..a50e770 100644
--- a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/retention/HdfsSnapshotEvictor.java
+++ b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/retention/HdfsSnapshotEvictor.java
@@ -60,8 +60,10 @@
     @Override
     public int run(String[] args) throws Exception {
         CommandLine cmd = getCommand(args);
-        DistributedFileSystem sourceFs = HdfsSnapshotUtil.getSourceFileSystem(cmd);
-        DistributedFileSystem targetFs = HdfsSnapshotUtil.getTargetFileSystem(cmd);
+        DistributedFileSystem sourceFs = HdfsSnapshotUtil.getSourceFileSystem(cmd,
+                new Configuration(getConf()));
+        DistributedFileSystem targetFs = HdfsSnapshotUtil.getTargetFileSystem(cmd,
+                new Configuration(getConf()));
 
         String sourceDir = cmd.getOptionValue(HdfsSnapshotMirrorProperties.SOURCE_SNAPSHOT_DIR.getName());
         String targetDir = cmd.getOptionValue(HdfsSnapshotMirrorProperties.TARGET_SNAPSHOT_DIR.getName());
diff --git a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/util/HdfsSnapshotUtil.java b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/util/HdfsSnapshotUtil.java
index 5196791..88f6fd9 100644
--- a/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/util/HdfsSnapshotUtil.java
+++ b/addons/hdfs-snapshot-mirroring/src/main/java/org/apache/falcon/snapshots/util/HdfsSnapshotUtil.java
@@ -19,6 +19,7 @@
 package org.apache.falcon.snapshots.util;
 
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.extensions.mirroring.hdfsSnapshot.HdfsSnapshotMirrorProperties;
@@ -37,29 +38,33 @@
 
     private HdfsSnapshotUtil() {}
 
-    public static DistributedFileSystem getSourceFileSystem(CommandLine cmd) throws FalconException {
+    public static DistributedFileSystem getSourceFileSystem(CommandLine cmd,
+                                                            Configuration conf) throws FalconException {
         String sourceStorageUrl = cmd.getOptionValue(HdfsSnapshotMirrorProperties.SOURCE_NN.getName());
         String sourceExecuteEndpoint = cmd.getOptionValue(HdfsSnapshotMirrorProperties.SOURCE_EXEC_URL.getName());
         String sourcePrincipal = parseKerberosPrincipal(cmd.getOptionValue(
                 HdfsSnapshotMirrorProperties.SOURCE_NN_KERBEROS_PRINCIPAL.getName()));
-        Configuration sourceConf = ClusterHelper.getConfiguration(sourceStorageUrl,
+
+        Configuration sourceConf = ClusterHelper.getConfiguration(conf, sourceStorageUrl,
                 sourceExecuteEndpoint, sourcePrincipal);
         return HadoopClientFactory.get().createDistributedProxiedFileSystem(sourceConf);
     }
 
-    public static DistributedFileSystem getTargetFileSystem(CommandLine cmd) throws FalconException {
+    public static DistributedFileSystem getTargetFileSystem(CommandLine cmd,
+                                                            Configuration conf) throws FalconException {
         String targetStorageUrl = cmd.getOptionValue(HdfsSnapshotMirrorProperties.TARGET_NN.getName());
         String taregtExecuteEndpoint = cmd.getOptionValue(HdfsSnapshotMirrorProperties.TARGET_EXEC_URL.getName());
         String targetPrincipal = parseKerberosPrincipal(cmd.getOptionValue(
                 HdfsSnapshotMirrorProperties.TARGET_NN_KERBEROS_PRINCIPAL.getName()));
 
-        Configuration targetConf = ClusterHelper.getConfiguration(targetStorageUrl,
+        Configuration targetConf = ClusterHelper.getConfiguration(conf, targetStorageUrl,
                 taregtExecuteEndpoint, targetPrincipal);
         return HadoopClientFactory.get().createDistributedProxiedFileSystem(targetConf);
     }
 
     public static String parseKerberosPrincipal(String principal) {
-        if (principal.equals(HdfsSnapshotMirroringExtension.EMPTY_KERBEROS_PRINCIPAL)) {
+        if (StringUtils.isEmpty(principal)
+                || principal.equals(HdfsSnapshotMirroringExtension.EMPTY_KERBEROS_PRINCIPAL)) {
             return null;
         }
         return principal;
diff --git a/addons/hdfs-snapshot-mirroring/src/test/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicatorTest.java b/addons/hdfs-snapshot-mirroring/src/test/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicatorTest.java
index 7924214..fe7ced5 100644
--- a/addons/hdfs-snapshot-mirroring/src/test/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicatorTest.java
+++ b/addons/hdfs-snapshot-mirroring/src/test/java/org/apache/falcon/snapshots/replication/HdfsSnapshotReplicatorTest.java
@@ -25,7 +25,7 @@
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.extensions.mirroring.hdfsSnapshot.HdfsSnapshotMirrorProperties;
-import org.apache.falcon.hadoop.HadoopClientFactory;
+import org.apache.falcon.snapshots.util.HdfsSnapshotUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
@@ -72,6 +72,7 @@
 
     @BeforeClass
     public void init() throws Exception {
+        this.setConf(new Configuration());
         baseDir = Files.createTempDirectory("test_snapshot-replication").toFile().getAbsoluteFile();
         miniDFSCluster = MiniHdfsClusterUtil.initMiniDfs(MiniHdfsClusterUtil.SNAPSHOT_REPL_TEST_PORT, baseDir);
         miniDfs = miniDFSCluster.getFileSystem();
@@ -100,14 +101,13 @@
 
     @Test
     public void replicationTest() throws Exception {
-        Configuration sourceConf = ClusterHelper.getConfiguration(sourceCluster);
-        this.setConf(sourceConf);
-        Configuration targetConf = ClusterHelper.getConfiguration(targetCluster);
         sourceStorageUrl = ClusterHelper.getStorageUrl(sourceCluster);
         targetStorageUrl = ClusterHelper.getStorageUrl(targetCluster);
 
-        DistributedFileSystem sourceFs = HadoopClientFactory.get().createDistributedProxiedFileSystem(sourceConf);
-        DistributedFileSystem targetFs = HadoopClientFactory.get().createDistributedProxiedFileSystem(targetConf);
+        DistributedFileSystem sourceFs = HdfsSnapshotUtil.getSourceFileSystem(cmd,
+                new Configuration(getConf()));
+        DistributedFileSystem targetFs = HdfsSnapshotUtil.getTargetFileSystem(cmd,
+                new Configuration(getConf()));
 
         // create dir1, create snapshot, invoke copy, check file in target, create snapshot on target
         Path dir1 = new Path(sourceDir, "dir1");
diff --git a/addons/hivedr/pom.xml b/addons/hivedr/pom.xml
index f380012..e2f0c7f 100644
--- a/addons/hivedr/pom.xml
+++ b/addons/hivedr/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <artifactId>falcon-hive-replication</artifactId>
@@ -37,56 +37,47 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-common</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-hdfs</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-core</artifactId>
-            <scope>compile</scope>
             <version>${hadoop.version}</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-mapreduce-client-common</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-yarn-server-nodemanager</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-auth</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-distcp</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRArgs.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRArgs.java
index 71b9043..d891487 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRArgs.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRArgs.java
@@ -30,8 +30,9 @@
     SOURCE_CLUSTER("sourceCluster", "source cluster"),
     SOURCE_METASTORE_URI("sourceMetastoreUri", "source meta store uri"),
     SOURCE_HS2_URI("sourceHiveServer2Uri", "source HS2 uri"),
-    SOURCE_DATABASE("sourceDatabase", "comma source databases"),
-    SOURCE_TABLE("sourceTable", "comma source tables"),
+    SOURCE_DATABASES("sourceDatabases", "comma source databases"),
+    SOURCE_DATABASE("sourceDatabase", "First source database"),
+    SOURCE_TABLES("sourceTables", "comma source tables"),
     SOURCE_STAGING_PATH("sourceStagingPath", "source staging path for data", false),
 
     // source hadoop endpoints
@@ -70,7 +71,7 @@
     // Map Bandwidth
     DISTCP_MAP_BANDWIDTH("distcpMapBandwidth", "map bandwidth in mb", false),
 
-    JOB_NAME("jobName", "unique job name"),
+    JOB_NAME("hiveJobName", "unique job name"),
 
     CLUSTER_FOR_JOB_RUN("clusterForJobRun", "cluster where job runs"),
     JOB_CLUSTER_NN("clusterForJobRunWriteEP", "write end point of cluster where job runs"),
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDROptions.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDROptions.java
index 0096727..215be35 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDROptions.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDROptions.java
@@ -63,21 +63,29 @@
     }
 
     public List<String> getSourceDatabases() {
-        return Arrays.asList(context.get(HiveDRArgs.SOURCE_DATABASE).trim().split(","));
+        return Arrays.asList(context.get(HiveDRArgs.SOURCE_DATABASES).trim().split(","));
     }
 
     public List<String> getSourceTables() {
-        return Arrays.asList(context.get(HiveDRArgs.SOURCE_TABLE).trim().split(","));
+        return Arrays.asList(context.get(HiveDRArgs.SOURCE_TABLES).trim().split(","));
     }
 
     public String getSourceStagingPath() {
+        return context.get(HiveDRArgs.SOURCE_STAGING_PATH);
+    }
+
+
+    public void setSourceStagingPath() {
         String stagingPath = context.get(HiveDRArgs.SOURCE_STAGING_PATH);
-        if (StringUtils.isNotBlank(stagingPath)) {
-            stagingPath = StringUtils.removeEnd(stagingPath, File.separator);
-            return stagingPath + File.separator + getJobName();
+        String srcStagingPath;
+        if ("NA".equalsIgnoreCase(stagingPath)) {
+            stagingPath = StringUtils.removeEnd(FileUtils.DEFAULT_EVENT_STORE_PATH, File.separator);
+            srcStagingPath = stagingPath + File.separator + getJobName();
         } else {
-            return FileUtils.DEFAULT_EVENT_STORE_PATH + getJobName();
+            stagingPath = StringUtils.removeEnd(stagingPath, File.separator);
+            srcStagingPath = stagingPath + File.separator + getJobName();
         }
+        context.put(HiveDRArgs.SOURCE_STAGING_PATH, srcStagingPath);
     }
 
     public String getSourceWriteEP() {
@@ -109,13 +117,20 @@
     }
 
     public String getTargetStagingPath() {
+        return context.get(HiveDRArgs.TARGET_STAGING_PATH);
+    }
+
+    public void setTargetStagingPath() {
         String stagingPath = context.get(HiveDRArgs.TARGET_STAGING_PATH);
-        if (StringUtils.isNotBlank(stagingPath)) {
-            stagingPath = StringUtils.removeEnd(stagingPath, File.separator);
-            return stagingPath + File.separator + getJobName();
+        String targetStagingPath;
+        if ("NA".equalsIgnoreCase(stagingPath)) {
+            stagingPath = StringUtils.removeEnd(FileUtils.DEFAULT_EVENT_STORE_PATH, File.separator);
+            targetStagingPath = stagingPath + File.separator + getJobName();
         } else {
-            return FileUtils.DEFAULT_EVENT_STORE_PATH + getJobName();
+            stagingPath = StringUtils.removeEnd(stagingPath, File.separator);
+            targetStagingPath = stagingPath + File.separator + getJobName();
         }
+        context.put(HiveDRArgs.TARGET_STAGING_PATH, targetStagingPath);
     }
 
     public String getReplicationMaxMaps() {
@@ -151,7 +166,7 @@
     }
 
     public static HiveDROptions create(String[] args) throws ParseException {
-        Map<HiveDRArgs, String> options = new HashMap<HiveDRArgs, String>();
+        Map<HiveDRArgs, String> options = new HashMap<>();
 
         CommandLine cmd = getCommand(args);
         for (HiveDRArgs arg : HiveDRArgs.values()) {
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRTool.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRTool.java
index 17eec22..e45b0d8 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRTool.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/HiveDRTool.java
@@ -136,6 +136,13 @@
         inputOptions = parseOptions(args);
         LOG.info("Input Options: {}", inputOptions);
 
+        // Update the source staging path
+        inputOptions.setSourceStagingPath();
+        inputOptions.setTargetStagingPath();
+
+        LOG.info("srcStaginPath: {}", inputOptions.getSourceStagingPath());
+        LOG.info("tgtStaginPath: {}", inputOptions.getTargetStagingPath());
+
         Configuration sourceConf = FileUtils.getConfiguration(inputOptions.getSourceWriteEP(),
                 inputOptions.getSourceNNKerberosPrincipal());
         sourceClusterFS = FileSystem.get(sourceConf);
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/LastReplicatedEvents.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/LastReplicatedEvents.java
index bae6c9e..a603deb 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/LastReplicatedEvents.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/LastReplicatedEvents.java
@@ -71,8 +71,8 @@
         FileUtils.validatePath(jobFS, new Path(DRStatusStore.BASE_DEFAULT_STORE_PATH));
 
         if (!jobFS.exists(dir)) {
-            if (!jobFS.mkdirs(dir)) {
-                throw new Exception("Creating directory failed: " + dir);
+            if (!FileSystem.mkdirs(jobFS, dir, FileUtils.DEFAULT_DIR_PERMISSION)) {
+                throw new IOException("Creating directory failed: " + dir);
             }
         }
 
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyMapper.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyMapper.java
index 08e0551..5cd7e74 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyMapper.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyMapper.java
@@ -28,10 +28,12 @@
 import org.apache.hadoop.mapreduce.Mapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
 
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Map class for Hive DR.
@@ -40,6 +42,7 @@
 
     private static final Logger LOG = LoggerFactory.getLogger(CopyMapper.class);
     private EventUtils eventUtils;
+    ScheduledThreadPoolExecutor timer;
 
     @Override
     protected void setup(Context context) throws IOException, InterruptedException {
@@ -54,15 +57,22 @@
 
     @Override
     protected void map(LongWritable key, Text value,
-                       Context context) throws IOException, InterruptedException {
+                       final Context context) throws IOException, InterruptedException {
         LOG.debug("Processing Event value: {}", value.toString());
-
+        timer = new ScheduledThreadPoolExecutor(1);
+        timer.scheduleAtFixedRate(new Runnable() {
+            public void run() {
+                System.out.println("Hive DR copy mapper progress heart beat");
+                context.progress();
+            }
+        }, 0, 30, TimeUnit.SECONDS);
         try {
             eventUtils.processEvents(value.toString());
         } catch (Exception e) {
             LOG.error("Exception in processing events:", e);
             throw new IOException(e);
         } finally {
+            timer.shutdownNow();
             cleanup(context);
         }
         List<ReplicationStatus> replicationStatusList = eventUtils.getListReplicationStatus();
@@ -75,7 +85,7 @@
         // In case of export stage, populate custom counters
         if (context.getConfiguration().get(HiveDRArgs.EXECUTION_STAGE.getName())
                 .equalsIgnoreCase(HiveDRUtils.ExecutionStage.EXPORT.name())
-                && !eventUtils.isCountersMapEmtpy()) {
+                && !eventUtils.isCountersMapEmpty()) {
             context.getCounter(ReplicationJobCountersList.BYTESCOPIED).increment(
                     eventUtils.getCounterValue(ReplicationJobCountersList.BYTESCOPIED.getName()));
             context.getCounter(ReplicationJobCountersList.COPY).increment(
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyReducer.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyReducer.java
index 50cb4b2..f4bb31c 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyReducer.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/mapreduce/CopyReducer.java
@@ -35,12 +35,15 @@
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Reducer class for Hive DR.
  */
 public class CopyReducer extends Reducer<Text, Text, Text, Text> {
     private DRStatusStore hiveDRStore;
+    private ScheduledThreadPoolExecutor timer;
 
     @Override
     protected void setup(Context context) throws IOException, InterruptedException {
@@ -62,9 +65,18 @@
     }
 
     @Override
-    protected void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
+    protected void reduce(Text key, Iterable<Text> values, final Context context)
+            throws IOException, InterruptedException {
         List<ReplicationStatus> replStatusList = new ArrayList<ReplicationStatus>();
         ReplicationStatus rs;
+        timer = new ScheduledThreadPoolExecutor(1);
+        timer.scheduleAtFixedRate(new Runnable() {
+            public void run() {
+                System.out.println("Hive DR copy reducer progress heart beat");
+                context.progress();
+            }
+        }, 0, 30, TimeUnit.SECONDS);
+
         try {
             for (Text value : values) {
                 String[] fields = (value.toString()).split("\t");
@@ -76,6 +88,8 @@
             hiveDRStore.updateReplicationStatus(key.toString(), sortStatusList(replStatusList));
         } catch (HiveReplicationException e) {
             throw new IOException(e);
+        } finally {
+            timer.shutdownNow();
         }
     }
 
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventSourcerUtils.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventSourcerUtils.java
index fb695d0..3d3badf 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventSourcerUtils.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventSourcerUtils.java
@@ -63,8 +63,8 @@
         FileUtils.validatePath(jobFS, new Path(DRStatusStore.BASE_DEFAULT_STORE_PATH));
 
         if (!jobFS.exists(dir)) {
-            if (!jobFS.mkdirs(dir)) {
-                throw new Exception("Creating directory failed: " + dir);
+            if (!FileSystem.mkdirs(jobFS, dir, FileUtils.DEFAULT_DIR_PERMISSION)) {
+                throw new IOException("Creating directory failed: " + dir);
             }
         }
 
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventUtils.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventUtils.java
index 3b088f7..590a7e3 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventUtils.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/EventUtils.java
@@ -37,7 +37,6 @@
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.sql.Connection;
@@ -95,17 +94,15 @@
         sourceDatabase = conf.get(HiveDRArgs.SOURCE_DATABASE.getName());
         sourceNN = conf.get(HiveDRArgs.SOURCE_NN.getName());
         sourceNNKerberosPrincipal = conf.get(HiveDRArgs.SOURCE_NN_KERBEROS_PRINCIPAL.getName());
-        sourceStagingPath = conf.get(HiveDRArgs.SOURCE_STAGING_PATH.getName())
-                + File.separator + conf.get(HiveDRArgs.JOB_NAME.getName());
+        sourceStagingPath = conf.get(HiveDRArgs.SOURCE_STAGING_PATH.getName());
         jobNN = conf.get(HiveDRArgs.JOB_CLUSTER_NN.getName());
         jobNNKerberosPrincipal = conf.get(HiveDRArgs.JOB_CLUSTER_NN_KERBEROS_PRINCIPAL.getName());
         targetHiveServer2Uri = conf.get(HiveDRArgs.TARGET_HS2_URI.getName());
-        targetStagingPath = conf.get(HiveDRArgs.TARGET_STAGING_PATH.getName())
-                + File.separator + conf.get(HiveDRArgs.JOB_NAME.getName());
+        targetStagingPath = conf.get(HiveDRArgs.TARGET_STAGING_PATH.getName());
         targetNN = conf.get(HiveDRArgs.TARGET_NN.getName());
         targetNNKerberosPrincipal = conf.get(HiveDRArgs.TARGET_NN_KERBEROS_PRINCIPAL.getName());
-        sourceCleanUpList = new ArrayList<Path>();
-        targetCleanUpList = new ArrayList<Path>();
+        sourceCleanUpList = new ArrayList<>();
+        targetCleanUpList = new ArrayList<>();
         countersMap = new HashMap<>();
     }
 
@@ -169,7 +166,7 @@
     }
 
     public void processEvents(String event) throws Exception {
-        listReplicationStatus = new ArrayList<ReplicationStatus>();
+        listReplicationStatus = new ArrayList<>();
         String[] eventSplit = event.split(DelimiterUtils.FIELD_DELIM);
         String dbName = new String(Base64.decodeBase64(eventSplit[0]), "UTF-8");
         String tableName = new String(Base64.decodeBase64(eventSplit[1]), "UTF-8");
@@ -203,7 +200,7 @@
                                  List<Path> cleanUpList, boolean isImportStatements)
         throws SQLException, HiveReplicationException, IOException {
         String[] commandList = eventStr.split(DelimiterUtils.NEWLINE_DELIM);
-        List<Command> deserializeCommand = new ArrayList<Command>();
+        List<Command> deserializeCommand = new ArrayList<>();
         for (String command : commandList) {
             Command cmd = ReplicationUtils.deserializeCommand(command);
             deserializeCommand.add(cmd);
@@ -269,7 +266,7 @@
     }
 
     private static List<Path> getCleanUpPaths(List<String> cleanupLocations) {
-        List<Path> cleanupLocationPaths = new ArrayList<Path>();
+        List<Path> cleanupLocationPaths = new ArrayList<>();
         for (String cleanupLocation : cleanupLocations) {
             cleanupLocationPaths.add(new Path(cleanupLocation));
         }
@@ -330,7 +327,7 @@
 
     public DistCpOptions getDistCpOptions() {
         // DistCpOptions expects the first argument to be a file OR a list of Paths
-        List<Path> sourceUris=new ArrayList<Path>();
+        List<Path> sourceUris=new ArrayList<>();
         sourceUris.add(new Path(sourceStagingUri));
         DistCpOptions distcpOptions = new DistCpOptions(sourceUris, new Path(targetStagingUri));
 
@@ -350,8 +347,8 @@
         return countersMap.get(counterKey);
     }
 
-    public boolean isCountersMapEmtpy() {
-        return countersMap.size() == 0 ? true : false;
+    public boolean isCountersMapEmpty() {
+        return countersMap.size() == 0;
     }
 
     public void cleanEventsDirectory() throws IOException {
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/FileUtils.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/FileUtils.java
index 001d10a..8b5c865 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/FileUtils.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/FileUtils.java
@@ -33,9 +33,11 @@
  */
 public final class FileUtils {
 
-    public static final String DEFAULT_EVENT_STORE_PATH = DRStatusStore.BASE_DEFAULT_STORE_PATH
-            + File.separator + "Events";
+    public static final String DEFAULT_EVENT_STORE_PATH = StringUtils.removeEnd(DRStatusStore
+            .BASE_DEFAULT_STORE_PATH,  File.separator) + File.separator + "Events" + File.separator;
     public static final FsPermission FS_PERMISSION_700 = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE);
+    public static final FsPermission DEFAULT_DIR_PERMISSION =
+            new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.NONE);
 
 
     private FileUtils() {}
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRStatusStore.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRStatusStore.java
index 900afe8..44f0989 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRStatusStore.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRStatusStore.java
@@ -32,6 +32,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -48,7 +49,10 @@
     private static final Logger LOG = LoggerFactory.getLogger(DRStatusStore.class);
     private FileSystem fileSystem;
 
-    private static final String DEFAULT_STORE_PATH = BASE_DEFAULT_STORE_PATH + "hiveReplicationStatusStore/";
+    private static final String DEFAULT_STORE_PATH = StringUtils.removeEnd
+            (DRStatusStore.BASE_DEFAULT_STORE_PATH,  File.separator) + File.separator
+            + "hiveReplicationStatusStore" + File.separator;
+
     private static final FsPermission DEFAULT_STATUS_DIR_PERMISSION =
             new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.NONE);
 
@@ -71,6 +75,8 @@
         Path basePath = new Path(BASE_DEFAULT_STORE_PATH);
         FileUtils.validatePath(fileSystem, basePath);
 
+        // Current limitation is that only users who belong to DRStatusStore.storeGroup can submit HiveDR jobs.
+        // BaseDir for status store is created with permissions 770 so that all eligible users can access statusStore.
         Path storePath = new Path(DEFAULT_STORE_PATH);
         if (!fileSystem.exists(storePath)) {
             if (!FileSystem.mkdirs(fileSystem, storePath, DEFAULT_STORE_PERMISSION)) {
@@ -163,10 +169,11 @@
     private DBReplicationStatus getDbReplicationStatus(String source, String target, String jobName,
                                                        String database) throws HiveReplicationException{
         DBReplicationStatus dbReplicationStatus = null;
+        Path statusDbDirPath = getStatusDbDirPath(database);
         Path statusDirPath = getStatusDirPath(database, jobName);
+
         // check if database name or jobName can contain chars not allowed by hdfs dir/file naming.
         // if yes, use md5 of the same for dir names. prefer to use actual db names for readability.
-
         try {
             if (fileSystem.exists(statusDirPath)) {
                 dbReplicationStatus = readStatusFile(statusDirPath);
@@ -176,6 +183,15 @@
                 ReplicationStatus initDbStatus = new ReplicationStatus(source, target, jobName,
                         database, null, ReplicationStatus.Status.INIT, -1);
                 dbReplicationStatus = new DBReplicationStatus(initDbStatus);
+
+                // Create parent dir first with default status store permissions. FALCON-2057
+                if (!fileSystem.exists(statusDbDirPath)) {
+                    if (!FileSystem.mkdirs(fileSystem, statusDbDirPath, DEFAULT_STATUS_DIR_PERMISSION)) {
+                        String error = "mkdir failed for " + statusDbDirPath.toString();
+                        LOG.error(error);
+                        throw new HiveReplicationException(error);
+                    }
+                }
                 if (!FileSystem.mkdirs(fileSystem, statusDirPath, DEFAULT_STATUS_DIR_PERMISSION)) {
                     String error = "mkdir failed for " + statusDirPath.toString();
                     LOG.error(error);
@@ -197,7 +213,11 @@
     }
 
     public Path getStatusDirPath(String database, String jobName) {
-        return new Path(DEFAULT_STORE_PATH + "/" + database.toLowerCase() + "/" + jobName);
+        return new Path(getStatusDbDirPath(database), jobName);
+    }
+
+    public Path getStatusDbDirPath(String dbName) {
+        return new Path(new Path(BASE_DEFAULT_STORE_PATH), dbName.toLowerCase());
     }
 
     private void writeStatusFile(DBReplicationStatus dbReplicationStatus) throws HiveReplicationException {
@@ -271,7 +291,7 @@
     public void checkForReplicationConflict(String newSource, String jobName,
                                              String database, String table) throws HiveReplicationException {
         try {
-            Path globPath = new Path(DEFAULT_STORE_PATH + "/" + database.toLowerCase() + "/*/latest.json");
+            Path globPath = new Path(getStatusDbDirPath(database), "*" + File.separator + "latest.json");
             FileStatus[] files = fileSystem.globStatus(globPath);
             for(FileStatus file : files) {
                 DBReplicationStatus dbFileStatus = new DBReplicationStatus(IOUtils.toString(
diff --git a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRUtils.java b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRUtils.java
index d5d3bc5..b21acc7 100644
--- a/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRUtils.java
+++ b/addons/hivedr/src/main/java/org/apache/falcon/hive/util/HiveDRUtils.java
@@ -70,13 +70,16 @@
 
     public static Configuration getDefaultConf() throws IOException {
         Configuration conf = new Configuration();
-        Path confPath = new Path("file:///", System.getProperty("oozie.action.conf.xml"));
 
-        final boolean actionConfExists = confPath.getFileSystem(conf).exists(confPath);
-        LOG.info("Oozie Action conf {} found ? {}", confPath, actionConfExists);
-        if (actionConfExists) {
-            LOG.info("Oozie Action conf found, adding path={}, conf={}", confPath, conf.toString());
-            conf.addResource(confPath);
+        if (System.getProperty("oozie.action.conf.xml") != null) {
+            Path confPath = new Path("file:///", System.getProperty("oozie.action.conf.xml"));
+
+            final boolean actionConfExists = confPath.getFileSystem(conf).exists(confPath);
+            LOG.info("Oozie Action conf {} found ? {}", confPath, actionConfExists);
+            if (actionConfExists) {
+                LOG.info("Oozie Action conf found, adding path={}, conf={}", confPath, conf.toString());
+                conf.addResource(confPath);
+            }
         }
 
         String tokenFile = System.getenv("HADOOP_TOKEN_FILE_LOCATION");
diff --git a/addons/hivedr/src/test/java/org/apache/falcon/hive/DRTest.java b/addons/hivedr/src/test/java/org/apache/falcon/hive/DRTest.java
index 1f44b62..a9c5661 100644
--- a/addons/hivedr/src/test/java/org/apache/falcon/hive/DRTest.java
+++ b/addons/hivedr/src/test/java/org/apache/falcon/hive/DRTest.java
@@ -25,8 +25,8 @@
     public void testHiveDr(String[] args) {
         String[] testArgs = {
             "-sourceMetastoreUri", "thrift://localhost:9083",
-            "-sourceDatabase", "default",
-            "-sourceTable", "test",
+            "-sourceDatabases", "default",
+            "-sourceTables", "test",
             "-sourceStagingPath", "/apps/hive/tools/dr",
             "-sourceNN", "hdfs://localhost:8020",
             "-sourceRM", "local",
diff --git a/archival/pom.xml b/archival/pom.xml
index b117d9d..dcace9d 100644
--- a/archival/pom.xml
+++ b/archival/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-archival</artifactId>
     <description>Apache Falcon Archival Module</description>
diff --git a/build-tools/pom.xml b/build-tools/pom.xml
index 6c8801e..8abdef6 100644
--- a/build-tools/pom.xml
+++ b/build-tools/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <groupId>org.apache.falcon</groupId>
     <artifactId>build-tools</artifactId>
diff --git a/cli/pom.xml b/cli/pom.xml
index e77cb46..8acb905 100644
--- a/cli/pom.xml
+++ b/cli/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-cli</artifactId>
     <description>Apache Falcon CLI client</description>
@@ -39,7 +39,6 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-auth</artifactId>
-            <scope>compile</scope>
         </dependency>
 
         <dependency>
diff --git a/client/pom.xml b/client/pom.xml
index 8bc77fa..3eb0a8b 100644
--- a/client/pom.xml
+++ b/client/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-client</artifactId>
     <description>Apache Falcon Java client</description>
@@ -117,6 +117,7 @@
                             <goal>copy-dependencies</goal>
                         </goals>
                         <configuration>
+                            <excludeScope>provided</excludeScope>
                             <outputDirectory>${project.build.directory}/dependency</outputDirectory>
                         </configuration>
                     </execution>
diff --git a/common/pom.xml b/common/pom.xml
index debb615..846202c 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-common</artifactId>
     <description>Apache Falcon Common Module</description>
diff --git a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
index 9e16fa4..f89def3 100644
--- a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
@@ -75,9 +75,8 @@
         return conf;
     }
 
-    public static Configuration getConfiguration(String storageUrl, String executeEndPoint,
-                                                 String kerberosPrincipal) {
-        Configuration conf = new Configuration();
+    public static Configuration getConfiguration(Configuration conf, String storageUrl,
+                                                 String executeEndPoint, String kerberosPrincipal) {
         conf.set(HadoopClientFactory.FS_DEFAULT_NAME_KEY, storageUrl);
         conf.set(HadoopClientFactory.MR_JT_ADDRESS_KEY, executeEndPoint);
         conf.set(HadoopClientFactory.YARN_RM_ADDRESS_KEY, executeEndPoint);
diff --git a/common/src/main/java/org/apache/falcon/entity/FileSystemStorage.java b/common/src/main/java/org/apache/falcon/entity/FileSystemStorage.java
index ece8b5d..eb15585 100644
--- a/common/src/main/java/org/apache/falcon/entity/FileSystemStorage.java
+++ b/common/src/main/java/org/apache/falcon/entity/FileSystemStorage.java
@@ -358,13 +358,20 @@
 
     private FileStatus[] findFilesForFeed(FileSystem fs, String feedBasePath) throws IOException {
         Matcher matcher = FeedDataPath.PATTERN.matcher(feedBasePath);
+        boolean regexMatchFound = false;
         while (matcher.find()) {
+            regexMatchFound = true;
             String var = feedBasePath.substring(matcher.start(), matcher.end());
             feedBasePath = feedBasePath.replaceAll(Pattern.quote(var), "*");
             matcher = FeedDataPath.PATTERN.matcher(feedBasePath);
         }
-        LOG.info("Searching for {}", feedBasePath);
-        return fs.globStatus(new Path(feedBasePath));
+        if (regexMatchFound) {
+            LOG.info("Searching for {}", feedBasePath);
+            return fs.globStatus(new Path(feedBasePath));
+        } else {
+            LOG.info("Ignoring static path {}", feedBasePath);
+            return null;
+        }
     }
 
     private boolean isDateInRange(Date date, Date start) {
diff --git a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
index f0d6073..c58be64 100644
--- a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
+++ b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
@@ -34,9 +34,6 @@
 public class WorkflowNameBuilder<T extends Entity> {
     private static final String PREFIX = "FALCON";
 
-    // Oozie JMS message property name that holds the workflow app name
-    private static final String OOZIE_JMS_MSG_APPNAME_PROP = "appName";
-
     private T entity;
     private Tag tag;
     private List<String> suffixes;
@@ -156,9 +153,5 @@
             }
             return null;
         }
-
-        public static String getJMSFalconSelector() {
-            return String.format("%s like '%s%s%%'", OOZIE_JMS_MSG_APPNAME_PROP, PREFIX, SEPARATOR);
-        }
     }
 }
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/ClusterEntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/ClusterEntityParser.java
index 96ba748..3f35962 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/ClusterEntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/ClusterEntityParser.java
@@ -45,8 +45,6 @@
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -236,14 +234,13 @@
     }
 
     protected void validateSparkMasterInterface(Cluster cluster) throws ValidationException {
-        final String sparkMasterUrl = ClusterHelper.getSparkMasterEndPoint(cluster);
-        if (StringUtils.isNotEmpty(sparkMasterUrl)) {
-            SparkConf sparkConf = new SparkConf();
-            sparkConf.setMaster(sparkMasterUrl).setAppName("Falcon Spark");
-
-            JavaSparkContext sc = new JavaSparkContext(sparkConf);
-            if (sc.startTime() == null) {
-                throw new ValidationException("Unable to reach Spark master URL:" + sparkMasterUrl);
+        final String sparkMasterEndPoint = ClusterHelper.getSparkMasterEndPoint(cluster);
+        LOG.info("Validating spark interface: {}", sparkMasterEndPoint);
+        if (StringUtils.isNotEmpty(sparkMasterEndPoint)) {
+            if (!("yarn-cluster".equalsIgnoreCase(sparkMasterEndPoint)
+                    || "yarn-client".equalsIgnoreCase(sparkMasterEndPoint)
+                    || "local".equalsIgnoreCase(sparkMasterEndPoint))) {
+                throw new ValidationException("Invalid Spark Interface End Point:" + sparkMasterEndPoint);
             }
         }
     }
diff --git a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
index 7f2b172..52feab7 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
@@ -69,6 +69,10 @@
     private static final Logger LOG = LoggerFactory.getLogger(ConfigurationStore.class);
     private static final Logger AUDIT = LoggerFactory.getLogger("AUDIT");
     private static final String UTF_8 = CharEncoding.UTF_8;
+    private static final String LOAD_ENTITIES_THREADS = "config.store.num.threads.load.entities";
+    private static final String TIMEOUT_MINS_LOAD_ENTITIES = "config.store.start.timeout.minutes";
+    private int numThreads;
+    private int restoreTimeOutInMins;
     private final boolean shouldPersist;
 
     private static final FsPermission STORE_PERMISSION =
@@ -149,6 +153,21 @@
 
     @Override
     public void init() throws FalconException {
+        try {
+            numThreads = Integer.parseInt(StartupProperties.get().getProperty(LOAD_ENTITIES_THREADS, "100"));
+            LOG.info("Number of threads used to restore entities: {}", restoreTimeOutInMins);
+        } catch (NumberFormatException nfe) {
+            throw new FalconException("Invalid value specified for start up property \""
+                    + LOAD_ENTITIES_THREADS + "\".Please provide an integer value");
+        }
+        try {
+            restoreTimeOutInMins = Integer.parseInt(StartupProperties.get().
+                    getProperty(TIMEOUT_MINS_LOAD_ENTITIES, "30"));
+            LOG.info("TimeOut to load Entities is taken as {} mins", restoreTimeOutInMins);
+        } catch (NumberFormatException nfe) {
+            throw new FalconException("Invalid value specified for start up property \""
+                    + TIMEOUT_MINS_LOAD_ENTITIES + "\".Please provide an integer value");
+        }
         String listenerClassNames = StartupProperties.get().
                 getProperty("configstore.listeners", "org.apache.falcon.entity.v0.EntityGraph");
         for (String listenerClassName : listenerClassNames.split(",")) {
@@ -172,7 +191,8 @@
             final ConcurrentHashMap<String, Entity> entityMap = dictionary.get(type);
             FileStatus[] files = fs.globStatus(new Path(storePath, type.name() + Path.SEPARATOR + "*"));
             if (files != null) {
-                final ExecutorService service = Executors.newFixedThreadPool(100);
+
+                final ExecutorService service = Executors.newFixedThreadPool(numThreads);
                 for (final FileStatus file : files) {
                     service.execute(new Runnable() {
                         @Override
@@ -183,6 +203,7 @@
                                 // ".xml"
                                 String entityName = URLDecoder.decode(encodedEntityName, UTF_8);
                                 Entity entity = restore(type, entityName);
+                                LOG.info("Restored configuration {}/{}", type, entityName);
                                 entityMap.put(entityName, entity);
                             } catch (IOException | FalconException e) {
                                 LOG.error("Unable to restore entity of", file);
@@ -191,10 +212,10 @@
                     });
                 }
                 service.shutdown();
-                if (service.awaitTermination(10, TimeUnit.MINUTES)) {
+                if (service.awaitTermination(restoreTimeOutInMins, TimeUnit.MINUTES)) {
                     LOG.info("Restored Configurations for entity type: {} ", type.name());
                 } else {
-                    LOG.warn("Time out happened while waiting for all threads to finish while restoring entities "
+                    LOG.warn("Timed out while waiting for all threads to finish while restoring entities "
                             + "for type: {}", type.name());
                 }
                 // Checking if all entities were loaded
@@ -336,6 +357,7 @@
                 } catch (IOException e) {
                     throw new StoreAccessException(e);
                 }
+                LOG.info("Restored configuration {}/{}", type, name);
                 entityMap.put(name, entity);
                 return entity;
             } else {
@@ -445,7 +467,6 @@
             throw new StoreAccessException("Unable to un-marshall xml definition for " + type + "/" + name, e);
         } finally {
             in.close();
-            LOG.info("Restored configuration {}/{}", type, name);
         }
     }
 
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
index e4d9385..acb570e 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
@@ -200,6 +200,16 @@
                 feedEdges.add(dbNode);
                 dbEdges.add(feedNode);
             }
+
+            if (FeedHelper.isExportEnabled(cluster)) {
+                Node dbNode = new Node(EntityType.DATASOURCE, FeedHelper.getExportDatasourceName(cluster));
+                if (!nodeEdges.containsKey(dbNode)) {
+                    nodeEdges.put(dbNode, new HashSet<Node>());
+                }
+                Set<Node> dbEdges = nodeEdges.get(dbNode);
+                feedEdges.add(dbNode);
+                dbEdges.add(feedNode);
+            }
         }
         return nodeEdges;
     }
diff --git a/common/src/main/java/org/apache/falcon/hadoop/HadoopClientFactory.java b/common/src/main/java/org/apache/falcon/hadoop/HadoopClientFactory.java
index d70c4b9..e30f51e 100644
--- a/common/src/main/java/org/apache/falcon/hadoop/HadoopClientFactory.java
+++ b/common/src/main/java/org/apache/falcon/hadoop/HadoopClientFactory.java
@@ -262,7 +262,9 @@
 
         try {
             if (UserGroupInformation.isSecurityEnabled()) {
-                ugi.checkTGTAndReloginFromKeytab();
+                LOG.debug("Revalidating Auth Token with auth method {}",
+                        UserGroupInformation.getLoginUser().getAuthenticationMethod().name());
+                UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
             }
         } catch (IOException ioe) {
             throw new FalconException("Exception while getting FileSystem. Unable to check TGT for user "
diff --git a/common/src/main/java/org/apache/falcon/metadata/GraphUpdateUtils.java b/common/src/main/java/org/apache/falcon/metadata/GraphUpdateUtils.java
new file mode 100644
index 0000000..a256e46
--- /dev/null
+++ b/common/src/main/java/org/apache/falcon/metadata/GraphUpdateUtils.java
@@ -0,0 +1,113 @@
+/**
+ * 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.falcon.metadata;
+
+import com.tinkerpop.blueprints.Graph;
+import com.tinkerpop.blueprints.util.io.graphson.GraphSONReader;
+import com.tinkerpop.blueprints.util.io.graphson.GraphSONWriter;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.io.FileUtils;
+import org.apache.falcon.FalconException;
+
+import java.io.File;
+
+/**
+ * Utility class for graph operations.
+ */
+public final class GraphUpdateUtils {
+
+    private static final String BANNER_MSG =
+            "Before running this utility please make sure that Falcon startup properties "
+                    + "has the right configuration settings for the graph database, "
+                    + "Falcon server is stopped and no other access to the graph database is being performed.";
+
+    private static final String IMPORT = "import";
+    private static final String EXPORT = "export";
+    private static final String INSTANCE_JSON_FILE = "instanceMetadata.json";
+
+    private GraphUpdateUtils() {
+    }
+
+    public static void main(String[] args) {
+        if (args.length != 2) {
+            usage();
+            System.exit(1);
+        }
+        System.out.println(BANNER_MSG);
+        String operation = args[0].toLowerCase();
+        if (!(operation.equals(EXPORT) || operation.equals(IMPORT))) {
+            usage();
+            System.exit(1);
+        }
+        String utilsDir = args[1];
+        File utilsDirFile = new File(utilsDir);
+        if (!utilsDirFile.isDirectory()) {
+            System.err.println(utilsDir + " is not a valid directory");
+            System.exit(1);
+        }
+        String jsonFile = new File(utilsDirFile, INSTANCE_JSON_FILE).getAbsolutePath();
+        try {
+            Graph graph;
+            if (operation.equals(EXPORT)) {
+                graph = MetadataMappingService.initializeGraphDB();
+                GraphSONWriter.outputGraph(graph, jsonFile);
+                System.out.println("Exported instance metadata to " + jsonFile);
+            } else {
+                // Backup existing graphDB dir
+                Configuration graphConfig = MetadataMappingService.getConfiguration();
+                String graphStore = (String) graphConfig.getProperty("storage.directory");
+                File graphStoreFile = new File(graphStore);
+                File graphDirBackup = new File(graphStore + "_backup");
+                if (graphDirBackup.exists()) {
+                    FileUtils.deleteDirectory(graphDirBackup);
+                }
+                FileUtils.copyDirectory(graphStoreFile, graphDirBackup);
+
+                // delete graph dir first and then init graphDB to ensure IMPORT happens into empty DB.
+                FileUtils.deleteDirectory(graphStoreFile);
+                graph = MetadataMappingService.initializeGraphDB();
+
+                // Import, if there is an exception restore backup.
+                try {
+                    GraphSONReader.inputGraph(graph, jsonFile);
+                    System.out.println("Imported instance metadata to " + jsonFile);
+                } catch (Exception ex) {
+                    String errorMsg = ex.getMessage();
+                    if (graphStoreFile.exists()) {
+                        FileUtils.deleteDirectory(graphStoreFile);
+                    }
+                    FileUtils.copyDirectory(graphDirBackup, graphStoreFile);
+                    throw new FalconException(errorMsg);
+                }
+            }
+        } catch (Exception e) {
+            System.err.println("Error " + operation + "ing JSON data to " + jsonFile + ", " + e.getMessage());
+            e.printStackTrace(System.out);
+            System.exit(1);
+        }
+        System.exit(0);
+    }
+
+    public static void usage() {
+        StringBuilder usageMessage = new StringBuilder(1024);
+        usageMessage.append("usage: java ").append(GraphUpdateUtils.class.getName())
+                .append(" {").append(EXPORT).append('|').append(IMPORT).append("} <directory>");
+        System.err.println(usageMessage);
+    }
+}
diff --git a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
index 66a3a58..225e44a 100644
--- a/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
+++ b/common/src/main/java/org/apache/falcon/metadata/MetadataMappingService.java
@@ -35,6 +35,7 @@
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.falcon.FalconException;
+import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
@@ -48,6 +49,9 @@
 import org.apache.falcon.workflow.WorkflowExecutionContext;
 import org.apache.falcon.workflow.WorkflowExecutionListener;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -69,7 +73,26 @@
      * Constant for the configuration property that indicates the prefix.
      */
     private static final String FALCON_PREFIX = "falcon.graph.";
-
+    /**
+     * Constant for the configuration property that indicates the storage backend.
+     */
+    public static final String PROPERTY_KEY_STORAGE_BACKEND = "storage.backend";
+    public static final String STORAGE_BACKEND_HBASE = "hbase";
+    public static final String STORAGE_BACKEND_BDB = "berkeleyje";
+    /**
+     * HBase configuration properties.
+     */
+    public static final String PROPERTY_KEY_STORAGE_HOSTNAME = "storage.hostname";
+    public static final String PROPERTY_KEY_STORAGE_TABLE = "storage.hbase.table";
+    public static final Set<String> PROPERTY_KEYS_HBASE = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            PROPERTY_KEY_STORAGE_HOSTNAME, PROPERTY_KEY_STORAGE_TABLE)));
+    /**
+     * Berkeley DB configuration properties.
+     */
+    public static final String PROPERTY_KEY_STORAGE_DIRECTORY = "storage.directory";
+    public static final String PROPERTY_KEY_SERIALIZE_PATH = "serialize.path";
+    public static final Set<String> PROPERTY_KEYS_BDB = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            PROPERTY_KEY_STORAGE_DIRECTORY, PROPERTY_KEY_SERIALIZE_PATH)));
 
     private Graph graph;
     private Set<String> vertexIndexedKeys;
@@ -116,13 +139,58 @@
         }
     }
 
-    protected Graph initializeGraphDB() {
+    public static Graph initializeGraphDB() {
         LOG.info("Initializing graph db");
-
         Configuration graphConfig = getConfiguration();
+        validateConfiguration(graphConfig);
         return GraphFactory.open(graphConfig);
     }
 
+    private static void validateConfiguration(Configuration graphConfig) {
+        // check if storage backend if configured
+        if (!graphConfig.containsKey(PROPERTY_KEY_STORAGE_BACKEND)) {
+            throw new FalconRuntimException("Titan GraphDB storage backend is not configured. "
+                    + "You need to choose either hbase or berkeleydb."
+                    + "Please check Configuration twiki or "
+                    + "the section Graph Database Properties in startup.properties "
+                    + "on how to configure Titan GraphDB backend.");
+        }
+
+        String backend = graphConfig.getString(PROPERTY_KEY_STORAGE_BACKEND);
+        switch (backend) {
+        case STORAGE_BACKEND_BDB:
+            // check required parameter for Berkeley DB backend
+            for (String key : PROPERTY_KEYS_BDB) {
+                if (!graphConfig.containsKey(key)) {
+                    throw new FalconRuntimException("Required parameter " + FALCON_PREFIX + key
+                            + " not found in startup.properties."
+                            + "Please check Configuration twiki or "
+                            + "the section Graph Database Properties in startup.properties "
+                            + "on how to configure Berkeley DB storage backend.");
+                }
+            }
+            break;
+        case STORAGE_BACKEND_HBASE:
+            // check required parameter for HBase backend
+            for (String key : PROPERTY_KEYS_HBASE) {
+                if (!graphConfig.containsKey(key)) {
+                    throw new FalconRuntimException("Required parameter " + FALCON_PREFIX + key
+                            + " not found in startup.properties."
+                            + "Please check Configuration twiki or "
+                            + "the section Graph Database Properties in startup.properties "
+                            + "on how to configure HBase storage backend.");
+                }
+            }
+            break;
+        default:
+            throw new FalconRuntimException("Invalid graph storage backend: " + backend + ". "
+                    + "You need to choose either hbase or berkeleydb."
+                    + "Please check Configuration twiki or "
+                    + "the section Graph Database Properties in startup.properties "
+                    + "on how to configure Titan GraphDB backend.");
+        }
+    }
+
     public static Configuration getConfiguration() {
         Configuration graphConfig = new BaseConfiguration();
 
diff --git a/common/src/main/java/org/apache/falcon/security/AuthenticationInitializationService.java b/common/src/main/java/org/apache/falcon/security/AuthenticationInitializationService.java
index f7b2155..31be07a 100644
--- a/common/src/main/java/org/apache/falcon/security/AuthenticationInitializationService.java
+++ b/common/src/main/java/org/apache/falcon/security/AuthenticationInitializationService.java
@@ -67,6 +67,7 @@
 
     private Timer timer = new Timer();
     private static final String SERVICE_NAME = "Authentication initialization service";
+    private static final long DEFAULT_VALIDATE_FREQUENCY_SECS = 86300;
 
     @Override
     public String getName() {
@@ -83,8 +84,13 @@
             String authTokenValidity = StartupProperties.get().getProperty(AUTH_TOKEN_VALIDITY_SECONDS);
             long validateFrequency;
             try {
+                // -100 so that revalidation is done before expiry.
                 validateFrequency = (StringUtils.isNotEmpty(authTokenValidity))
-                        ? Long.parseLong(authTokenValidity) : 86400;
+                        ? (Long.parseLong(authTokenValidity) - 100) : DEFAULT_VALIDATE_FREQUENCY_SECS;
+                if (validateFrequency < 0) {
+                    throw new NumberFormatException("Value provided for startup property \""
+                            + AUTH_TOKEN_VALIDITY_SECONDS + "\" should be greater than 100.");
+                }
             } catch (NumberFormatException nfe) {
                 throw new FalconException("Invalid value provided for startup property \""
                         + AUTH_TOKEN_VALIDITY_SECONDS + "\", please provide a valid long number", nfe);
@@ -149,12 +155,12 @@
         @Override
         public void run() {
             try {
-                LOG.info("Validating Auth Token: {}", new Date());
-                initializeKerberos();
+                LOG.debug("Revalidating Auth Token at : {} with auth method {}", new Date(),
+                        UserGroupInformation.getLoginUser().getAuthenticationMethod().name());
+                UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
             } catch (Throwable t) {
-                LOG.error("Error in Auth Token Validation task: ", t);
-                GenericAlert.initializeKerberosFailed(
-                        "Exception in Auth Token Validation : ", t);
+                LOG.error("Error in Auth Token revalidation task: ", t);
+                GenericAlert.initializeKerberosFailed("Exception in Auth Token revalidation : ", t);
             }
         }
     }
diff --git a/common/src/main/java/org/apache/falcon/security/DefaultAuthorizationProvider.java b/common/src/main/java/org/apache/falcon/security/DefaultAuthorizationProvider.java
index 5eb6a25..a4410af 100644
--- a/common/src/main/java/org/apache/falcon/security/DefaultAuthorizationProvider.java
+++ b/common/src/main/java/org/apache/falcon/security/DefaultAuthorizationProvider.java
@@ -59,6 +59,7 @@
 
     private static final Set<String> RESOURCES = new HashSet<String>(
             Arrays.asList(new String[]{"admin", "entities", "instance", "metadata", "extension", }));
+    private static final String LIST_OPERATION = "list";
 
     /**
      * Constant for the configuration property that indicates the prefix.
@@ -170,7 +171,11 @@
                     authorizeAdminResource(authenticatedUGI, action);
                 }
             } else if ("entities".equals(resource) || "instance".equals(resource)) {
-                authorizeEntityResource(authenticatedUGI, entityName, entityType, action);
+                if ("entities".equals(resource) && LIST_OPERATION.equals(action)) {
+                    LOG.info("Skipping authorization for entity list operations");
+                } else {
+                    authorizeEntityResource(authenticatedUGI, entityName, entityType, action);
+                }
             } else if ("metadata".equals(resource)) {
                 authorizeMetadataResource(authenticatedUGI, action);
             }
@@ -296,7 +301,6 @@
                                            String entityName, String entityType,
                                            String action)
         throws AuthorizationException, EntityNotRegisteredException {
-
         Validate.notEmpty(entityType, "Entity type cannot be empty or null");
         LOG.debug("Authorizing authenticatedUser={} against entity/instance action={}, "
                 + "entity name={}, entity type={}",
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
index 9b1e1f4..9b011b8 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowExecutionContext.java
@@ -304,11 +304,11 @@
     }
 
     public int getWorkflowRunId() {
-        return Integer.parseInt(getValue(WorkflowExecutionArgs.RUN_ID));
+        return Integer.parseInt(getValue(WorkflowExecutionArgs.RUN_ID, "0"));
     }
 
     public String getWorkflowRunIdString() {
-        return String.valueOf(Integer.parseInt(getValue(WorkflowExecutionArgs.RUN_ID)));
+        return String.valueOf(Integer.parseInt(getValue(WorkflowExecutionArgs.RUN_ID, "0")));
     }
 
     public String getWorkflowUser() {
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
index b692258..6d1332e 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowJobEndNotificationService.java
@@ -155,6 +155,7 @@
     private boolean updateContextFromWFConf(WorkflowExecutionContext context) throws FalconException {
         Properties wfProps = contextMap.get(context.getWorkflowId());
         if (wfProps == null) {
+            wfProps = new Properties();
             Entity entity = null;
             try {
                 entity = EntityUtil.getEntity(context.getEntityType(), context.getEntityName());
@@ -166,11 +167,12 @@
                 return false;
             }
             for (String cluster : EntityUtil.getClustersDefinedInColos(entity)) {
+                wfProps.setProperty(WorkflowExecutionArgs.CLUSTER_NAME.getName(), cluster);
                 try {
                     InstancesResult.Instance[] instances = WorkflowEngineFactory.getWorkflowEngine(entity)
                             .getJobDetails(cluster, context.getWorkflowId()).getInstances();
                     if (instances != null && instances.length > 0) {
-                        wfProps = getWFProps(instances[0].getWfParams());
+                        wfProps.putAll(getWFProps(instances[0].getWfParams()));
                         // Required by RetryService. But, is not part of conf.
                         wfProps.setProperty(WorkflowExecutionArgs.RUN_ID.getName(),
                                 Integer.toString(instances[0].getRunId()));
@@ -299,7 +301,7 @@
         }
         Long duration = (endTime.getTime() - startTime.getTime()) * 1000000;
 
-        if (context.hasWorkflowFailed()) {
+        if (!context.hasWorkflowSucceeded()) {
             GenericAlert.instrumentFailedInstance(clusterName, entityType,
                     entityName, nominalTime, workflowId, workflowUser, runId, operation,
                     SchemaHelper.formatDateUTC(startTime), "", "", duration);
diff --git a/common/src/main/resources/startup.properties b/common/src/main/resources/startup.properties
index 2229edf..0990035 100644
--- a/common/src/main/resources/startup.properties
+++ b/common/src/main/resources/startup.properties
@@ -34,6 +34,7 @@
 *.application.services=org.apache.falcon.security.AuthenticationInitializationService,\
                         org.apache.falcon.workflow.WorkflowJobEndNotificationService, \
                         org.apache.falcon.service.ProcessSubscriberService,\
+                        org.apache.falcon.extensions.ExtensionService,\
                         org.apache.falcon.service.FeedSLAMonitoringService,\
                         org.apache.falcon.service.LifecyclePolicyMap,\
                         org.apache.falcon.entity.store.ConfigurationStore,\
@@ -43,8 +44,7 @@
                         org.apache.falcon.service.LogCleanupService,\
                         org.apache.falcon.service.GroupsService,\
                         org.apache.falcon.service.ProxyUserService,\
-                        org.apache.falcon.service.FalconJPAService,\
-                        org.apache.falcon.extensions.ExtensionService
+                        org.apache.falcon.service.FalconJPAService
 ##Add if you want to send data to graphite
 #                        org.apache.falcon.metrics.MetricNotificationService\
 ## Add if you want to use Falcon Azure integration ##
@@ -156,9 +156,20 @@
 *.falcon.graph.blueprints.graph=com.thinkaurelius.titan.core.TitanFactory
 
 # Graph Storage
-*.falcon.graph.storage.directory=${user.dir}/target/graphdb
-*.falcon.graph.storage.backend=berkeleyje
-*.falcon.graph.serialize.path=${user.dir}/target/graphdb
+# IMPORTANT:   Please enable one of the graph db backend: hbase or berkeleydb, per instructions below.
+
+# Enable the following for Berkeley DB.  Make sure je-5.0.73.jar is downloaded and available
+# under Falcon webapp directory or under falcon server classpath.
+#*.falcon.graph.storage.backend=berkeleyje
+#*.falcon.graph.storage.directory=/${falcon.home}/data/graphdb
+#*.falcon.graph.serialize.path=${user.dir}/target/graphdb
+
+# Enable the following for HBase
+#*.falcon.graph.storage.backend=hbase
+# For standalone mode , set hostname to localhost; for distributed mode, set to the zookeeper quorum
+# @see http://s3.thinkaurelius.com/docs/titan/current/hbase.html#_remote_server_mode_2
+#*.falcon.graph.storage.hostname=localhost
+#*.falcon.graph.storage.hbase.table=falcon_titan
 
 # Avoid acquiring read lock when iterating over large graphs
 # See http://s3.thinkaurelius.com/docs/titan/0.5.4/bdb.html
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
index c642fb8..f9aad19 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
@@ -977,7 +977,8 @@
         }
     }
 
-    @Test
+    // disable this test due to its validation of dummy s3 url no longer supported by latest hdfs (2.7.2 or above)
+    @Test (enabled = false)
     public void testValidateACLForArchiveReplication() throws Exception {
         StartupProperties.get().setProperty("falcon.security.authorization.enabled", "true");
         Assert.assertTrue(Boolean.valueOf(
diff --git a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
index 23f69d7..b41cc03 100644
--- a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
@@ -20,17 +20,22 @@
 
 import org.apache.falcon.entity.AbstractTestBase;
 import org.apache.falcon.entity.store.ConfigurationStore;
+import org.apache.falcon.entity.v0.feed.Feed;
+import org.apache.falcon.entity.v0.feed.Load;
 import org.apache.falcon.entity.v0.feed.Argument;
 import org.apache.falcon.entity.v0.feed.Arguments;
 import org.apache.falcon.entity.v0.feed.Clusters;
 import org.apache.falcon.entity.v0.feed.ClusterType;
 import org.apache.falcon.entity.v0.feed.Extract;
 import org.apache.falcon.entity.v0.feed.ExtractMethod;
-import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.feed.FieldsType;
 import org.apache.falcon.entity.v0.feed.FieldIncludeExclude;
 import org.apache.falcon.entity.v0.feed.Import;
 import org.apache.falcon.entity.v0.feed.MergeType;
+import org.apache.falcon.entity.v0.feed.Export;
+import org.apache.falcon.entity.v0.feed.LoadMethod;
+
+
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.datasource.Datasource;
 import org.apache.falcon.entity.v0.process.Input;
@@ -184,6 +189,36 @@
         return imp;
     }
 
+    private Feed addFeedExport(String feed, Cluster cluster, Datasource ds) {
+
+        Feed f1 = new Feed();
+        f1.setName(feed);
+        org.apache.falcon.entity.v0.feed.Cluster feedCluster =
+                new org.apache.falcon.entity.v0.feed.Cluster();
+        feedCluster.setName(cluster.getName());
+        feedCluster.setType(ClusterType.SOURCE);
+        Clusters clusters = new Clusters();
+        clusters.getClusters().add(feedCluster);
+        f1.setClusters(clusters);
+
+        Export exp = getAnExport(LoadMethod.UPDATEONLY, ds);
+        f1.getClusters().getClusters().get(0).setExport(exp);
+        return f1;
+    }
+
+    private Export getAnExport(LoadMethod loadMethod, Datasource ds) {
+
+        org.apache.falcon.entity.v0.feed.Datasource target = new org.apache.falcon.entity.v0.feed.Datasource();
+        target.setName(ds.getName());
+        target.setTableName("test-table");
+        Load load = new Load();
+        load.setType(loadMethod);
+        target.setLoad(load);
+        Export exp = new Export();
+        exp.setTarget(target);
+        return exp;
+    }
+
     private void attachInput(Process process, Feed feed) {
         if (process.getInputs() == null) {
             process.setInputs(new Inputs());
@@ -382,6 +417,42 @@
     }
 
     @Test
+    public void testOnAddExport() throws Exception {
+
+        Datasource ds = new Datasource();
+        ds.setName("test-db");
+        ds.setColo("c1");
+
+        Cluster cluster = new Cluster();
+        cluster.setName("ci1");
+        cluster.setColo("c1");
+
+        Feed f1 = addFeedExport("fe1", cluster, ds);
+
+        store.publish(EntityType.CLUSTER, cluster);
+        store.publish(EntityType.DATASOURCE, ds);
+        store.publish(EntityType.FEED, f1);
+
+        Set<Entity> entities = graph.getDependents(cluster);
+        Assert.assertEquals(entities.size(), 1);
+        Assert.assertTrue(entities.contains(f1));
+
+        entities = graph.getDependents(ds);
+        Assert.assertEquals(entities.size(), 1);
+        Assert.assertTrue(entities.contains(f1));
+
+        entities = graph.getDependents(f1);
+        Assert.assertEquals(entities.size(), 2);
+        Assert.assertTrue(entities.contains(cluster));
+        Assert.assertTrue(entities.contains(ds));
+
+        store.remove(EntityType.FEED, "fe1");
+        store.remove(EntityType.DATASOURCE, "test-db");
+        store.remove(EntityType.CLUSTER, "ci1");
+    }
+
+
+    @Test
     public void testOnRemoveDatasource() throws Exception {
 
         Datasource ds = new Datasource();
diff --git a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
index c0ae5fc..62db501 100644
--- a/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/metadata/MetadataMappingServiceTest.java
@@ -122,8 +122,10 @@
         configStore = ConfigurationStore.get();
 
         Services.get().register(new WorkflowJobEndNotificationService());
-        StartupProperties.get().setProperty("falcon.graph.storage.directory",
-                "target/graphdb-" + System.currentTimeMillis());
+        StartupProperties.get().setProperty("falcon.graph.storage.backend", "berkeleyje");
+        String graphDBDir = "target/graphdb-" + System.currentTimeMillis();
+        StartupProperties.get().setProperty("falcon.graph.storage.directory", graphDBDir);
+        StartupProperties.get().setProperty("falcon.graph.serialize.path", graphDBDir);
         StartupProperties.get().setProperty("falcon.graph.preserve.history", "true");
         service = new MetadataMappingService();
         service.init();
diff --git a/common/src/test/java/org/apache/falcon/security/DefaultAuthorizationProviderTest.java b/common/src/test/java/org/apache/falcon/security/DefaultAuthorizationProviderTest.java
index 3a6d8c0..2196ad1 100644
--- a/common/src/test/java/org/apache/falcon/security/DefaultAuthorizationProviderTest.java
+++ b/common/src/test/java/org/apache/falcon/security/DefaultAuthorizationProviderTest.java
@@ -315,7 +315,7 @@
                 "admin", realUser, new String[]{"admin", });
 
         DefaultAuthorizationProvider provider = new DefaultAuthorizationProvider();
-        provider.authorizeResource("entities", "list", null, "primary-cluster", proxyUgi);
+        provider.authorizeResource("instance", "list", null, "sample-process", proxyUgi);
         Assert.fail("Bad entity type");
     }
 
@@ -328,7 +328,7 @@
                 "admin", realUser, new String[]{"admin", });
 
         DefaultAuthorizationProvider provider = new DefaultAuthorizationProvider();
-        provider.authorizeResource("entities", "list", "clusterz", "primary-cluster", proxyUgi);
+        provider.authorizeResource("instance", "list", "processz", "sample-process", proxyUgi);
         Assert.fail("Bad entity type");
     }
 
diff --git a/distro/pom.xml b/distro/pom.xml
index 4351400..858e692 100644
--- a/distro/pom.xml
+++ b/distro/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.falcon</groupId>
     <artifactId>falcon-main</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
   </parent>
   <artifactId>falcon-distro</artifactId>
   <description>Apache Falcon Distro</description>
@@ -52,6 +52,7 @@
                 <descriptor>../src/main/assemblies/src-package.xml</descriptor>
               </descriptors>
               <finalName>apache-falcon-distributed-${project.version}</finalName>
+              <tarLongFileMode>posix</tarLongFileMode>
             </configuration>
             <executions>
               <execution>
@@ -126,6 +127,7 @@
             <descriptor>../src/main/assemblies/src-package.xml</descriptor>
           </descriptors>
           <finalName>apache-falcon-${project.version}</finalName>
+          <tarLongFileMode>posix</tarLongFileMode>
         </configuration>
         <executions>
           <execution>
diff --git a/docs/license/animate-LICENSE.txt b/docs/license/animate-LICENSE.txt
new file mode 100644
index 0000000..76ba6f5
--- /dev/null
+++ b/docs/license/animate-LICENSE.txt
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c) 2015 Daniel Eden
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
diff --git a/docs/license/cabin-font-LICENSE.txt b/docs/license/cabin-font-LICENSE.txt
new file mode 100644
index 0000000..03b1c52
--- /dev/null
+++ b/docs/license/cabin-font-LICENSE.txt
@@ -0,0 +1,95 @@
+Copyright (c) 2011, Pablo Impallari (www.impallari.com|impallari@gmail.com),

+Copyright (c) 2011, Igino Marini. (www.ikern.com|mail@iginomarini.com),

+with Reserved Font Name Cabin.

+

+This Font Software is licensed under the SIL Open Font License, Version 1.1.

+This license is copied below, and is also available with a FAQ at:

+http://scripts.sil.org/OFL

+

+

+-----------------------------------------------------------

+SIL OPEN FONT LICENSE Version 1.1 - 26 February 2007

+-----------------------------------------------------------

+

+PREAMBLE

+The goals of the Open Font License (OFL) are to stimulate worldwide

+development of collaborative font projects, to support the font creation

+efforts of academic and linguistic communities, and to provide a free and

+open framework in which fonts may be shared and improved in partnership

+with others.

+

+The OFL allows the licensed fonts to be used, studied, modified and

+redistributed freely as long as they are not sold by themselves. The

+fonts, including any derivative works, can be bundled, embedded, 

+redistributed and/or sold with any software provided that any reserved

+names are not used by derivative works. The fonts and derivatives,

+however, cannot be released under any other type of license. The

+requirement for fonts to remain under this license does not apply

+to any document created using the fonts or their derivatives.

+

+DEFINITIONS

+"Font Software" refers to the set of files released by the Copyright

+Holder(s) under this license and clearly marked as such. This may

+include source files, build scripts and documentation.

+

+"Reserved Font Name" refers to any names specified as such after the

+copyright statement(s).

+

+"Original Version" refers to the collection of Font Software components as

+distributed by the Copyright Holder(s).

+

+"Modified Version" refers to any derivative made by adding to, deleting,

+or substituting -- in part or in whole -- any of the components of the

+Original Version, by changing formats or by porting the Font Software to a

+new environment.

+

+"Author" refers to any designer, engineer, programmer, technical

+writer or other person who contributed to the Font Software.

+

+PERMISSION & CONDITIONS

+Permission is hereby granted, free of charge, to any person obtaining

+a copy of the Font Software, to use, study, copy, merge, embed, modify,

+redistribute, and sell modified and unmodified copies of the Font

+Software, subject to the following conditions:

+

+1) Neither the Font Software nor any of its individual components,

+in Original or Modified Versions, may be sold by itself.

+

+2) Original or Modified Versions of the Font Software may be bundled,

+redistributed and/or sold with any software, provided that each copy

+contains the above copyright notice and this license. These can be

+included either as stand-alone text files, human-readable headers or

+in the appropriate machine-readable metadata fields within text or

+binary files as long as those fields can be easily viewed by the user.

+

+3) No Modified Version of the Font Software may use the Reserved Font

+Name(s) unless explicit written permission is granted by the corresponding

+Copyright Holder. This restriction only applies to the primary font name as

+presented to the users.

+

+4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font

+Software shall not be used to promote, endorse or advertise any

+Modified Version, except to acknowledge the contribution(s) of the

+Copyright Holder(s) and the Author(s) or with their explicit written

+permission.

+

+5) The Font Software, modified or unmodified, in part or in whole,

+must be distributed entirely under this license, and must not be

+distributed under any other license. The requirement for fonts to

+remain under this license does not apply to any document created

+using the Font Software.

+

+TERMINATION

+This license becomes null and void if any of the above conditions are

+not met.

+

+DISCLAIMER

+THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,

+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF

+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT

+OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE

+COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,

+INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL

+DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING

+FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM

+OTHER DEALINGS IN THE FONT SOFTWARE.

diff --git a/docs/license/ngMask-LICENSE.txt b/docs/license/ngMask-LICENSE.txt
new file mode 100644
index 0000000..8fb8ea2
--- /dev/null
+++ b/docs/license/ngMask-LICENSE.txt
@@ -0,0 +1,13 @@
+Copyright (c) 2014, Carlos André Oliveira <candreoliveira@gmail.com>
+
+Permission to use, copy, modify, and/or distribute this software for any
+purpose with or without fee is hereby granted, provided that the above
+copyright notice and this permission notice appear in all copies.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
+WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
+ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
+WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
+ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
+OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
diff --git a/docs/license/ngTagsInput-LICENSE.txt b/docs/license/ngTagsInput-LICENSE.txt
new file mode 100644
index 0000000..2e766e3
--- /dev/null
+++ b/docs/license/ngTagsInput-LICENSE.txt
@@ -0,0 +1,20 @@
+The MIT License (MIT)
+
+Copyright (c) 2013 Michael Benford
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/docs/license/normalize-LICENSE.txt b/docs/license/normalize-LICENSE.txt
new file mode 100644
index 0000000..a9dc817
--- /dev/null
+++ b/docs/license/normalize-LICENSE.txt
@@ -0,0 +1,22 @@
+The MIT License
+
+Copyright © Nicolas Gallagher and Jonathan Neal
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
diff --git a/docs/pom.xml b/docs/pom.xml
index 766f408..e3556e8 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-docs</artifactId>
     <description>Apache Falcon Documentation</description>
diff --git a/docs/src/site/twiki/Configuration.twiki b/docs/src/site/twiki/Configuration.twiki
index bfca3d8..98acb83 100644
--- a/docs/src/site/twiki/Configuration.twiki
+++ b/docs/src/site/twiki/Configuration.twiki
@@ -103,9 +103,6 @@
 explained below. Falcon post processing feature continues to only send user notifications so enabling Oozie
 JMS notification is important.
 
-*NOTE : If Oozie JMS notification is not enabled, the Falcon features such as failure retry, late data handling and metadata
-service will be disabled for all entities on the server.*
-
 ---+++Enable Oozie JMS notification
 
    * Please add/change the following properties in oozie-site.xml in the oozie installation dir.
@@ -325,10 +322,16 @@
 For details on the same, refer to [[FalconNativeScheduler][Falcon Native Scheduler]]
 
 ---+++Titan GraphDB backend
-You can either choose to use 5.0.73 version of berkeleydb (the default for Falcon for the last few releases) or 1.1.x or later version HBase as the backend database. Falcon in its release distributions will have the titan storage plugins for both BerkeleyDB and HBase.
+GraphDB backend needs to be configured to properly start Falcon server.
+You can either choose to use 5.0.73 version of berkeleydb (the default for Falcon for the last few releases) or 1.1.x or later version HBase as the backend database.
+Falcon in its release distributions will have the titan storage plugins for both BerkeleyDB and HBase.
 
 ----++++Using BerkeleyDB backend
-Falcon distributions may not package berkeley db artifacts (je-5.0.73.jar) based on build profiles.  If Berkeley DB is not packaged, you can download the Berkeley DB jar file from the URL: <verbatim>http://download.oracle.com/otn/berkeley-db/je-5.0.73.zip</verbatim>.   The following properties describe an example berkeley db graph storage backend that can be specified in the configuration file <verbatim>$FALCON_HOME/conf/startup.properties</verbatim>.
+Falcon distributions may not package berkeley db artifacts (je-5.0.73.jar) based on build profiles.
+If Berkeley DB is not packaged, you can download the Berkeley DB jar file from the URL:
+<verbatim>http://download.oracle.com/otn/berkeley-db/je-5.0.73.zip</verbatim>.
+The following properties describe an example berkeley db graph storage backend that can be specified in the configuration file
+<verbatim>$FALCON_HOME/conf/startup.properties</verbatim>.
 
 <verbatim>
 # Graph Storage
@@ -337,13 +340,14 @@
 *.falcon.graph.serialize.path=${user.dir}/target/graphdb
 </verbatim>
 
-----++++Using HBase backend
+---++++Using HBase backend
 
-To use HBase as the backend it is recommended that a HBase cluster be provisioned with distributed mode confiuguratoin, primarily because of the support of kerberos enabled clusters and HA considerations.  Based on build profile, a standalone hbase version can be packaged with the Falcon binary distribution.   Along with this, a template for <verbatim>hbase-site.xml</verbatim> is provided, which can be used to start the standalone mode HBase enviornment for development/testing purposes.
+To use HBase as the backend it is recommended that a HBase cluster be provisioned with distributed mode configuration primarily because of the support of kerberos enabled clusters and HA considerations.  Based on build profile, a standalone hbase version can be packaged with the Falcon binary distribution.   Along with this, a template for <verbatim>hbase-site.xml</verbatim> is provided, which can be used to start the standalone mode HBase enviornment for development/testing purposes.
 
-Basic configuration
+---++++ Basic configuration
 
 <verbatim>
+##### Falcon startup.properties
 *.falcon.graph.storage.backend=hbase
 #For standalone mode , specify localhost
 #for distributed mode, specify zookeeper quorum here - For more information refer http://s3.thinkaurelius.com/docs/titan/current/hbase.html#_remote_server_mode_2
@@ -362,7 +366,45 @@
 *.falcon.graph.storage.hbase.table=falcon_titan
 </verbatim>
 
-Permissions
+---++++Starting standalone HBase for testing
+
+HBase can be started in stand alone mode for testing as a backend for Titan. The following steps outline the config changes required:
+<verbatim>
+1. Build Falcon as below to package hbase binaries
+   $ export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=256m" && mvn clean assembly:assembly -Ppackage-standalone-hbase
+2. Configure HBase
+   a. When falcon tar file is expanded, HBase binaries are under ${FALCON_HOME}/hbase
+   b. Copy ${FALCON_HOME}/conf/hbase-site.xml.template into hbase conf dir in ${FALCON_HOME}/hbase/conf/hbase-site.xml
+   c. Set {hbase_home} property to point to a local dir
+   d. Standalone HBase starts zookeeper on the default port (2181). This port can be changed by adding the following to hbase-site.xml
+       <property>
+            <name>hbase.zookeeper.property.clientPort</name>
+            <value>2223</value>
+       </property>
+
+       <property>
+            <name>hbase.zookeeper.quorum</name>
+            <value>localhost</value>
+       </property>
+    e. set JAVA_HOME to point to Java 1.7 or above
+    f. Start hbase as ${FALCON_HOME}/hbase/bin/start-hbase.sh
+3. Configure Falcon
+   a. In ${FALCON_HOME}/conf/startup.properties, uncomment the following to enable HBase as the backend
+      *.falcon.graph.storage.backend=hbase
+      ### specify the zookeeper host and port name with which standalone hbase is started (see step 2)
+      ### by default, it will be localhost and port 2181
+      *.falcon.graph.storage.hostname=<zookeeper-host-name>:<zookeeper-host-port>
+      *.falcon.graph.serialize.path=${user.dir}/target/graphdb
+      *.falcon.graph.storage.hbase.table=falcon_titan
+      *.falcon.graph.storage.transactions=false
+4. Add HBase jars to Falcon classpath in ${FALCON_HOME}/conf/falcon-env.sh as:
+      FALCON_EXTRA_CLASS_PATH=`${FALCON_HOME}/hbase/bin/hbase classpath`
+5. Set the following in ${FALCON_HOME}/conf/startup.properties to disable SSL if needed
+      *.falcon.enableTLS=false
+6. Start Falcon
+</verbatim>
+
+---++++Permissions
 
 When Falcon is configured with HBase as the storage backend Titan needs to have sufficient authorizations to create and access an HBase table.  In a secure cluster it may be necessary to grant permissions to the <verbatim>falcon</verbatim> user for the <verbatim>falcon_titan</verbatim> table (or whateven tablename was specified for the property <verbatim>*.falcon.graph.storage.hbase.table</verbatim>
 
@@ -376,7 +418,7 @@
    echo "grant 'falcon', 'RWXCA', 'falcon_titan'" | hbase shell
 </verbatim>
 
-Advanced configuration
+---++++Advanced configuration
 
 HBase storage backend support in Titan has a few other configurations and they can be set in <verbatim>$FALCON_HOME/conf/startup.properties</verbatim>, by prefixing the Titan property with <verbatim>*.falcon.graph</verbatim> prefix.
 
diff --git a/docs/src/site/twiki/DataReplicationAzure.twiki b/docs/src/site/twiki/DataReplicationAzure.twiki
new file mode 100644
index 0000000..24e543b
--- /dev/null
+++ b/docs/src/site/twiki/DataReplicationAzure.twiki
@@ -0,0 +1,61 @@
+---+ Data Replication between On-premise Hadoop Clusters and Azure Cloud
+
+---++ Overview
+Falcon provides an easy way to replicate data between on-premise Hadoop clusters and Azure cloud.
+With this feature, users would be able to build a hybrid data pipeline,
+e.g. processing sensitive data on-premises for privacy and compliance reasons
+while leverage cloud for elastic scale and online services (e.g. Azure machine learning) with non-sensitive data.
+
+---++ Use Case
+1. Copy data from on-premise Hadoop clusters to Azure cloud
+2. Copy data from Azure cloud to on-premise Hadoop clusters
+3. Copy data within Azure cloud (i.e. from one Azure location to another).
+
+---++ Usage
+---+++ Set Up Azure Blob Credentials
+To move data to/from Azure blobs, we need to add Azure blob credentials in HDFS.
+This can be done by adding the credential property through Ambari HDFS configs, and HDFS needs to be restarted after adding the credential.
+You can also add the credential property to core-site.xml directly, but make sure you restart HDFS from command line instead of Ambari.
+Otherwise, Ambari will take the previous HDFS configuration without your Azure blob credentials.
+<verbatim>
+<property>
+      <name>fs.azure.account.key.{AZURE_BLOB_ACCOUNT_NAME}.blob.core.windows.net</name>
+      <value>{AZURE_BLOB_ACCOUNT_KEY}</value>
+</property>
+</verbatim>
+
+To verify you set up Azure credential properly, you can check if you are able to access Azure blob through HDFS, e.g.
+<verbatim>
+hadoop fs ­ls wasb://{AZURE_BLOB_CONTAINER}@{AZURE_BLOB_ACCOUNT_NAME}.blob.core.windows.net/
+</verbatim>
+
+---+++ Replication Feed
+[[EntitySpecification][Falcon replication feed]] can be used for data replication to/from Azure cloud.
+You can specify WASB (i.e. Windows Azure Storage Blob) url in source or target locations.
+See below for an example of data replication from Hadoop cluster to Azure blob.
+Note that the clusters for the source and the target need to be different.
+Analogously, if you want to copy data from Azure blob, you can add Azure blob location to the source.
+<verbatim>
+<?xml version="1.0" encoding="UTF-8"?>
+<feed name="AzureReplication" xmlns="uri:falcon:feed:0.1">
+    <frequency>months(1)</frequency>
+    <clusters>
+        <cluster name="SampleCluster1" type="source">
+            <validity start="2010-06-01T00:00Z" end="2010-06-02T00:00Z"/>
+            <retention limit="days(90)" action="delete"/>
+        </cluster>
+        <cluster name="SampleCluster2" type="target">
+            <validity start="2010-06-01T00:00Z" end="2010-06-02T00:00Z"/>
+            <retention limit="days(90)" action="delete"/>
+            <locations>
+                <location type="data" path="wasb://replication-test@mystorage.blob.core.windows.net/replicated-${YEAR}-${MONTH}"/>
+            </locations>
+        </cluster>
+    </clusters>
+    <locations>
+        <location type="data" path="/apps/falcon/demo/data-${YEAR}-${MONTH}" />
+    </locations>
+    <ACL owner="ambari-qa" group="users" permission="0755"/>
+    <schema location="hcat" provider="hcat"/>
+</feed>
+</verbatim>
diff --git a/docs/src/site/twiki/Embedded-mode.twiki b/docs/src/site/twiki/Embedded-mode.twiki
index d5c37a1..47acab4 100644
--- a/docs/src/site/twiki/Embedded-mode.twiki
+++ b/docs/src/site/twiki/Embedded-mode.twiki
@@ -178,6 +178,7 @@
 <verbatim>
 $bin/falcon entity -submitAndSchedule -type process -file examples/entity/filesystem/oozie-mr-process.xml
 $bin/falcon entity -submitAndSchedule -type process -file examples/entity/filesystem/pig-process.xml
+$bin/falcon entity -submitAndSchedule -type process -file examples/entity/spark/spark-process.xml
 </verbatim>
 Generate input data:
 <verbatim>
@@ -189,7 +190,7 @@
 </verbatim>
 
 HCat based example entities are in examples/entity/hcat.
-
+Spark based example entities are in examples/entity/spark.
 
 ---+++Stopping Falcon Server
 <verbatim>
diff --git a/docs/src/site/twiki/EntitySpecification.twiki b/docs/src/site/twiki/EntitySpecification.twiki
index b27e341..9f9e210 100644
--- a/docs/src/site/twiki/EntitySpecification.twiki
+++ b/docs/src/site/twiki/EntitySpecification.twiki
@@ -52,6 +52,11 @@
 Falcon uses this interface to register/de-register partitions for a given database and table. Also,
 uses this information to schedule data availability events based on partitions in the workflow engine.
 Although Hive metastore supports both RPC and HTTP, Falcon comes with an implementation for RPC over thrift.
+For Hive HA mode, make sure the uris are separated with comma and you only add protocol "thrift://" at the beginning.
+See below for an example of Hive HA mode:
+<verbatim>
+<interface type="registry" endpoint="thrift://c6402.ambari.apache.org:9083,c6403.ambari.apache.org:9083" version="0.11.0" />
+</verbatim>
 
 <verbatim>
 <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true" version="5.4.6" />
@@ -828,13 +833,13 @@
 The properties defined in the cluster and cluster properties(nameNode and jobTracker) will also
 be available for the workflow.
 
-There are 3 engines supported today.
+There are 4 engines supported today.
 
 ---++++ Oozie
 
 As part of oozie workflow engine support, users can embed a oozie workflow.
-Refer to oozie [[http://oozie.apache.org/docs/4.0.1/DG_Overview.html][workflow overview]] and
-[[http://oozie.apache.org/docs/4.0.1/WorkflowFunctionalSpec.html][workflow specification]] for details.
+Refer to oozie [[http://oozie.apache.org/docs/4.2.0/DG_Overview.html][workflow overview]] and
+[[http://oozie.apache.org/docs/4.2.0/WorkflowFunctionalSpec.html][workflow specification]] for details.
 
 Syntax:
 <verbatim>
@@ -897,6 +902,54 @@
 Feeds with Hive table storage will send one more parameter apart from the general ones:
 <verbatim>$input_filter</verbatim>
 
+---++++ Spark
+Falcon also adds the Spark engine as part of Spark Integration which enables users to run the Java/Python Spark application as a process.
+When "spark" workflow engine is mentioned spark related parameters must be provided through <spark-attributes>
+Examples:
+<verbatim>
+<process name="spark-process">
+...
+    <workflow engine="spark" path="/resources/action">
+    <spark-attributes>
+          <master>local</master>
+          <name>Spark WordCount</name>
+          <class>org.examples.WordCount</class>
+          <jar>/resources/action/lib/spark-application.jar</jar>
+          <spark-opts>--num-executors 1 --driver-memory 512m</spark-opts>
+    </spark-attributes>
+...
+</process>
+</verbatim>
+
+This defines the workflow engine to be spark and Java/Python Spark application must be defined with "jar" option that need to be executed.
+There is flexibility to override the Spark master through process entity either to "yarn-client" or "yarn-cluster", if spark interface is already defined in cluster entity.
+Input and Output data to the Spark application will be set as argument when Spark workflow will be generated, if input and output feed entity is defined in the process entity.
+In the set of arguments, first argument will always correspond to input feed, second argument will always correspond to output feed and then user's provided argument will be set.
+
+
+For running the Spark SQL process entity, that read and write the data stored on Hive, the datanucleus jars under the $HIVE_HOME/lib directory and hive-site.xml
+under $SPARK_HOME/conf/ directory need to be available on the driver and all executors launched by the YARN cluster.
+The convenient way to do this is adding them through the --jars option and --file option of the spark-opts attribute.
+Example:
+<verbatim>
+<process name="spark-process">
+...
+    <workflow engine="spark" path="/resources/action">
+    <spark-attributes>
+        <master>local</master>
+        <name>Spark SQL</name>
+        <class>org.examples.SparkSQLProcessTable</class>
+        <jar>/resources/action/lib/spark-application.jar</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --jars /usr/local/hive/lib/datanucleus-rdbms.jar,/usr/local/hive/lib/datanucleus-core.jar,/usr/local/hive/lib/datanucleus-api-jdo.jar --files /usr/local/spark/conf/hive-site.xml</spark-opts>
+    </spark-attributes>
+...
+</process>
+</verbatim>
+
+Input and Output to the Spark SQL application will be set as argument when Spark workflow will be generated, if input and output feed entity is defined in the process entity.
+If input feed is of table type, then input table partition, table name and database name will be set as input arguments. If output feed is of table type, then output table partition, table name and database name will be set as output arguments.
+Once input and output arguments is set, then user's provided argument will be set.
+
 ---+++ Retry
 Retry policy defines how the workflow failures should be handled. Three retry policies are defined: periodic, exp-backoff(exponential backoff) and final. Depending on the delay and number of attempts, the workflow is re-tried after specific intervals. If user sets the onTimeout attribute to "true", retries will happen for TIMED_OUT instances.
 Syntax:
diff --git a/docs/src/site/twiki/Extensions.twiki b/docs/src/site/twiki/Extensions.twiki
index cf88c87..a3fed4e 100644
--- a/docs/src/site/twiki/Extensions.twiki
+++ b/docs/src/site/twiki/Extensions.twiki
@@ -53,6 +53,7 @@
 config value: org.apache.falcon.extensions.ExtensionService
 </verbatim>
 
+ExtensionService should be added before ConfigurationStore in startup properties for application services configuration.
 For manual installation user is expected to update "extension.store.uri" property defined in startup properties with
 HDFS path where the extension artifacts will be copied to.
 Extension artifacts in addons/extensions are packaged in falcon. For manual installation once the Falcon Server is setup user is expected to copy the extension artifacts under {falcon-server-dir}/extensions to HDFS at "extension.store.uri" path defined in startup properties and then restart Falcon.
@@ -60,5 +61,5 @@
 ---++ Migration
 Recipes framework and HDFS mirroring capability was added in Apache Falcon 0.6.0 release and it was client side logic. With 0.10 release its moved to server side and renamed as server side extensions. Client side recipes only had CLI support and expected certain pre steps to get it working. This is no longer required in 0.10 release as new CLI and REST API support has been provided.
 
-If user is migrating to 0.10 release and above then old Recipe setup and CLI's won't work. For manual installation user is expected to copy Extension artifacts to HDFS. Please refer "Packaging and installation" section above for more details.
+Migrating to 0.10 release and above is not backward compatible for Recipes. If user is migrating to 0.10 release and above then old Recipe setup and CLI's won't work. For manual installation user is expected to copy Extension artifacts to HDFS. Please refer "Packaging and installation" section above for more details.
 Please Refer to [[falconcli/FalconCLI][Falcon CLI]] and [[restapi/ResourceList][REST API]] for more details on usage of CLI and REST API's for extension jobs and instances management.
diff --git a/docs/src/site/twiki/FalconDocumentation.twiki b/docs/src/site/twiki/FalconDocumentation.twiki
index 4848746..fe1c0de 100644
--- a/docs/src/site/twiki/FalconDocumentation.twiki
+++ b/docs/src/site/twiki/FalconDocumentation.twiki
@@ -447,9 +447,11 @@
 
 ---+++ Archival as Replication
 
-Falcon allows users to archive data from on-premice to cloud, either Azure WASB or S3.
+Falcon allows users to archive data from on-premise to cloud, either Azure WASB or S3.
 It uses the underlying replication for archiving data from source to target. The archival URI is
 specified as the overridden location for the target cluster.
+Note that for data replication between on-premise and Azure cloud, Azure credentials need to be added to core-site.xml.
+Please refer to [[DataReplicationAzure][AzureDataReplication]] for details and examples.
 
 *Example:*
 <verbatim>
diff --git a/docs/src/site/twiki/GraphiteMetricCollection.twiki b/docs/src/site/twiki/GraphiteMetricCollection.twiki
new file mode 100644
index 0000000..c76e68d
--- /dev/null
+++ b/docs/src/site/twiki/GraphiteMetricCollection.twiki
@@ -0,0 +1,22 @@
+---++Graphite Metric Collection
+
+Graphite Metric Collection currently allows to collect the following metrics at process level :
+
+1. Processing time the process spent in the running state in seconds (workflow_end_time - workflow_start_time)
+2. Wait time that the process spent in the waiting/ready state. (workflow_start_time - workflow_nominal_time)
+3. Number of instances that are failed for a process.
+
+To send data to graphite we need to intialize metricNotificationService in startup.properties:
+
+*.application.services= org.apache.falcon.metrics.MetricNotificationService,
+
+
+Add following properties for graphiteNotificationPlugin :
+
+Graphite properties
+   * *.falcon.graphite.hostname=localhost
+   * *.falcon.graphite.port=2003
+   * *.falcon.graphite.frequency=1
+   * *.falcon.graphite.prefix=falcon
+
+The falcon.graphite.frequency is in seconds and all the time that is being sent to graphite is in seconds.
\ No newline at end of file
diff --git a/docs/src/site/twiki/InstallationSteps.twiki b/docs/src/site/twiki/InstallationSteps.twiki
index 93b1eab..297d88e 100644
--- a/docs/src/site/twiki/InstallationSteps.twiki
+++ b/docs/src/site/twiki/InstallationSteps.twiki
@@ -27,16 +27,15 @@
 </verbatim>
 It builds and installs the package into the local repository, for use as a dependency in other projects locally.
 
-[optionally -Dhadoop.version=<<hadoop.version>> can be appended to build for a specific version of Hadoop]
+[optionally -Dhadoop.version=<<hadoop.version>> can be appended to build for a specific version of hadoop]
 
-*NOTE:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
-[optionally -Doozie.version=<<oozie version>> can be appended to build with a specific version of Oozie. Oozie versions
->= 4 are supported]
-NOTE: Falcon builds with JDK 1.7/1.8 using -noverify option
-      To compile Falcon with Hive Replication, optionally "-P hadoop-2,hivedr" can be appended. For this Hive >= 1.2.0
-      and Oozie >= 4.2.0 should be available.
+*Note 1:* Falcon drops support for Hadoop-1 and only supports Hadoop-2 from Falcon 0.6 onwards
+          Falcon build with JDK 1.7 using -noverify option
 
-
+*Note 2:* To compile Falcon with addon extensions, append additional profiles to build command using syntax -P<<profile1,profile2>>
+          For Hive Mirroring extension, use profile"hivedr". Hive >= 1.2.0 and Oozie >= 4.2.0 is required
+          For HDFS Snapshot mirroring extension, use profile "hdfs-snapshot-mirroring". Hadoop >= 2.7.0 is required
+          For ADF integration, use profile "adf"
 
 ---+++Step 3 - Package and Deploy Falcon
 
diff --git a/docs/src/site/twiki/MigrationInstructions.twiki b/docs/src/site/twiki/MigrationInstructions.twiki
index 7c0e027..a11dbc4 100644
--- a/docs/src/site/twiki/MigrationInstructions.twiki
+++ b/docs/src/site/twiki/MigrationInstructions.twiki
@@ -1,15 +1,32 @@
 ---+ Migration Instructions
 
----++ Migrate from 0.5-incubating to 0.6-incubating
+---++ Migrate from 0.9 to 0.10
 
-This is a placeholder wiki for migration instructions from falcon 0.5-incubating to 0.6-incubating.
+FALCON-1333 (Instance Search feature) requires Falcon to use titan-berkeleyje version 0.5.4 to support indexing.
+Up until version 0.9 - Falcon used titan-berkeleyje-jre6 version 0.4.2. GraphDB created by version 0.4.2 cannot be
+read by version 0.5.4. The solution is to migrate the GraphDB to be compatible with Falcon 0.10 release. Please make
+sure that no falcon server is running while performing the migration.
 
----+++ Update Entities
+---+++ 1. Install Falcon 0.10
+Install Falcon 0.10 by following the [[InstallationSteps][Installation Steps]]. Do not start the falcon server yet.
+The tool to migrate graphDB is packaged with 0.10 Falcon server in falcon-common-0.10.jar.
 
----+++ Change cluster dir permissions
+---+++ 2. Export GraphDB to JSON file using Falcon 0.9
+Please run the following command to generate the JSON file.
 
----+++ Enable/Disable TLS
+<verbatim>
+ $FALCON_HOME/bin/graphdbutil.sh export <<java_home> <<hadoop_home>> <<falcon_0.9_home>> <<path_to_falcon-common-0.10.jar>> /jsonFile/dir/
+</verbatim>
 
----+++ Authorization
+This command will create /jsonFile/dir/instanceMetadata.json
 
+---+++ 3. Import GraphDB from JSON file using Falcon 0.10
+Please run the following command to import graphDB the JSON file. The location of graphDB will be based on property
+"*.falcon.graph.storage.directory" set in startup.properties file.
+
+<verbatim>
+  $FALCON_HOME/bin/graphdbutil.sh export <<java_home> <<hadoop_home>> <<falcon_0.10_home>> <<path_to_falcon-common-0.10.jar>> /jsonFile/dir/
+</verbatim>
+
+This command will import from /jsonFile/dir/instanceMetadata.json, now start the Falcon 0.10 server.
 
diff --git a/docs/src/site/twiki/Operability.twiki b/docs/src/site/twiki/Operability.twiki
index 616af36..2bccb51 100644
--- a/docs/src/site/twiki/Operability.twiki
+++ b/docs/src/site/twiki/Operability.twiki
@@ -223,3 +223,8 @@
 
 Users may also extend the Falcon Audit plugin to send audits to systems like Apache Argus, etc. by
 extending org.apache.falcon.plugin.AuditingPlugin interface.
+
+
+---++ Metrics Collection In Graphite
+
+Falcon has support to send metrics to graphite more details regarding this can be found on [[GraphiteMetricCollection][Graphite Metric Collection]]
\ No newline at end of file
diff --git a/docs/src/site/twiki/restapi/EntityList.twiki b/docs/src/site/twiki/restapi/EntityList.twiki
index 2c2a734..a439dc7 100644
--- a/docs/src/site/twiki/restapi/EntityList.twiki
+++ b/docs/src/site/twiki/restapi/EntityList.twiki
@@ -1,4 +1,4 @@
----++  GET /api/entities/list/:entity-type?fields=:fields
+---++  GET /api/entities/list/{:entity-type}
    * <a href="#Description">Description</a>
    * <a href="#Parameters">Parameters</a>
    * <a href="#Results">Results</a>
@@ -8,7 +8,7 @@
 Get list of the entities.
 
 ---++ Parameters
-   * :entity-type Comma-separated entity types. Can be empty. Valid entity types are cluster, feed or process.
+   * :entity-type <optional param> Comma-separated entity types. Valid entity types are cluster, feed or process.
    * fields <optional param> Fields of entity that the user wants to view, separated by commas.
       * Valid options are STATUS, TAGS, PIPELINES, CLUSTERS.
    * nameseq <optional param> Subsequence of entity name. Not case sensitive.
@@ -38,27 +38,6 @@
 ---++ Examples
 ---+++ Rest Call
 <verbatim>
-GET http://localhost:15000/api/entities/list/feed
-</verbatim>
----+++ Result
-<verbatim>
-{
-    "totalResults":"2”,
-    "entity": [
-        {
-            "name": "SampleOutput",
-            "type": "feed"
-        },
-        {
-            "name": "SampleInput",
-            "type": "feed"
-        }
-    ]
-}
-</verbatim>
-
----+++ Rest Call
-<verbatim>
 GET http://localhost:15000/api/entities/list
 </verbatim>
 ---+++ Result
@@ -88,6 +67,27 @@
 
 ---+++ Rest Call
 <verbatim>
+GET http://localhost:15000/api/entities/list/feed
+</verbatim>
+---+++ Result
+<verbatim>
+{
+    "totalResults":"2”,
+    "entity": [
+        {
+            "name": "SampleOutput",
+            "type": "feed"
+        },
+        {
+            "name": "SampleInput",
+            "type": "feed"
+        }
+    ]
+}
+</verbatim>
+
+---+++ Rest Call
+<verbatim>
 GET http://localhost:15000/api/entities/list/feed?fields=status
 </verbatim>
 ---+++ Result
diff --git a/docs/src/site/twiki/restapi/ResourceList.twiki b/docs/src/site/twiki/restapi/ResourceList.twiki
index aadd14f..5a4c253 100644
--- a/docs/src/site/twiki/restapi/ResourceList.twiki
+++ b/docs/src/site/twiki/restapi/ResourceList.twiki
@@ -54,7 +54,7 @@
 | DELETE      | [[EntityDelete][api/entities/delete/:entity-type/:entity-name]]             | Delete the entity                  |
 | GET         | [[EntityStatus][api/entities/status/:entity-type/:entity-name]]             | Get the status of the entity       |
 | GET         | [[EntityDefinition][api/entities/definition/:entity-type/:entity-name]]     | Get the definition of the entity   |
-| GET         | [[EntityList][api/entities/list/:entity-type]]                              | Get the list of entities           |
+| GET         | [[EntityList][api/entities/list/{:entity-type}]]                            | Get the list of entities           |
 | GET         | [[EntitySummary][api/entities/summary/:entity-type/:cluster]]               | Get instance summary of all entities |
 | GET         | [[EntityDependencies][api/entities/dependencies/:entity-type/:entity-name]] | Get the dependencies of the entity |
 | GET         | [[FeedSLA][api/entities/sla-alert/:entity-type]]                            | Get pending feed instances which missed sla |
diff --git a/examples/app/spark/wordcount.py b/examples/app/spark/wordcount.py
new file mode 100644
index 0000000..b6ba426
--- /dev/null
+++ b/examples/app/spark/wordcount.py
@@ -0,0 +1,52 @@
+#/**
+# * 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.
+# */
+
+from __future__ import print_function
+
+import sys
+from operator import add
+
+from pyspark import SparkContext
+
+if __name__ == "__main__":
+    if len(sys.argv) < 2:
+        print("Usage: wordcount <file>", file=sys.stderr)
+        exit(-1)
+    sc = SparkContext(appName="Python WordCount")
+    
+    # Read input and output path
+    inputPath = sys.argv[1]
+    print ('Path of input file ->' + inputPath)
+    outputPath = sys.argv[2]
+    print ('Path of output file ->' + outputPath)
+    
+    distFile = sc.textFile(inputPath)
+    
+    def flatMap(line):
+        return line.split(",")
+    
+    def map(word):
+        return (word,1)
+    
+    def reduce(a,b):
+        return a+b
+    
+    
+    counts = distFile.flatMap(flatMap).map(map).reduceByKey(reduce)
+    
+    counts.saveAsTextFile(outputPath)
diff --git a/examples/entity/spark/pyspark-process.xml b/examples/entity/spark/pyspark-process.xml
new file mode 100644
index 0000000..de4b5cd
--- /dev/null
+++ b/examples/entity/spark/pyspark-process.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<process name="pyspark-process" xmlns="uri:falcon:process:0.1">
+    <clusters>
+        <cluster name="local">
+            <validity start="2013-11-15T00:05Z" end="2013-11-15T01:05Z"/>
+        </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>minutes(5)</frequency>
+    <timezone>UTC</timezone>
+
+    <inputs>
+        <!-- In the workflow, the input paths will be available in a variable 'inpaths' -->
+        <input name="inpaths" feed="in" start="now(0,-5)" end="now(0,-1)"/>
+    </inputs>
+
+    <outputs>
+        <!-- In the workflow, the output path will be available in a variable 'outpath' -->
+        <output name="outpath" feed="out" instance="now(0,0)"/>
+    </outputs>
+
+    <workflow engine="spark" path="/app/spark"/>
+    <spark-attributes>
+        <master>local</master>
+        <name>Python Spark Wordcount</name>
+        <jar>/app/spark/wordcount.py</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1</spark-opts>
+    </spark-attributes>
+
+    <retry policy="periodic" delay="minutes(3)" attempts="3"/>
+
+</process>
diff --git a/examples/entity/spark/spark-process-pi.xml b/examples/entity/spark/spark-process-pi.xml
new file mode 100644
index 0000000..65c81cf
--- /dev/null
+++ b/examples/entity/spark/spark-process-pi.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<process name="spark-pi" xmlns="uri:falcon:process:0.1">
+    <clusters>
+        <cluster name="local">
+            <validity start="2013-11-15T00:05Z" end="2013-11-15T01:05Z"/>
+        </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>minutes(5)</frequency>
+    <timezone>UTC</timezone>
+
+    <workflow engine="spark" path="/app/spark/"/>
+    <spark-attributes>
+        <master>local</master>
+        <name>Spark PI</name>
+        <class>org.apache.falcon.example.spark.SparkPI</class>
+        <jar>/app/spark/lib/falcon-examples.jar</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1</spark-opts>
+	    <arg>2</arg>
+    </spark-attributes>
+
+    <retry policy="periodic" delay="minutes(3)" attempts="3"/>
+
+</process>
diff --git a/examples/entity/spark/spark-process.xml b/examples/entity/spark/spark-process.xml
new file mode 100644
index 0000000..b9ecc98
--- /dev/null
+++ b/examples/entity/spark/spark-process.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+
+<process name="spark-process" xmlns="uri:falcon:process:0.1">
+    <clusters>
+        <cluster name="local">
+            <validity start="2013-11-15T00:05Z" end="2013-11-15T01:05Z"/>
+        </cluster>
+    </clusters>
+
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>minutes(5)</frequency>
+    <timezone>UTC</timezone>
+
+    <inputs>
+        <!-- In the workflow, the input paths will be available in a variable 'inpaths' -->
+        <input name="inpaths" feed="in" start="now(0,-5)" end="now(0,-1)"/>
+    </inputs>
+
+    <outputs>
+        <!-- In the workflow, the output path will be available in a variable 'outpath' -->
+        <output name="outpath" feed="out" instance="now(0,0)"/>
+    </outputs>
+
+    <workflow engine="spark" path="/app/spark"/>
+    <spark-attributes>
+        <master>local</master>
+        <name>Java Spark Wordcount</name>
+        <class>org.apache.falcon.example.spark.SparkWordCount</class>
+        <jar>/app/spark/lib/falcon-examples.jar</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1</spark-opts>
+    </spark-attributes>
+
+    <retry policy="periodic" delay="minutes(3)" attempts="3"/>
+
+</process>
diff --git a/examples/entity/spark/spark-sql-process.xml b/examples/entity/spark/spark-sql-process.xml
new file mode 100644
index 0000000..cdd2ccc
--- /dev/null
+++ b/examples/entity/spark/spark-sql-process.xml
@@ -0,0 +1,55 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+<process name="spark-sql-process" xmlns="uri:falcon:process:0.1">
+    <!-- where -->
+    <clusters>
+        <cluster name="hcat-local">
+            <validity start="2013-11-15T00:05Z" end="2013-11-15T01:05Z"/>
+        </cluster>
+    </clusters>
+
+    <!-- when -->
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>minutes(5)</frequency>
+    <timezone>UTC</timezone>
+
+    <!-- what -->
+    <inputs>
+        <!-- In the workflow, the input paths will be available in a variable 'inpaths' -->
+        <input name="inparts" feed="hcat-in" start="now(0,-5)" end="now(0,-1)"/>
+    </inputs>
+
+    <outputs>
+        <!-- In the workflow, the output path will be available in a variable 'outpath' -->
+        <output name="outpart" feed="hcat-out" instance="now(0,0)"/>
+    </outputs>
+
+    <workflow engine="spark" path="/app/spark"/>
+    <spark-attributes>
+        <master>local</master>
+        <name>Spark SQL</name>
+        <class>org.apache.falcon.example.spark.SparkSQLProcessTable</class>
+        <jar>/app/spark/lib/falcon-examples.jar</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1</spark-opts>
+    </spark-attributes>
+
+    <retry policy="periodic" delay="minutes(3)" attempts="3"/>
+
+</process>
\ No newline at end of file
diff --git a/examples/pom.xml b/examples/pom.xml
index 2ff3011..a1aedf8 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -23,11 +23,12 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-examples</artifactId>
     <description>Apache Falcon Examples</description>
     <name>Apache Falcon Examples</name>
+    <packaging>jar</packaging>
 
     <dependencies>
         <dependency>
@@ -35,6 +36,44 @@
             <artifactId>hadoop-client</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_2.10</artifactId>
+            <version>${spark.version}</version>
+            <scope>compile</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.mesos</groupId>
+                    <artifactId>mesos</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>jul-to-slf4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>jcl-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_2.10</artifactId>
+            <version>${spark.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-hive_2.10</artifactId>
+            <version>${spark.version}</version>
+        </dependency>
+        <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>
diff --git a/examples/src/main/java/org/apache/falcon/example/spark/SparkPI.java b/examples/src/main/java/org/apache/falcon/example/spark/SparkPI.java
new file mode 100644
index 0000000..7ae0b6b
--- /dev/null
+++ b/examples/src/main/java/org/apache/falcon/example/spark/SparkPI.java
@@ -0,0 +1,72 @@
+/**
+ * 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.falcon.example.spark;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Computes an approximation to pi.
+ * Usage: JavaSparkPi [slices]
+ */
+public final class SparkPI {
+
+    private SparkPI() {
+    }
+
+    public static void main(String[] args) throws Exception {
+        SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi");
+        JavaSparkContext jsc = new JavaSparkContext(sparkConf);
+
+        int slices = (args.length == 1) ? Integer.parseInt(args[0]) : 2;
+        int n = 1 * slices;
+        System.out.println("n:"+n+"\tslices:"+slices);
+        List<Integer> l = new ArrayList<Integer>(n);
+        for (int i = 0; i < n; i++) {
+            l.add(i);
+        }
+
+        JavaRDD<Integer> dataSet = jsc.parallelize(l, slices);
+
+        int count = dataSet.map(new Function<Integer, Integer>() {
+            @Override
+            public Integer call(Integer integer) {
+                double x = Math.random() * 2 - 1;
+                double y = Math.random() * 2 - 1;
+                return (x * x + y * y < 1) ? 1 : 0;
+            }
+        }).reduce(new Function2<Integer, Integer, Integer>() {
+            @Override
+            public Integer call(Integer integer, Integer integer2) {
+                return integer + integer2;
+            }
+        });
+
+        System.out.println("Pi is roughly " + 4.0 * count / n);
+
+        jsc.stop();
+    }
+}
+
diff --git a/examples/src/main/java/org/apache/falcon/example/spark/SparkSQLProcessTable.java b/examples/src/main/java/org/apache/falcon/example/spark/SparkSQLProcessTable.java
new file mode 100644
index 0000000..5e9f092
--- /dev/null
+++ b/examples/src/main/java/org/apache/falcon/example/spark/SparkSQLProcessTable.java
@@ -0,0 +1,51 @@
+/**
+ * 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.falcon.example.spark;
+
+import org.apache.spark.SparkContext;
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.hive.HiveContext;
+
+/**
+ * Spark SQL Example.
+ */
+
+public final class SparkSQLProcessTable {
+
+    private SparkSQLProcessTable() {
+    }
+    public static void main(String[] args) {
+        if (args.length < 1) {
+            System.out.println("Arguments must contain details for input or output table");
+            System.exit(0);
+        }
+
+        SparkConf conf = new SparkConf().setAppName("SparkSQL example");
+        SparkContext sc = new SparkContext(conf);
+        HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc);
+
+        String sqlQuery = "FROM " +args[2]+"."+args[1]+ " INSERT OVERWRITE TABLE " +args[5]+"."+args[4]
+                +" PARTITION("+args[3]+")  SELECT word, SUM(cnt) AS cnt WHERE "+args[0]+" GROUP BY word";
+
+        DataFrame df = sqlContext.sql(sqlQuery);
+        df.show();
+    }
+}
+
diff --git a/examples/src/main/java/org/apache/falcon/example/spark/SparkWordCount.java b/examples/src/main/java/org/apache/falcon/example/spark/SparkWordCount.java
new file mode 100644
index 0000000..f74a536
--- /dev/null
+++ b/examples/src/main/java/org/apache/falcon/example/spark/SparkWordCount.java
@@ -0,0 +1,74 @@
+/**
+ * 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.falcon.example.spark;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import scala.Tuple2;
+
+import java.util.Arrays;
+
+/**
+ * Spark Word Count example.
+ */
+public final class SparkWordCount {
+
+    private SparkWordCount() {
+    }
+    protected static final FlatMapFunction<String, String> WORDS_EXTRACTOR =
+            new FlatMapFunction<String, String>() {
+                public Iterable<String> call(String s) throws Exception {
+                    return Arrays.asList(s.split(" "));
+                }
+            };
+
+    protected static final PairFunction<String, String, Integer> WORDS_MAPPER =
+            new PairFunction<String, String, Integer>() {
+                public Tuple2<String, Integer> call(String s) throws Exception {
+                    return new Tuple2<String, Integer>(s, 1);
+                }
+            };
+
+    protected static final Function2<Integer, Integer, Integer> WORDS_REDUCER =
+            new Function2<Integer, Integer, Integer>() {
+                public Integer call(Integer a, Integer b) throws Exception {
+                    return a + b;
+                }
+            };
+
+    public static void main(String[] args) {
+        if (args.length < 1) {
+            System.err.println("Please provide the input file full path as argument");
+            System.exit(0);
+        }
+
+        SparkConf conf = new SparkConf().setAppName("Java WordCount");
+        JavaSparkContext context = new JavaSparkContext(conf);
+        JavaRDD<String> file = context.textFile(args[0]);
+        JavaRDD<String> words = file.flatMap(WORDS_EXTRACTOR);
+        JavaPairRDD<String, Integer> pairs = words.mapToPair(WORDS_MAPPER);
+        JavaPairRDD<String, Integer> counter = pairs.reduceByKey(WORDS_REDUCER);
+        counter.saveAsTextFile(args[1]);
+    }
+}
diff --git a/extensions/pom.xml b/extensions/pom.xml
index 4243dee..09d4249 100644
--- a/extensions/pom.xml
+++ b/extensions/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-extensions</artifactId>
     <description>Apache Falcon server side extensions Module</description>
diff --git a/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtension.java b/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtension.java
index 949aea5..9222e0a 100644
--- a/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtension.java
+++ b/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtension.java
@@ -39,6 +39,7 @@
     private static final String ALL_TABLES = "*";
     private static final String COMMA_DELIMITER = ",";
     private static final String SECURE_RESOURCE = "-secure";
+    private static final String NOT_APPLICABLE = "NA";
 
     @Override
     public String getName() {
@@ -122,6 +123,12 @@
         additionalProperties.put(HiveMirroringExtensionProperties.HIVE_MIRRORING_JOB_NAME.getName(),
                 jobName);
 
+        // Get the first source DB
+        additionalProperties.put(HiveMirroringExtensionProperties.SOURCE_DATABASE.getName(),
+                extensionProperties.getProperty(HiveMirroringExtensionProperties.SOURCE_DATABASES
+                .getName()).trim().split(",")[0]
+        );
+
         String clusterName = extensionProperties.getProperty(ExtensionProperties.CLUSTER_NAME.getName());
         // Add required properties of cluster where job should run
         additionalProperties.put(HiveMirroringExtensionProperties.CLUSTER_FOR_JOB_RUN.getName(),
@@ -210,7 +217,7 @@
         String replicationMaxMaps =
                 extensionProperties.getProperty(HiveMirroringExtensionProperties.MAX_MAPS.getName());
         if (StringUtils.isBlank(replicationMaxMaps)) {
-            additionalProperties.put(HiveMirroringExtensionProperties.MAX_MAPS.getName(), "5");
+            additionalProperties.put(HiveMirroringExtensionProperties.MAX_MAPS.getName(), "2");
         }
 
         String distcpMaxMaps = extensionProperties.getProperty(
@@ -230,6 +237,16 @@
             additionalProperties.put(HiveMirroringExtensionProperties.TDE_ENCRYPTION_ENABLED.getName(), "false");
         }
 
+        if (StringUtils.isBlank(
+                extensionProperties.getProperty(HiveMirroringExtensionProperties.SOURCE_STAGING_PATH.getName()))) {
+            additionalProperties.put(HiveMirroringExtensionProperties.SOURCE_STAGING_PATH.getName(), NOT_APPLICABLE);
+        }
+
+        if (StringUtils.isBlank(
+                extensionProperties.getProperty(HiveMirroringExtensionProperties.TARGET_STAGING_PATH.getName()))) {
+            additionalProperties.put(HiveMirroringExtensionProperties.TARGET_STAGING_PATH.getName(), NOT_APPLICABLE);
+        }
+
         return additionalProperties;
     }
 }
diff --git a/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtensionProperties.java b/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtensionProperties.java
index 6c4f58d..828817b 100644
--- a/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtensionProperties.java
+++ b/extensions/src/main/java/org/apache/falcon/extensions/mirroring/hive/HiveMirroringExtensionProperties.java
@@ -27,6 +27,7 @@
     SOURCE_METASTORE_URI("sourceMetastoreUri", "Source Hive metastore uri", false),
     SOURCE_HS2_URI("sourceHiveServer2Uri", "Source HS2 uri"),
     SOURCE_DATABASES("sourceDatabases", "List of databases to replicate"),
+    SOURCE_DATABASE("sourceDatabase", "Database to verify the setup connection", false),
     SOURCE_TABLES("sourceTables", "List of tables to replicate", false),
     SOURCE_STAGING_PATH("sourceStagingPath", "Location of source staging path", false),
     SOURCE_NN("sourceNN", "Source name node", false),
@@ -50,13 +51,13 @@
     MAX_EVENTS("maxEvents", "Maximum events to replicate", false),
     MAX_MAPS("replicationMaxMaps", "Maximum number of maps used during replication", false),
     DISTCP_MAX_MAPS("distcpMaxMaps", "Maximum number of maps used during distcp", false),
-    MAP_BANDWIDTH_IN_MB("distcpMapBandwidth", "Bandwidth in MB/s used by each mapper during replication"),
+    MAP_BANDWIDTH_IN_MB("distcpMapBandwidth", "Bandwidth in MB/s used by each mapper during replication", false),
     CLUSTER_FOR_JOB_RUN("clusterForJobRun", "Cluster on which replication job runs", false),
-    CLUSTER_FOR_JOB_NN_KERBEROS_PRINCIPAL("Job cluster kerberos principal",
-            "Write EP of cluster on which replication job runs", false),
+    CLUSTER_FOR_JOB_NN_KERBEROS_PRINCIPAL("clusterForJobNNKerberosPrincipal", "Job cluster kerberos principal",
+            false),
     CLUSTER_FOR_JOB_RUN_WRITE_EP("clusterForJobRunWriteEP", "Write EP of cluster on which replication job runs", false),
     TDE_ENCRYPTION_ENABLED("tdeEncryptionEnabled", "Set to true if TDE encryption is enabled", false),
-    HIVE_MIRRORING_JOB_NAME("jobName", "Unique hive replication job name", false);
+    HIVE_MIRRORING_JOB_NAME("hiveJobName", "Unique hive replication job name", false);
 
     private final String name;
     private final String description;
diff --git a/falcon-regression/merlin-core/pom.xml b/falcon-regression/merlin-core/pom.xml
index fa3c939..367ef07 100644
--- a/falcon-regression/merlin-core/pom.xml
+++ b/falcon-regression/merlin-core/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon.regression</groupId>
         <artifactId>falcon-regression</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-merlin-core</artifactId>
     <description>merlin-core - utilities for Apache Falcon regression suite</description>
diff --git a/falcon-regression/merlin/pom.xml b/falcon-regression/merlin/pom.xml
index 73be13e..33012fc 100644
--- a/falcon-regression/merlin/pom.xml
+++ b/falcon-regression/merlin/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <groupId>org.apache.falcon.regression</groupId>
         <artifactId>falcon-regression</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-merlin</artifactId>
     <description>Merlin - Regression test suite for Apache Falcon</description>
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedLateRerunTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedLateRerunTest.java
index ec2b877..6405b30 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedLateRerunTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedLateRerunTest.java
@@ -56,7 +56,7 @@
  * On adding further late data it checks whether the data has been replicated correctly in the given late cut-off time.
  * Assuming that late frequency set in server is 3 minutes. Although value can be changed according to requirement.
  */
-@Test(groups = { "distributed", "embedded", "sanity" })
+@Test(groups = { "distributed", "embedded", "sanity", "multiCluster" })
 public class FeedLateRerunTest extends BaseTestClass {
 
     private ColoHelper cluster1 = servers.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
index dad0dc2..3367817 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/FeedReplicationTest.java
@@ -60,7 +60,7 @@
  * feed replication test.
  * Replicates empty directories as well as directories containing data.
  */
-@Test(groups = { "distributed", "embedded", "sanity" })
+@Test(groups = { "distributed", "embedded", "sanity", "multiCluster" })
 public class FeedReplicationTest extends BaseTestClass {
 
     private ColoHelper cluster1 = servers.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
index df1716f..dbb93eb 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/InstanceSummaryTest.java
@@ -173,7 +173,7 @@
     /**
      * Adjust multi-cluster process. Submit and schedule it. Get its instances summary.
      */
-    @Test(enabled = true, timeOut = 1200000)
+    @Test(enabled = true, timeOut = 1200000, groups = "multiCluster")
     public void testSummaryMultiClusterProcess() throws JAXBException,
             ParseException, IOException, URISyntaxException, AuthenticationException,
             InterruptedException {
@@ -208,7 +208,7 @@
     /**
      *  Adjust multi-cluster feed. Submit and schedule it. Get its instances summary.
      */
-    @Test(enabled = true, timeOut = 1200000)
+    @Test(enabled = true, timeOut = 1200000, groups = "multiCluster")
     public void testSummaryMultiClusterFeed() throws JAXBException, ParseException, IOException,
             URISyntaxException, OozieClientException, AuthenticationException,
             InterruptedException {
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
index 20f8f46..ce4c903 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/ProcessInstanceColoMixedTest.java
@@ -48,7 +48,7 @@
 /**
  * Process instance mixed colo tests.
  */
-@Test(groups = { "distributed", "embedded" })
+@Test(groups = { "distributed", "embedded", "multiCluster" })
 public class ProcessInstanceColoMixedTest extends BaseTestClass {
 
     private final String baseTestHDFSDir = cleanAndGetTestDir();
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
index eb20d7c..db020ec 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatFeedOperationsTest.java
@@ -119,7 +119,7 @@
      *
      * @throws Exception
      */
-    @Test(groups = {"singleCluster"})
+    @Test(groups = {"multiCluster"})
     public void submitFeedWhenTableDoesNotExist() throws Exception {
         Bundle.submitCluster(bundles[1]);
         feed = bundles[1].getInputFeedFromBundle();
@@ -159,7 +159,7 @@
      *
      * @throws Exception
      */
-    @Test
+    @Test(groups = {"multiCluster"})
     public void submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget() throws Exception {
         Bundle.submitCluster(bundles[0], bundles[1]);
         final String startDate = "2010-01-01T20:00Z";
@@ -192,7 +192,7 @@
      *
      * @throws Exception
      */
-    @Test
+    @Test(groups = {"multiCluster"})
     public void suspendAndResumeReplicationFeed() throws Exception {
 
         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
@@ -215,7 +215,7 @@
      *
      * @throws Exception
      */
-    @Test
+    @Test(groups = {"multiCluster"})
     public void deleteReplicationFeed() throws Exception {
         submitAndScheduleReplicationFeedWhenTableExistsOnSourceAndTarget();
 
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
index a96b17e..6643ce5 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hcat/HCatReplicationTest.java
@@ -64,7 +64,7 @@
 /**
  * Tests for replication with hcat.
  */
-@Test(groups = "embedded")
+@Test(groups = {"embedded", "multiCluster"})
 public class HCatReplicationTest extends BaseTestClass {
 
     private static final Logger LOGGER = Logger.getLogger(HCatReplicationTest.class);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HdfsRecipeTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HdfsRecipeTest.java
index 4a2d913..07996d5 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HdfsRecipeTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HdfsRecipeTest.java
@@ -50,7 +50,7 @@
 /**
  * Hdfs recipe test.
  */
-@Test(groups = "embedded")
+@Test(groups = {"embedded", "multiCluster"})
 public class HdfsRecipeTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(HdfsRecipeTest.class);
     private final ColoHelper cluster = servers.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDRTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDRTest.java
index 4dab0db..7cd71e1 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDRTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDRTest.java
@@ -65,7 +65,7 @@
 /**
  * Hive DR Testing.
  */
-@Test(groups = "embedded")
+@Test(groups = {"embedded", "multiCluster"})
 public class HiveDRTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(HiveDRTest.class);
     private static final String DB_NAME = "hdr_sdb1";
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDbDRTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDbDRTest.java
index 5efd69f..e281bee 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDbDRTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/hive/dr/HiveDbDRTest.java
@@ -59,7 +59,7 @@
 /**
  * Hive DR Testing for Hive database replication.
  */
-@Test(groups = "embedded")
+@Test(groups = {"embedded", "multiCluster"})
 public class HiveDbDRTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(HiveDbDRTest.class);
     private final ColoHelper cluster = servers.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/EntitySummaryTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/EntitySummaryTest.java
index 849f67a..ac91d59 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/EntitySummaryTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/EntitySummaryTest.java
@@ -128,7 +128,7 @@
      * Get status of 7 feeds and 7 instances of each feed the call should give correct information,
      * instance info must be recent.
      */
-    @Test
+    @Test(groups = "multiCluster")
     public void getFeedSummary() throws Exception {
         //prepare feed template.
         bundles[0].setInputFeedPeriodicity(5, Frequency.TimeUnit.minutes);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/ListFeedInstancesTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/ListFeedInstancesTest.java
index 4f86594..93efbac 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/ListFeedInstancesTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/lineage/ListFeedInstancesTest.java
@@ -59,7 +59,7 @@
  * expected instance statuses which are being compared with actual result of -list request
  * with different parameters in different order, variation, etc.
  */
-@Test(groups = { "distributed", "embedded", "sanity" })
+@Test(groups = { "distributed", "embedded", "sanity", "multiCluster"})
 public class ListFeedInstancesTest extends BaseTestClass {
     private static final Logger LOGGER = Logger.getLogger(ListFeedInstancesTest.class);
     private OozieClient cluster2OC = serverOC.get(1);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeScheduleTest.java
index 54e7805..81e0a7e 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/nativeScheduler/NativeScheduleTest.java
@@ -184,7 +184,7 @@
      * Successfully schedule process via native scheduler through prism and server on multiple cluster.
      * Schedule the same process on oozie. It should fail.
      */
-    @Test(groups = {"prism", "0.2"})
+    @Test(groups = {"prism", "0.2", "multiCluster"})
     public void scheduleProcessWithNativeOnTwoClusters() throws Exception {
 
         ProcessMerlin processMerlinNative = bundles[0].getProcessObject();
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
index 405725d..0a5e9ce 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedReplicationUpdateTest.java
@@ -49,7 +49,7 @@
 /**
  * Update replication feed tests.
  */
-@Test(groups = { "distributed", "embedded" })
+@Test(groups = { "distributed", "embedded", "multiCluster" })
 public class PrismFeedReplicationUpdateTest extends BaseTestClass {
 
     private ColoHelper cluster1 = servers.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
index 137ef6f..1a1dc98 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedScheduleTest.java
@@ -38,7 +38,7 @@
 /**
  * Schedule feed via prism tests.
  */
-@Test(groups = { "distributed", "embedded" })
+@Test(groups = { "distributed", "embedded", "multiCluster"})
 public class PrismFeedScheduleTest extends BaseTestClass {
 
     private OozieClient cluster1OC = serverOC.get(0);
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
index a5220e3..6caac9f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismFeedUpdateTest.java
@@ -96,7 +96,7 @@
      * Set 2 processes with common output feed. Second one is zero-input process. Update feed
      * queue. TODO : complete test case
      */
-    @Test(enabled = true, timeOut = 1200000)
+    @Test(enabled = true, timeOut = 1200000 , groups = "multiCluster")
     public void updateFeedQueueDependentMultipleProcessOneProcessZeroInput() throws Exception {
         //cluster1colo and cluster2colo are source. feed01 on cluster1colo target cluster2colo,
         // feed02 on cluster2colo target cluster1colo
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
index 4aa7189..63f793f 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/prism/PrismProcessSnSTest.java
@@ -80,7 +80,7 @@
      * Submit and schedule process2 on cluster2. Check that process2 is running and process1 is
      * not running on cluster2.
      */
-    @Test(groups = {"prism", "0.2", "embedded"})
+    @Test(groups = {"prism", "0.2", "embedded", "multiCluster"})
     public void testProcessSnSOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitAndScheduleProcess();
@@ -100,7 +100,7 @@
      * on cluster2. Submit process2 but schedule process1 once more. Check that process1 is running
      * on cluster1 but not on cluster2.
      */
-    @Test(groups = {"prism", "0.2", "embedded"})
+    @Test(groups = {"prism", "0.2", "embedded", "multiCluster"})
     public void testProcessSnSForSubmittedProcessOnBothColos() throws Exception {
         //schedule both bundles
         bundles[0].submitProcess(true);
@@ -122,7 +122,7 @@
      * once more and check that it is still running on cluster1 but process2 isn't running on
      * cluster2.
      */
-    @Test(groups = {"prism", "0.2", "embedded"})
+    @Test(groups = {"prism", "0.2", "embedded", "multiCluster"})
     public void testProcessSnSForSubmittedProcessOnBothColosUsingColoHelper()
         throws Exception {
         bundles[0].submitProcess(true);
@@ -228,7 +228,7 @@
      * running. Delete both of them. Submit and schedule them once more. Check that they are
      * running again.
      */
-    @Test(groups = {"prism", "0.2", "embedded"})
+    @Test(groups = {"prism", "0.2", "embedded", "multiCluster"})
     public void testSnSDeletedProcessOnBothColos() throws Exception {
         //schedule both bundles
         final String cluster1Running = cluster1.getClusterHelper().getColoName() + "/RUNNING";
diff --git a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/searchUI/MirrorTest.java b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/searchUI/MirrorTest.java
index e99202b..a7887da 100644
--- a/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/searchUI/MirrorTest.java
+++ b/falcon-regression/merlin/src/test/java/org/apache/falcon/regression/searchUI/MirrorTest.java
@@ -56,7 +56,7 @@
 import java.util.Arrays;
 
 /** UI tests for Mirror Setup Wizard. */
-@Test(groups = "search-ui")
+@Test(groups = {"search-ui", "multiCluster"})
 public class MirrorTest extends BaseUITestClass {
     private static final Logger LOGGER = Logger.getLogger(MirrorTest.class);
     private final String baseTestDir = cleanAndGetTestDir();
diff --git a/falcon-regression/pom.xml b/falcon-regression/pom.xml
index daa88cb..987ddaa 100644
--- a/falcon-regression/pom.xml
+++ b/falcon-regression/pom.xml
@@ -24,11 +24,11 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <groupId>org.apache.falcon.regression</groupId>
     <artifactId>falcon-regression</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Regression Framework for Falcon</description>
     <name>Apache Falcon Regression</name>
     <packaging>pom</packaging>
diff --git a/falcon-ui/app/css/img/user.svg b/falcon-ui/app/css/img/user.svg
index 60ac6c5..fb534c2 100644
--- a/falcon-ui/app/css/img/user.svg
+++ b/falcon-ui/app/css/img/user.svg
@@ -1,4 +1,20 @@
 <?xml version="1.0" encoding="utf-8"?>
+<!--
+   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.
+  -->
 <!-- Generator: Adobe Illustrator 16.0.3, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
 <!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
 <svg version="1.1" id="User" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" x="0px" y="0px"
diff --git a/falcon-ui/app/css/styles/autocomplete-tags.less b/falcon-ui/app/css/styles/autocomplete-tags.less
index c2f5dc2..4c6fc96 100644
--- a/falcon-ui/app/css/styles/autocomplete-tags.less
+++ b/falcon-ui/app/css/styles/autocomplete-tags.less
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
+
 .top-buffer{
   padding-top:20px;
 }
@@ -67,4 +85,4 @@
 }
 .suggestions-list:focus{
   outline:none;
-}
\ No newline at end of file
+}
diff --git a/falcon-ui/app/js/lib/popover.js b/falcon-ui/app/js/lib/popover.js
index e26c870..57814df 100644
--- a/falcon-ui/app/js/lib/popover.js
+++ b/falcon-ui/app/js/lib/popover.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
+
 (function(window, angular, undefined){
   'use strict';
 
@@ -460,4 +478,4 @@
       };
     }
   ]);
-})(window, window.angular);
\ No newline at end of file
+})(window, window.angular);
diff --git a/falcon-ui/app/test/e2e/protractor.js b/falcon-ui/app/test/e2e/protractor.js
index 37d6e65..6c2d97e 100644
--- a/falcon-ui/app/test/e2e/protractor.js
+++ b/falcon-ui/app/test/e2e/protractor.js
@@ -1,7 +1,25 @@
+/**
+ * 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.
+ */
+
 exports.config = {
   chromeDriver: '../../../node_modules/protractor/selenium/chromedriver',
 
   jasmineNodeOpts: {
     showColors: true
   }
-};
\ No newline at end of file
+};
diff --git a/falcon-ui/karma.conf.js b/falcon-ui/karma.conf.js
index 11189e8..776249f 100644
--- a/falcon-ui/karma.conf.js
+++ b/falcon-ui/karma.conf.js
@@ -1,3 +1,21 @@
+/**
+ * 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.
+ */
+
 // Karma configuration
 // Generated on Wed Sep 24 2014 21:15:41 GMT-0500 (CDT)
 
diff --git a/falcon-ui/pom.xml b/falcon-ui/pom.xml
index 93ed9bb..bbb917c 100644
--- a/falcon-ui/pom.xml
+++ b/falcon-ui/pom.xml
@@ -21,7 +21,7 @@
   <parent>
     <groupId>org.apache.falcon</groupId>
     <artifactId>falcon-main</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
   </parent>
   <artifactId>falcon-ui</artifactId>
   <packaging>pom</packaging>
diff --git a/hadoop-dependencies/pom.xml b/hadoop-dependencies/pom.xml
index e2529f1..384cceb 100644
--- a/hadoop-dependencies/pom.xml
+++ b/hadoop-dependencies/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-hadoop-dependencies</artifactId>
     <description>Apache Falcon Hadoop Dependencies Module</description>
diff --git a/lifecycle/pom.xml b/lifecycle/pom.xml
index 725f1e6..b940796 100644
--- a/lifecycle/pom.xml
+++ b/lifecycle/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-feed-lifecycle</artifactId>
     <description>Apache Falcon Lifecycle Module</description>
diff --git a/lifecycle/src/main/resources/action/feed/eviction-action.xml b/lifecycle/src/main/resources/action/feed/eviction-action.xml
index 4ab67d2..bded1d6 100644
--- a/lifecycle/src/main/resources/action/feed/eviction-action.xml
+++ b/lifecycle/src/main/resources/action/feed/eviction-action.xml
@@ -31,7 +31,7 @@
             <!-- HCatalog jars -->
             <property>
                 <name>oozie.action.sharelib.for.java</name>
-                <value>hcatalog</value>
+                <value>hcatalog,hive</value>
             </property>
             <property>
                 <name>oozie.launcher.oozie.libpath</name>
diff --git a/messaging/pom.xml b/messaging/pom.xml
index 667c5d1..3baddd9 100644
--- a/messaging/pom.xml
+++ b/messaging/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-messaging</artifactId>
     <description>Apache Falcon JMS messaging Module</description>
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
index 8b48e93..90bbdd3 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/JMSMessageConsumer.java
@@ -92,8 +92,7 @@
 
             topicSession = (TopicSession) connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
             Topic destination = topicSession.createTopic(topicName);
-            topicSubscriber = topicSession.createDurableSubscriber(destination, FALCON_CLIENT_ID,
-                    WorkflowNameBuilder.WorkflowName.getJMSFalconSelector(), false);
+            topicSubscriber = topicSession.createDurableSubscriber(destination, FALCON_CLIENT_ID);
             topicSubscriber.setMessageListener(this);
 
             connection.setExceptionListener(this);
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
index 0ba9464..cffdb59 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/JMSMessageConsumerTest.java
@@ -83,11 +83,6 @@
 
     public void sendMessages(String topic, WorkflowExecutionContext.Type type)
         throws JMSException, FalconException, IOException {
-        sendMessages(topic, type, true);
-    }
-
-    public void sendMessages(String topic, WorkflowExecutionContext.Type type, boolean isFalconWF)
-        throws JMSException, FalconException, IOException {
         ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(BROKER_URL);
         Connection connection = connectionFactory.createConnection();
         connection.start();
@@ -105,10 +100,10 @@
                 message = getMockFalconMessage(i, session);
                 break;
             case WORKFLOW_JOB:
-                message = getMockOozieMessage(i, session, isFalconWF);
+                message = getMockOozieMessage(i, session);
                 break;
             case COORDINATOR_ACTION:
-                message = getMockOozieCoordMessage(i, session, isFalconWF);
+                message = getMockOozieCoordMessage(i, session);
             default:
                 break;
             }
@@ -117,15 +112,10 @@
         }
     }
 
-    private Message getMockOozieMessage(int i, Session session, boolean isFalconWF)
-        throws FalconException, JMSException {
+    private Message getMockOozieMessage(int i, Session session) throws FalconException, JMSException {
         TextMessage message = session.createTextMessage();
         message.setStringProperty("appType", "WORKFLOW_JOB");
-        if (isFalconWF) {
-            message.setStringProperty("appName", "FALCON_PROCESS_DEFAULT_process1");
-        } else {
-            message.setStringProperty("appName", "OozieSampleShellWF");
-        }
+        message.setStringProperty("appName", "FALCON_PROCESS_DEFAULT_process1");
         message.setStringProperty("user", "falcon");
         switch(i % 4) {
         case 0:
@@ -152,15 +142,11 @@
         return message;
     }
 
-    private Message getMockOozieCoordMessage(int i, Session session, boolean isFalconWF)
+    private Message getMockOozieCoordMessage(int i, Session session)
         throws FalconException, JMSException {
         TextMessage message = session.createTextMessage();
         message.setStringProperty("appType", "COORDINATOR_ACTION");
-        if (isFalconWF) {
-            message.setStringProperty("appName", "FALCON_PROCESS_DEFAULT_process1");
-        } else {
-            message.setStringProperty("appName", "OozieSampleShellWF");
-        }
+        message.setStringProperty("appName", "FALCON_PROCESS_DEFAULT_process1");
         message.setStringProperty("user", "falcon");
         switch(i % 5) {
         case 0:
@@ -245,15 +231,10 @@
             sendMessages(TOPIC_NAME, WorkflowExecutionContext.Type.POST_PROCESSING);
 
             final BrokerView adminView = broker.getAdminView();
-
-            Assert.assertEquals(adminView.getTotalDequeueCount(), 0);
-//            Assert.assertEquals(adminView.getTotalEnqueueCount(), 10);
             Assert.assertEquals(adminView.getTotalConsumerCount(), 2);
 
             sendMessages(SECONDARY_TOPIC_NAME, WorkflowExecutionContext.Type.POST_PROCESSING);
 
-//            Assert.assertEquals(adminView.getTotalEnqueueCount(), 18);
-            Assert.assertEquals(adminView.getTotalDequeueCount(), 0);
             Assert.assertEquals(adminView.getTotalConsumerCount(), 3);
         } catch (Exception e) {
             Assert.fail("This should not have thrown an exception.", e);
@@ -265,9 +246,6 @@
         sendMessages(TOPIC_NAME, WorkflowExecutionContext.Type.WORKFLOW_JOB);
 
         final BrokerView adminView = broker.getAdminView();
-
-        Assert.assertEquals(adminView.getTotalDequeueCount(), 0);
-//        Assert.assertEquals(adminView.getTotalEnqueueCount(), 10);
         Assert.assertEquals(adminView.getTotalConsumerCount(), 2);
 
         // Async operations. Give some time for messages to be processed.
@@ -283,9 +261,6 @@
         sendMessages(TOPIC_NAME, WorkflowExecutionContext.Type.COORDINATOR_ACTION);
 
         final BrokerView adminView = broker.getAdminView();
-
-        Assert.assertEquals(adminView.getTotalDequeueCount(), 0);
-//        Assert.assertEquals(adminView.getTotalEnqueueCount(), 12);
         Assert.assertEquals(adminView.getTotalConsumerCount(), 2);
 
         // Async operations. Give some time for messages to be processed.
@@ -303,24 +278,4 @@
         broker.stop();
         subscriber.closeSubscriber();
     }
-
-    @Test
-    public void testJMSMessagesFromOozieForNonFalconWF() throws Exception {
-        sendMessages(TOPIC_NAME, WorkflowExecutionContext.Type.WORKFLOW_JOB, false /* isFalconWF */);
-
-        final BrokerView adminView = broker.getAdminView();
-
-        Assert.assertEquals(adminView.getTotalDequeueCount(), 0);
-        Assert.assertEquals(adminView.getTotalEnqueueCount(), 10);
-        Assert.assertEquals(adminView.getTotalConsumerCount(), 2);
-        Assert.assertEquals(adminView.getTotalMessageCount(), 0);
-
-        Thread.sleep(100);
-        Mockito.verify(jobEndService, Mockito.never()).notifyStart(Mockito.any(WorkflowExecutionContext.class));
-        Mockito.verify(jobEndService, Mockito.never()).notifySuccess(Mockito.any(WorkflowExecutionContext.class));
-        Mockito.verify(jobEndService, Mockito.never()).notifySuspend(Mockito.any(WorkflowExecutionContext.class));
-        Mockito.verify(jobEndService, Mockito.never()).notifyWait(Mockito.any(WorkflowExecutionContext.class));
-        Mockito.verify(jobEndService, Mockito.never()).notifyFailure(Mockito.any(WorkflowExecutionContext.class));
-    }
-
 }
diff --git a/metrics/pom.xml b/metrics/pom.xml
index 48cf80d..6266d0d 100644
--- a/metrics/pom.xml
+++ b/metrics/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-metrics</artifactId>
     <description>Apache Falcon Metrics</description>
diff --git a/oozie-el-extensions/pom.xml b/oozie-el-extensions/pom.xml
index 71ae45d..afbb2e3 100644
--- a/oozie-el-extensions/pom.xml
+++ b/oozie-el-extensions/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-oozie-el-extension</artifactId>
     <description>Apache Falcon Oozie EL Extension</description>
diff --git a/oozie-el-extensions/src/main/java/org/apache/oozie/extensions/OozieELExtensions.java b/oozie-el-extensions/src/main/java/org/apache/oozie/extensions/OozieELExtensions.java
index a6ff487..f0cb7cd 100644
--- a/oozie-el-extensions/src/main/java/org/apache/oozie/extensions/OozieELExtensions.java
+++ b/oozie-el-extensions/src/main/java/org/apache/oozie/extensions/OozieELExtensions.java
@@ -100,7 +100,7 @@
                     String emptyDir = (String) eval.getVariable(dataInName + ".empty-dir");
                     XLog.getLog(OozieELExtensions.class).debug("No instances could be resolved. Passing empty dir : "
                             + emptyDir);
-                    uristr = emptyDir;
+                    return emptyDir;
                 }
             } catch (Exception e) {
                 throw new RuntimeException("Failed to resolve instance range for " + dataInName, e);
diff --git a/oozie-el-extensions/src/test/java/org/apache/oozie/extensions/TestOozieELExtensions.java b/oozie-el-extensions/src/test/java/org/apache/oozie/extensions/TestOozieELExtensions.java
index b9bf594..2be8603 100644
--- a/oozie-el-extensions/src/test/java/org/apache/oozie/extensions/TestOozieELExtensions.java
+++ b/oozie-el-extensions/src/test/java/org/apache/oozie/extensions/TestOozieELExtensions.java
@@ -168,6 +168,8 @@
                 "*/US", "_DONE", },
             // With availability flag. All instances missing
             {"hdfs://localhost:8020/projects/falcon/staging/EMPTY_DIR_DONT_DELETE", "null", "_FINISH"},
+            // With availability flag and partitions. All instances missing
+            {"hdfs://localhost:8020/projects/falcon/staging/EMPTY_DIR_DONT_DELETE", "*", "_FINISH"},
             // No availability flag. One instance missing
             {"hdfs://localhost:8020/clicks/2009/09/02/09", "null", ""},
             // With availability flag. One instance missing.
diff --git a/oozie/pom.xml b/oozie/pom.xml
index c83daf6..a784c5a 100644
--- a/oozie/pom.xml
+++ b/oozie/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-oozie-adaptor</artifactId>
     <description>Apache Falcon Oozie Adaptor Module</description>
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
index b0e46f0..07d293c 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationCoordinatorBuilder.java
@@ -275,11 +275,6 @@
             Path scriptPath = new Path(buildPath, "scripts");
             copyHiveScript(fs, scriptPath, IMPORT_HQL);
             copyHiveScript(fs, scriptPath, EXPORT_HQL);
-
-            // create hive conf to stagingDir
-            Path confPath = new Path(buildPath + "/conf");
-            persistHiveConfiguration(fs, confPath, srcCluster, "falcon-source-");
-            persistHiveConfiguration(fs, confPath, trgCluster, "falcon-target-");
         } catch (IOException e) {
             throw new FalconException("Unable to create hive conf files", e);
         }
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationWorkflowBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationWorkflowBuilder.java
index 5a62130..010446b 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationWorkflowBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/FeedReplicationWorkflowBuilder.java
@@ -24,7 +24,6 @@
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.FeedHelper;
-import org.apache.falcon.entity.HiveUtil;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
@@ -79,11 +78,7 @@
         marshal(cluster, workflow, buildPath);
         Properties props = getProperties(buildPath, wfName);
         props.putAll(createDefaultConfiguration(cluster));
-        if (EntityUtil.isTableStorageType(cluster, entity)) {
-            // todo: kludge send source hcat creds for coord dependency check to pass
-            props.putAll(HiveUtil.getHiveCredentials(srcCluster));
-            props.putAll(HiveUtil.getHiveCredentials(cluster));
-        }
+
         props.putAll(getWorkflowProperties(entity));
         props.putAll(FeedHelper.getUserWorkflowProperties(getLifecycle()));
         // Write out the config to config-default.xml
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/feed/HCatReplicationWorkflowBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/feed/HCatReplicationWorkflowBuilder.java
index 629485d..3da97d3 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/feed/HCatReplicationWorkflowBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/feed/HCatReplicationWorkflowBuilder.java
@@ -22,15 +22,18 @@
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
+import org.apache.falcon.entity.HiveUtil;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.oozie.workflow.ACTION;
 import org.apache.falcon.oozie.workflow.WORKFLOWAPP;
+import org.apache.falcon.oozie.workflow.CONFIGURATION;
 import org.apache.falcon.util.OozieUtils;
 import org.apache.falcon.workflow.WorkflowExecutionArgs;
 
 import javax.xml.bind.JAXBElement;
 import java.util.Arrays;
+import java.util.Map;
 import java.util.Properties;
 
 /**
@@ -60,6 +63,15 @@
         //Add pre-processing
         if (shouldPreProcess()) {
             ACTION action = getPreProcessingAction(false, Tag.REPLICATION);
+            Properties hiveConf = HiveUtil.getHiveCredentials(src);
+            for (Map.Entry<Object, Object> e : hiveConf.entrySet()) {
+                CONFIGURATION.Property prop = new CONFIGURATION.Property();
+                prop.setName((String) e.getKey());
+                prop.setValue((String) e.getValue());
+                LOG.info("Adding config to replication hive preprocessing action : key = {}  value = {}",
+                        e.getKey(), e.getValue());
+                action.getJava().getConfiguration().getProperty().add(prop);
+            }
             addHDFSServersConfig(action, src, target);
             addTransition(action, EXPORT_ACTION_NAME, FAIL_POSTPROCESS_ACTION_NAME);
             workflow.getDecisionOrForkOrJoin().add(action);
@@ -72,6 +84,16 @@
             OozieUtils.unMarshalHiveAction(export);
         org.apache.falcon.oozie.hive.ACTION hiveExportAction = exportActionJaxbElement.getValue();
         addHDFSServersConfig(hiveExportAction, src, target);
+        Properties hiveConf = HiveUtil.getHiveCredentials(src);
+        for (Map.Entry<Object, Object> e : hiveConf.entrySet()) {
+            org.apache.falcon.oozie.hive.CONFIGURATION.Property prop =
+                    new org.apache.falcon.oozie.hive.CONFIGURATION.Property();
+            prop.setName((String) e.getKey());
+            prop.setValue((String) e.getValue());
+            LOG.info("Adding config to replication hive export action : key = {}  value = {}",
+                    e.getKey(), e.getValue());
+            hiveExportAction.getConfiguration().getProperty().add(prop);
+        }
         OozieUtils.marshalHiveAction(export, exportActionJaxbElement);
         addTransition(export, REPLICATION_ACTION_NAME, FAIL_POSTPROCESS_ACTION_NAME);
         workflow.getDecisionOrForkOrJoin().add(export);
@@ -89,6 +111,16 @@
             OozieUtils.unMarshalHiveAction(importAction);
         org.apache.falcon.oozie.hive.ACTION hiveImportAction = importActionJaxbElement.getValue();
         addHDFSServersConfig(hiveImportAction, src, target);
+        Properties hiveConf2 = HiveUtil.getHiveCredentials(target);
+        for (Map.Entry<Object, Object> e : hiveConf2.entrySet()) {
+            org.apache.falcon.oozie.hive.CONFIGURATION.Property prop =
+                    new org.apache.falcon.oozie.hive.CONFIGURATION.Property();
+            prop.setName((String) e.getKey());
+            prop.setValue((String) e.getValue());
+            LOG.info("Adding config to replication hive import action : key = {}  value = {}",
+                    e.getKey(), e.getValue());
+            hiveImportAction.getConfiguration().getProperty().add(prop);
+        }
         OozieUtils.marshalHiveAction(importAction, importActionJaxbElement);
         addTransition(importAction, CLEANUP_ACTION_NAME, FAIL_POSTPROCESS_ACTION_NAME);
         workflow.getDecisionOrForkOrJoin().add(importAction);
@@ -133,8 +165,8 @@
                 (org.apache.falcon.oozie.workflow.ACTION) object;
             String actionName = action.getName();
             if (PREPROCESS_ACTION_NAME.equals(actionName)) {
+
                 // add reference to hive-site conf to each action
-                action.getJava().setJobXml("${wf:appPath()}/conf/falcon-source-hive-site.xml");
 
                 if (isSecurityEnabled) { // add a reference to credential in the action
                     action.setCred(SOURCE_HIVE_CREDENTIAL_NAME);
diff --git a/oozie/src/main/java/org/apache/falcon/oozie/process/SparkProcessWorkflowBuilder.java b/oozie/src/main/java/org/apache/falcon/oozie/process/SparkProcessWorkflowBuilder.java
index dc5a491..51db75d 100644
--- a/oozie/src/main/java/org/apache/falcon/oozie/process/SparkProcessWorkflowBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/oozie/process/SparkProcessWorkflowBuilder.java
@@ -30,12 +30,10 @@
 import org.apache.falcon.entity.v0.process.Input;
 import org.apache.falcon.entity.v0.process.Output;
 import org.apache.falcon.entity.v0.process.Process;
-import org.apache.falcon.hadoop.HadoopClientFactory;
 import org.apache.falcon.oozie.spark.CONFIGURATION.Property;
 import org.apache.falcon.oozie.workflow.ACTION;
 import org.apache.falcon.oozie.workflow.CONFIGURATION;
 import org.apache.falcon.util.OozieUtils;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 import javax.xml.bind.JAXBElement;
@@ -46,6 +44,7 @@
  */
 public class SparkProcessWorkflowBuilder extends ProcessExecutionWorkflowBuilder {
     private static final String ACTION_TEMPLATE = "/action/process/spark-action.xml";
+    private static final String FALCON_PREFIX = "falcon_";
 
     public SparkProcessWorkflowBuilder(Process entity) {
         super(entity);
@@ -58,7 +57,7 @@
         org.apache.falcon.oozie.spark.ACTION sparkAction = actionJaxbElement.getValue();
 
         String sparkMasterURL = entity.getSparkAttributes().getMaster();
-        String sparkFilePath = entity.getSparkAttributes().getJar();
+        Path sparkJarFilePath = new Path(entity.getSparkAttributes().getJar());
         String sparkJobName = entity.getSparkAttributes().getName();
         String sparkOpts = entity.getSparkAttributes().getSparkOpts();
         String sparkClassName = entity.getSparkAttributes().getClazz();
@@ -89,21 +88,32 @@
             argList.addAll(sparkArgs);
         }
 
-        addInputFeedsAsArgument(argList, cluster);
+        //Adding output first so that final order must have input and then output followed by user's arguments.
         addOutputFeedsAsArgument(argList, cluster);
+        addInputFeedsAsArgument(argList, cluster);
 
-        sparkAction.setJar(addUri(sparkFilePath, cluster));
-
-        setSparkLibFileToWorkflowLib(sparkFilePath, entity);
+        // In Oozie spark action, value for jar is either Java jar file path or Python file path.
+        validateSparkJarFilePath(sparkJarFilePath);
+        sparkAction.setJar(sparkJarFilePath.getName());
+        setSparkLibFileToWorkflowLib(sparkJarFilePath.toString(), entity);
         propagateEntityProperties(sparkAction);
 
         OozieUtils.marshalSparkAction(action, actionJaxbElement);
         return action;
     }
 
-    private void setSparkLibFileToWorkflowLib(String sparkFile, Process entity) {
+    private void setSparkLibFileToWorkflowLib(String sparkJarFilePath, Process entity) {
         if (StringUtils.isEmpty(entity.getWorkflow().getLib())) {
-            entity.getWorkflow().setLib(sparkFile);
+            entity.getWorkflow().setLib(sparkJarFilePath);
+        } else {
+            String workflowLib = entity.getWorkflow().getLib() + "," + sparkJarFilePath;
+            entity.getWorkflow().setLib(workflowLib);
+        }
+    }
+
+    private void validateSparkJarFilePath(Path sparkJarFilePath) throws FalconException {
+        if (!sparkJarFilePath.isAbsolute()) {
+            throw new FalconException("Spark jar file path must be absolute:"+sparkJarFilePath);
         }
     }
 
@@ -145,6 +155,7 @@
             return;
         }
 
+        //Adding to the 0th index and getting the args shifted as arguments are added to get the desired effect.
         int numInputFeed = entity.getInputs().getInputs().size();
         while (numInputFeed > 0) {
             Input input = entity.getInputs().getInputs().get(numInputFeed-1);
@@ -153,6 +164,10 @@
             final String inputName = input.getName();
             if (storage.getType() == Storage.TYPE.FILESYSTEM) {
                 argList.add(0, "${" + inputName + "}");
+            } else if (storage.getType() == Storage.TYPE.TABLE) {
+                argList.add(0, "${" + FALCON_PREFIX+inputName+"_database" + "}");
+                argList.add(0, "${" + FALCON_PREFIX+inputName+"_table" + "}");
+                argList.add(0, "${" + FALCON_PREFIX+inputName+"_partition_filter_hive" + "}");
             }
             numInputFeed--;
         }
@@ -163,26 +178,24 @@
             return;
         }
 
-        for(Output output : entity.getOutputs().getOutputs()) {
+        //Adding to the 0th index and getting the args shifted as arguments are added to get the desired effect.
+        int numOutputFeed = entity.getOutputs().getOutputs().size();
+        while (numOutputFeed > 0) {
+            Output output = entity.getOutputs().getOutputs().get(numOutputFeed-1);
             Feed feed = EntityUtil.getEntity(EntityType.FEED, output.getFeed());
             Storage storage = FeedHelper.createStorage(cluster, feed);
             final String outputName = output.getName();
             if (storage.getType() == Storage.TYPE.FILESYSTEM) {
-                argList.add(argList.size(), "${" + outputName + "}");
+                argList.add(0, "${" + outputName + "}");
+            } else if (storage.getType() == Storage.TYPE.TABLE) {
+                argList.add(0, "${" + FALCON_PREFIX+outputName+"_database" + "}");
+                argList.add(0, "${" + FALCON_PREFIX+outputName+"_table" + "}");
+                argList.add(0, "${" + FALCON_PREFIX+outputName+"_partitions_hive" + "}");
             }
+            numOutputFeed--;
         }
     }
 
-    private String addUri(String jarFile, Cluster cluster) throws FalconException {
-        FileSystem fs = HadoopClientFactory.get().createProxiedFileSystem(
-                ClusterHelper.getConfiguration(cluster));
-        Path jarFilePath = new Path(jarFile);
-        if (jarFilePath.isAbsoluteAndSchemeAuthorityNull()) {
-            return fs.makeQualified(jarFilePath).toString();
-        }
-        return jarFile;
-    }
-
     private String getClusterEntitySparkMaster(Cluster cluster) {
         return ClusterHelper.getSparkMasterEndPoint(cluster);
     }
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
index 4961896..ea914f6 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
@@ -48,6 +48,14 @@
         // serialize the context to HDFS under logs dir before sending the message
         context.serialize();
 
+        boolean systemNotificationEnabled = Boolean.parseBoolean(context.
+            getValue(WorkflowExecutionArgs.SYSTEM_JMS_NOTIFICATION_ENABLED, "true"));
+
+        if (systemNotificationEnabled) {
+            LOG.info("Sending Falcon message {} ", context);
+            invokeFalconMessageProducer(context);
+        }
+
         String userBrokerUrl = context.getValue(WorkflowExecutionArgs.USER_BRKR_URL);
         boolean userNotificationEnabled = Boolean.parseBoolean(context.
                 getValue(WorkflowExecutionArgs.USER_JMS_NOTIFICATION_ENABLED, "true"));
@@ -72,6 +80,13 @@
         jmsMessageProducer.sendMessage(WorkflowExecutionContext.USER_MESSAGE_ARGS);
     }
 
+    private void invokeFalconMessageProducer(WorkflowExecutionContext context) throws Exception {
+        JMSMessageProducer jmsMessageProducer = JMSMessageProducer.builder(context)
+            .type(JMSMessageProducer.MessageType.FALCON)
+            .build();
+        jmsMessageProducer.sendMessage();
+    }
+
     private void invokeLogProducer(WorkflowExecutionContext context) {
         // todo: need to move this out to Falcon in-process
         if (UserGroupInformation.isSecurityEnabled()) {
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
index 25f7180..9a09f18 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
@@ -694,9 +694,9 @@
                 if (action.equals(JobAction.STATUS) && Boolean.TRUE.equals(allAttempts)) {
                     try {
                         performAction(cluster, action, coordinatorAction, props, instance, isForced);
-                        if (instance.getRunId() > 0) {
-                            instanceList = getAllInstances(cluster, coordinatorAction, nominalTimeStr);
-                        } else {
+                        instanceList = getAllInstances(cluster, coordinatorAction, nominalTimeStr);
+                        // Happens when the action is in READY/WAITING, when no workflow is kicked off yet.
+                        if (instanceList.isEmpty() || StringUtils.isBlank(coordinatorAction.getExternalId())) {
                             instanceList.add(instance);
                         }
                     } catch (FalconException e) {
@@ -883,8 +883,8 @@
     private List<InstancesResult.Instance> getAllInstances(String cluster, CoordinatorAction coordinatorAction,
                                                            String nominalTimeStr) throws FalconException {
         List<InstancesResult.Instance> instanceList = new ArrayList<>();
-        if (StringUtils.isNotBlank(coordinatorAction.getExternalId())) {
-            List<WorkflowJob> workflowJobList = getWfsForCoordAction(cluster, coordinatorAction.getExternalId());
+        if (StringUtils.isNotBlank(coordinatorAction.getId())) {
+            List<WorkflowJob> workflowJobList = getWfsForCoordAction(cluster, coordinatorAction.getId());
             if (workflowJobList != null && workflowJobList.size()>0) {
                 for (WorkflowJob workflowJob : workflowJobList) {
                     InstancesResult.Instance newInstance = new InstancesResult.Instance(cluster, nominalTimeStr, null);
@@ -892,7 +892,7 @@
                     if (wfJob!=null) {
                         newInstance.startTime = wfJob.getStartTime();
                         newInstance.endTime = wfJob.getEndTime();
-                        newInstance.logFile = coordinatorAction.getId();
+                        newInstance.logFile = wfJob.getConsoleUrl();
                         populateInstanceActions(cluster, wfJob, newInstance);
                         newInstance.status = WorkflowStatus.valueOf(mapActionStatus(wfJob.getStatus().name()));
                         instanceList.add(newInstance);
@@ -912,7 +912,7 @@
             status = jobInfo.getStatus().name();
             instance.startTime = jobInfo.getStartTime();
             instance.endTime = jobInfo.getEndTime();
-            instance.logFile = coordinatorAction.getId();
+            instance.logFile = jobInfo.getConsoleUrl();
             instance.runId = jobInfo.getRun();
         }
 
@@ -1603,6 +1603,8 @@
             } else if (jobId.endsWith("-B")) {
                 BundleJob bundle = client.getBundleJobInfo(jobId);
                 return bundle.getStatus().name();
+            } else if (jobId.contains("-C@")) {
+                return client.getCoordActionInfo(jobId).getStatus().name();
             }
             throw new IllegalArgumentException("Unhandled jobs id: " + jobId);
         } catch (Exception e) {
diff --git a/oozie/src/main/resources/action/feed/eviction-action.xml b/oozie/src/main/resources/action/feed/eviction-action.xml
index 4ab67d2..bded1d6 100644
--- a/oozie/src/main/resources/action/feed/eviction-action.xml
+++ b/oozie/src/main/resources/action/feed/eviction-action.xml
@@ -31,7 +31,7 @@
             <!-- HCatalog jars -->
             <property>
                 <name>oozie.action.sharelib.for.java</name>
-                <value>hcatalog</value>
+                <value>hcatalog,hive</value>
             </property>
             <property>
                 <name>oozie.launcher.oozie.libpath</name>
diff --git a/oozie/src/main/resources/action/feed/table-export.xml b/oozie/src/main/resources/action/feed/table-export.xml
index fcf1a1a..6bc214b 100644
--- a/oozie/src/main/resources/action/feed/table-export.xml
+++ b/oozie/src/main/resources/action/feed/table-export.xml
@@ -26,7 +26,6 @@
         <prepare>
             <delete path="${distcpSourcePaths}"/>
         </prepare>
-        <job-xml>${wf:appPath()}/conf/falcon-source-hive-site.xml</job-xml>
         <configuration>
             <property>
                 <name>mapred.job.queue.name</name>
diff --git a/oozie/src/main/resources/action/feed/table-import.xml b/oozie/src/main/resources/action/feed/table-import.xml
index 6e9a073..450e68a 100644
--- a/oozie/src/main/resources/action/feed/table-import.xml
+++ b/oozie/src/main/resources/action/feed/table-import.xml
@@ -20,7 +20,6 @@
     <hive xmlns="uri:oozie:hive-action:0.2">
         <job-tracker>${falconTargetJobTracker}</job-tracker>
         <name-node>${falconTargetNameNode}</name-node>
-        <job-xml>${wf:appPath()}/conf/falcon-target-hive-site.xml</job-xml>
         <configuration>
             <property>
                 <name>mapred.job.queue.name</name>
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
index b0fc230..2040748 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/feed/OozieFeedWorkflowBuilderTest.java
@@ -561,9 +561,6 @@
         Assert.assertTrue(fs.exists(new Path(wfPath + "/scripts/falcon-table-export.hql")));
         Assert.assertTrue(fs.exists(new Path(wfPath + "/scripts/falcon-table-import.hql")));
 
-        Assert.assertTrue(fs.exists(new Path(wfPath + "/conf")));
-        Assert.assertTrue(fs.exists(new Path(wfPath + "/conf/falcon-source-hive-site.xml")));
-        Assert.assertTrue(fs.exists(new Path(wfPath + "/conf/falcon-target-hive-site.xml")));
 
         HashMap<String, String> props = getCoordProperties(coord);
 
@@ -626,11 +623,7 @@
     private void assertReplicationHCatCredentials(WORKFLOWAPP wf, String wfPath) throws IOException {
         FileSystem fs = trgMiniDFS.getFileSystem();
 
-        Path hiveConfPath = new Path(wfPath, "conf/falcon-source-hive-site.xml");
-        Assert.assertTrue(fs.exists(hiveConfPath));
 
-        hiveConfPath = new Path(wfPath, "conf/falcon-target-hive-site.xml");
-        Assert.assertTrue(fs.exists(hiveConfPath));
 
         boolean isSecurityEnabled = SecurityUtil.isSecurityEnabled();
         if (isSecurityEnabled) {
@@ -651,7 +644,6 @@
             }
 
             if ("recordsize".equals(actionName)) {
-                Assert.assertEquals(action.getJava().getJobXml(), "${wf:appPath()}/conf/falcon-source-hive-site.xml");
                 if (isSecurityEnabled) {
                     Assert.assertNotNull(action.getCred());
                     Assert.assertEquals(action.getCred(), "falconSourceHiveAuth");
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
index 85100e7..a692d0c 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/process/OozieProcessWorkflowBuilderTest.java
@@ -326,6 +326,80 @@
     }
 
     @Test
+    public void testSparkSQLProcess() throws Exception {
+        URL resource = this.getClass().getResource("/config/feed/hive-table-feed.xml");
+        Feed inFeed = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(resource);
+        ConfigurationStore.get().publish(EntityType.FEED, inFeed);
+
+        resource = this.getClass().getResource("/config/feed/hive-table-feed-out.xml");
+        Feed outFeed = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(resource);
+        ConfigurationStore.get().publish(EntityType.FEED, outFeed);
+
+        resource = this.getClass().getResource("/config/process/spark-sql-process.xml");
+        Process process = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(resource);
+        ConfigurationStore.get().publish(EntityType.PROCESS, process);
+
+        prepare(process);
+        OozieEntityBuilder builder = OozieEntityBuilder.get(process);
+        Path bundlePath = new Path("/falcon/staging/workflows", process.getName());
+        builder.build(cluster, bundlePath);
+        assertTrue(fs.exists(bundlePath));
+
+        BUNDLEAPP bundle = getBundle(fs, bundlePath);
+        assertEquals(EntityUtil.getWorkflowName(process).toString(), bundle.getName());
+        assertEquals(1, bundle.getCoordinator().size());
+        assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString(),
+                bundle.getCoordinator().get(0).getName());
+        String coordPath = bundle.getCoordinator().get(0).getAppPath().replace("${nameNode}", "");
+
+        COORDINATORAPP coord = getCoordinator(fs, new Path(coordPath));
+        HashMap<String, String> props = getCoordProperties(coord);
+        HashMap<String, String> wfProps = getWorkflowProperties(fs, coord);
+
+        verifyEntityProperties(process, cluster,
+                WorkflowExecutionContext.EntityOperations.GENERATE, wfProps);
+        verifyBrokerProperties(cluster, wfProps);
+
+        // verify table and hive props
+        Map<String, String> expected = getExpectedProperties(inFeed, outFeed, process);
+        expected.putAll(ClusterHelper.getHiveProperties(cluster));
+        for (Map.Entry<String, String> entry : props.entrySet()) {
+            if (expected.containsKey(entry.getKey())) {
+                Assert.assertEquals(entry.getValue(), expected.get(entry.getKey()));
+            }
+        }
+
+        String wfPath = coord.getAction().getWorkflow().getAppPath().replace("${nameNode}", "");
+        WORKFLOWAPP parentWorkflow = getWorkflowapp(fs, new Path(wfPath, "workflow.xml"));
+        testParentWorkflow(process, parentWorkflow);
+        assertEquals(process.getWorkflow().getLib(), "/resources/action/lib/falcon-examples.jar");
+
+        ACTION sparkNode = getAction(parentWorkflow, "user-action");
+
+        JAXBElement<org.apache.falcon.oozie.spark.ACTION> actionJaxbElement =
+                OozieUtils.unMarshalSparkAction(sparkNode);
+        org.apache.falcon.oozie.spark.ACTION sparkAction = actionJaxbElement.getValue();
+
+        assertEquals(sparkAction.getMaster(), "local");
+        assertEquals(sparkAction.getJar(), "falcon-examples.jar");
+
+        Assert.assertTrue(Storage.TYPE.TABLE == ProcessHelper.getStorageType(cluster, process));
+        List<String> argsList = sparkAction.getArg();
+
+        Input input = process.getInputs().getInputs().get(0);
+        Output output = process.getOutputs().getOutputs().get(0);
+
+        assertEquals(argsList.get(0), "${falcon_"+input.getName()+"_partition_filter_hive}");
+        assertEquals(argsList.get(1), "${falcon_"+input.getName()+"_table}");
+        assertEquals(argsList.get(2), "${falcon_"+input.getName()+"_database}");
+        assertEquals(argsList.get(3), "${falcon_"+output.getName()+"_partitions_hive}");
+        assertEquals(argsList.get(4), "${falcon_"+output.getName()+"_table}");
+        assertEquals(argsList.get(5), "${falcon_"+output.getName()+"_database}");
+
+        ConfigurationStore.get().remove(EntityType.PROCESS, process.getName());
+    }
+
+    @Test
     public void testSparkProcess() throws Exception {
 
         URL resource = this.getClass().getResource(SPARK_PROCESS_XML);
@@ -357,6 +431,7 @@
         String wfPath = coord.getAction().getWorkflow().getAppPath().replace("${nameNode}", "");
         WORKFLOWAPP parentWorkflow = getWorkflowapp(fs, new Path(wfPath, "workflow.xml"));
         testParentWorkflow(process, parentWorkflow);
+        assertEquals(process.getWorkflow().getLib(), "/resources/action/lib/spark-wordcount.jar");
 
         ACTION sparkNode = getAction(parentWorkflow, "user-action");
 
@@ -364,7 +439,7 @@
                 OozieUtils.unMarshalSparkAction(sparkNode);
         org.apache.falcon.oozie.spark.ACTION sparkAction = actionJaxbElement.getValue();
         assertEquals(sparkAction.getMaster(), "local");
-        assertEquals(sparkAction.getJar(), "jail://testCluster:00/resources/action/lib/spark-wordcount.jar");
+        assertEquals(sparkAction.getJar(), "spark-wordcount.jar");
         List<String> argsList = sparkAction.getArg();
         Input input = process.getInputs().getInputs().get(0);
         Output output = process.getOutputs().getOutputs().get(0);
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
index 1c50a97..4132c3a 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
@@ -100,6 +100,9 @@
 
     @AfterClass
     public void tearDown() throws Exception {
+        if (broker.isStopped()) {
+            broker.start(true);
+        }
         broker.deleteAllMessages();
         broker.stop();
     }
diff --git a/oozie/src/test/resources/config/process/spark-sql-process.xml b/oozie/src/test/resources/config/process/spark-sql-process.xml
new file mode 100644
index 0000000..55ff89b
--- /dev/null
+++ b/oozie/src/test/resources/config/process/spark-sql-process.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+<process name="spark-sql-process" xmlns="uri:falcon:process:0.1">
+    <!-- where -->
+    <clusters>
+        <cluster name="corp">
+            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z"/>
+        </cluster>
+    </clusters>
+
+    <!-- when -->
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>hours(1)</frequency>
+    <timezone>UTC</timezone>
+
+    <!-- what -->
+    <inputs>
+        <input name="input" feed="clicks-raw-table" start="yesterday(0,0)" end="yesterday(20,0)"/>
+    </inputs>
+
+    <outputs>
+        <output name="output" feed="clicks-summary-table" instance="today(0,0)"/>
+    </outputs>
+
+    <workflow engine="spark" path="/resources/action"/>
+    <spark-attributes>
+        <master>local</master>
+        <name>Spark SQL</name>
+        <class>org.apache.falcon.example.spark.SparkSQLProcessTable</class>
+        <jar>/resources/action/lib/falcon-examples.jar</jar>
+        <spark-opts>--num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1</spark-opts>
+    </spark-attributes>
+
+    <retry policy="periodic" delay="minutes(3)" attempts="3"/>
+
+</process>
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 0d804b2..8e836da 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon</groupId>
     <artifactId>falcon-main</artifactId>
-    <version>0.10-SNAPSHOT</version>
+    <version>0.10</version>
     <description>Data Management and Processing Platform over Hadoop</description>
     <name>Apache Falcon</name>
     <packaging>pom</packaging>
@@ -98,7 +98,7 @@
         <oozie.version>4.2.0</oozie.version>
         <oozie.buildversion>${oozie.version}-falcon</oozie.buildversion>
         <oozie.forcebuild>false</oozie.forcebuild>
-        <activemq.version>5.12.0</activemq.version>
+        <activemq.version>5.13.3</activemq.version>
         <tinkerpop.version>2.6.0</tinkerpop.version>
         <titan.version>0.5.4</titan.version>
         <hbase.version>1.1.5</hbase.version>
@@ -106,7 +106,9 @@
         <spark.version>1.6.1</spark.version>
         <jetty.version>6.1.26</jetty.version>
         <jersey.version>1.9</jersey.version>
+        <jackson.version>2.2.3</jackson.version>
         <quartz.version>2.2.1</quartz.version>
+        <c3p0.version>0.9.5.2</c3p0.version>
         <joda.version>2.8.2</joda.version>
         <mockito.version>1.9.5</mockito.version>
         <openjpa.version>2.4.0</openjpa.version>
@@ -216,6 +218,7 @@
                                 <exclude>**/db1.script</exclude>
                                 <exclude>**/credential_provider.jceks</exclude>
                                 <exclude>**/*.json</exclude>
+                                <exclude>**/falcon-cli-hist.log</exclude>
                             </excludes>
                         </configuration>
                         <executions>
@@ -630,15 +633,21 @@
             </dependency>
 
             <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-core-asl</artifactId>
-                <version>1.9.2</version>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-annotations</artifactId>
+                <version>${jackson.version}</version>
             </dependency>
 
             <dependency>
-                <groupId>org.codehaus.jackson</groupId>
-                <artifactId>jackson-mapper-asl</artifactId>
-                <version>1.9.2</version>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-core</artifactId>
+                <version>${jackson.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-databind</artifactId>
+                <version>${jackson.version}</version>
             </dependency>
 
             <dependency>
@@ -748,12 +757,6 @@
             </dependency>
 
             <dependency>
-                <groupId>com.vividsolutions</groupId>
-                <artifactId>jts</artifactId>
-                <version>1.13</version>
-            </dependency>
-
-            <dependency>
                 <groupId>org.apache.falcon</groupId>
                 <artifactId>falcon-hadoop-dependencies</artifactId>
                 <version>${project.version}</version>
diff --git a/prism/pom.xml b/prism/pom.xml
index 2eddbc1..e7ec531 100644
--- a/prism/pom.xml
+++ b/prism/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../pom.xml</relativePath>
     </parent>
     <artifactId>falcon-prism</artifactId>
@@ -98,11 +98,13 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-metrics</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-rerun</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>
@@ -249,6 +251,14 @@
                         <include>org/apache/falcon/plugin/ChainableMonitoringPluginTest.java</include>
                         <include>org/apache/falcon/aspect/GenericAlert.java</include>
                         <include>org/apache/falcon/aspect/GenericAlertTest.java</include>
+                        <include>org/apache/falcon/security/FalconAuditFilter.java</include>
+                        <include>org/apache/falcon/resource/metadata/LineageMetadataResource</include>
+                        <include>org/apache/falcon/messaging/JMSMessageConsumer</include>
+                        <include>org/apache/falcon/service/LogCleanupService</include>
+                        <include>org/apache/falcon/cleanup/LogCleanupServiceTest</include>
+                        <include>org/apache/falcon/security/AuthenticationInitializationService</include>
+                        <include>org/apache/falcon/security/AuthenticationInitializationServiceTest</include>
+                        <include>org/apache/falcon/messaging/JMSMessageConsumerTest</include>
                     </includes>
                     <weaveDependencies>
                         <weaveDependency>
diff --git a/prism/src/main/java/org/apache/falcon/plugin/GraphiteNotificationPlugin.java b/prism/src/main/java/org/apache/falcon/plugin/GraphiteNotificationPlugin.java
index 9d46b0d..abe6777 100644
--- a/prism/src/main/java/org/apache/falcon/plugin/GraphiteNotificationPlugin.java
+++ b/prism/src/main/java/org/apache/falcon/plugin/GraphiteNotificationPlugin.java
@@ -45,34 +45,39 @@
         MetricNotificationService metricNotificationService =
                 Services.get().getService(MetricNotificationService.SERVICE_NAME);
         try {
-            String entityType = message.getDimensions().get("entity-type");
-            String entityName = message.getDimensions().get("entity-name");
+            String entityType = StringUtils.isNotBlank(message.getDimensions().get("entityType"))
+                    ? message.getDimensions().get("entityType") :message.getDimensions().get("entity-type");
+            String entityName = StringUtils.isNotBlank(message.getDimensions().get("entityName"))
+                    ? message.getDimensions().get("entityName") :message.getDimensions().get("entity-name");
             String prefix = StartupProperties.get().getProperty("falcon.graphite.prefix");
-            if (entityType.equals(EntityType.PROCESS.name())) {
+            String separator = ".";
+            LOG.debug("message:" + message.getAction());
+            if (entityType.equalsIgnoreCase(EntityType.PROCESS.name())) {
                 Entity entity = ConfigurationStore.get().get(EntityType.PROCESS, entityName);
                 Process process = (Process) entity;
                 String pipeline =  StringUtils.isNotBlank(process.getPipelines()) ? process.getPipelines() : "default";
 
-
                 if ((message.getAction().equals("wf-instance-succeeded"))) {
                     Long timeTaken =  message.getExecutionTime() / 1000000000;
-                    String metricsName = prefix + message.getDimensions().get("cluster") + pipeline
-                            + ".GENERATE." + entityName + ".processing_time";
-                    metricNotificationService.publish(metricsName, timeTaken);
+                    StringBuilder processingMetric = new StringBuilder(prefix).append(".").append(message.
+                            getDimensions().get("cluster")).append(".").append(pipeline).append(".GENERATE.")
+                            .append(entityName).append(".processing_time");
+                    metricNotificationService.publish(processingMetric.toString(), timeTaken);
 
                     DateTime nominalTime = new DateTime(message.getDimensions().get("nominal-time"));
                     DateTime startTime = new DateTime(message.getDimensions().get("start-time"));
-                    metricsName = prefix + message.getDimensions().get("cluster") + pipeline
-                            + ".GENERATE." + entityName + ".start_delay";
-                    metricNotificationService.publish(metricsName,
-                        (long)Seconds.secondsBetween(nominalTime, startTime).getSeconds());
+                    StringBuilder startTimeMetric = new StringBuilder(prefix).append(".").append(message.
+                            getDimensions().get("cluster")).append(".").append(pipeline).append(".GENERATE.").
+                            append(entityName).append(".start_delay");
+                    metricNotificationService.publish(startTimeMetric.toString(),
+                            (long)Seconds.secondsBetween(nominalTime, startTime).getSeconds());
                 }
 
                 if (message.getAction().equals("wf-instance-failed")){
-                    String metricName =  prefix + message.getDimensions().get("cluster") + pipeline
-                            + ".GENERATE." +  entityName + ".failure"
-                        + message.getDimensions().get("error-message");
-                    metricNotificationService.publish(metricName, (long) 1);
+                    StringBuilder metricName = new StringBuilder(prefix).append(".").append(message.
+                            getDimensions().get("cluster")).append(".").append(pipeline).append(".GENERATE.").
+                            append(entityName).append(".failure").append(message.getDimensions().get("error-message"));
+                    metricNotificationService.publish(metricName.toString(), (long) 1);
                 }
             }
         } catch (Exception e) {
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index 5fa345d..8ba9b82 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -168,15 +168,8 @@
             Set<String> clusters = EntityUtil.getClustersDefined(entity);
             Set<String> colos = new HashSet<String>();
             for (String cluster : clusters) {
-                try{
-                    Cluster clusterEntity = EntityUtil.getEntity(EntityType.CLUSTER, cluster);
-                    colos.add(clusterEntity.getColo());
-                } catch (EntityNotRegisteredException e){
-                    LOG.warn(e.getMessage(), e);
-                }
-            }
-            if (colos.isEmpty()) {
-                throw new EntityNotRegisteredException(entity.getName()  + " (" + type + ") not found");
+                Cluster clusterEntity = EntityUtil.getEntity(EntityType.CLUSTER, cluster);
+                colos.add(clusterEntity.getColo());
             }
             return colos;
         } catch (FalconException e) {
diff --git a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
index 788df58..f69b566 100644
--- a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
+++ b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
@@ -35,6 +35,7 @@
         server.addConnector(connector);
 
         WebAppContext application = new WebAppContext(path, "/");
+        application.setParentLoaderPriority(true);
         server.setHandler(application);
     }
 
diff --git a/prism/src/test/java/org/apache/falcon/resource/metadata/MetadataTestContext.java b/prism/src/test/java/org/apache/falcon/resource/metadata/MetadataTestContext.java
index 47d6ba1..0fc708d 100644
--- a/prism/src/test/java/org/apache/falcon/resource/metadata/MetadataTestContext.java
+++ b/prism/src/test/java/org/apache/falcon/resource/metadata/MetadataTestContext.java
@@ -95,6 +95,10 @@
         Services.get().register(new WorkflowJobEndNotificationService());
         Assert.assertTrue(Services.get().isRegistered(WorkflowJobEndNotificationService.SERVICE_NAME));
 
+        StartupProperties.get().setProperty("falcon.graph.storage.backend", "berkeleyje");
+        String graphDBDir = "target/graphdb-" + System.currentTimeMillis();
+        StartupProperties.get().setProperty("falcon.graph.storage.directory", graphDBDir);
+        StartupProperties.get().setProperty("falcon.graph.serialize.path", graphDBDir);
         StartupProperties.get().setProperty("falcon.graph.preserve.history", "true");
         service = new MetadataMappingService();
         service.init();
diff --git a/release-docs/0.10/CHANGES.0.10.md b/release-docs/0.10/CHANGES.0.10.md
new file mode 100644
index 0000000..bfe3a27
--- /dev/null
+++ b/release-docs/0.10/CHANGES.0.10.md
@@ -0,0 +1,220 @@
+# Apache Falcon Changelog
+
+## Release 0.10 - 2016-07-26
+
+### INCOMPATIBLE CHANGES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-1926](https://issues.apache.org/jira/browse/FALCON-1926) | Filter out effectively non-falcon related JMS messages from Oozie |  Major | messaging | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1858](https://issues.apache.org/jira/browse/FALCON-1858) | Support HBase as a storage backend for Falcon Titan graphDB |  Major | . | Ying Zheng | Venkat Ranganathan |
+| [FALCON-1852](https://issues.apache.org/jira/browse/FALCON-1852) | Optional Input for a process not truly optional |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1844](https://issues.apache.org/jira/browse/FALCON-1844) | Falcon feed replication leaves behind old files when a feed instance is re-run |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1835](https://issues.apache.org/jira/browse/FALCON-1835) | Falcon should do coord rerun rather than workflow rerun to ensure concurrency |  Major | . | Pallavi Rao | Pallavi Rao |
+
+
+### NEW FEATURES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-1956](https://issues.apache.org/jira/browse/FALCON-1956) | Graphite Plugin for monitoring |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1919](https://issues.apache.org/jira/browse/FALCON-1919) | Provide user the option to store sensitive information with Hadoop credential provider |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1865](https://issues.apache.org/jira/browse/FALCON-1865) | Persist Feed sla data to database |  Major | . | Ajay Yadava | Praveen Adlakha |
+| [FALCON-1861](https://issues.apache.org/jira/browse/FALCON-1861) | Support HDFS Snapshot based replication in Falcon |  Major | replication | Balu Vellanki | Balu Vellanki |
+| [FALCON-1763](https://issues.apache.org/jira/browse/FALCON-1763) | Create a spark execution engine for Falcon |  Major | . | Venkat Ranganathan | Peeyush Bishnoi |
+| [FALCON-1627](https://issues.apache.org/jira/browse/FALCON-1627) | Provider integration with Azure Data Factory pipelines |  Major | . | Venkat Ranganathan | Ying Zheng |
+| [FALCON-1623](https://issues.apache.org/jira/browse/FALCON-1623) | Implement Safe Mode in Falcon |  Major | . | sandeep samudrala | Balu Vellanki |
+| [FALCON-1333](https://issues.apache.org/jira/browse/FALCON-1333) | Support instance search of a group of entities |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-634](https://issues.apache.org/jira/browse/FALCON-634) | Add server side extensions in Falcon |  Major | . | Venkatesh Seetharam | Sowmya Ramesh |
+| [FALCON-141](https://issues.apache.org/jira/browse/FALCON-141) | Support cluster updates |  Major | . | Shwetha G S | Balu Vellanki |
+| [FALCON-36](https://issues.apache.org/jira/browse/FALCON-36) | Ability to ingest data from databases |  Major | acquisition | Venkatesh Seetharam | Venkatesan Ramachandran |
+
+
+### IMPROVEMENTS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-2020](https://issues.apache.org/jira/browse/FALCON-2020) | Changes in Graphite Notification Plugin |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1981](https://issues.apache.org/jira/browse/FALCON-1981) | Remove runtime superfluous jar dependencies - pom.xml cleanup |  Major | build-tools | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1979](https://issues.apache.org/jira/browse/FALCON-1979) | Update HttpClient versions to close security vulnerabilities |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-1963](https://issues.apache.org/jira/browse/FALCON-1963) | Falcon CLI should provide detailed hints if the user's command is invalid |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1942](https://issues.apache.org/jira/browse/FALCON-1942) | Allow Falcon server and client classpath to be customizable |  Major | . | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-1916](https://issues.apache.org/jira/browse/FALCON-1916) | Allow RM principal to be specified in Cluster entity |  Major | common | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-1895](https://issues.apache.org/jira/browse/FALCON-1895) | Refactoring of FalconCLI and FalconClient |  Major | client | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1841](https://issues.apache.org/jira/browse/FALCON-1841) | Grouping test in falcon for running nightly regression |  Major | regression | Pragya Mittal | Pragya Mittal |
+| [FALCON-1836](https://issues.apache.org/jira/browse/FALCON-1836) | Ingest to Hive |  Major | . | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1802](https://issues.apache.org/jira/browse/FALCON-1802) | Workflow Builder for scheduling based on Data for Process in case of Native Scheduler |  Major | . | pavan kumar kolamuri | pavan kumar kolamuri |
+| [FALCON-1774](https://issues.apache.org/jira/browse/FALCON-1774) | Better message for api not allowed on server |  Major | . | Sanjeev T | Praveen Adlakha |
+| [FALCON-1751](https://issues.apache.org/jira/browse/FALCON-1751) | Support assembly:single mojo |  Minor | . | ruoyu wang | ruoyu wang |
+| [FALCON-887](https://issues.apache.org/jira/browse/FALCON-887) | Support for multiple lib paths in falcon process |  Minor | process | Akshay Goyal | Sowmya Ramesh |
+| [FALCON-625](https://issues.apache.org/jira/browse/FALCON-625) | Documentation improvements |  Major | . | Paul Isaychuk | Ajay Yadava |
+
+
+### BUG FIXES:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-2107](https://issues.apache.org/jira/browse/FALCON-2107) | NPE in FalconWorkflowEngine::isActive() method |  Blocker | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-2104](https://issues.apache.org/jira/browse/FALCON-2104) | Loss of data in GraphDB when upgrading Falcon from 0.9 to 0.10 |  Blocker | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-2100](https://issues.apache.org/jira/browse/FALCON-2100) | Remove dependency on com.vividsolutions.jts |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-2090](https://issues.apache.org/jira/browse/FALCON-2090) | HDFS Snapshot failed with UnknownHostException when scheduling in HA Mode |  Critical | replication | Murali Ramasami | Balu Vellanki |
+| [FALCON-2088](https://issues.apache.org/jira/browse/FALCON-2088) | Entity submission fails with EntityNotRegisteredException in distributed mode |  Blocker | feed, prism, process | Pragya Mittal | Praveen Adlakha |
+| [FALCON-2084](https://issues.apache.org/jira/browse/FALCON-2084) | HCatReplicationTest are failing in secure mode |  Critical | replication | Murali Ramasami | Venkat Ranganathan |
+| [FALCON-2081](https://issues.apache.org/jira/browse/FALCON-2081) | ExtensionManagerIT fails occassionally |  Blocker | tests | Balu Vellanki | Balu Vellanki |
+| [FALCON-2076](https://issues.apache.org/jira/browse/FALCON-2076) | Server fails to start since extension.store.uri is not defined in startup.properties |  Major | prism | Pragya Mittal | Balu Vellanki |
+| [FALCON-2075](https://issues.apache.org/jira/browse/FALCON-2075) | Falcon HiveDR tasks do not report progress and can get killed |  Critical | . | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-2071](https://issues.apache.org/jira/browse/FALCON-2071) | Falcon Spark SQL failing with Yarn Client Mode |  Critical | process | Murali Ramasami | Peeyush Bishnoi |
+| [FALCON-2061](https://issues.apache.org/jira/browse/FALCON-2061) | Falcon CLI shows hadoop classpath loading info in the console |  Major | client | Murali Ramasami | Balu Vellanki |
+| [FALCON-2060](https://issues.apache.org/jira/browse/FALCON-2060) | Retry does not happen if instance timedout |  Major | . | Pragya Mittal | Pallavi Rao |
+| [FALCON-2058](https://issues.apache.org/jira/browse/FALCON-2058) | s3 tests with dummy url no longer compatible with latest HDFS |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2057](https://issues.apache.org/jira/browse/FALCON-2057) | HiveDR not working with multiple users and same DB |  Major | replication | Murali Ramasami | Balu Vellanki |
+| [FALCON-2056](https://issues.apache.org/jira/browse/FALCON-2056) | HiveDR doesn't work with multiple users |  Major | replication | Murali Ramasami | Sowmya Ramesh |
+| [FALCON-2051](https://issues.apache.org/jira/browse/FALCON-2051) | Falcon post-processing services are not getting invoked |  Blocker | general | Peeyush Bishnoi | Venkatesan Ramachandran |
+| [FALCON-2050](https://issues.apache.org/jira/browse/FALCON-2050) | Configure jetty parent classloader to be prioritized over webapp classloader |  Major | common | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-2049](https://issues.apache.org/jira/browse/FALCON-2049) | Feed Replication with Empty Directories are failing |  Blocker | feed | Murali Ramasami | Balu Vellanki |
+| [FALCON-2048](https://issues.apache.org/jira/browse/FALCON-2048) | Cluster submission failed in yarn-cluster mode |  Critical | general | Murali Ramasami | Peeyush Bishnoi |
+| [FALCON-2046](https://issues.apache.org/jira/browse/FALCON-2046) | HDFS Replication failing in secure Mode |  Critical | replication | Murali Ramasami | Sowmya Ramesh |
+| [FALCON-2045](https://issues.apache.org/jira/browse/FALCON-2045) | Enhance document on registry point in cluster specifiction for Hive HA mode |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2038](https://issues.apache.org/jira/browse/FALCON-2038) | When all Optional input instances are missing, we should not suffix partition |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-2037](https://issues.apache.org/jira/browse/FALCON-2037) | HiveDR Extension tests are failed in Secure mode with clusterForJobNNKerberosPrincipal not found |  Critical | replication | Murali Ramasami | Sowmya Ramesh |
+| [FALCON-2036](https://issues.apache.org/jira/browse/FALCON-2036) | Update twiki on entity list operation with up-to-date REST API path |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2035](https://issues.apache.org/jira/browse/FALCON-2035) | Entity list operation without type parameter doesn't work when authorization is enabled |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2034](https://issues.apache.org/jira/browse/FALCON-2034) | Make numThreads and timeOut configurable In ConfigurationStore init |  Critical | . | Pallavi Rao | sandeep samudrala |
+| [FALCON-2032](https://issues.apache.org/jira/browse/FALCON-2032) | Update the extension documentation to add ExtensionService before ConfigurationStore in startup properties |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-2031](https://issues.apache.org/jira/browse/FALCON-2031) | Hcat Retention test cases are failing with NoClassDefFoundError |  Blocker | retention | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-2027](https://issues.apache.org/jira/browse/FALCON-2027) | Enhance documentation on data replication from HDP to Azure |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2025](https://issues.apache.org/jira/browse/FALCON-2025) | Periodic revalidation of kerberos credentials should be done on loginUser |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-2023](https://issues.apache.org/jira/browse/FALCON-2023) | Feed eviction fails when feed locations "stats" and "meta" does not have time pattern. |  Blocker | feed | Balu Vellanki | Venkatesan Ramachandran |
+| [FALCON-2018](https://issues.apache.org/jira/browse/FALCON-2018) | WorkflowJobNotification sends incorrect message for killed instances |  Major | . | Pragya Mittal | Praveen Adlakha |
+| [FALCON-2017](https://issues.apache.org/jira/browse/FALCON-2017) | Fix HiveDR extension issues |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-2016](https://issues.apache.org/jira/browse/FALCON-2016) | maven assembly:single fails on MacOS |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-2010](https://issues.apache.org/jira/browse/FALCON-2010) | Fix UT errors due to ActiveMQ upgrade |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-2007](https://issues.apache.org/jira/browse/FALCON-2007) | Hive DR Replication failing with "Can not create a Path from a null string" |  Critical | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1984](https://issues.apache.org/jira/browse/FALCON-1984) | Provide proper hint and documentation if required titan storage backend is not configured in startup.properties |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1983](https://issues.apache.org/jira/browse/FALCON-1983) | Upgrade jackson core and databind versions to fix dependency incompatibility with higher-version Hive |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1982](https://issues.apache.org/jira/browse/FALCON-1982) | Document use of  HBase in standalone mode for GraphDB |  Major | docs | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1978](https://issues.apache.org/jira/browse/FALCON-1978) | Fix flaky unit test - MetadataMappingServiceTest |  Major | tests | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1976](https://issues.apache.org/jira/browse/FALCON-1976) | Remove hadoop-2 profile |  Major | . | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-1975](https://issues.apache.org/jira/browse/FALCON-1975) | Getting NoSuchMethodError when calling isNoneEmpty |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1974](https://issues.apache.org/jira/browse/FALCON-1974) | Cluster update : Allow superuser to update bundle/coord of dependent entities |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-1973](https://issues.apache.org/jira/browse/FALCON-1973) | Falcon build failure due checkstyle issue |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1972](https://issues.apache.org/jira/browse/FALCON-1972) | Handling cases when Extension service or "extension.store.uri" is not present in startup proeprties |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1969](https://issues.apache.org/jira/browse/FALCON-1969) | Provide server-side error details on CLI, if any |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1965](https://issues.apache.org/jira/browse/FALCON-1965) | Update ActiveMQ version to 5.13.3 to avoid Falcon start error after rolling upgrade |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1964](https://issues.apache.org/jira/browse/FALCON-1964) | Should delete temporary JKS file after IT tests for credential provider alias |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1962](https://issues.apache.org/jira/browse/FALCON-1962) | Extension related bugs |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1961](https://issues.apache.org/jira/browse/FALCON-1961) | Should return error if an extension job doesn't exist for delete/suspend/resume/schedule operations |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1957](https://issues.apache.org/jira/browse/FALCON-1957) | Documentation on using Hadoop credential provider for sensitive properties |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1953](https://issues.apache.org/jira/browse/FALCON-1953) | Build fails when profiles hivedr and test-patch is used together |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-1943](https://issues.apache.org/jira/browse/FALCON-1943) | Extension API/CLI fails when authorization is enabled |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1941](https://issues.apache.org/jira/browse/FALCON-1941) | HiveDR fails with NN-HA enabled on both the source and target clusters |  Critical | . | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-1939](https://issues.apache.org/jira/browse/FALCON-1939) | Avoid creating multiple falcon\*.tar.gz during falcon build |  Major | build-tools | Balu Vellanki | Balu Vellanki |
+| [FALCON-1936](https://issues.apache.org/jira/browse/FALCON-1936) | Extensions related files are not available in $FALCON\_HOM/extensions/ directory |  Critical | . | Peeyush Bishnoi | Sowmya Ramesh |
+| [FALCON-1935](https://issues.apache.org/jira/browse/FALCON-1935) | Falcon fails to start with default startup.properties |  Blocker | . | Ying Zheng | Praveen Adlakha |
+| [FALCON-1934](https://issues.apache.org/jira/browse/FALCON-1934) | Document safemode in Falcon Server |  Major | docs | Balu Vellanki | Balu Vellanki |
+| [FALCON-1932](https://issues.apache.org/jira/browse/FALCON-1932) | Extension CLI should support common options |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1931](https://issues.apache.org/jira/browse/FALCON-1931) | multiCluster tag is missing for Multiple Cluster scenarios |  Major | regression | Murali Ramasami | Murali Ramasami |
+| [FALCON-1928](https://issues.apache.org/jira/browse/FALCON-1928) | FalconJPAService missing in default startup.properties |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1924](https://issues.apache.org/jira/browse/FALCON-1924) | Falcon Coordinator rerun return old workflow id |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1909](https://issues.apache.org/jira/browse/FALCON-1909) | Catalog instance triage action fails with null pointer exception. |  Major | feed | Balu Vellanki | Balu Vellanki |
+| [FALCON-1908](https://issues.apache.org/jira/browse/FALCON-1908) | Document HDFS snapshot based mirroring extension |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-1907](https://issues.apache.org/jira/browse/FALCON-1907) | Package new CLI module added |  Major | client | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1896](https://issues.apache.org/jira/browse/FALCON-1896) | Failure in Falcon build in distro module |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1894](https://issues.apache.org/jira/browse/FALCON-1894) | HDFS Data replication cannot be initiated independent of Oozie server location |  Minor | general | Alex Bush | Sowmya Ramesh |
+| [FALCON-1886](https://issues.apache.org/jira/browse/FALCON-1886) | Feed sla monitoring does not work across restarts |  Major | . | Ajay Yadava | Ajay Yadava |
+| [FALCON-1885](https://issues.apache.org/jira/browse/FALCON-1885) | SLA monitoring API throws ResultNotFoundException |  Major | feed | Pragya Mittal | Praveen Adlakha |
+| [FALCON-1883](https://issues.apache.org/jira/browse/FALCON-1883) | Falcon regression build fails with minor checkstyle issues |  Major | regression | Murali Ramasami | Murali Ramasami |
+| [FALCON-1882](https://issues.apache.org/jira/browse/FALCON-1882) | Instance status api not working via prism |  Major | prism | Pragya Mittal | Praveen Adlakha |
+| [FALCON-1881](https://issues.apache.org/jira/browse/FALCON-1881) | Database Export should not expect fields list in the feed entity specification |  Major | acquisition | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1880](https://issues.apache.org/jira/browse/FALCON-1880) | To support TDE encryption : Add --skipcrccheck to distcp options for HiveDR |  Major | replication | Balu Vellanki | Balu Vellanki |
+| [FALCON-1877](https://issues.apache.org/jira/browse/FALCON-1877) | Falcon webUI returns 413 (Full head - Request entity too large) error when TLS is enabled in a secure cluster with AD integration |  Major | . | Venkat Ranganathan | Venkat Ranganathan |
+| [FALCON-1874](https://issues.apache.org/jira/browse/FALCON-1874) | Import and Export fails with HDFS as src/dest |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1867](https://issues.apache.org/jira/browse/FALCON-1867) | hardcoded query names in JDBCStateStore |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1866](https://issues.apache.org/jira/browse/FALCON-1866) | Bug in JDBCStateStore |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1864](https://issues.apache.org/jira/browse/FALCON-1864) | Retry event does not get removed from delay queue even after the instance succeeds |  Major | rerun | Pallavi Rao | Pallavi Rao |
+| [FALCON-1859](https://issues.apache.org/jira/browse/FALCON-1859) | Database Export instances are not added graph db for lineage tracking |  Major | general | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1855](https://issues.apache.org/jira/browse/FALCON-1855) | Falcon regression build fails with checkstyle issues |  Major | regression | Pragya Mittal | Murali Ramasami |
+| [FALCON-1854](https://issues.apache.org/jira/browse/FALCON-1854) | Fixing PrismProcessScheduleTest and NoOutputProcessTest |  Major | regression | Murali Ramasami | Murali Ramasami |
+| [FALCON-1848](https://issues.apache.org/jira/browse/FALCON-1848) | Late rerun is not working due to failnodes set to true |  Major | rerun | Pragya Mittal | Pallavi Rao |
+| [FALCON-1847](https://issues.apache.org/jira/browse/FALCON-1847) | Execution order not honored when instances are suspended/resumed |  Major | scheduler | Pallavi Rao | Pallavi Rao |
+| [FALCON-1846](https://issues.apache.org/jira/browse/FALCON-1846) | Fixing EntityDryRunTest |  Major | regression | Pragya Mittal | Pragya Mittal |
+| [FALCON-1845](https://issues.apache.org/jira/browse/FALCON-1845) | Retries Stopped happening  for all entities when one entity was deleted during rerun of instance |  Major | rerun | pavan kumar kolamuri | pavan kumar kolamuri |
+| [FALCON-1842](https://issues.apache.org/jira/browse/FALCON-1842) | Falcon build failed in Jenkins at org.apache.falcon.oozie.feed.OozieFeedWorkflowBuilderTest |  Major | falcon-unit | Balu Vellanki | Balu Vellanki |
+| [FALCON-1840](https://issues.apache.org/jira/browse/FALCON-1840) | Archive older definition in case of update |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1838](https://issues.apache.org/jira/browse/FALCON-1838) | Export instances are not added graph db for lineage tracking |  Major | . | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1826](https://issues.apache.org/jira/browse/FALCON-1826) | Execution order not honoured when instances are KILLED |  Major | scheduler | Pragya Mittal | Pallavi Rao |
+| [FALCON-1825](https://issues.apache.org/jira/browse/FALCON-1825) | Process end time inclusive in case of Native Scheduler |  Major | scheduler | pavan kumar kolamuri | pavan kumar kolamuri |
+| [FALCON-1823](https://issues.apache.org/jira/browse/FALCON-1823) |  wrong permissions on hadoolibs and conf folder in distributed mode deb |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1819](https://issues.apache.org/jira/browse/FALCON-1819) | Improve test class entity cleanup logic |  Major | merlin | Paul Isaychuk | Paul Isaychuk |
+| [FALCON-1816](https://issues.apache.org/jira/browse/FALCON-1816) | Fix findbugs-exclude.xml path and hadoop version in falcon-regression pom |  Major | merlin | Paul Isaychuk | Paul Isaychuk |
+| [FALCON-1811](https://issues.apache.org/jira/browse/FALCON-1811) | Status API does not honour start option |  Major | client | Pragya Mittal | Praveen Adlakha |
+| [FALCON-1796](https://issues.apache.org/jira/browse/FALCON-1796) | [HOTFIX] Incorrect parent pom in distro module |  Major | . | Ajay Yadava | Ajay Yadava |
+| [FALCON-1795](https://issues.apache.org/jira/browse/FALCON-1795) | Kill api does not kill waiting/ready instances |  Major | oozie | Pragya Mittal | sandeep samudrala |
+| [FALCON-1793](https://issues.apache.org/jira/browse/FALCON-1793) | feed element action="archive" is submittable via command line tool falcon |  Major | feed | Margus Roo | Deepak Barr |
+| [FALCON-1792](https://issues.apache.org/jira/browse/FALCON-1792) | Upgrade hadoop.version to 2.6.2 |  Major | hadoop | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1787](https://issues.apache.org/jira/browse/FALCON-1787) | Ooozie pig-action.xml requires hive sharelib for HCatalog use |  Major | oozie | Mark Greene | Sowmya Ramesh |
+| [FALCON-1784](https://issues.apache.org/jira/browse/FALCON-1784) | Add regression test for for FALCON-1647 |  Major | merlin | Paul Isaychuk | Paul Isaychuk |
+| [FALCON-1783](https://issues.apache.org/jira/browse/FALCON-1783) | Fix ProcessUpdateTest and SearchApiTest to use prism |  Major | merlin | Paul Isaychuk | Paul Isaychuk |
+| [FALCON-1766](https://issues.apache.org/jira/browse/FALCON-1766) | Add CLI metrics check for HiveDR, HDFS and feed replication |  Major | merlin | Paul Isaychuk | Paul Isaychuk |
+| [FALCON-1743](https://issues.apache.org/jira/browse/FALCON-1743) | Entity summary does not work via prism |  Major | client | Pragya Mittal | Ajay Yadava |
+| [FALCON-1724](https://issues.apache.org/jira/browse/FALCON-1724) | Falcon CLI.twiki in docs folder is not pointed by index page |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1721](https://issues.apache.org/jira/browse/FALCON-1721) | Move checkstyle artifacts under parent |  Major | . | Shwetha G S | sandeep samudrala |
+| [FALCON-1621](https://issues.apache.org/jira/browse/FALCON-1621) | Lifecycle of entity gets missed when prism and falcon server communicates |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1584](https://issues.apache.org/jira/browse/FALCON-1584) | Falcon allows invalid hadoop queue name for schedulable feed entities |  Major | . | Venkatesan Ramachandran | Venkatesan Ramachandran |
+
+
+### SUB-TASKS:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-2099](https://issues.apache.org/jira/browse/FALCON-2099) | Update Installation-steps.txt and NOTICE.txt for 0.10 release |  Major | ease | Balu Vellanki | Balu Vellanki |
+| [FALCON-2000](https://issues.apache.org/jira/browse/FALCON-2000) | Create branch 0.10 |  Major | general | Balu Vellanki | Balu Vellanki |
+| [FALCON-1996](https://issues.apache.org/jira/browse/FALCON-1996) | Upgrade falcon POM for 0.10 release |  Major | build-tools | Balu Vellanki | Balu Vellanki |
+| [FALCON-1993](https://issues.apache.org/jira/browse/FALCON-1993) | Update JIRA fix versions |  Major | general | Balu Vellanki | Balu Vellanki |
+| [FALCON-1980](https://issues.apache.org/jira/browse/FALCON-1980) | Change input and output argument order for Spark process workflow |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1954](https://issues.apache.org/jira/browse/FALCON-1954) | Steps to configure Oozie JMS for Falcon |  Major | messaging | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1938](https://issues.apache.org/jira/browse/FALCON-1938) | Add support to execute Spark SQL process |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1937](https://issues.apache.org/jira/browse/FALCON-1937) | Add documentation for cluster update. |  Major | . | Balu Vellanki | Balu Vellanki |
+| [FALCON-1929](https://issues.apache.org/jira/browse/FALCON-1929) | Extension job management: IT tests for CLIs |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1905](https://issues.apache.org/jira/browse/FALCON-1905) | Extension Job Management: IT tests for REST APIs and CLIs |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1904](https://issues.apache.org/jira/browse/FALCON-1904) | Extension Job Management: documentation for REST APIs and CLI |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1902](https://issues.apache.org/jira/browse/FALCON-1902) | Server side extension repository management CLI support |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1897](https://issues.apache.org/jira/browse/FALCON-1897) | Extension Job Management: CLI support |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1893](https://issues.apache.org/jira/browse/FALCON-1893) | Add documentation and examples for spark workflow engine |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1892](https://issues.apache.org/jira/browse/FALCON-1892) | Remove client side Recipe logic |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1860](https://issues.apache.org/jira/browse/FALCON-1860) | ADFProviderService should be optional as default setting |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1853](https://issues.apache.org/jira/browse/FALCON-1853) | Add spark process workflow builder |  Major | oozie | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1839](https://issues.apache.org/jira/browse/FALCON-1839) | Test case for APIs for entities scheduled on native scheduler |  Major | scheduler | Pragya Mittal | Pragya Mittal |
+| [FALCON-1831](https://issues.apache.org/jira/browse/FALCON-1831) | Flaky WorkflowExecutionContextTest.testWorkflowStartEnd |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1829](https://issues.apache.org/jira/browse/FALCON-1829) | Add regression for submit and schedule process on native scheduler (time based) |  Major | scheduler | Pragya Mittal | Pragya Mittal |
+| [FALCON-1817](https://issues.apache.org/jira/browse/FALCON-1817) | Update xsd for Spark execution engine |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1801](https://issues.apache.org/jira/browse/FALCON-1801) | Update CHANGES.txt in trunk to mark 0.9 as released |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1790](https://issues.apache.org/jira/browse/FALCON-1790) | CLI support for instance search |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1789](https://issues.apache.org/jira/browse/FALCON-1789) | Extension Job Management: REST API |  Major | . | Sowmya Ramesh | Ying Zheng |
+| [FALCON-1767](https://issues.apache.org/jira/browse/FALCON-1767) | Improve Falcon retention policy documentation |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1729](https://issues.apache.org/jira/browse/FALCON-1729) | Database ingest to support password alias via keystore file |  Major | acquisition | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1646](https://issues.apache.org/jira/browse/FALCON-1646) | Ability to export to database - Entity Definition |  Major | acquisition | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1496](https://issues.apache.org/jira/browse/FALCON-1496) | Flaky FalconPostProcessingTest |  Major | . | Pallavi Rao | Pallavi Rao |
+| [FALCON-1335](https://issues.apache.org/jira/browse/FALCON-1335) | Backend support of instance search of a group of entities |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1334](https://issues.apache.org/jira/browse/FALCON-1334) | Improve search performance with Titan graph database indexing |  Major | . | Ying Zheng | Ying Zheng |
+| [FALCON-1111](https://issues.apache.org/jira/browse/FALCON-1111) | Instance update on titan DB based on JMS notifications on workflow jobs |  Major | common, messaging | Sowmya Ramesh | Ying Zheng |
+| [FALCON-1107](https://issues.apache.org/jira/browse/FALCON-1107) | Move trusted recipe processing to server side |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1106](https://issues.apache.org/jira/browse/FALCON-1106) | Documentation for extension |  Major | . | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1105](https://issues.apache.org/jira/browse/FALCON-1105) | Server side extension repository management REST API support |  Major | client | Sowmya Ramesh | Sowmya Ramesh |
+| [FALCON-1085](https://issues.apache.org/jira/browse/FALCON-1085) | Allow cluster entities to be updated |  Major | . | Ajay Yadava | Balu Vellanki |
+
+
+### OTHER:
+
+| JIRA | Summary | Priority | Component | Reporter | Contributor |
+|:---- |:---- | :--- |:---- |:---- |:---- |
+| [FALCON-2063](https://issues.apache.org/jira/browse/FALCON-2063) | Add change log for 0.10 |  Major | . | Ajay Yadava | Ajay Yadava |
+| [FALCON-1765](https://issues.apache.org/jira/browse/FALCON-1765) | Move to github pull request model |  Major | . | Ajay Yadava | Ajay Yadava |
+| [FALCON-2008](https://issues.apache.org/jira/browse/FALCON-2008) | Add documentation for Graphite Notification Plugin |  Major | . | Praveen Adlakha | Praveen Adlakha |
+| [FALCON-1948](https://issues.apache.org/jira/browse/FALCON-1948) | Document steps to configure Oozie for Falcon |  Major | docs | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1899](https://issues.apache.org/jira/browse/FALCON-1899) | Create examples artifact module in Falcon |  Major | . | Peeyush Bishnoi | Peeyush Bishnoi |
+| [FALCON-1888](https://issues.apache.org/jira/browse/FALCON-1888) | Falcon JMS Notification details and documentation |  Major | docs | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1818](https://issues.apache.org/jira/browse/FALCON-1818) | Minor doc update for tar package locations after FALCON-1751 |  Minor | . | Deepak Barr | Deepak Barr |
+| [FALCON-1806](https://issues.apache.org/jira/browse/FALCON-1806) | Update documentation for Import and Export |  Major | . | Venkatesan Ramachandran | Venkatesan Ramachandran |
+| [FALCON-1567](https://issues.apache.org/jira/browse/FALCON-1567) | Test case for Lifecycle feature |  Major | merlin | Pragya Mittal | Pragya Mittal |
+| [FALCON-1566](https://issues.apache.org/jira/browse/FALCON-1566) | Add test for SLA monitoring API |  Major | merlin | Pragya Mittal | Pragya Mittal |
diff --git a/replication/pom.xml b/replication/pom.xml
index 1e80173..6e8b4b1 100644
--- a/replication/pom.xml
+++ b/replication/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-distcp-replication</artifactId>
     <description>Apache Falcon Distcp Replication Module</description>
@@ -52,6 +52,11 @@
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-metrics</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
+        </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
diff --git a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
index a8da51d..0906bd5 100644
--- a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
+++ b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
@@ -184,12 +184,13 @@
         return new GnuParser().parse(options, args);
     }
 
-    protected DistCpOptions getDistCpOptions(CommandLine cmd) {
+    protected DistCpOptions getDistCpOptions(CommandLine cmd) throws FalconException, IOException {
         String[] paths = cmd.getOptionValue("sourcePaths").trim().split(",");
         List<Path> srcPaths = getPaths(paths);
-        String trgPath = cmd.getOptionValue("targetPath").trim();
+        String targetPathString = cmd.getOptionValue("targetPath").trim();
+        Path targetPath = new Path(targetPathString);
 
-        DistCpOptions distcpOptions = new DistCpOptions(srcPaths, new Path(trgPath));
+        DistCpOptions distcpOptions = new DistCpOptions(srcPaths, targetPath);
         distcpOptions.setBlocking(true);
         distcpOptions.setMaxMaps(Integer.parseInt(cmd.getOptionValue("maxMaps")));
         distcpOptions.setMapBandwidth(Integer.parseInt(cmd.getOptionValue("mapBandwidth")));
@@ -214,8 +215,16 @@
         // Removing deleted files by default - FALCON-1844
         String removeDeletedFiles = cmd.getOptionValue(
                 ReplicationDistCpOption.DISTCP_OPTION_REMOVE_DELETED_FILES.getName(), "true");
-        distcpOptions.setDeleteMissing(Boolean.parseBoolean(removeDeletedFiles));
-
+        boolean deleteMissing = Boolean.parseBoolean(removeDeletedFiles);
+        distcpOptions.setDeleteMissing(deleteMissing);
+        if (deleteMissing) {
+            // DistCP will fail with InvalidInputException if deleteMissing is set to true and
+            // if targetPath does not exist. Create targetPath to avoid failures.
+            FileSystem fs = HadoopClientFactory.get().createProxiedFileSystem(targetPath.toUri(), getConf());
+            if (!fs.exists(targetPath)) {
+                fs.mkdirs(targetPath);
+            }
+        }
 
         String preserveBlockSize = cmd.getOptionValue(
                 ReplicationDistCpOption.DISTCP_OPTION_PRESERVE_BLOCK_SIZE.getName());
diff --git a/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java b/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
index e7e177e..2662ade 100644
--- a/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
+++ b/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.falcon.replication;
 
+import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.commons.cli.CommandLine;
 import org.apache.falcon.entity.Storage;
 import org.apache.hadoop.fs.Path;
@@ -32,6 +33,8 @@
  */
 public class FeedReplicatorTest {
 
+    private String defaultPath = "jail://FeedReplicatorTest:00/tmp";
+
     @Test
     public void testArguments() throws Exception {
         /*
@@ -42,21 +45,26 @@
          * <arg>-sourcePaths</arg><arg>${distcpSourcePaths}</arg>
          * <arg>-targetPath</arg><arg>${distcpTargetPaths}</arg>
          */
+
+        // creates jailed cluster in which DistCpOtions command can be tested.
+        EmbeddedCluster cluster =  EmbeddedCluster.newCluster("FeedReplicatorTest");
+
         final String[] args = {
             "true",
             "-maxMaps", "3",
             "-mapBandwidth", "4",
-            "-sourcePaths", "hdfs://localhost:8020/tmp/",
-            "-targetPath", "hdfs://localhost1:8020/tmp/",
+            "-sourcePaths", defaultPath,
+            "-targetPath", defaultPath,
             "-falconFeedStorageType", Storage.TYPE.FILESYSTEM.name(),
         };
 
         FeedReplicator replicator = new FeedReplicator();
         CommandLine cmd = replicator.getCommand(args);
+        replicator.setConf(cluster.getConf());
         DistCpOptions options = replicator.getDistCpOptions(cmd);
 
         List<Path> srcPaths = new ArrayList<Path>();
-        srcPaths.add(new Path("hdfs://localhost:8020/tmp/"));
+        srcPaths.add(new Path(defaultPath));
         validateMandatoryArguments(options, srcPaths, true);
         Assert.assertTrue(options.shouldDeleteMissing());
     }
@@ -82,8 +90,8 @@
             "true",
             "-maxMaps", "3",
             "-mapBandwidth", "4",
-            "-sourcePaths", "hdfs://localhost:8020/tmp/",
-            "-targetPath", "hdfs://localhost1:8020/tmp/",
+            "-sourcePaths", defaultPath,
+            "-targetPath", defaultPath,
             "-falconFeedStorageType", Storage.TYPE.FILESYSTEM.name(),
             "-overwrite", "true",
             "-ignoreErrors", "false",
@@ -99,7 +107,7 @@
         DistCpOptions options = replicator.getDistCpOptions(cmd);
 
         List<Path> srcPaths = new ArrayList<Path>();
-        srcPaths.add(new Path("hdfs://localhost:8020/tmp/"));
+        srcPaths.add(new Path(defaultPath));
         validateMandatoryArguments(options, srcPaths, false);
         validateOptionalArguments(options);
     }
@@ -108,7 +116,7 @@
         Assert.assertEquals(options.getMaxMaps(), 3);
         Assert.assertEquals(options.getMapBandwidth(), 4);
         Assert.assertEquals(options.getSourcePaths(), srcPaths);
-        Assert.assertEquals(options.getTargetPath(), new Path("hdfs://localhost1:8020/tmp/"));
+        Assert.assertEquals(options.getTargetPath(), new Path(defaultPath));
         Assert.assertEquals(options.shouldSyncFolder(), shouldSyncFolder);
     }
 
diff --git a/rerun/pom.xml b/rerun/pom.xml
index 8694e1d..c61dfa4 100644
--- a/rerun/pom.xml
+++ b/rerun/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-rerun</artifactId>
     <description>Apache Falcon Rerun Handler</description>
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
index 4c763c2..3cad362 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
@@ -59,9 +59,9 @@
                     (message.getRunId() + 1), message.getAttempts(), message.getEntityName(), message.getInstance(),
                     message.getWfId(), SchemaHelper.formatDateUTC(new Date(System.currentTimeMillis())));
             // Use coord action id for rerun if available
-            String id = message.getParentId();
-            if (StringUtils.isBlank(id)) {
-                id = message.getWfId();
+            String id = message.getWfId();
+            if (!id.contains("-C@") && StringUtils.isNotBlank(message.getParentId())) {
+                id = message.getParentId();
             }
             handler.getWfEngine(entityType, entityName).reRun(message.getClusterName(), id, null, false);
         } catch (Exception e) {
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
index c691922..b8adeef 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
@@ -110,17 +110,17 @@
 
     @Override
     public void onFailure(WorkflowExecutionContext context) throws FalconException {
-        // Re-run does not make sense when killed by user.
-        if (context.isWorkflowKilledManually()) {
-            LOG.debug("Workflow: {} Instance: {} Entity: {}, killed manually by user. Will not retry.",
-                    context.getWorkflowId(), context.getNominalTimeAsISO8601(), context.getEntityName());
-            return;
-        } else if (context.hasWorkflowTimedOut()) {
+        if (context.hasWorkflowTimedOut()) {
             Entity entity = EntityUtil.getEntity(context.getEntityType(), context.getEntityName());
             Retry retry = getRetry(entity);
             if (!retry.isOnTimeout()) {
                 return;
             }
+        // Re-run does not make sense when killed by user.
+        } else if (context.isWorkflowKilledManually()) {
+            LOG.debug("Workflow: {} Instance: {} Entity: {}, killed manually by user. Will not retry.",
+                    context.getWorkflowId(), context.getNominalTimeAsISO8601(), context.getEntityName());
+            return;
         }
         handleRerun(context.getClusterName(), context.getEntityType(),
                 context.getEntityName(), context.getNominalTimeAsISO8601(),
diff --git a/retention/pom.xml b/retention/pom.xml
index d59ed81..0eb19d4 100644
--- a/retention/pom.xml
+++ b/retention/pom.xml
@@ -25,7 +25,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-retention</artifactId>
     <description>Apache Falcon Retention Module</description>
diff --git a/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
index 72447da..98936ae 100644
--- a/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
+++ b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
@@ -462,6 +462,43 @@
         }
     }
 
+    @Test
+    public void testEvictionStatsMetaWithNoPattern() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair = createTestData("/data");
+            createDir("/stats");
+            createDir("/meta");
+            createTestData("/tmp");
+            final String storageUrl = cluster.getConf().get(HadoopClientFactory.FS_DEFAULT_NAME_KEY);
+            FeedEvictor.main(new String[] {
+                "-feedBasePath",
+                getFeedBasePath(LocationType.DATA, storageUrl) + "#"
+                    + getStatsOrMetaPath(LocationType.STATS, storageUrl)
+                    + "#" + getStatsOrMetaPath(LocationType.META, storageUrl)
+                    + "#" + getFeedBasePath(LocationType.TMP, storageUrl),
+                "-retentionType", "instance",
+                "-retentionLimit", "months(5)",
+                "-timeZone", "UTC",
+                "-frequency", "hourly",
+                "-logFile", conf.get(HadoopClientFactory.FS_DEFAULT_NAME_KEY)
+                + "/falcon/staging/feed/2012-01-01-04-00", "-falconFeedStorageType",
+                Storage.TYPE.FILESYSTEM.name(),
+            });
+
+            // should not throw exception
+            // stats and meta dir should not be deleted
+            Assert.assertTrue(isDirPresent("/stats"));
+            Assert.assertTrue(isDirPresent("/meta"));
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
 
     private Pair<List<String>, List<String>> createTestData(String locationType) throws Exception {
         Configuration conf = cluster.getConf();
@@ -482,6 +519,12 @@
         return Pair.of(inRange, outOfRange);
     }
 
+    private void createDir(String locationType) throws Exception {
+        Configuration conf = cluster.getConf();
+        FileSystem fs = FileSystem.get(conf);
+        touch(fs, locationType, false);
+    }
+
     private Pair<List<String>, List<String>> createTestData(String feed, String mask,
                                                             int period, TimeUnit timeUnit,
                                                             String locationType) throws Exception {
@@ -542,11 +585,21 @@
         }
     }
 
+    private boolean isDirPresent(String path) throws Exception {
+        FileSystem fs = FileSystem.get(cluster.getConf());
+        return fs.exists(new Path(path));
+    }
+
     private String getFeedBasePath(LocationType locationType, String storageUrl) {
         return locationType.name() + "=" + storageUrl
                 + "/" + locationType.name().toLowerCase() + "/data/YYYY/feed3/dd/MM/?{MONTH}/more/?{HOUR}";
     }
 
+    private String getStatsOrMetaPath(LocationType locationType, String storageUrl) {
+        return locationType.name() + "=" + storageUrl
+                + "/" + locationType.name().toLowerCase();
+    }
+
     private static class InMemoryWriter extends PrintStream {
 
         private final StringBuffer buffer = new StringBuffer();
diff --git a/scheduler/pom.xml b/scheduler/pom.xml
index f69dc93..62dd290 100644
--- a/scheduler/pom.xml
+++ b/scheduler/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-scheduler</artifactId>
     <description>Apache Falcon Scheduler Module</description>
@@ -75,6 +75,18 @@
 	    <groupId>org.quartz-scheduler</groupId>
 	    <artifactId>quartz</artifactId>
 	    <version>${quartz.version}</version>
+            <exclusions>
+              <exclusion>
+                <groupId>c3p0</groupId>
+                <artifactId>c3p0</artifactId>
+              </exclusion>
+            </exclusions>
+	</dependency>
+
+	<dependency>
+	    <groupId>com.mchange</groupId>
+	    <artifactId>c3p0</artifactId>
+	    <version>${c3p0.version}</version>
 	</dependency>
 
         <dependency>
diff --git a/scheduler/src/main/java/org/apache/falcon/workflow/engine/FalconWorkflowEngine.java b/scheduler/src/main/java/org/apache/falcon/workflow/engine/FalconWorkflowEngine.java
index 6dbec0c..82a1bdf 100644
--- a/scheduler/src/main/java/org/apache/falcon/workflow/engine/FalconWorkflowEngine.java
+++ b/scheduler/src/main/java/org/apache/falcon/workflow/engine/FalconWorkflowEngine.java
@@ -108,12 +108,17 @@
 
     @Override
     public boolean isActive(Entity entity) throws FalconException {
-        EntityID id = new EntityID(entity);
-        // Ideally state store should have all entities, but, check anyway.
-        if (STATE_STORE.entityExists(id)) {
-            return STATE_STORE.getEntity(id).getCurrentState() != EntityState.STATE.SUBMITTED;
+        try {
+            EntityID id = new EntityID(entity);
+            // Ideally state store should have all entities, but, check anyway.
+            if (STATE_STORE.entityExists(id)) {
+                return STATE_STORE.getEntity(id).getCurrentState() != EntityState.STATE.SUBMITTED;
+            }
+            return false;
+        } catch (NullPointerException npe) {
+            // FalconJPAService is not always used, so catch NPE and return false
+            return false;
         }
-        return false;
     }
 
     @Override
diff --git a/src/bin/graphdbutil.sh b/src/bin/graphdbutil.sh
new file mode 100644
index 0000000..151ec2f
--- /dev/null
+++ b/src/bin/graphdbutil.sh
@@ -0,0 +1,118 @@
+#!/bin/sh
+#
+# Licensed 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. See accompanying LICENSE file.
+#
+
+
+usage() {
+  echo "usage: $0  operation java-home hadoop-home falcon-home falcon-common-jar input/out-dir"
+  echo "  where operation is either export OR import"
+  echo "        java-home is the java installation location"
+  echo "        hadoop-home is the hadoop installation location"
+  echo "        falcon-home is the falcon home installation location"
+  echo "        falcon-common-jar is the falcon-common-<version>.jar location with GraphUtils"
+  echo "        input/output dir is the directory for the graph data"
+  exit 1
+}
+
+if [ $# != 6 ]; then
+  usage
+fi
+
+operation=$1
+java_home=$2
+hadoop_home=$3
+falcon_home=$4
+falcon_common_jar=$5
+util_dir=$6
+
+export=0
+import=0
+keep_temp=Y
+
+case $operation in
+   import) import=1
+           ;;
+   export) export=1
+           ;;
+   *)     echo "Unknown operation $operation"
+          usage
+esac
+
+if [ -d  $java_home -a -f $java_home/bin/java -a -f $java_home/bin/jar ] ; then
+  :
+else
+  echo "Invalid java home directory $java_home"
+  usage
+fi
+
+if [ -d  $hadoop_home -a -f $hadoop_home/bin/hadoop ] ; then
+  :
+else
+  echo "Invalid hadoop home directory $hadoop_home"
+  usage
+fi
+
+if [ -d  $falcon_home -a -f $falcon_home/bin/falcon ] ; then
+  :
+else
+  echo "Invalid falcon home directory $falcon_home"
+  usage
+fi
+
+falcon_war=$falcon_home/server/webapp/falcon.war
+if [ ! -f $falcon_war ]; then
+  echo "Falcon war file $falcon_war not available"
+  usage
+fi
+
+if [ ! -f $falcon_common_jar ]; then
+  echo "Falcon commons jar file $falcon_common_jar not available"
+  usage
+fi
+
+
+util_tmpdir=/tmp/falcon-graphutil-tmp-$$
+echo "Using $util_tmpdir as temporary directory"
+trap "rm -rf $util.tmpdir" 0 2 3 15
+rm -rf $util_tmpdir
+mkdir -p $util_tmpdir
+
+if [ ! -d $util_dir ]; then
+   echo "Directory $util_dir does not exist"
+   usage
+fi
+
+if [ x$import = x1 ]; then
+   if [ ! -f $metadata_file ]; then
+      echo "Directory $util_dir does not exist or $metadata_file not present"
+      usage
+   fi
+fi
+
+cd $util_tmpdir
+jar -xf $falcon_war
+rm ./WEB-INF/lib/jackson*  ./WEB-INF/lib/falcon-common*.jar ./WEB-INF/lib/slf4j* ./WEB-INF/lib/activemq*
+cp $falcon_common_jar ./WEB-INF/lib/
+
+JAVA_HOME=$java_home
+export PATH=$JAVA_HOME/bin:$PATH
+export CLASSPATH="$falcon_home/conf:./WEB-INF/lib/*:`$hadoop_home/bin/hadoop classpath`"
+echo "Using classpath $CLASSPATH"
+java -Dfalcon.log.dir=/tmp/ org.apache.falcon.metadata.GraphUpdateUtils $operation $util_dir
+
+if [ x$keep_temp = xY ]; then
+  :
+else
+  rm -rf $util_tmpdir
+fi
\ No newline at end of file
diff --git a/src/conf/hbase-site.xml.template b/src/conf/hbase-site.xml.template
index 2c72617..aa83889 100644
--- a/src/conf/hbase-site.xml.template
+++ b/src/conf/hbase-site.xml.template
@@ -19,7 +19,7 @@
 <configuration>
   <property>
     <name>hbase.rootdir</name>
-    <value>file://${hbase_home}/root</value>
+    <value>file:///${hbase_home}/root</value>
   </property>
   <property>
     <name>hbase.zookeeper.property.dataDir</name>
diff --git a/src/conf/startup.properties b/src/conf/startup.properties
index d732013..345c192 100644
--- a/src/conf/startup.properties
+++ b/src/conf/startup.properties
@@ -42,6 +42,7 @@
 *.application.services=org.apache.falcon.security.AuthenticationInitializationService,\
                         org.apache.falcon.workflow.WorkflowJobEndNotificationService, \
                         org.apache.falcon.service.ProcessSubscriberService,\
+                        org.apache.falcon.extensions.ExtensionService,\
                         org.apache.falcon.service.LifecyclePolicyMap,\
                         org.apache.falcon.entity.store.ConfigurationStore,\
                         org.apache.falcon.rerun.service.RetryService,\
@@ -49,8 +50,8 @@
                         org.apache.falcon.metadata.MetadataMappingService,\
                         org.apache.falcon.service.LogCleanupService,\
                         org.apache.falcon.service.GroupsService,\
-                        org.apache.falcon.service.ProxyUserService,\
-                        org.apache.falcon.extensions.ExtensionService
+                        org.apache.falcon.service.ProxyUserService
+
 ##For feed SLA monitoring enable these two
 #                        org.apache.falcon.service.FalconJPAService,\
 #                        org.apache.falcon.service.FeedSLAMonitoringService,\
@@ -123,7 +124,7 @@
 *.config.store.uri=file://${falcon.home}/data/${falcon.app.type}-store
 
 # HDFS location of extension store for extension artifacts
-*.extension.store.uri=
+*.extension.store.uri=file://${falcon.home}/extensions/
 
 #Location to store state of Feed SLA monitoring service
 *.feed.sla.service.store.uri = file://${falcon.home}/data/sla/pendingfeedinstances
@@ -135,6 +136,12 @@
 
 *.falcon.cleanup.service.frequency=days(1)
 
+# Default number of threads to be used to restore entities.
+*.config.store.num.threads.load.entities=100
+
+# Default timeout in minutes to load entities
+*.config.store.start.timeout.minutes=30
+
 ######### Properties for Feed SLA Monitoring #########
 # frequency of serialization for the state of FeedSLAMonitoringService - 1 hour
 *.feed.sla.serialization.frequency.millis=3600000
@@ -173,25 +180,19 @@
 *.falcon.graph.blueprints.graph=com.thinkaurelius.titan.core.TitanFactory
 
 # Graph Storage
-# IMPORTANT:   Please enable one of hbase or berkeleydb backends are enabled
-#  after the backend requirements are provisioned as needed.
+# IMPORTANT:   Please enable one of the graph db backend: hbase or berkeleydb, per instructions below.
 
-# Enable the following for Berkeley DB.  Make sure je-5.0.73.jar is
-# downloaded and available under Falcon webapp directory or under falcon
-# server classpath.
-
-#*.falcon.graph.storage.directory=/${falcon.home}/data/graphdb
+# Enable the following for Berkeley DB.  Make sure je-5.0.73.jar is downloaded and available
+# under Falcon webapp directory or under falcon server classpath.
 #*.falcon.graph.storage.backend=berkeleyje
-
+#*.falcon.graph.storage.directory=/${falcon.home}/data/graphdb
+#*.falcon.graph.serialize.path=${user.dir}/target/graphdb
 
 # Enable the following for HBase
 #*.falcon.graph.storage.backend=hbase
-#For standalone mode , set hostname to localhost
-#for distributed mode, set to the zookeeper quorum
+# For standalone mode , set hostname to localhost; for distributed mode, set to the zookeeper quorum
 # @see http://s3.thinkaurelius.com/docs/titan/current/hbase.html#_remote_server_mode_2
-
 #*.falcon.graph.storage.hostname=localhost
-#*.falcon.graph.serialize.path=${user.dir}/target/graphdb
 #*.falcon.graph.storage.hbase.table=falcon_titan
 
 # Avoid acquiring read lock when iterating over large graphs
diff --git a/src/main/assemblies/distributed-package.xml b/src/main/assemblies/distributed-package.xml
index 1b5353d..4683a81 100644
--- a/src/main/assemblies/distributed-package.xml
+++ b/src/main/assemblies/distributed-package.xml
@@ -337,6 +337,13 @@
             <outputDirectory>extensions/hive-mirroring</outputDirectory>
             <fileMode>0755</fileMode>
         </file>
+
+        <file>
+            <source>../examples/target/falcon-examples-${project.version}.jar</source>
+            <outputDirectory>examples/app/spark/lib</outputDirectory>
+            <destName>falcon-examples.jar</destName>
+            <fileMode>0644</fileMode>
+        </file>
     </files>
 </assembly>
     
diff --git a/src/main/assemblies/standalone-package.xml b/src/main/assemblies/standalone-package.xml
index 54d89ce..c291697 100644
--- a/src/main/assemblies/standalone-package.xml
+++ b/src/main/assemblies/standalone-package.xml
@@ -305,6 +305,13 @@
         </file>
 
         <file>
+            <source>../examples/target/falcon-examples-${project.version}.jar</source>
+            <outputDirectory>examples/app/spark/lib</outputDirectory>
+            <destName>falcon-examples.jar</destName>
+            <fileMode>0644</fileMode>
+        </file>
+
+        <file>
             <source>../webapp/target/falcon-webapp-${project.version}.war</source>
             <outputDirectory>server/webapp</outputDirectory>
             <destName>falcon.war</destName>
diff --git a/test-tools/hadoop-webapp/pom.xml b/test-tools/hadoop-webapp/pom.xml
index 213668e..e887d82 100644
--- a/test-tools/hadoop-webapp/pom.xml
+++ b/test-tools/hadoop-webapp/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../..</relativePath>
     </parent>
     <artifactId>falcon-hadoop-webapp</artifactId>
diff --git a/test-tools/hcatalog-sharelib/pom.xml b/test-tools/hcatalog-sharelib/pom.xml
index 46d6c96..034f3d8 100644
--- a/test-tools/hcatalog-sharelib/pom.xml
+++ b/test-tools/hcatalog-sharelib/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../..</relativePath>
     </parent>
     <artifactId>falcon-sharelib-hcatalog</artifactId>
diff --git a/test-tools/hive-sharelib/pom.xml b/test-tools/hive-sharelib/pom.xml
index 7cea8ed..aa2049a 100644
--- a/test-tools/hive-sharelib/pom.xml
+++ b/test-tools/hive-sharelib/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../..</relativePath>
     </parent>
     <artifactId>falcon-sharelib-hive</artifactId>
diff --git a/test-tools/oozie-sharelib/pom.xml b/test-tools/oozie-sharelib/pom.xml
index 37bc062..9f10e08 100644
--- a/test-tools/oozie-sharelib/pom.xml
+++ b/test-tools/oozie-sharelib/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../..</relativePath>
     </parent>
     <artifactId>falcon-sharelib-oozie</artifactId>
diff --git a/test-tools/pig-sharelib/pom.xml b/test-tools/pig-sharelib/pom.xml
index ae6de96..a96c52a 100644
--- a/test-tools/pig-sharelib/pom.xml
+++ b/test-tools/pig-sharelib/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
         <relativePath>../..</relativePath>
     </parent>
     <artifactId>falcon-sharelib-pig</artifactId>
diff --git a/test-tools/pom.xml b/test-tools/pom.xml
index 1ec7bbe..ea78ee3 100644
--- a/test-tools/pom.xml
+++ b/test-tools/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-test-tools</artifactId>
     <description>Apache Falcon Test Tools - Test Cluster</description>
diff --git a/test-util/pom.xml b/test-util/pom.xml
index baff8c3..0f65115 100644
--- a/test-util/pom.xml
+++ b/test-util/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-test-util</artifactId>
     <description>Apache Falcon Test Utility</description>
diff --git a/titan/pom.xml b/titan/pom.xml
index 1409cfa..0f71acc 100644
--- a/titan/pom.xml
+++ b/titan/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <artifactId>falcon-main</artifactId>
         <groupId>org.apache.falcon</groupId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-titan</artifactId>
     <description>Titan HBase 1.0 shaded libraries for 0.5.4 for Falcon</description>
@@ -48,11 +48,6 @@
         </dependency>
 
         <dependency>
-            <groupId>com.vividsolutions</groupId>
-            <artifactId>jts</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>com.thinkaurelius.titan</groupId>
             <artifactId>titan-es</artifactId>
         </dependency>
diff --git a/unit/pom.xml b/unit/pom.xml
index 4902fcc..1ca4533 100644
--- a/unit/pom.xml
+++ b/unit/pom.xml
@@ -23,7 +23,7 @@
     <parent>
         <artifactId>falcon-main</artifactId>
         <groupId>org.apache.falcon</groupId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
     <artifactId>falcon-unit</artifactId>
diff --git a/webapp/pom.xml b/webapp/pom.xml
index f98c5e3..90ba089 100644
--- a/webapp/pom.xml
+++ b/webapp/pom.xml
@@ -24,7 +24,7 @@
     <parent>
         <groupId>org.apache.falcon</groupId>
         <artifactId>falcon-main</artifactId>
-        <version>0.10-SNAPSHOT</version>
+        <version>0.10</version>
     </parent>
     <artifactId>falcon-webapp</artifactId>
     <description>Apache Falcon Web Application</description>
@@ -182,11 +182,13 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-metrics</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-rerun</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>
@@ -387,6 +389,7 @@
                         </goals>
                         <configuration>
                             <outputDirectory>${project.build.directory}/dependency</outputDirectory>
+                            <excludeScope>provided</excludeScope>
                         </configuration>
                     </execution>
                     <execution>
diff --git a/webapp/src/test/java/org/apache/falcon/resource/ExtensionManagerIT.java b/webapp/src/test/java/org/apache/falcon/resource/ExtensionManagerIT.java
index b352be4..6efe884 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/ExtensionManagerIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/ExtensionManagerIT.java
@@ -18,6 +18,7 @@
 
 package org.apache.falcon.resource;
 
+import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.extensions.ExtensionProperties;
 import org.apache.falcon.extensions.mirroring.hdfs.HdfsMirroringExtensionProperties;
 import org.apache.falcon.extensions.store.AbstractTestExtensionStore;
@@ -30,8 +31,9 @@
 import java.util.Map;
 
 /**
- * Unit tests for org.apache.falcon.extensions.ExtensionManager.
+ * IT tests for org.apache.falcon.extensions.ExtensionManager.
  */
+@Test (enabled = false)
 public class ExtensionManagerIT extends AbstractTestExtensionStore {
     private static final String HDFS_MIRRORING_PROPERTY_TEMPLATE = "/hdfs-mirroring-property-template.txt";
     private static final String JOB_NAME_1 = "hdfs-mirroring-job-1";
@@ -56,7 +58,7 @@
         TestContext.deleteEntitiesFromStore();
     }
 
-    @Test
+    @Test (enabled = false)
     public void testTrustedExtensionJob() throws Exception {
         Map<String, String> overlay = context.getUniqueOverlay();
         String endTime = context.getProcessEndTime();
@@ -110,12 +112,15 @@
                 "extension -instances -jobName " + JOB_NAME_2 + " -fields status,clusters,tags"), 0);
 
         // validate instance list results
+        context.waitForInstancesToStart(EntityType.PROCESS.name(), JOB_NAME_1, 10000);
         ExtensionInstanceList instanceList = context.getExtensionInstances(JOB_NAME_1, START_TIME_1, endTime, "RUNNING",
                 null, null, null, null, null, null);
         System.out.println("Validate running instances of extension job " + JOB_NAME_1 + ": \n"
                 + instanceList.toString());
         Assert.assertEquals(instanceList.numEntities, 1);
         Assert.assertEquals(instanceList.entitySummary.get(0).instances.length, 1);
+
+        context.waitForInstancesToStart(EntityType.PROCESS.name(), JOB_NAME_2, 10000);
         instanceList = context.getExtensionInstances(JOB_NAME_2, START_TIME_1, endTime, "RUNNING",
                 null, null, null, null, null, null);
         System.out.println("Validate running instances of extension job " + JOB_NAME_2 + ": \n"
diff --git a/webapp/src/test/java/org/apache/falcon/resource/InstanceSchedulerManagerJerseyIT.java b/webapp/src/test/java/org/apache/falcon/resource/InstanceSchedulerManagerJerseyIT.java
index 6d6d40b..00dbf7a 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/InstanceSchedulerManagerJerseyIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/InstanceSchedulerManagerJerseyIT.java
@@ -28,6 +28,7 @@
 /**
  * Tests for Instance operations using Falcon Native Scheduler.
  */
+@Test (enabled = false)
 public class InstanceSchedulerManagerJerseyIT extends AbstractSchedulerManagerJerseyIT {
 
 
@@ -40,7 +41,7 @@
         super.setup();
     }
 
-    @Test
+    @Test (enabled = false)
     public void testProcessInstanceExecution() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
@@ -64,7 +65,7 @@
 
     }
 
-    @Test
+    @Test (enabled = false)
     public void testKillAndRerunInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
@@ -98,7 +99,7 @@
 
     }
 
-    @Test
+    @Test (enabled = false)
     public void testSuspendResumeInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
@@ -128,7 +129,7 @@
                 START_INSTANCE, InstancesResult.WorkflowStatus.RUNNING);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testListInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
@@ -149,7 +150,7 @@
         Assert.assertEquals(result.getInstances()[2].getInstance(), START_INSTANCE);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testInstanceSummary() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
@@ -173,7 +174,7 @@
         Assert.assertEquals(result.getInstancesSummary()[0].getSummaryMap().get("READY").longValue(), 1L);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testProcessWithInputs() throws Exception {
         UnitTestContext context = new UnitTestContext();
         Map<String, String> overlay = context.getUniqueOverlay();
diff --git a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerIT.java b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerIT.java
index 4687520..37f8b99 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerIT.java
@@ -37,6 +37,7 @@
 /**
  * Test class for Process Instance REST API.
  */
+@Test (enabled = false)
 public class ProcessInstanceManagerIT extends AbstractSchedulerManagerJerseyIT {
 
     private static final String START_INSTANCE = "2012-04-20T00:00Z";
@@ -71,7 +72,7 @@
         OozieTestUtils.waitForProcessWFtoStart(context);
     }
 
-    //@Test
+    @Test (enabled = false)
     public void testGetRunningInstances() throws Exception {
         TestContext context = new TestContext();
         schedule(context);
@@ -86,7 +87,7 @@
         assertInstance(response.getInstances()[0], START_INSTANCE, WorkflowStatus.RUNNING);
     }
 
-    //@Test
+    @Test (enabled = false)
     public void testGetRunningInstancesPagination()  throws Exception {
         TestContext context = new TestContext();
         schedule(context, 4);
@@ -117,7 +118,7 @@
         Assert.assertEquals(processInstance.getStatus(), status);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testGetInstanceStatus() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
@@ -131,7 +132,7 @@
         Assert.assertEquals(response.getInstances()[0].getStatus(), WorkflowStatus.RUNNING);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testGetInstanceStatusPagination() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
@@ -146,7 +147,7 @@
         Assert.assertEquals(response.getInstances()[0].getStatus(), WorkflowStatus.RUNNING);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testKillInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
@@ -172,7 +173,7 @@
         Assert.assertEquals(response.getInstances()[0].getStatus(), WorkflowStatus.KILLED);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testReRunInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
@@ -201,7 +202,7 @@
         Assert.assertEquals(response.getInstances()[0].getStatus(), WorkflowStatus.RUNNING);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testSuspendInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
@@ -220,7 +221,7 @@
         Assert.assertEquals(response.getInstances()[0].getStatus(), WorkflowStatus.SUSPENDED);
     }
 
-    @Test
+    @Test (enabled = false)
     public void testResumesInstances() throws Exception {
         UnitTestContext context = new UnitTestContext();
         schedule(context);
diff --git a/webapp/src/test/java/org/apache/falcon/resource/TestContext.java b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
index 5412608..8201506 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
@@ -324,6 +324,21 @@
         return response.getEntity(ExtensionInstanceList.class);
     }
 
+    public void waitForInstancesToStart(String entityType, String entityName, long timeout) {
+        long mustEnd = System.currentTimeMillis() + timeout;
+        WebResource resource = this.service.path("api/instance/running/" + entityType + "/" + entityName);
+        InstancesResult instancesResult;
+        while (System.currentTimeMillis() < mustEnd) {
+            ClientResponse response = resource.header("Cookie", AUTH_COOKIE_EQ + authenticationToken)
+                    .accept(MediaType.APPLICATION_JSON).type(MediaType.TEXT_XML)
+                    .method(HttpMethod.GET, ClientResponse.class);
+            instancesResult = response.getEntity(InstancesResult.class);
+            if (instancesResult.getInstances() != null && instancesResult.getInstances().length > 0) {
+                break;
+            }
+        }
+    }
+
     public ClientResponse submitAndSchedule(String template, Map<String, String> overlay, EntityType entityType)
         throws Exception {
         return submitAndSchedule(template, overlay, entityType, null, "", null);
diff --git a/webapp/src/test/resources/startup.properties b/webapp/src/test/resources/startup.properties
index fd62bc4..1cfebab 100644
--- a/webapp/src/test/resources/startup.properties
+++ b/webapp/src/test/resources/startup.properties
@@ -32,6 +32,7 @@
 *.application.services=org.apache.falcon.security.AuthenticationInitializationService,\
                         org.apache.falcon.workflow.WorkflowJobEndNotificationService, \
                         org.apache.falcon.service.ProcessSubscriberService,\
+                        org.apache.falcon.extensions.ExtensionService,\
                         org.apache.falcon.service.FalconJPAService,\
                         org.apache.falcon.entity.store.ConfigurationStore,\
                         org.apache.falcon.rerun.service.RetryService,\
@@ -42,8 +43,7 @@
                         org.apache.falcon.notification.service.impl.SchedulerService,\
                         org.apache.falcon.notification.service.impl.AlarmService,\
                         org.apache.falcon.notification.service.impl.DataAvailabilityService,\
-                        org.apache.falcon.execution.FalconExecutionService,\
-                        org.apache.falcon.extensions.ExtensionService
+                        org.apache.falcon.execution.FalconExecutionService
 
 ##### Falcon Configuration Store Change listeners #####
 *.configstore.listeners=org.apache.falcon.entity.v0.EntityGraph,\