You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@falcon.apache.org by sr...@apache.org on 2013/08/19 19:37:11 UTC

[1/2] FALCON-62 Falcon compilation with hadoop 2.0 libs. Contributed by Shwetha GS

Updated Branches:
  refs/heads/master 33a68b7a6 -> 416a358b0


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
----------------------------------------------------------------------
diff --git a/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java b/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
index 22a3191..36365c2 100644
--- a/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
+++ b/test-util/src/main/java/org/apache/falcon/cluster/util/EmbeddedCluster.java
@@ -18,21 +18,23 @@
 
 package org.apache.falcon.cluster.util;
 
-import org.apache.falcon.entity.v0.cluster.*;
+import java.io.File;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.falcon.entity.v0.cluster.Cluster;
+import org.apache.falcon.entity.v0.cluster.Interface;
+import org.apache.falcon.entity.v0.cluster.Interfaces;
+import org.apache.falcon.entity.v0.cluster.Interfacetype;
+import org.apache.falcon.entity.v0.cluster.Location;
+import org.apache.falcon.entity.v0.cluster.Locations;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.log4j.Logger;
 
-import java.io.File;
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
 /**
  * A utility class that doles out an embedded Hadoop cluster with DFS and/or MR.
  */
@@ -45,31 +47,28 @@ public class EmbeddedCluster {
 
     private Configuration conf = new Configuration();
     private MiniDFSCluster dfsCluster;
-    private MiniMRCluster mrCluster;
     protected Cluster clusterEntity;
 
     public Configuration getConf() {
         return conf;
     }
 
-    public static EmbeddedCluster newCluster(final String name, final boolean withMR) throws Exception {
-        return createClusterAsUser(name, withMR);
+    public static EmbeddedCluster newCluster(final String name) throws Exception {
+        return createClusterAsUser(name);
     }
 
     public static EmbeddedCluster newCluster(final String name,
-                                             final boolean withMR,
                                              final String user) throws Exception {
         UserGroupInformation hdfsUser = UserGroupInformation.createRemoteUser(user);
         return hdfsUser.doAs(new PrivilegedExceptionAction<EmbeddedCluster>() {
             @Override
             public EmbeddedCluster run() throws Exception {
-                return createClusterAsUser(name, withMR);
+                return createClusterAsUser(name);
             }
         });
     }
 
-    private static EmbeddedCluster createClusterAsUser(String name,
-                                                       boolean withMR) throws IOException {
+    private static EmbeddedCluster createClusterAsUser(String name) throws IOException {
         EmbeddedCluster cluster = new EmbeddedCluster();
         File target = new File("webapp/target");
         if (!target.exists()) {
@@ -78,7 +77,6 @@ public class EmbeddedCluster {
         } else {
             System.setProperty("test.build.data", "webapp/target/" + name + "/data");
         }
-        String user = System.getProperty("user.name");
         cluster.conf.set("hadoop.tmp.dir", target.getAbsolutePath());
         cluster.conf.set("hadoop.log.dir", new File(target, "tmp").getAbsolutePath());
         cluster.conf.set("hadoop.proxyuser.oozie.groups", "*");
@@ -91,25 +89,6 @@ public class EmbeddedCluster {
         ProxyUsers.refreshSuperUserGroupsConfiguration(cluster.conf);
         String hdfsUrl = cluster.conf.get("fs.default.name");
         LOG.info("Cluster Namenode = " + hdfsUrl);
-        if (withMR) {
-            System.setProperty("hadoop.log.dir", "/tmp");
-            System.setProperty("org.apache.hadoop.mapred.TaskTracker", "/tmp");
-            cluster.conf.set("org.apache.hadoop.mapred.TaskTracker", "/tmp");
-            cluster.conf.set("org.apache.hadoop.mapred.TaskTracker", "/tmp");
-            cluster.conf.set("mapreduce.jobtracker.staging.root.dir", "/user");
-            Path path = new Path("/tmp/hadoop-" + user, "mapred");
-            FileSystem.get(cluster.conf).mkdirs(path);
-            FileSystem.get(cluster.conf).setPermission(path, new FsPermission((short) 511));
-            cluster.mrCluster = new MiniMRCluster(1,
-                    hdfsUrl, 1);
-            Configuration mrConf = cluster.mrCluster.createJobConf();
-            cluster.conf.set("mapred.job.tracker",
-                    mrConf.get("mapred.job.tracker"));
-            cluster.conf.set("mapred.job.tracker.http.address",
-                    mrConf.get("mapred.job.tracker.http.address"));
-            LOG.info("Cluster JobTracker = " + cluster.conf.
-                    get("mapred.job.tracker"));
-        }
         cluster.buildClusterObject(name);
         return cluster;
     }
@@ -162,9 +141,6 @@ public class EmbeddedCluster {
     }
 
     public void shutdown() {
-        if (mrCluster != null) {
-            mrCluster.shutdown();
-        }
         dfsCluster.shutdown();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/webapp/pom.xml
----------------------------------------------------------------------
diff --git a/webapp/pom.xml b/webapp/pom.xml
index 86242c2..185a453 100644
--- a/webapp/pom.xml
+++ b/webapp/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -35,6 +35,12 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-common</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
 
         <dependency>
@@ -82,6 +88,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
@@ -401,6 +408,18 @@
                 </plugins>
             </build>
         </profile>
+
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+            </dependencies>
+        </profile>
     </profiles>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/webapp/src/test/java/org/apache/falcon/logging/LogProviderIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/logging/LogProviderIT.java b/webapp/src/test/java/org/apache/falcon/logging/LogProviderIT.java
index 25e0979..4c3ce97 100644
--- a/webapp/src/test/java/org/apache/falcon/logging/LogProviderIT.java
+++ b/webapp/src/test/java/org/apache/falcon/logging/LogProviderIT.java
@@ -55,6 +55,7 @@ public class LogProviderIT {
     public void setup() throws Exception {
         Map<String, String> overlay = new HashMap<String, String>();
         overlay.put("cluster", "logProviderTest");
+        overlay.put("colo", "gs");
         TestContext context = new TestContext();
         String file = context.
                 overlayParametersOverTemplate(context.CLUSTER_TEMPLATE, overlay);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/webapp/src/test/java/org/apache/falcon/resource/ClusterEntityValidationIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/ClusterEntityValidationIT.java b/webapp/src/test/java/org/apache/falcon/resource/ClusterEntityValidationIT.java
index 58e9c08..8f2b6e4 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/ClusterEntityValidationIT.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/ClusterEntityValidationIT.java
@@ -18,11 +18,12 @@
 
 package org.apache.falcon.resource;
 
-import org.apache.activemq.broker.BrokerService;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Map;
+
 import org.apache.falcon.entity.ClusterHelper;
-import org.apache.falcon.entity.parser.ClusterEntityParser;
-import org.apache.falcon.entity.parser.EntityParserFactory;
-import org.apache.falcon.entity.parser.ValidationException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.cluster.Interface;
@@ -32,20 +33,13 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
-import javax.xml.bind.Marshaller;
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.util.Map;
+import com.sun.jersey.api.client.ClientResponse;
 
 /**
  * Tests cluster entity validation to verify if each of the specified
  * interface endpoints are valid.
  */
 public class ClusterEntityValidationIT {
-    private final ClusterEntityParser parser =
-            (ClusterEntityParser) EntityParserFactory.getParser(EntityType.CLUSTER);
-
     private final TestContext context = new TestContext();
     private Map<String, String> overlay;
 
@@ -53,18 +47,6 @@ public class ClusterEntityValidationIT {
     @BeforeClass
     public void setup() throws Exception {
         TestContext.prepare();
-        startBroker();
-
-        overlay = context.getUniqueOverlay();
-    }
-
-    private void startBroker() throws Exception {
-        BrokerService broker = new BrokerService();
-        broker.setUseJmx(false);
-        broker.setDataDirectory("target/data");
-        broker.addConnector("vm://localhost");
-        broker.addConnector("tcp://localhost:61616");
-        broker.start();
     }
 
     /**
@@ -74,25 +56,17 @@ public class ClusterEntityValidationIT {
      */
     @Test
     public void testClusterEntityWithValidInterfaces() throws Exception {
-
-        String filePath = context.overlayParametersOverTemplate(TestContext.CLUSTER_TEMPLATE, overlay);
-        InputStream stream = new FileInputStream(filePath);
-        Cluster cluster = parser.parse(stream);
-        Assert.assertNotNull(cluster);
-        cluster.setColo("default");  // validations will be ignored if not default & tests fail
-
-        StringWriter stringWriter = new StringWriter();
-        Marshaller marshaller = EntityType.CLUSTER.getMarshaller();
-        marshaller.marshal(cluster, stringWriter);
-        System.out.println(stringWriter.toString());
-        parser.parseAndValidate(stringWriter.toString());
+        overlay = context.getUniqueOverlay();
+        overlay.put("colo", "default");
+        ClientResponse response = context.submitToFalcon(TestContext.CLUSTER_TEMPLATE, overlay, EntityType.CLUSTER);
+        context.assertSuccessful(response);
     }
 
 
     @DataProvider(name = "interfaceToInvalidURLs")
     public Object[][] createInterfaceToInvalidURLData() {
         return new Object[][] {
-            // todo FileSystem validates invalid hftp url, does NOT fail
+            // TODO FileSystem validates invalid hftp url, does NOT fail
             // {Interfacetype.READONLY, "hftp://localhost:41119"},
             {Interfacetype.READONLY, ""},
             {Interfacetype.READONLY, "localhost:41119"},
@@ -106,25 +80,22 @@ public class ClusterEntityValidationIT {
         };
     }
 
-    @Test (dataProvider = "interfaceToInvalidURLs",
-           expectedExceptions = {ValidationException.class, IllegalArgumentException.class})
+    @Test (dataProvider = "interfaceToInvalidURLs")
     public void testClusterEntityWithInvalidInterfaces(Interfacetype interfacetype, String endpoint)
         throws Exception {
+        overlay = context.getUniqueOverlay();
         String filePath = context.overlayParametersOverTemplate(TestContext.CLUSTER_TEMPLATE, overlay);
         InputStream stream = new FileInputStream(filePath);
-        Cluster cluster = parser.parse(stream);
+        Cluster cluster = (Cluster) EntityType.CLUSTER.getUnmarshaller().unmarshal(stream);
         Assert.assertNotNull(cluster);
         cluster.setColo("default");  // validations will be ignored if not default & tests fail
 
         Interface anInterface = ClusterHelper.getInterface(cluster, interfacetype);
         anInterface.setEndpoint(endpoint);
 
-        StringWriter stringWriter = new StringWriter();
-        Marshaller marshaller = EntityType.CLUSTER.getMarshaller();
-        marshaller.marshal(cluster, stringWriter);
-        System.out.println(stringWriter.toString());
-        parser.parseAndValidate(stringWriter.toString());
-        Assert.fail("Validation exception must have been thrown for an invalid interface: "
-                + interfacetype + ", URL: " + endpoint);
+        File tmpFile = context.getTempFile();
+        EntityType.CLUSTER.getMarshaller().marshal(cluster, tmpFile);
+        ClientResponse response = context.submitFileToFalcon(EntityType.CLUSTER, tmpFile.getAbsolutePath());
+        context.assertFailure(response);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
----------------------------------------------------------------------
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 a7ee75c..f762c4b 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/TestContext.java
@@ -291,7 +291,7 @@ public class TestContext {
         return submitFileToFalcon(entityType, tmpFile);
     }
 
-    private ClientResponse submitFileToFalcon(EntityType entityType, String tmpFile) throws IOException {
+    public ClientResponse submitFileToFalcon(EntityType entityType, String tmpFile) throws IOException {
 
         ServletInputStream rawlogStream = getServletInputStream(tmpFile);
 
@@ -445,6 +445,7 @@ public class TestContext {
         long time = System.currentTimeMillis();
         clusterName = "cluster" + time;
         overlay.put("cluster", clusterName);
+        overlay.put("colo", "gs");
         overlay.put("inputFeedName", "in" + time);
         //only feeds with future dates can be scheduled
         Date endDate = new Date(System.currentTimeMillis() + 15 * 60 * 1000);
@@ -460,6 +461,7 @@ public class TestContext {
 
         Map<String, String> overlay = new HashMap<String, String>();
         overlay.put("cluster", RandomStringUtils.randomAlphabetic(5));
+        overlay.put("colo", "gs");
         TestContext context = new TestContext();
         String file = context.
                 overlayParametersOverTemplate(TestContext.CLUSTER_TEMPLATE, overlay);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/webapp/src/test/resources/cluster-template.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/cluster-template.xml b/webapp/src/test/resources/cluster-template.xml
index c71e08d..11aaedf 100644
--- a/webapp/src/test/resources/cluster-template.xml
+++ b/webapp/src/test/resources/cluster-template.xml
@@ -17,7 +17,7 @@
   limitations under the License.
   -->
 
-<cluster colo="gs" description="" name="##cluster##" xmlns="uri:falcon:cluster:0.1"
+<cluster colo="##colo##" description="" name="##cluster##" xmlns="uri:falcon:cluster:0.1"
         >
     <interfaces>
         <interface type="readonly" endpoint="hftp://localhost:41110"


[2/2] git commit: FALCON-62 Falcon compilation with hadoop 2.0 libs. Contributed by Shwetha GS

Posted by sr...@apache.org.
FALCON-62 Falcon compilation with hadoop 2.0 libs. Contributed by Shwetha GS


Project: http://git-wip-us.apache.org/repos/asf/incubator-falcon/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-falcon/commit/416a358b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-falcon/tree/416a358b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-falcon/diff/416a358b

Branch: refs/heads/master
Commit: 416a358b0c2c2ad148295c1a0eb831061acbfdfb
Parents: 33a68b7
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Mon Aug 19 23:06:38 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Mon Aug 19 23:06:38 2013 +0530

----------------------------------------------------------------------
 .gitignore                                      |   2 +
 CHANGES.txt                                     |   2 +
 common/pom.xml                                  |  71 +++++--
 .../entity/parser/ClusterEntityParser.java      |  12 +-
 .../falcon/cleanup/LogCleanupServiceTest.java   |  23 +--
 .../entity/parser/ClusterEntityParserTest.java  |  19 +-
 .../falcon/entity/parser/FeedUpdateTest.java    |   2 +-
 .../entity/parser/ProcessEntityParserTest.java  |  23 ++-
 .../apache/falcon/update/UpdateHelperTest.java  |   8 +-
 feed/pom.xml                                    |   1 +
 .../falcon/converter/OozieFeedMapperTest.java   |  20 +-
 hadoop-webapp/pom.xml                           |  64 ++++++-
 .../falcon/listener/HadoopStartupListener.java  | 127 ++++++++++---
 hadoop-webapp/src/main/resources/yarn-site.xml  |  64 +++++++
 messaging/pom.xml                               | 110 ++++++-----
 .../falcon/messaging/FeedProducerTest.java      |  17 +-
 oozie-el-extensions/pom.xml                     |  45 ++++-
 oozie/pom.xml                                   |  52 +++--
 pom.xml                                         | 190 +++++++++++++++----
 prism/pom.xml                                   |   7 +
 process/pom.xml                                 |  69 +++++--
 .../OozieProcessMapperLateProcessTest.java      |   2 +-
 .../converter/OozieProcessMapperTest.java       |  37 ++--
 replication/pom.xml                             |  49 ++++-
 rerun/pom.xml                                   |   1 +
 retention/pom.xml                               |   1 +
 .../apache/falcon/retention/FeedEvictor.java    |  41 ++--
 .../falcon/retention/FeedEvictorTest.java       |  29 ++-
 src/bin/falcon-start                            |   2 +-
 test-util/pom.xml                               |  68 +++++--
 .../falcon/cluster/util/EmbeddedCluster.java    |  52 ++---
 webapp/pom.xml                                  |  23 ++-
 .../apache/falcon/logging/LogProviderIT.java    |   1 +
 .../resource/ClusterEntityValidationIT.java     |  65 ++-----
 .../org/apache/falcon/resource/TestContext.java |   4 +-
 webapp/src/test/resources/cluster-template.xml  |   2 +-
 36 files changed, 927 insertions(+), 378 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index c22e2b3..510bcae 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,6 +27,8 @@ target
 .classpath
 .project
 .settings
+.externalToolBuilders
+maven-eclipse.xml
 
 #ActiveMQ
 activemq-data

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9ec2b36..48a4cfd 100755
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -5,6 +5,8 @@ Trunk (Unreleased)
   INCOMPATIBLE CHANGES
 
   NEW FEATURES
+    FALCON-62 Falcon compilation with hadoop 2.0 libs. (Shwetha GS
+    via Srikanth Sundarrajan)
 
   IMPROVEMENTS
     FALCON-80 Option in falcon-start to not start embedded activemq.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 498320a..0e192fc 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -15,9 +15,10 @@
    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.
- -->
+  -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
@@ -31,6 +32,46 @@
     <name>Apache Falcon Commons</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-test</artifactId>
+                    <scope>test</scope>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
             <groupId>commons-beanutils</groupId>
@@ -43,12 +84,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-test</artifactId>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-client</artifactId>
         </dependency>
@@ -59,13 +94,18 @@
         </dependency>
 
         <dependency>
-            <groupId>javax.xml.bind</groupId>
-            <artifactId>jaxb-api</artifactId>
+            <groupId>commons-el</groupId>
+            <artifactId>commons-el</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>javax.servlet.jsp</groupId>
+            <artifactId>jsp-api</artifactId>
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
+            <groupId>javax.xml.bind</groupId>
+            <artifactId>jaxb-api</artifactId>
         </dependency>
 
         <dependency>
@@ -75,6 +115,11 @@
 
         <dependency>
             <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
             <artifactId>slf4j-api</artifactId>
         </dependency>
 
@@ -96,6 +141,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
@@ -131,5 +177,4 @@
             </plugin>
         </plugins>
     </build>
-
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/main/java/org/apache/falcon/entity/parser/ClusterEntityParser.java
----------------------------------------------------------------------
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 242fdfb..b4e4a95 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
@@ -18,6 +18,10 @@
 
 package org.apache.falcon.entity.parser;
 
+import java.io.IOException;
+
+import javax.jms.ConnectionFactory;
+
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.StoreAccessException;
@@ -33,10 +37,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.log4j.Logger;
-
-import javax.jms.ConnectionFactory;
-import java.io.IOException;
-
 /**
  * Parser that parses cluster entity definition.
  */
@@ -114,7 +114,9 @@ public class ClusterEntityParser extends EntityParser<Cluster> {
         try {
             JobConf jobConf = new JobConf();
             jobConf.set("mapred.job.tracker", executeUrl);
-            new JobClient(jobConf);
+            jobConf.set("yarn.resourcemanager.address", executeUrl);
+            JobClient jobClient = new JobClient(jobConf);
+            jobClient.getClusterStatus().getMapTasks();
         } catch (IOException e) {
             throw new ValidationException("Invalid Execute server or port: " + executeUrl, e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java b/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
index 6324448..fa21a90 100644
--- a/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.falcon.cleanup;
 
+import java.io.IOException;
+
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.AbstractTestBase;
@@ -31,8 +33,6 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.io.IOException;
-
 /**
  * Test for log cleanup service.
  */
@@ -41,19 +41,19 @@ public class LogCleanupServiceTest extends AbstractTestBase {
     private FileSystem fs;
     private FileSystem tfs;
     private EmbeddedCluster targetDfsCluster;
-    private Path instanceLogPath = new Path("/projects/falcon/staging/falcon/workflows/process/"
+    private final Path instanceLogPath = new Path("/projects/falcon/staging/falcon/workflows/process/"
         + "sample" + "/logs/job-2010-01-01-01-00/000");
-    private Path instanceLogPath1 = new Path("/projects/falcon/staging/falcon/workflows/process/"
+    private final Path instanceLogPath1 = new Path("/projects/falcon/staging/falcon/workflows/process/"
         + "sample" + "/logs/job-2010-01-01-01-00/001");
-    private Path instanceLogPath2 = new Path("/projects/falcon/staging/falcon/workflows/process/"
+    private final Path instanceLogPath2 = new Path("/projects/falcon/staging/falcon/workflows/process/"
         + "sample" + "/logs/job-2010-01-01-02-00/001");
-    private Path instanceLogPath3 = new Path("/projects/falcon/staging/falcon/workflows/process/"
+    private final Path instanceLogPath3 = new Path("/projects/falcon/staging/falcon/workflows/process/"
         + "sample2" + "/logs/job-2010-01-01-01-00/000");
-    private Path instanceLogPath4 = new Path("/projects/falcon/staging/falcon/workflows/process/"
+    private final Path instanceLogPath4 = new Path("/projects/falcon/staging/falcon/workflows/process/"
         + "sample" + "/logs/latedata/2010-01-01-01-00");
-    private Path feedInstanceLogPath = new Path("/projects/falcon/staging/falcon/workflows/feed/"
+    private final Path feedInstanceLogPath = new Path("/projects/falcon/staging/falcon/workflows/feed/"
         + "impressionFeed" + "/logs/job-2010-01-01-01-00/testCluster/000");
-    private Path feedInstanceLogPath1 = new Path("/projects/falcon/staging/falcon/workflows/feed/"
+    private final Path feedInstanceLogPath1 = new Path("/projects/falcon/staging/falcon/workflows/feed/"
         + "impressionFeed2" + "/logs/job-2010-01-01-01-00/testCluster/000");
 
 
@@ -63,15 +63,16 @@ public class LogCleanupServiceTest extends AbstractTestBase {
         this.targetDfsCluster.shutdown();
     }
 
+    @Override
     @BeforeClass
     public void setup() throws Exception {
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         conf = dfsCluster.getConf();
         fs = dfsCluster.getFileSystem();
 
         storeEntity(EntityType.CLUSTER, "testCluster");
         System.setProperty("test.build.data", "target/tdfs/data" + System.currentTimeMillis());
-        this.targetDfsCluster = EmbeddedCluster.newCluster("backupCluster", false);
+        this.targetDfsCluster = EmbeddedCluster.newCluster("backupCluster");
         conf = targetDfsCluster.getConf();
 
         storeEntity(EntityType.CLUSTER, "backupCluster");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/test/java/org/apache/falcon/entity/parser/ClusterEntityParserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/ClusterEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/ClusterEntityParserTest.java
index f7d03e3..20d14e8 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/ClusterEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/ClusterEntityParserTest.java
@@ -18,6 +18,15 @@
 
 package org.apache.falcon.entity.parser;
 
+import static org.testng.AssertJUnit.assertEquals;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.AbstractTestBase;
@@ -31,14 +40,6 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-
-import static org.testng.AssertJUnit.assertEquals;
-
 /**
  * Test for validating cluster entity parsing.
  */
@@ -109,7 +110,7 @@ public class ClusterEntityParserTest extends AbstractTestBase {
 
     @BeforeClass
     public void init() throws Exception {
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         this.conf = dfsCluster.getConf();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/test/java/org/apache/falcon/entity/parser/FeedUpdateTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/FeedUpdateTest.java b/common/src/test/java/org/apache/falcon/entity/parser/FeedUpdateTest.java
index e887fee..f39f300 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/FeedUpdateTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/FeedUpdateTest.java
@@ -44,7 +44,7 @@ public class FeedUpdateTest extends AbstractTestBase {
 
     @BeforeClass
     public void init() throws Exception {
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         this.conf = dfsCluster.getConf();
         setup();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/test/java/org/apache/falcon/entity/parser/ProcessEntityParserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/ProcessEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/ProcessEntityParserTest.java
index fd15062..39acc53 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/ProcessEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/ProcessEntityParserTest.java
@@ -18,6 +18,15 @@
 
 package org.apache.falcon.entity.parser;
 
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.AbstractTestBase;
@@ -32,14 +41,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * Tests for validating process entity parser.
  */
@@ -55,7 +56,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
 
     @BeforeClass
     public void init() throws Exception {
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         this.conf = dfsCluster.getConf();
     }
 
@@ -64,6 +65,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
         this.dfsCluster.shutdown();
     }
 
+    @Override
     @BeforeMethod
     public void setup() throws Exception {
         storeEntity(EntityType.CLUSTER, "testCluster");
@@ -156,7 +158,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
 
     @Test(expectedExceptions = ValidationException.class)
     public void applyValidationInvalidProcess() throws Exception {
-        Process process = (Process) parser.parseAndValidate(getClass().getResourceAsStream(PROCESS_XML));
+        Process process = parser.parseAndValidate(getClass().getResourceAsStream(PROCESS_XML));
         process.getClusters().getClusters().get(0).setName("invalid cluster");
         parser.validate(process);
     }
@@ -173,6 +175,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
 
         for (int i = 0; i < 3; i++) {
             threadList.add(new Thread() {
+                @Override
                 public void run() {
                     try {
                         EntityParser parser = EntityParserFactory.getParser(EntityType.PROCESS);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java b/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
index 23fa148..d4b5a2a 100644
--- a/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
+++ b/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
@@ -27,7 +27,11 @@ import org.apache.falcon.entity.parser.ProcessEntityParser;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.entity.v0.SchemaHelper;
-import org.apache.falcon.entity.v0.feed.*;
+import org.apache.falcon.entity.v0.feed.Feed;
+import org.apache.falcon.entity.v0.feed.LocationType;
+import org.apache.falcon.entity.v0.feed.Partition;
+import org.apache.falcon.entity.v0.feed.Properties;
+import org.apache.falcon.entity.v0.feed.Property;
 import org.apache.falcon.entity.v0.process.PolicyType;
 import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
@@ -47,7 +51,7 @@ public class UpdateHelperTest extends AbstractTestBase {
 
     @BeforeClass
     public void init() throws Exception {
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         this.conf = dfsCluster.getConf();
         setup();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/feed/pom.xml
----------------------------------------------------------------------
diff --git a/feed/pom.xml b/feed/pom.xml
index 27e17e7..f578791 100644
--- a/feed/pom.xml
+++ b/feed/pom.xml
@@ -51,6 +51,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
----------------------------------------------------------------------
diff --git a/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java b/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
index fb45c68..c3d083f 100644
--- a/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
+++ b/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
@@ -17,7 +17,15 @@
  */
 package org.apache.falcon.converter;
 
+import static org.testng.Assert.assertEquals;
+
+import java.util.Collection;
+import java.util.List;
+
+import javax.xml.bind.Unmarshaller;
+
 import junit.framework.Assert;
+
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -35,19 +43,13 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import javax.xml.bind.Unmarshaller;
-import java.util.Collection;
-import java.util.List;
-
-import static org.testng.Assert.assertEquals;
-
 /**
  * Tests for Oozie workflow definition for feed replication & retention.
  */
 public class OozieFeedMapperTest {
     private EmbeddedCluster srcMiniDFS;
     private EmbeddedCluster trgMiniDFS;
-    private ConfigurationStore store = ConfigurationStore.get();
+    private final ConfigurationStore store = ConfigurationStore.get();
     private Cluster srcCluster;
     private Cluster trgCluster;
     private Feed feed;
@@ -58,10 +60,10 @@ public class OozieFeedMapperTest {
 
     @BeforeClass
     public void setUpDFS() throws Exception {
-        srcMiniDFS = EmbeddedCluster.newCluster("cluster1", false);
+        srcMiniDFS = EmbeddedCluster.newCluster("cluster1");
         String srcHdfsUrl = srcMiniDFS.getConf().get("fs.default.name");
 
-        trgMiniDFS = EmbeddedCluster.newCluster("cluster2", false);
+        trgMiniDFS = EmbeddedCluster.newCluster("cluster2");
         String trgHdfsUrl = trgMiniDFS.getConf().get("fs.default.name");
 
         cleanupStore();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/hadoop-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-webapp/pom.xml b/hadoop-webapp/pom.xml
index 6c1e2ac..0d175dc 100644
--- a/hadoop-webapp/pom.xml
+++ b/hadoop-webapp/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -18,7 +18,7 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -31,15 +31,65 @@
     <name>Apache Falcon Embedded Hadoop - Test Cluster</name>
     <packaging>war</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <classifier>tests</classifier>
+                </dependency>
+                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                </dependency>
+                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+                </dependency>                    
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-oozie-el-extension</artifactId>
         </dependency>
 
         <dependency>
-            <groupId>org.apache.falcon</groupId>
-            <artifactId>falcon-oozie-el-extension</artifactId>
+            <groupId>org.apache.activemq</groupId>
+            <artifactId>activemq-core</artifactId>
         </dependency>
     </dependencies>
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/hadoop-webapp/src/main/java/org/apache/falcon/listener/HadoopStartupListener.java
----------------------------------------------------------------------
diff --git a/hadoop-webapp/src/main/java/org/apache/falcon/listener/HadoopStartupListener.java b/hadoop-webapp/src/main/java/org/apache/falcon/listener/HadoopStartupListener.java
index 28013f8..1ce709d 100644
--- a/hadoop-webapp/src/main/java/org/apache/falcon/listener/HadoopStartupListener.java
+++ b/hadoop-webapp/src/main/java/org/apache/falcon/listener/HadoopStartupListener.java
@@ -18,22 +18,25 @@
 
 package org.apache.falcon.listener;
 
+import java.io.File;
+
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+
+import org.apache.activemq.broker.BrokerService;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobTracker;
-import org.apache.hadoop.mapred.TaskTracker;
-
-import javax.servlet.ServletContextEvent;
-import javax.servlet.ServletContextListener;
-import java.io.File;
+import org.apache.log4j.Logger;
 
 /**
  * Listener for bootstrapping embedded hadoop cluster for integration tests.
  */
 public class HadoopStartupListener implements ServletContextListener {
+    private static final Logger LOG = Logger.getLogger(HadoopStartupListener.class);
+    private BrokerService broker;
 
     @Override
     public void contextInitialized(ServletContextEvent sce) {
@@ -43,37 +46,101 @@ public class HadoopStartupListener implements ServletContextListener {
 
             NameNode.format(conf);
             final String[] emptyArgs = {};
-            NameNode.createNameNode(emptyArgs, conf);
-            DataNode.createDataNode(emptyArgs, conf);
-            final JobTracker jobTracker = JobTracker.startTracker(new JobConf(conf));
-            new Thread(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        jobTracker.offerService();
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }
-            }).start();
-            new Thread(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        TaskTracker taskTracker = new TaskTracker(new JobConf(conf));
-                        taskTracker.run();
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }
-            }).start();
+            String hadoopProfle = System.getProperty("hadoop.profile", "1");
+            if (hadoopProfle.equals("1")) {
+                NameNode.createNameNode(emptyArgs, conf);
+                DataNode.createDataNode(emptyArgs, conf);
+                JobConf jobConf = new JobConf(conf);
+//                JobTracker jt = JobTracker.startTracker(jobConf);
+//                jt.offerService();
+//                TaskTracker tt = new TaskTracker(jobConf);
+//                tt.run();
+                Object jt = Class.forName("org.apache.hadoop.mapred.JobTracker")
+                                .getMethod("startTracker", JobConf.class).invoke(null, jobConf);
+                startService(jt, "offerService");
+                Object tt = Class.forName("org.apache.hadoop.mapred.TaskTracker")
+                                .getConstructor(JobConf.class).newInstance(jobConf);
+                startService(tt, "run");
+            } else if (hadoopProfle.equals("2")) {
+//                DefaultMetricsSystem.setMiniClusterMode(true);
+//                ResourceManager resourceManager = new ResourceManager(new MemStore());
+//                YarnConfiguration yarnConf = new YarnConfiguration(conf);
+//                resourceManager.init(yarnConf);
+//                resourceManager.start();
+//                NodeManager nodeManager = new NodeManager();
+//                nodeManager.init(yarnConf);
+//                nodeManager.start();
+                Class.forName("org.apache.hadoop.metrics2.lib.DefaultMetricsSystem")
+                                .getMethod("setMiniClusterMode", boolean.class).invoke(null, true);
+                NameNode.createNameNode(emptyArgs, conf);
+                DataNode.createDataNode(emptyArgs, conf);
+
+                Object memStore = instance("org.apache.hadoop.yarn.server.resourcemanager.recovery.MemStore");
+                Object resourceManager = Class.forName("org.apache.hadoop.yarn.server.resourcemanager.ResourceManager")
+                        .getConstructor(Class.forName("org.apache.hadoop.yarn.server.resourcemanager.recovery.Store"))
+                        .newInstance(memStore);
+                Object yarnConf = Class.forName("org.apache.hadoop.yarn.conf.YarnConfiguration")
+                        .getConstructor(Configuration.class).newInstance(conf);
+                invoke(resourceManager, "init", Configuration.class, yarnConf);
+                startService(resourceManager, "start");
+                Object nodeManager = instance("org.apache.hadoop.yarn.server.nodemanager.NodeManager");
+                invoke(nodeManager, "init", Configuration.class, yarnConf);
+                startService(nodeManager, "start");
+            } else {
+                throw new RuntimeException("Unhandled hadoop profile " + hadoopProfle);
+            }
+            startBroker();
         } catch (Exception e) {
             e.printStackTrace();
+            LOG.error("Unable to start hadoop cluster", e);
             throw new RuntimeException("Unable to start hadoop cluster", e);
         }
     }
 
+    private void startBroker() throws Exception {
+        broker = new BrokerService();
+        broker.setUseJmx(false);
+        broker.setDataDirectory("target/data");
+        broker.addConnector("vm://localhost");
+        broker.addConnector("tcp://localhost:61616");
+        broker.start();
+    }
+
+    private Object instance(String clsName) throws Exception {
+        return Class.forName(clsName).newInstance();
+    }
+
+    @SuppressWarnings("rawtypes")
+    private void invoke(Object service, String methodName, Class argCls, Object arg) throws Exception {
+        if (argCls == null) {
+            service.getClass().getMethod(methodName).invoke(service);
+        } else {
+            service.getClass().getMethod(methodName, argCls).invoke(service, arg);
+        }
+    }
+
+    private void startService(final Object service, final String method) {
+        new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    LOG.info("Starting service " + service.getClass().getName());
+                    invoke(service, method, null, null);
+                } catch(Exception e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        }).start();
+    }
+
     @Override
     public void contextDestroyed(ServletContextEvent sce) {
+        try {
+            if (broker != null) {
+                broker.stop();
+            }
+        } catch(Exception e) {
+            LOG.warn("Failed to stop activemq", e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/hadoop-webapp/src/main/resources/yarn-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-webapp/src/main/resources/yarn-site.xml b/hadoop-webapp/src/main/resources/yarn-site.xml
new file mode 100644
index 0000000..587f1c5
--- /dev/null
+++ b/hadoop-webapp/src/main/resources/yarn-site.xml
@@ -0,0 +1,64 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+  <property>
+    <description>The address of the applications manager interface in the RM.</description>
+    <name>yarn.resourcemanager.address</name>
+    <value>0.0.0.0:41021</value>
+  </property>
+
+  <property>
+    <name>mapreduce.framework.name</name>
+    <value>yarn</value>
+  </property>
+
+  <property>
+    <name>yarn.resourcemanager.resource-tracker.address</name>
+    <value>0.0.0.0:41025</value>
+  </property>
+
+  <property>
+    <description>The address of the RM web application.</description>
+    <name>yarn.resourcemanager.webapp.address</name>
+    <value>0.0.0.0:41130</value>
+  </property>
+
+  <property>
+    <description>The address of the scheduler interface.</description>
+    <name>yarn.resourcemanager.scheduler.address</name>
+    <value>0.0.0.0:41030</value>
+  </property>
+
+  <property>
+    <description>Address where the localizer IPC is.</description>
+    <name>yarn.nodemanager.localizer.address</name>
+    <value>0.0.0.0:41040</value>
+  </property>
+
+  <property>
+    <description>The address of the RM admin interface.</description>
+    <name>yarn.resourcemanager.admin.address</name>
+    <value>0.0.0.0:41140</value>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/messaging/pom.xml
----------------------------------------------------------------------
diff --git a/messaging/pom.xml b/messaging/pom.xml
index 820734d..4cbf06f 100644
--- a/messaging/pom.xml
+++ b/messaging/pom.xml
@@ -1,19 +1,24 @@
 <?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. -->
+<!--
+   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
 
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+       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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -26,6 +31,46 @@
     <name>Apache Falcon Messaging</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-test</artifactId>
+                    <scope>test</scope>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                 <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    </profiles>
+
+
     <dependencies>
         <dependency>
             <groupId>org.apache.falcon</groupId>
@@ -39,14 +84,8 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-test</artifactId>
-            <scope>test</scope>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
         </dependency>
 
         <dependency>
@@ -66,33 +105,4 @@
             <artifactId>testng</artifactId>
         </dependency>
     </dependencies>
-    <build>
-        <plugins>
-            <!--
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <version>1.5</version>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                        <configuration>
-                        <finalName>falcon-messaging-execution-jar</finalName>
-                            <transformers>
-                                <transformer
-                                    implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                    <mainClass>org.apache.falcon.messaging.MessageProducer</mainClass>
-                                </transformer>
-                            </transformers>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            -->
-        </plugins>
-    </build>
-
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
index 32f731f..6a6dc35 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
@@ -18,6 +18,17 @@
 
 package org.apache.falcon.messaging;
 
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import javax.jms.Connection;
+import javax.jms.ConnectionFactory;
+import javax.jms.Destination;
+import javax.jms.JMSException;
+import javax.jms.MapMessage;
+import javax.jms.MessageConsumer;
+import javax.jms.Session;
+
 import org.apache.activemq.ActiveMQConnectionFactory;
 import org.apache.activemq.broker.BrokerService;
 import org.apache.activemq.util.ByteArrayInputStream;
@@ -32,10 +43,6 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import javax.jms.*;
-import java.io.InputStream;
-import java.io.OutputStream;
-
 /**
  * Test for feed message producer.
  */
@@ -58,7 +65,7 @@ public class FeedProducerTest {
     @BeforeClass
     public void setup() throws Exception {
 
-        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster");
         conf = dfsCluster.getConf();
         logFile = new Path(conf.get("fs.default.name"),
                 "/falcon/feed/agg-logs/instance-2012-01-01-10-00.csv");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/oozie-el-extensions/pom.xml
----------------------------------------------------------------------
diff --git a/oozie-el-extensions/pom.xml b/oozie-el-extensions/pom.xml
index 8bd7da2..6cdd871 100644
--- a/oozie-el-extensions/pom.xml
+++ b/oozie-el-extensions/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -18,7 +18,7 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -31,6 +31,40 @@
     <name>Apache Falcon Oozie EL Extension</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.oozie</groupId>
@@ -38,11 +72,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/oozie/pom.xml
----------------------------------------------------------------------
diff --git a/oozie/pom.xml b/oozie/pom.xml
index a0499e3..d3c9dff 100644
--- a/oozie/pom.xml
+++ b/oozie/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -18,7 +18,7 @@
   -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -31,6 +31,40 @@
     <name>Apache Falcon Oozie Adaptor</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.falcon</groupId>
@@ -50,6 +84,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
@@ -58,11 +93,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>
@@ -88,7 +118,7 @@
                                 <includeSchema>oozie-coordinator-0.3.xsd</includeSchema>
                             </includeSchemas>
                             <excludeBindings>
-                                <exluceBinding>jaxb-binding.xjb</exluceBinding>
+                                <excludeBinding>jaxb-binding.xjb</excludeBinding>
                             </excludeBindings>
                         </configuration>
                     </execution>
@@ -105,7 +135,7 @@
                                 <includeSchema>oozie-workflow-0.3.xsd</includeSchema>
                             </includeSchemas>
                             <excludeBindings>
-                                <exluceBinding>jaxb-binding.xjb</exluceBinding>
+                                <excludeBinding>jaxb-binding.xjb</excludeBinding>
                             </excludeBindings>
                         </configuration>
                     </execution>
@@ -122,7 +152,7 @@
                                 <includeSchema>oozie-bundle-0.1.xsd</includeSchema>
                             </includeSchemas>
                             <excludeBindings>
-                                <exluceBinding>jaxb-binding.xjb</exluceBinding>
+                                <excludeBinding>jaxb-binding.xjb</excludeBinding>
                             </excludeBindings>
                         </configuration>
                     </execution>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0c4028c..7668bb9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -14,8 +14,10 @@
    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.
- -->
-<project>
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
     <modelVersion>4.0.0</modelVersion>
     <groupId>org.apache.falcon</groupId>
     <artifactId>falcon-main</artifactId>
@@ -94,7 +96,7 @@
 
         <include.prism>true</include.prism>
 
-        <hadoop.version>1.1.2</hadoop.version>
+        <slf4j.version>1.2</slf4j.version>
         <oozie.version>3.2.2</oozie.version>
         <activemq.version>5.4.3</activemq.version>
         <hadoop-distcp.version>0.3</hadoop-distcp.version>
@@ -103,6 +105,117 @@
         <skipCheck>false</skipCheck>
     </properties>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <properties>
+                <hadoop.profile>1</hadoop.profile>
+                <hadoop.version>1.1.2</hadoop.version>
+            </properties>
+            <dependencyManagement>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-core</artifactId>
+                        <version>${hadoop.version}</version>
+                        <exclusions>
+                            <exclusion>
+                                <groupId>org.eclipse.jdt</groupId>
+                                <artifactId>core</artifactId>
+                            </exclusion>
+                            <exclusion>
+                                <groupId>tomcat</groupId>
+                                <artifactId>jasper-runtime</artifactId>
+                            </exclusion>
+                            <exclusion>
+                                <groupId>tomcat</groupId>
+                                <artifactId>jasper-compiler</artifactId>
+                            </exclusion>
+                        </exclusions>
+                    </dependency>
+        
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-test</artifactId>
+                        <version>${hadoop.version}</version>
+                        <exclusions>
+                            <exclusion>
+                                <groupId>org.apache.ftpserver</groupId>
+                                <artifactId>ftpserver-core</artifactId>
+                            </exclusion>
+                            <exclusion>
+                                <groupId>org.apache.ftpserver</groupId>
+                                <artifactId>ftpserver-deprecated</artifactId>
+                            </exclusion>
+                        </exclusions>
+                    </dependency>
+                </dependencies>
+            </dependencyManagement>
+        </profile>
+
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <properties>
+                <hadoop.profile>2</hadoop.profile>
+                <hadoop.version>2.0.2-alpha</hadoop.version>
+            </properties>
+            <dependencyManagement>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-client</artifactId>
+                        <version>${hadoop.version}</version>
+                        <exclusions>
+                            <exclusion>
+                                <groupId>org.glassfish</groupId>
+                                <artifactId>javax.servlet</artifactId>
+                            </exclusion>
+                        </exclusions>
+                    </dependency>
+                    
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-hdfs</artifactId>
+                        <version>${hadoop.version}</version>
+                        <classifier>tests</classifier>
+                    </dependency>
+                    
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-common</artifactId>
+                        <version>${hadoop.version}</version>
+                        <classifier>tests</classifier>
+                    </dependency>
+                    
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                        <version>${hadoop.version}</version>
+                    </dependency>
+                    
+                    <dependency>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+                        <version>${hadoop.version}</version>
+                    </dependency>                    
+                </dependencies>
+            </dependencyManagement>
+        </profile>
+    </profiles>
+
     <modules>
         <module>client</module>
         <module>metrics</module>
@@ -171,7 +284,7 @@
         </repository>
         <repository>
             <id>cdh.repo</id>
-            <url>https://repository.cloudera.com/content/groups/cloudera-repos</url>
+            <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
             <name>Cloudera Repository</name>
             <snapshots>
                 <enabled>false</enabled>
@@ -212,8 +325,14 @@
 
             <dependency>
                 <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-log4j12</artifactId>
+                <version>${slf4j.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-simple</artifactId>
-                <version>1.2</version>
+                <version>${slf4j.version}</version>
             </dependency>
 
             <dependency>
@@ -342,7 +461,7 @@
             <dependency>
                 <groupId>org.slf4j</groupId>
                 <artifactId>slf4j-api</artifactId>
-                <version>1.2</version>
+                <version>${slf4j.version}</version>
             </dependency>
 
             <dependency>
@@ -451,57 +570,58 @@
             </dependency>
 
             <dependency>
-                <groupId>org.apache.oozie</groupId>
-                <artifactId>oozie-core</artifactId>
-                <version>${oozie.version}</version>
+                <groupId>commons-el</groupId>
+                <artifactId>commons-el</artifactId>
+                <version>1.0</version>
             </dependency>
 
             <dependency>
-                <groupId>org.apache.oozie</groupId>
-                <artifactId>oozie-webapp</artifactId>
-                <version>${oozie.version}</version>
-                <type>war</type>
+                <groupId>javax.servlet.jsp</groupId>
+                <artifactId>jsp-api</artifactId>
+                <version>2.0</version>
             </dependency>
 
             <dependency>
-                <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-core</artifactId>
-                <version>${hadoop.version}</version>
+                <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-core</artifactId>
+                <version>${oozie.version}</version>
                 <exclusions>
                     <exclusion>
-                        <groupId>org.eclipse.jdt</groupId>
-                        <artifactId>core</artifactId>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-auth</artifactId>
                     </exclusion>
                     <exclusion>
-                        <groupId>tomcat</groupId>
-                        <artifactId>jasper-runtime</artifactId>
+                        <groupId>javax.servlet</groupId>
+                        <artifactId>servlet-api</artifactId>
                     </exclusion>
                     <exclusion>
-                        <groupId>tomcat</groupId>
-                        <artifactId>jasper-compiler</artifactId>
+                        <groupId>javax.servlet.jsp</groupId>
+                        <artifactId>jsp-api</artifactId>
                     </exclusion>
                 </exclusions>
             </dependency>
 
             <dependency>
-                <groupId>org.apache.hadoop.inmobi.tools</groupId>
-                <artifactId>hadoop-distcp</artifactId>
-                <version>${hadoop-distcp.version}</version>
+                <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-webapp</artifactId>
+                <version>${oozie.version}</version>
+                <type>war</type>
             </dependency>
 
             <dependency>
                 <groupId>org.apache.hadoop</groupId>
-                <artifactId>hadoop-test</artifactId>
+                <artifactId>hadoop-auth</artifactId>
                 <version>${hadoop.version}</version>
-                <!--<scope>test</scope>-->
+            </dependency>
+
+            <dependency>
+                <groupId>org.apache.hadoop.inmobi.tools</groupId>
+                <artifactId>hadoop-distcp</artifactId>
+                <version>${hadoop-distcp.version}</version>
                 <exclusions>
                     <exclusion>
-                        <groupId>org.apache.ftpserver</groupId>
-                        <artifactId>ftpserver-core</artifactId>
-                    </exclusion>
-                    <exclusion>
-                        <groupId>org.apache.ftpserver</groupId>
-                        <artifactId>ftpserver-deprecated</artifactId>
+                        <groupId>org.apache.hadoop</groupId>
+                        <artifactId>hadoop-core</artifactId>
                     </exclusion>
                 </exclusions>
             </dependency>
@@ -836,6 +956,8 @@
                         <exclude>**/.settings/**</exclude>
                         <exclude>**/test-output/**</exclude>
                         <exclude>**/data.txt</exclude>
+                        <exclude>**/maven-eclipse.xml</exclude>
+                        <exclude>**/.externalToolBuilders/**</exclude>
                     </excludes>
                 </configuration>
                 <executions>
@@ -881,7 +1003,7 @@
                 <groupId>org.codehaus.mojo</groupId>
                 <artifactId>findbugs-maven-plugin</artifactId>
                 <configuration>
-                    <!--debug>true</debug-->
+                    <!--debug>true</debug -->
                     <xmlOutput>true</xmlOutput>
                     <excludeFilterFile>${basedir}/../build-tools/src/main/resources/falcon/findbugs-exclude.xml</excludeFilterFile>
                     <failOnError>true</failOnError>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/prism/pom.xml
----------------------------------------------------------------------
diff --git a/prism/pom.xml b/prism/pom.xml
index 561df89..55439c1 100644
--- a/prism/pom.xml
+++ b/prism/pom.xml
@@ -35,11 +35,18 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-common</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>javax.servlet.jsp</groupId>
+                    <artifactId>jsp-api</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
 
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/process/pom.xml
----------------------------------------------------------------------
diff --git a/process/pom.xml b/process/pom.xml
index b024a70..9cf028a 100644
--- a/process/pom.xml
+++ b/process/pom.xml
@@ -1,18 +1,24 @@
 <?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. -->
+<!--
+   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.
+  -->
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -25,6 +31,36 @@
     <name>Apache Falcon Process</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
             <groupId>commons-cli</groupId>
@@ -49,6 +85,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
@@ -57,16 +94,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-test</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-jms</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
----------------------------------------------------------------------
diff --git a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
index 34bfe7f..fbda0ea 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
@@ -50,7 +50,7 @@ public class OozieProcessMapperLateProcessTest {
 
         cleanupStore();
 
-        dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        dfsCluster = EmbeddedCluster.newCluster("testCluster");
         Configuration conf = dfsCluster.getConf();
         String hdfsUrl = conf.get("fs.default.name");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
----------------------------------------------------------------------
diff --git a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
index abf664c..af20397 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
@@ -18,7 +18,25 @@
 
 package org.apache.falcon.converter;
 
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.InputStreamReader;
+import java.util.Collections;
+import java.util.List;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
 import junit.framework.Assert;
+
 import org.apache.falcon.Tag;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -50,22 +68,6 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.InputStreamReader;
-import java.util.Collections;
-import java.util.List;
-
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertTrue;
-
 /**
  * Test for the Falcon entities mapping into Oozie artifacts.
  */
@@ -75,7 +77,7 @@ public class OozieProcessMapperTest extends AbstractTestBase {
 
     @BeforeClass
     public void setUpDFS() throws Exception {
-        EmbeddedCluster cluster = EmbeddedCluster.newCluster("testCluster", false);
+        EmbeddedCluster cluster = EmbeddedCluster.newCluster("testCluster");
         Configuration conf = cluster.getConf();
         hdfsUrl = conf.get("fs.default.name");
     }
@@ -281,6 +283,7 @@ public class OozieProcessMapperTest extends AbstractTestBase {
         return contents.toString();
     }
 
+    @Override
     @AfterClass
     public void cleanup() throws Exception {
         super.cleanup();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/replication/pom.xml
----------------------------------------------------------------------
diff --git a/replication/pom.xml b/replication/pom.xml
index 83a4cca..200cb2f 100644
--- a/replication/pom.xml
+++ b/replication/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -17,9 +17,8 @@
    limitations under the License.
   -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -32,8 +31,48 @@
     <name>Apache Falcon Replication</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                 <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.hadoop.inmobi.tools</groupId>
             <artifactId>hadoop-distcp</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/rerun/pom.xml
----------------------------------------------------------------------
diff --git a/rerun/pom.xml b/rerun/pom.xml
index 976775f..e054d22 100644
--- a/rerun/pom.xml
+++ b/rerun/pom.xml
@@ -41,6 +41,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/retention/pom.xml
----------------------------------------------------------------------
diff --git a/retention/pom.xml b/retention/pom.xml
index 8927013..763b752 100644
--- a/retention/pom.xml
+++ b/retention/pom.xml
@@ -46,6 +46,7 @@
         <dependency>
             <groupId>org.apache.falcon</groupId>
             <artifactId>falcon-test-util</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
----------------------------------------------------------------------
diff --git a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
index 6df7488..c8995b7 100644
--- a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
+++ b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
@@ -18,6 +18,29 @@
 
 package org.apache.falcon.retention;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.servlet.jsp.el.ELException;
+import javax.servlet.jsp.el.ExpressionEvaluator;
+
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.Option;
@@ -37,17 +60,6 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Logger;
 
-import javax.servlet.jsp.el.ELException;
-import javax.servlet.jsp.el.ExpressionEvaluator;
-import java.io.*;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
 /**
  * Feed Evictor is called only if the retention policy that applies
  * to the feed is that of delete.
@@ -77,7 +89,7 @@ public class FeedEvictor extends Configured implements Tool {
     }
 
     private FileSystem fs;
-    private Map<VARS, String> map = new TreeMap<VARS, String>();
+    private final Map<VARS, String> map = new TreeMap<VARS, String>();
     private final StringBuffer instancePaths = new StringBuffer("instancePaths=");
     private final StringBuffer buffer = new StringBuffer();
 
@@ -94,7 +106,7 @@ public class FeedEvictor extends Configured implements Tool {
 
         String[] feedLocs = feedBasePath.split("#");
         for (String path : feedLocs) {
-            evictor(path, retentionType, retentionLimit, timeZone, frequency, logFile);
+            evictor(path, retentionType, retentionLimit, timeZone, frequency);
         }
 
         logInstancePaths(new Path(logFile));
@@ -108,8 +120,7 @@ public class FeedEvictor extends Configured implements Tool {
     }
 
     private void evictor(String feedBasePath, String retentionType,
-                         String retentionLimit, String timeZone, String frequency,
-                         String logFile) throws IOException, ELException {
+                         String retentionLimit, String timeZone, String frequency) throws IOException, ELException {
         Path normalizedPath = new Path(feedBasePath);
         fs = normalizedPath.getFileSystem(getConf());
         feedBasePath = normalizedPath.toUri().getPath();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
----------------------------------------------------------------------
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 c5a2013..fc55e11 100644
--- a/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
+++ b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
@@ -18,6 +18,21 @@
 
 package org.apache.falcon.retention;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.falcon.Pair;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.hadoop.conf.Configuration;
@@ -29,24 +44,18 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import java.io.*;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Test for FeedEvictor.
  */
 public class FeedEvictorTest {
 
     private EmbeddedCluster cluster;
-    private InMemoryWriter stream = new InMemoryWriter(System.out);
-    private Map<String, String> map = new HashMap<String, String>();
+    private final InMemoryWriter stream = new InMemoryWriter(System.out);
+    private final Map<String, String> map = new HashMap<String, String>();
 
     @BeforeClass
     public void start() throws Exception {
-        cluster = EmbeddedCluster.newCluster("test", false);
+        cluster = EmbeddedCluster.newCluster("test");
         FeedEvictor.OUT.set(stream);
     }
 
@@ -384,7 +393,7 @@ public class FeedEvictorTest {
 
     private static class InMemoryWriter extends PrintStream {
 
-        private StringBuffer buffer = new StringBuffer();
+        private final StringBuffer buffer = new StringBuffer();
 
         public InMemoryWriter(OutputStream out) {
             super(out);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/src/bin/falcon-start
----------------------------------------------------------------------
diff --git a/src/bin/falcon-start b/src/bin/falcon-start
index c22d4c8..f7b6524 100755
--- a/src/bin/falcon-start
+++ b/src/bin/falcon-start
@@ -44,7 +44,7 @@ else
 fi
 
 pushd ${BASEDIR} > /dev/null
-JAVA_PROPERTIES="-Dconfig.location=${BASEDIR}/conf"
+JAVA_PROPERTIES="$FALCON_OPTS -Dconfig.location=${BASEDIR}/conf"
 while [[ ${1} =~ ^\-D ]]; do
   JAVA_PROPERTIES="${JAVA_PROPERTIES} ${1}"
   shift

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/416a358b/test-util/pom.xml
----------------------------------------------------------------------
diff --git a/test-util/pom.xml b/test-util/pom.xml
index 9ebc056..92bf9d1 100644
--- a/test-util/pom.xml
+++ b/test-util/pom.xml
@@ -7,9 +7,9 @@
    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.
@@ -17,8 +17,7 @@
    limitations under the License.
   -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
@@ -32,6 +31,57 @@
     <name>Apache Falcon Test Utility</name>
     <packaging>jar</packaging>
 
+    <profiles>
+        <profile>
+            <id>hadoop-1</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>1</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-core</artifactId>
+                </dependency>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-test</artifactId>
+                </dependency>
+            </dependencies>
+        </profile>
+
+        <profile>
+            <id>hadoop-2</id>
+            <activation>
+                <property>
+                    <name>hadoop.profile</name>
+                    <value>2</value>
+                </property>
+            </activation>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-client</artifactId>
+                </dependency>
+                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <classifier>tests</classifier>
+                </dependency>
+                                
+                <dependency>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-common</artifactId>
+                    <classifier>tests</classifier>
+                </dependency>                
+            </dependencies>
+        </profile>
+    </profiles>
+
     <dependencies>
         <dependency>
             <groupId>org.apache.falcon</groupId>
@@ -39,16 +89,6 @@
         </dependency>
 
         <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-test</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>