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/04/26 17:50:17 UTC

[01/47] git commit: Fix for inmemory rerun queue comparator

Updated Branches:
  refs/heads/master dfaf33f20 -> f15ef92a8


Fix for inmemory rerun queue comparator


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

Branch: refs/heads/master
Commit: ce2589e2b36786a4fe3e61163ad58b62ccbe0e37
Parents: dfaf33f
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Fri Apr 12 12:54:00 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Fri Apr 12 12:54:00 2013 +0530

----------------------------------------------------------------------
 .../org/apache/falcon/rerun/event/RerunEvent.java  |   16 +---
 .../falcon/rerun/queue/InMemoryQueueTest.java      |   66 +++++++++++++++
 2 files changed, 70 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ce2589e2/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
index 41cb488..9526e0a 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
@@ -17,6 +17,7 @@
  */
 package org.apache.falcon.rerun.event;
 
+import java.util.Date;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.TimeUnit;
 
@@ -80,18 +81,9 @@ public class RerunEvent implements Delayed {
 
 	@Override
 	public int compareTo(Delayed o) {
-		int ret = 0;
-		RerunEvent event = (RerunEvent) o;
-
-		if (this.delayInMilliSec < event.delayInMilliSec)
-			ret = -1;
-		else if (this.delayInMilliSec > event.delayInMilliSec)
-			ret = 1;
-		else if (this.msgInsertTime == event.msgInsertTime)
-			ret = 0;
-
-		return ret;
-
+        RerunEvent event = (RerunEvent) o;
+        return new Date(msgInsertTime + delayInMilliSec).
+                compareTo(new Date(event.msgInsertTime + event.delayInMilliSec));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ce2589e2/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
new file mode 100644
index 0000000..09779db
--- /dev/null
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
@@ -0,0 +1,66 @@
+package org.apache.falcon.rerun.queue;
+
+import org.apache.falcon.FalconException;
+import org.apache.falcon.rerun.event.RerunEvent;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.io.File;
+import java.util.LinkedList;
+
+public class InMemoryQueueTest {
+
+    @Test (timeOut = 10000)
+    public void testDelayedQueue() throws Exception {
+        runTest();
+    }
+
+    private void runTest() throws InterruptedException, FalconException {
+        InMemoryQueue<MyEvent> queue = new InMemoryQueue<MyEvent>(new File("target"));
+
+        LinkedList<MyEvent> events = new LinkedList<MyEvent>();
+
+        for (int index = 0; index < 5; index++) {
+            Thread.sleep(30);
+            long time = System.currentTimeMillis();
+            int delay = ((5 - index) / 2) * 50;
+            MyEvent event = new MyEvent("someCluster", Integer.toString(index),
+                    time, delay, "someType", "someName", "someInstance", 0);
+            queue.offer(event);
+            boolean inserted = false;
+            for (int posn = 0; posn < events.size(); posn++) {
+                MyEvent thisEvent = events.get(posn);
+                if (thisEvent.getDelayInMilliSec() + thisEvent.getMsgInsertTime() >
+                        event.getDelayInMilliSec() + event.getMsgInsertTime()) {
+                    events.add(posn, event);
+                    inserted = true;
+                    break;
+                }
+            }
+            if (!inserted) {
+                events.add(event);
+            }
+        }
+
+        for (MyEvent event : events) {
+            MyEvent queueEvent = queue.take();
+            Assert.assertEquals(queueEvent.getWfId(), event.getWfId());
+        }
+    }
+
+    private class MyEvent extends RerunEvent {
+
+        public MyEvent(String clusterName, String wfId,
+                       long msgInsertTime, long delay, String entityType,
+                       String entityName, String instance, int runId) {
+            super(clusterName, wfId, msgInsertTime, delay,
+                    entityType, entityName, instance, runId);
+        }
+
+        @Override
+        public RerunType getType() {
+            RerunType type = super.getType();
+            return type == null ? RerunType.RETRY : type;
+        }
+    }
+}


[08/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
index 19269c1..188c397 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
@@ -1,4 +1,3 @@
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -30,65 +29,65 @@ import org.apache.falcon.rerun.policy.RerunPolicyFactory;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 
 public class RetryHandler<M extends DelayedQueue<RetryEvent>> extends
-		AbstractRerunHandler<RetryEvent, M> {
+        AbstractRerunHandler<RetryEvent, M> {
 
-	@Override
-	public void handleRerun(String cluster, String entityType,
-			String entityName, String nominalTime, String runId, String wfId,
-			long msgReceivedTime) {
-		try {
-			Entity entity = getEntity(entityType, entityName);
-			Retry retry = getRetry(entity);
+    @Override
+    public void handleRerun(String cluster, String entityType,
+                            String entityName, String nominalTime, String runId, String wfId,
+                            long msgReceivedTime) {
+        try {
+            Entity entity = getEntity(entityType, entityName);
+            Retry retry = getRetry(entity);
 
-			if (retry == null) {
-				LOG.warn("Retry not configured for entity:" + entityType + "("
-						+ entity.getName() + "), ignoring failed retries");
-				return;
-			}
+            if (retry == null) {
+                LOG.warn("Retry not configured for entity:" + entityType + "("
+                        + entity.getName() + "), ignoring failed retries");
+                return;
+            }
 
-			int attempts = retry.getAttempts();
-			Frequency delay = retry.getDelay();
-			PolicyType policy = retry.getPolicy();
-			int intRunId = Integer.parseInt(runId);
+            int attempts = retry.getAttempts();
+            Frequency delay = retry.getDelay();
+            PolicyType policy = retry.getPolicy();
+            int intRunId = Integer.parseInt(runId);
 
-			if (attempts > intRunId) {
-				AbstractRerunPolicy rerunPolicy = RerunPolicyFactory
-						.getRetryPolicy(policy);
-				long delayTime = rerunPolicy.getDelay(delay,
-						Integer.parseInt(runId));
-				RetryEvent event = new RetryEvent(cluster, wfId,
-						msgReceivedTime, delayTime, entityType, entityName,
-						nominalTime, intRunId, attempts, 0);
-				offerToQueue(event);
-			} else {
-				LOG.warn("All retry attempt failed out of configured: "
-						+ attempts + " attempt for entity instance::"
-						+ entityName + ":" + nominalTime + " And WorkflowId: "
-						+ wfId);
+            if (attempts > intRunId) {
+                AbstractRerunPolicy rerunPolicy = RerunPolicyFactory
+                        .getRetryPolicy(policy);
+                long delayTime = rerunPolicy.getDelay(delay,
+                        Integer.parseInt(runId));
+                RetryEvent event = new RetryEvent(cluster, wfId,
+                        msgReceivedTime, delayTime, entityType, entityName,
+                        nominalTime, intRunId, attempts, 0);
+                offerToQueue(event);
+            } else {
+                LOG.warn("All retry attempt failed out of configured: "
+                        + attempts + " attempt for entity instance::"
+                        + entityName + ":" + nominalTime + " And WorkflowId: "
+                        + wfId);
 
-				GenericAlert.alertRetryFailed(entityType, entityName,
-						nominalTime, wfId, runId,
-						"All retry attempt failed out of configured: "
-								+ attempts + " attempt for entity instance::");
-			}
-		} catch (Exception e) {
-			LOG.error("Error during retry of entity instance " + entityName
-					+ ":" + nominalTime, e);
-			GenericAlert.alertRetryFailed(entityType, entityName, nominalTime,
-					wfId, runId, e.getMessage());
-		}
+                GenericAlert.alertRetryFailed(entityType, entityName,
+                        nominalTime, wfId, runId,
+                        "All retry attempt failed out of configured: "
+                                + attempts + " attempt for entity instance::");
+            }
+        } catch (Exception e) {
+            LOG.error("Error during retry of entity instance " + entityName
+                    + ":" + nominalTime, e);
+            GenericAlert.alertRetryFailed(entityType, entityName, nominalTime,
+                    wfId, runId, e.getMessage());
+        }
 
-	}
+    }
 
-	@Override
-	public void init(M queue) throws FalconException {
-		super.init(queue);
-		Thread daemon = new Thread(new RetryConsumer(this));
-		daemon.setName("RetryHandler");
-		daemon.setDaemon(true);
-		daemon.start();
-		LOG.info("RetryHandler  thread started");
+    @Override
+    public void init(M queue) throws FalconException {
+        super.init(queue);
+        Thread daemon = new Thread(new RetryConsumer(this));
+        daemon.setName("RetryHandler");
+        daemon.setDaemon(true);
+        daemon.start();
+        LOG.info("RetryHandler  thread started");
 
-	}
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
index ca6c354..4bfbef2 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
@@ -17,28 +17,28 @@
  */
 package org.apache.falcon.rerun.policy;
 
-import java.util.Date;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.expression.ExpressionHelper;
 
+import java.util.Date;
+
 public abstract class AbstractRerunPolicy {
 
-	public long getDurationInMilliSec(Frequency frequency)
-			throws FalconException {
-		ExpressionHelper helper = ExpressionHelper.get();
-		return helper.evaluate(frequency.toString(), Long.class);
+    public long getDurationInMilliSec(Frequency frequency)
+            throws FalconException {
+        ExpressionHelper helper = ExpressionHelper.get();
+        return helper.evaluate(frequency.toString(), Long.class);
 
-	}
+    }
 
-	public static Date addTime(Date date, int milliSecondsToAdd) {
-		return new Date(date.getTime() + milliSecondsToAdd);
-	}
+    public static Date addTime(Date date, int milliSecondsToAdd) {
+        return new Date(date.getTime() + milliSecondsToAdd);
+    }
 
-	public abstract long getDelay(Frequency delay, int eventNumber)
-			throws FalconException;
+    public abstract long getDelay(Frequency delay, int eventNumber)
+            throws FalconException;
 
-	public abstract long getDelay(Frequency delay, Date nominaltime,
-			Date cutOffTime) throws FalconException;
+    public abstract long getDelay(Frequency delay, Date nominaltime,
+                                  Date cutOffTime) throws FalconException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
index 31ff052..b23e014 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
@@ -17,43 +17,44 @@
  */
 package org.apache.falcon.rerun.policy;
 
-import java.util.Date;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.expression.ExpressionHelper;
 
+import java.util.Date;
+
 public class ExpBackoffPolicy extends AbstractRerunPolicy {
 
-	@Override
-	public long getDelay(Frequency delay, int eventNumber)
-			throws FalconException {
-		return (long) (getDurationInMilliSec(delay) * Math.pow(getPower(),
-				eventNumber));
-	}
-
-	@Override
-	public long getDelay(Frequency delay, Date nominalTime, Date cutOffTime)
-			throws FalconException {
-		ExpressionHelper evaluator = ExpressionHelper.get();
-		Date now = new Date();
-		Date lateTime = nominalTime;
-		long delayMilliSeconds = evaluator.evaluate(delay.toString(),
-				Long.class);
-		int factor = 1;
-		// TODO we can get rid of this using formula
-		while (lateTime.compareTo(now)<=0) {
-			lateTime = addTime(lateTime, (int) (factor * delayMilliSeconds));
-			factor *= getPower();
-		}
-		if (lateTime.after(cutOffTime))
-			lateTime = cutOffTime;
-		return (lateTime.getTime() - nominalTime.getTime());
-
-	}
-
-	protected int getPower() {
-		return 2;
-	}
+    @Override
+    public long getDelay(Frequency delay, int eventNumber)
+            throws FalconException {
+        return (long) (getDurationInMilliSec(delay) * Math.pow(getPower(),
+                eventNumber));
+    }
+
+    @Override
+    public long getDelay(Frequency delay, Date nominalTime, Date cutOffTime)
+            throws FalconException {
+        ExpressionHelper evaluator = ExpressionHelper.get();
+        Date now = new Date();
+        Date lateTime = nominalTime;
+        long delayMilliSeconds = evaluator.evaluate(delay.toString(),
+                Long.class);
+        int factor = 1;
+        // TODO we can get rid of this using formula
+        while (lateTime.compareTo(now) <= 0) {
+            lateTime = addTime(lateTime, (int) (factor * delayMilliSeconds));
+            factor *= getPower();
+        }
+        if (lateTime.after(cutOffTime)) {
+            lateTime = cutOffTime;
+        }
+        return (lateTime.getTime() - nominalTime.getTime());
+
+    }
+
+    protected int getPower() {
+        return 2;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
index c8c400b..19fe8f7 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
@@ -18,11 +18,11 @@
 
 package org.apache.falcon.rerun.policy;
 
-import java.util.Date;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Frequency;
 
+import java.util.Date;
+
 public class FinalPolicy extends AbstractRerunPolicy {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
index 83107ec..9bcca26 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
@@ -18,8 +18,8 @@
 package org.apache.falcon.rerun.policy;
 
 public class PeriodicPolicy extends ExpBackoffPolicy {
-	@Override
-	protected int getPower() {
-		return 1;
-	}
+    @Override
+    protected int getPower() {
+        return 1;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
index f7ac42f..5ee902a 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
@@ -28,7 +28,7 @@ public class RerunPolicyFactory {
     public static AbstractRerunPolicy getRetryPolicy(PolicyType latePolicy) {
         switch (latePolicy) {
             case PERIODIC:
-                return new  PeriodicPolicy();
+                return new PeriodicPolicy();
 
             case EXP_BACKOFF:
                 return new ExpBackoffPolicy();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
index 73de658..82dfc22 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
@@ -17,17 +17,6 @@
  */
 package org.apache.falcon.rerun.queue;
 
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import javax.jms.ConnectionFactory;
-import javax.jms.Destination;
-import javax.jms.JMSException;
-import javax.jms.MessageConsumer;
-import javax.jms.MessageProducer;
-import javax.jms.Session;
-import javax.jms.TextMessage;
-
 import org.apache.activemq.ActiveMQConnection;
 import org.apache.activemq.ActiveMQConnectionFactory;
 import org.apache.activemq.ScheduledMessage;
@@ -35,120 +24,124 @@ import org.apache.falcon.FalconException;
 import org.apache.falcon.rerun.event.RerunEvent;
 import org.apache.falcon.rerun.event.RerunEventFactory;
 
-public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
-
-	private ActiveMQConnection connection;
-	private String brokerUrl;
-	private String destinationName;
-	private Destination destination;
-	private MessageProducer producer;
-	private MessageConsumer consumer;
-
-	public ActiveMQueue(String brokerUrl, String destinationName) {
-		this.brokerUrl = brokerUrl;
-		this.destinationName = destinationName;
-	}
-
-	@Override
-	public boolean offer(T event) throws FalconException {
-		Session session;
-		try {
-			session = getSession();
-			TextMessage msg = session.createTextMessage(event.toString());
-			msg.setLongProperty(ScheduledMessage.AMQ_SCHEDULED_DELAY,
-					event.getDelay(TimeUnit.MILLISECONDS));
-			msg.setStringProperty("TYPE", event.getType().name());
-			producer.send(msg);
-			LOG.debug("Enqueued Message:" + event.toString() + "with delay "
-					+ event.getDelay(TimeUnit.MILLISECONDS) + " milli sec");
-			return true;
-		} catch (Exception e) {
-			LOG.error("Unable to offer event:" + event + " to activeMqueue", e);
-			throw new FalconException("Unable to offer event:" + event + " to activeMqueue", e);
-		}
-	}
-
-	private Session getSession() throws Exception {
-		if (connection == null) {
-			init();
-		}
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		return session;
-	}
-
-	@Override
-	public T take() throws FalconException {
-		try {
-			TextMessage textMessage = (TextMessage) consumer.receive();
-			T event = new RerunEventFactory<T>().getRerunEvent(
-					textMessage.getStringProperty("TYPE"),
-					textMessage.getText());
-			LOG.debug("Dequeued Message:" + event.toString());
-			return event;
-		} catch (Exception e) {
-			LOG.error("Error getting the messge from ActiveMqueue: ", e);
-			throw new FalconException("Error getting the messge from ActiveMqueue: ", e);
-		}
-	}
-
-	@Override
-	public void populateQueue(List<T> events) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public void init() {
-		try {
-			createAndStartConnection("", "", brokerUrl);
-			Session session = connection.createSession(false,
-					Session.AUTO_ACKNOWLEDGE);
-			destination = session.createQueue(destinationName);
-			producer = session.createProducer(destination);
-			consumer = session.createConsumer(destination);
-			LOG.info("Initialized Queue on activeMQ: " + destinationName);
-		} catch (Exception e) {
-			LOG.error(
-					"Error starting ActiveMQueue connection for dealyed queue",
-					e);
-			throw new RuntimeException(
-					"Error starting ActiveMQueue connection for delayed queue",
-					e);
-		}
-	}
-
-	private void createAndStartConnection(String userName, String password,
-			String url) throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				userName, password, url);
-		connection = (ActiveMQConnection) connectionFactory.createConnection();
-		connection.start();
-		LOG.info("Connected successfully to " + url);
-	}
-
-	@Override
-	public void reconnect() throws FalconException {
-		try {
-			LOG.info("Attempting to close producer");
-			producer.close();
-			LOG.info("Producer closed successfully");
-		} catch (Exception ignore) {
-		}
-		try {
-			LOG.info("Attempting to close consumer");
-			consumer.close();
-			LOG.info("Consumer closed successfully");
-		} catch (Exception ignore) {
-		}
-		try {
-			LOG.info("Attempting to close connection");
-			connection.close();
-			LOG.info("Connection closed successfully");
-		} catch (Exception ignore) {
-		}
+import javax.jms.*;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
 
-		init();
+public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
 
-	}
+    private ActiveMQConnection connection;
+    private String brokerUrl;
+    private String destinationName;
+    private Destination destination;
+    private MessageProducer producer;
+    private MessageConsumer consumer;
+
+    public ActiveMQueue(String brokerUrl, String destinationName) {
+        this.brokerUrl = brokerUrl;
+        this.destinationName = destinationName;
+    }
+
+    @Override
+    public boolean offer(T event) throws FalconException {
+        Session session;
+        try {
+            session = getSession();
+            TextMessage msg = session.createTextMessage(event.toString());
+            msg.setLongProperty(ScheduledMessage.AMQ_SCHEDULED_DELAY,
+                    event.getDelay(TimeUnit.MILLISECONDS));
+            msg.setStringProperty("TYPE", event.getType().name());
+            producer.send(msg);
+            LOG.debug("Enqueued Message:" + event.toString() + "with delay "
+                    + event.getDelay(TimeUnit.MILLISECONDS) + " milli sec");
+            return true;
+        } catch (Exception e) {
+            LOG.error("Unable to offer event:" + event + " to activeMqueue", e);
+            throw new FalconException("Unable to offer event:" + event + " to activeMqueue", e);
+        }
+    }
+
+    private Session getSession() throws Exception {
+        if (connection == null) {
+            init();
+        }
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        return session;
+    }
+
+    @Override
+    public T take() throws FalconException {
+        try {
+            TextMessage textMessage = (TextMessage) consumer.receive();
+            T event = new RerunEventFactory<T>().getRerunEvent(
+                    textMessage.getStringProperty("TYPE"),
+                    textMessage.getText());
+            LOG.debug("Dequeued Message:" + event.toString());
+            return event;
+        } catch (Exception e) {
+            LOG.error("Error getting the messge from ActiveMqueue: ", e);
+            throw new FalconException("Error getting the messge from ActiveMqueue: ", e);
+        }
+    }
+
+    @Override
+    public void populateQueue(List<T> events) {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void init() {
+        try {
+            createAndStartConnection("", "", brokerUrl);
+            Session session = connection.createSession(false,
+                    Session.AUTO_ACKNOWLEDGE);
+            destination = session.createQueue(destinationName);
+            producer = session.createProducer(destination);
+            consumer = session.createConsumer(destination);
+            LOG.info("Initialized Queue on activeMQ: " + destinationName);
+        } catch (Exception e) {
+            LOG.error(
+                    "Error starting ActiveMQueue connection for dealyed queue",
+                    e);
+            throw new RuntimeException(
+                    "Error starting ActiveMQueue connection for delayed queue",
+                    e);
+        }
+    }
+
+    private void createAndStartConnection(String userName, String password,
+                                          String url) throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                userName, password, url);
+        connection = (ActiveMQConnection) connectionFactory.createConnection();
+        connection.start();
+        LOG.info("Connected successfully to " + url);
+    }
+
+    @Override
+    public void reconnect() throws FalconException {
+        try {
+            LOG.info("Attempting to close producer");
+            producer.close();
+            LOG.info("Producer closed successfully");
+        } catch (Exception ignore) {
+        }
+        try {
+            LOG.info("Attempting to close consumer");
+            consumer.close();
+            LOG.info("Consumer closed successfully");
+        } catch (Exception ignore) {
+        }
+        try {
+            LOG.info("Attempting to close connection");
+            connection.close();
+            LOG.info("Connection closed successfully");
+        } catch (Exception ignore) {
+        }
+
+        init();
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
index 0c63f8e..7e55206 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
@@ -17,22 +17,23 @@
  */
 package org.apache.falcon.rerun.queue;
 
-import java.util.List;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.rerun.event.RerunEvent;
 import org.apache.log4j.Logger;
 
-public abstract class  DelayedQueue<T extends RerunEvent> {
-	public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
-	public abstract boolean offer(T event) throws FalconException;
-
-	public abstract T take() throws FalconException;
-	
-	public abstract void populateQueue(List<T> events);
-	
-	public abstract void init();
-	
-	public abstract void reconnect() throws FalconException;
+import java.util.List;
+
+public abstract class DelayedQueue<T extends RerunEvent> {
+    public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
+
+    public abstract boolean offer(T event) throws FalconException;
+
+    public abstract T take() throws FalconException;
+
+    public abstract void populateQueue(List<T> events);
+
+    public abstract void init();
+
+    public abstract void reconnect() throws FalconException;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
index ff92f3d..7243f4d 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
@@ -17,130 +17,125 @@
  */
 package org.apache.falcon.rerun.queue;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.DelayQueue;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.rerun.event.RerunEvent;
 import org.apache.falcon.rerun.event.RerunEventFactory;
 import org.apache.log4j.Logger;
 
+import java.io.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.DelayQueue;
+
 public class InMemoryQueue<T extends RerunEvent> extends DelayedQueue<T> {
-	public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
-	protected DelayQueue<T> QUEUE = new DelayQueue<T>();
-	private File serializeFilePath;
-
-	@Override
-	public boolean offer(T event) {
-		boolean flag = QUEUE.offer(event);
-		beforeRetry(event);
-		LOG.debug("Enqueued Message:" + event.toString());
-		return flag;
-	}
-
-	@Override
-	public T take() throws FalconException {
-		T event;
-		try {
-			event = QUEUE.take();
-			LOG.debug("Dequeued Message:" + event.toString());
-			afterRetry(event);
-		} catch (InterruptedException e) {
-			throw new FalconException(e);
-		}
-		return event;
-	}
-
-	public InMemoryQueue(File serializeFilePath) {
-		this.serializeFilePath = serializeFilePath;
-	}
-
-	public void populateQueue(List<T> events) {
-		for (T event : events) {
-			QUEUE.offer(event);
-		}
-
-	}
-
-	@Override
-	public void init() {
-		List<T> events = bootstrap();
-		populateQueue(events);
-
-	}
-
-	@Override
-	public void reconnect() throws FalconException {
-		//Do Nothing
-	}
-	
-	private void beforeRetry(T event) {
-		File retryFile = getRetryFile(serializeFilePath, event);
-		try {
-			BufferedWriter out = new BufferedWriter(new FileWriter(retryFile,
-					true));
-			out.write(event.toString());
-			out.newLine();
-			out.close();
-		} catch (IOException e) {
-			LOG.warn(
-					"Unable to write entry for process-instance: "
-							+ event.getEntityName() + ":"
-							+ event.getInstance(), e);
-		}
-	}
-
-	private File getRetryFile(File basePath, T event) {
-		return new File(basePath, (event.getType().name()) + "-"
-				+ event.getEntityName() + "-"
-				+ event.getInstance().replaceAll(":", "-"));
-	}
-
-	private void afterRetry(T event) {
-		File retryFile = getRetryFile(serializeFilePath, event);
-		if (!retryFile.exists()) {
-			LOG.warn("Rerun file deleted or renamed for process-instance: "
-					+ event.getEntityName() + ":" + event.getInstance());
-			GenericAlert.alertRetryFailed(event.getEntityType(),
-					event.getEntityName(), event.getInstance(),
-					event.getWfId(), Integer.toString(event.getRunId()),
-					"Rerun file deleted or renamed for process-instance:");
-		} else {
-			if (!retryFile.delete()) {
-				LOG.warn("Unable to remove rerun file " + event.getWfId());
-				retryFile.deleteOnExit();
-			}
-		}
-	}
-
-	private List<T> bootstrap() {
-		List<T> rerunEvents = new ArrayList<T>();
-		for (File rerunFile : this.serializeFilePath.listFiles()) {
-			try {
-				BufferedReader reader = new BufferedReader(new FileReader(
-						rerunFile));
-				String line;
-				while ((line = reader.readLine()) != null) {
-					line.split("");
-					T event = new RerunEventFactory<T>().getRerunEvent(
-							rerunFile.getName(), line);
-					rerunEvents.add(event);
-				}
-			} catch (Exception e) {
-				LOG.warn(
-						"Not able to read rerun entry "
-								+ rerunFile.getAbsolutePath(), e);
-			}
-		}
-		return rerunEvents;
-
-	}
+    public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
+    protected DelayQueue<T> QUEUE = new DelayQueue<T>();
+    private File serializeFilePath;
+
+    @Override
+    public boolean offer(T event) {
+        boolean flag = QUEUE.offer(event);
+        beforeRetry(event);
+        LOG.debug("Enqueued Message:" + event.toString());
+        return flag;
+    }
+
+    @Override
+    public T take() throws FalconException {
+        T event;
+        try {
+            event = QUEUE.take();
+            LOG.debug("Dequeued Message:" + event.toString());
+            afterRetry(event);
+        } catch (InterruptedException e) {
+            throw new FalconException(e);
+        }
+        return event;
+    }
+
+    public InMemoryQueue(File serializeFilePath) {
+        this.serializeFilePath = serializeFilePath;
+    }
+
+    public void populateQueue(List<T> events) {
+        for (T event : events) {
+            QUEUE.offer(event);
+        }
+
+    }
+
+    @Override
+    public void init() {
+        List<T> events = bootstrap();
+        populateQueue(events);
+
+    }
+
+    @Override
+    public void reconnect() throws FalconException {
+        //Do Nothing
+    }
+
+    private void beforeRetry(T event) {
+        File retryFile = getRetryFile(serializeFilePath, event);
+        try {
+            BufferedWriter out = new BufferedWriter(new FileWriter(retryFile,
+                    true));
+            out.write(event.toString());
+            out.newLine();
+            out.close();
+        } catch (IOException e) {
+            LOG.warn(
+                    "Unable to write entry for process-instance: "
+                            + event.getEntityName() + ":"
+                            + event.getInstance(), e);
+        }
+    }
+
+    private File getRetryFile(File basePath, T event) {
+        return new File(basePath, (event.getType().name()) + "-"
+                + event.getEntityName() + "-"
+                + event.getInstance().replaceAll(":", "-"));
+    }
+
+    private void afterRetry(T event) {
+        File retryFile = getRetryFile(serializeFilePath, event);
+        if (!retryFile.exists()) {
+            LOG.warn("Rerun file deleted or renamed for process-instance: "
+                    + event.getEntityName() + ":" + event.getInstance());
+            GenericAlert.alertRetryFailed(event.getEntityType(),
+                    event.getEntityName(), event.getInstance(),
+                    event.getWfId(), Integer.toString(event.getRunId()),
+                    "Rerun file deleted or renamed for process-instance:");
+        } else {
+            if (!retryFile.delete()) {
+                LOG.warn("Unable to remove rerun file " + event.getWfId());
+                retryFile.deleteOnExit();
+            }
+        }
+    }
+
+    private List<T> bootstrap() {
+        List<T> rerunEvents = new ArrayList<T>();
+        for (File rerunFile : this.serializeFilePath.listFiles()) {
+            try {
+                BufferedReader reader = new BufferedReader(new FileReader(
+                        rerunFile));
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    line.split("");
+                    T event = new RerunEventFactory<T>().getRerunEvent(
+                            rerunFile.getName(), line);
+                    rerunEvents.add(event);
+                }
+            } catch (Exception e) {
+                LOG.warn(
+                        "Not able to read rerun entry "
+                                + rerunFile.getAbsolutePath(), e);
+            }
+        }
+        return rerunEvents;
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java b/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
index d446a06..f8edfbc 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
@@ -17,8 +17,6 @@
  */
 package org.apache.falcon.rerun.service;
 
-import java.io.File;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.rerun.event.LaterunEvent;
 import org.apache.falcon.rerun.event.RerunEvent.RerunType;
@@ -29,42 +27,44 @@ import org.apache.falcon.service.FalconService;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.log4j.Logger;
 
+import java.io.File;
+
 public class LateRunService implements FalconService {
 
-	private static final Logger LOG = Logger.getLogger(LateRunService.class);
+    private static final Logger LOG = Logger.getLogger(LateRunService.class);
 
-	@Override
-	public String getName() {
-		return LateRunService.class.getName();
-	}
+    @Override
+    public String getName() {
+        return LateRunService.class.getName();
+    }
 
-	@Override
-	public void init() throws FalconException {
-		AbstractRerunHandler<LaterunEvent, ActiveMQueue<LaterunEvent>> rerunHandler = RerunHandlerFactory
-				.getRerunHandler(RerunType.LATE);
-		ActiveMQueue<LaterunEvent> queue = new ActiveMQueue<LaterunEvent>(
-				StartupProperties
-						.get()
-						.getProperty("broker.url",
-								"failover:(tcp://localhost:61616)?initialReconnectDelay=5000"),
-				"falcon.late.queue");
-		rerunHandler.init(queue);
-	}
+    @Override
+    public void init() throws FalconException {
+        AbstractRerunHandler<LaterunEvent, ActiveMQueue<LaterunEvent>> rerunHandler = RerunHandlerFactory
+                .getRerunHandler(RerunType.LATE);
+        ActiveMQueue<LaterunEvent> queue = new ActiveMQueue<LaterunEvent>(
+                StartupProperties
+                        .get()
+                        .getProperty("broker.url",
+                                "failover:(tcp://localhost:61616)?initialReconnectDelay=5000"),
+                "falcon.late.queue");
+        rerunHandler.init(queue);
+    }
 
-	@Override
-	public void destroy() throws FalconException {
-		LOG.info("LateRun  thread destroyed");
-	}
+    @Override
+    public void destroy() throws FalconException {
+        LOG.info("LateRun  thread destroyed");
+    }
 
-	private File getBasePath() {
-		File basePath = new File(StartupProperties.get().getProperty(
-				"rerun.recorder.path", "/tmp/falcon/rerun"));
-		if ((!basePath.exists() && !basePath.mkdirs())
-				|| (basePath.exists() && !basePath.canWrite())) {
-			throw new RuntimeException("Unable to initialize late recorder @"
-					+ basePath);
-		}
-		return basePath;
-	}
+    private File getBasePath() {
+        File basePath = new File(StartupProperties.get().getProperty(
+                "rerun.recorder.path", "/tmp/falcon/rerun"));
+        if ((!basePath.exists() && !basePath.mkdirs())
+                || (basePath.exists() && !basePath.canWrite())) {
+            throw new RuntimeException("Unable to initialize late recorder @"
+                    + basePath);
+        }
+        return basePath;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java b/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
index f0d0939..8a902be 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
@@ -17,8 +17,6 @@
  */
 package org.apache.falcon.rerun.service;
 
-import java.io.File;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.rerun.event.RerunEvent.RerunType;
 import org.apache.falcon.rerun.event.RetryEvent;
@@ -30,38 +28,40 @@ import org.apache.falcon.service.FalconService;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.log4j.Logger;
 
+import java.io.File;
+
 public class RetryService implements FalconService {
 
-	private static final Logger LOG = Logger.getLogger(RetryService.class);
+    private static final Logger LOG = Logger.getLogger(RetryService.class);
 
-	@Override
-	public String getName() {
-		return "Falcon Retry failed Instance";
-	}
+    @Override
+    public String getName() {
+        return "Falcon Retry failed Instance";
+    }
 
-	@Override
-	public void init() throws FalconException {
-		AbstractRerunHandler<RetryEvent, DelayedQueue<RetryEvent>> rerunHandler = RerunHandlerFactory
-				.getRerunHandler(RerunType.RETRY);
-		 InMemoryQueue<RetryEvent> queue = new InMemoryQueue<RetryEvent>(
-		 getBasePath());
-		rerunHandler.init(queue);
-	}
+    @Override
+    public void init() throws FalconException {
+        AbstractRerunHandler<RetryEvent, DelayedQueue<RetryEvent>> rerunHandler = RerunHandlerFactory
+                .getRerunHandler(RerunType.RETRY);
+        InMemoryQueue<RetryEvent> queue = new InMemoryQueue<RetryEvent>(
+                getBasePath());
+        rerunHandler.init(queue);
+    }
 
-	@Override
-	public void destroy() throws FalconException {
-		LOG.info("RetryHandler  thread destroyed");
-	}
+    @Override
+    public void destroy() throws FalconException {
+        LOG.info("RetryHandler  thread destroyed");
+    }
 
-	private File getBasePath() {
-		File basePath = new File(StartupProperties.get().getProperty(
-				"retry.recorder.path", "/tmp/falcon/retry"));
-		if ((!basePath.exists() && !basePath.mkdirs())
-				|| (basePath.exists() && !basePath.canWrite())) {
-			throw new RuntimeException("Unable to initialize retry recorder @"
-					+ basePath);
-		}
-		return basePath;
-	}
+    private File getBasePath() {
+        File basePath = new File(StartupProperties.get().getProperty(
+                "retry.recorder.path", "/tmp/falcon/retry"));
+        if ((!basePath.exists() && !basePath.mkdirs())
+                || (basePath.exists() && !basePath.canWrite())) {
+            throw new RuntimeException("Unable to initialize retry recorder @"
+                    + basePath);
+        }
+        return basePath;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java b/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
index 494ddc6..be74032 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
@@ -17,8 +17,6 @@
  */
 package org.apache.falcon.rerun;
 
-import java.util.Date;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.rerun.policy.AbstractRerunPolicy;
@@ -27,67 +25,69 @@ import org.apache.falcon.rerun.policy.PeriodicPolicy;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import java.util.Date;
+
 public class AbstractRerunPolicyTest {
 
-	@Test
-	public void TestGetDurationInMillis() throws FalconException {
-		AbstractRerunPolicy policy = new AbstractRerunPolicy() {
+    @Test
+    public void TestGetDurationInMillis() throws FalconException {
+        AbstractRerunPolicy policy = new AbstractRerunPolicy() {
+
+            @Override
+            public long getDelay(Frequency delay, Date nominaltime,
+                                 Date cutOffTime) throws FalconException {
+                // TODO Auto-generated method stub
+                return 0;
+            }
 
-			@Override
-			public long getDelay(Frequency delay, Date nominaltime,
-					Date cutOffTime) throws FalconException {
-				// TODO Auto-generated method stub
-				return 0;
-			}
+            @Override
+            public long getDelay(Frequency delay, int eventNumber)
+                    throws FalconException {
+                // TODO Auto-generated method stub
+                return 0;
+            }
+        };
 
-			@Override
-			public long getDelay(Frequency delay, int eventNumber)
-					throws FalconException {
-				// TODO Auto-generated method stub
-				return 0;
-			}
-		};
+        Frequency frequency = new Frequency("minutes(1)");
+        Assert.assertEquals(policy.getDurationInMilliSec(frequency), 60000);
+        frequency = new Frequency("minutes(15)");
+        Assert.assertEquals(policy.getDurationInMilliSec(frequency), 900000);
+        frequency = new Frequency("hours(2)");
+        Assert.assertEquals(policy.getDurationInMilliSec(frequency), 7200000);
+    }
 
-		Frequency frequency = new Frequency("minutes(1)");
-		Assert.assertEquals(policy.getDurationInMilliSec(frequency), 60000);
-		frequency = new Frequency("minutes(15)");
-		Assert.assertEquals(policy.getDurationInMilliSec(frequency), 900000);
-		frequency = new Frequency("hours(2)");
-		Assert.assertEquals(policy.getDurationInMilliSec(frequency), 7200000);
-	}
+    @Test
+    public void TestExpBackoffPolicy() throws FalconException {
+        AbstractRerunPolicy backoff = new ExpBackoffPolicy();
+        long delay = backoff.getDelay(new Frequency("minutes(2)"), 2);
+        Assert.assertEquals(delay, 480000);
 
-	@Test
-	public void TestExpBackoffPolicy() throws FalconException {
-		AbstractRerunPolicy backoff = new ExpBackoffPolicy();
-		long delay = backoff.getDelay(new Frequency("minutes(2)"), 2);
-		Assert.assertEquals(delay, 480000);
+        long currentTime = System.currentTimeMillis();
+        delay = backoff.getDelay(new Frequency("minutes(2)"), new Date(
+                currentTime - 1 * 4 * 60 * 1000), new Date(currentTime + 1 * 60
+                * 60 * 1000));
+        Assert.assertEquals(delay, 1 * 6 * 60 * 1000);
 
-		long currentTime = System.currentTimeMillis();
-		delay = backoff.getDelay(new Frequency("minutes(2)"), new Date(
-				currentTime - 1 * 4 * 60 * 1000), new Date(currentTime + 1 * 60
-				* 60 * 1000));
-		Assert.assertEquals(delay, 1 * 6 * 60 * 1000);
-		
-		currentTime = System.currentTimeMillis();
-		delay = backoff.getDelay(new Frequency("minutes(1)"), new Date(
-				currentTime - 1 * 9 * 60 * 1000), new Date(currentTime + 1 * 60
-				* 60 * 1000));
-		Assert.assertEquals(delay, 900000);
-	}
+        currentTime = System.currentTimeMillis();
+        delay = backoff.getDelay(new Frequency("minutes(1)"), new Date(
+                currentTime - 1 * 9 * 60 * 1000), new Date(currentTime + 1 * 60
+                * 60 * 1000));
+        Assert.assertEquals(delay, 900000);
+    }
 
-	@Test
-	public void TestPeriodicPolicy() throws FalconException, InterruptedException {
-		AbstractRerunPolicy periodic = new PeriodicPolicy();
-		long delay = periodic.getDelay(new Frequency("minutes(2)"), 2);
-		Assert.assertEquals(delay, 120000);
-		delay = periodic.getDelay(new Frequency("minutes(2)"), 5);
-		Assert.assertEquals(delay, 120000);
+    @Test
+    public void TestPeriodicPolicy() throws FalconException, InterruptedException {
+        AbstractRerunPolicy periodic = new PeriodicPolicy();
+        long delay = periodic.getDelay(new Frequency("minutes(2)"), 2);
+        Assert.assertEquals(delay, 120000);
+        delay = periodic.getDelay(new Frequency("minutes(2)"), 5);
+        Assert.assertEquals(delay, 120000);
 
-		long currentTime = System.currentTimeMillis();
-		//Thread.sleep(1000);
-		delay = periodic.getDelay(new Frequency("minutes(3)"), new Date(
-				currentTime), new Date(currentTime + 1 * 60
-				* 60 * 1000));
-		Assert.assertEquals(delay, 180000);
-	}
+        long currentTime = System.currentTimeMillis();
+        //Thread.sleep(1000);
+        delay = periodic.getDelay(new Frequency("minutes(3)"), new Date(
+                currentTime), new Date(currentTime + 1 * 60
+                * 60 * 1000));
+        Assert.assertEquals(delay, 180000);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
index b29f830..6028e10 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
@@ -19,7 +19,6 @@
 package org.apache.falcon.rerun.handler;
 
 import com.sun.jersey.api.client.WebResource;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -30,6 +29,7 @@ import org.apache.falcon.entity.v0.cluster.Interfacetype;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
 import org.mockito.MockitoAnnotations;
 import org.testng.annotations.BeforeClass;
 
@@ -40,32 +40,33 @@ import java.io.StringWriter;
 import java.util.Collection;
 
 public class TestLateData {
-	
-	protected static final String FEED_XML = "/feed-template.xml";
+
+    protected static final String FEED_XML = "/feed-template.xml";
     protected static String CLUSTER_XML = "/cluster-template.xml";
     protected static final String PROCESS_XML = "/process-template.xml";
     protected static final String PROCESS_XML2 = "/process-template2.xml";
-    
+
     protected WebResource service = null;
     protected Configuration conf = new Configuration();
 
     @BeforeClass
     public void initConfigStore() throws Exception {
-    	MockitoAnnotations.initMocks(this);
+        MockitoAnnotations.initMocks(this);
         cleanupStore();
         String listeners = StartupProperties.get().getProperty("configstore.listeners");
-        StartupProperties.get().setProperty("configstore.listeners", 
+        StartupProperties.get().setProperty("configstore.listeners",
                 listeners.replace("org.apache.falcon.service.SharedLibraryHostingService", ""));
         ConfigurationStore.get().init();
-        
+
     }
-    
+
     protected void cleanupStore() throws FalconException {
         ConfigurationStore store = ConfigurationStore.get();
-        for(EntityType type:EntityType.values()) {
+        for (EntityType type : EntityType.values()) {
             Collection<String> entities = store.getEntities(type);
-            for(String entity:entities)
+            for (String entity : entities) {
                 store.remove(type, entity);
+            }
         }
     }
 
@@ -73,8 +74,8 @@ public class TestLateData {
         Unmarshaller unmarshaller = type.getUnmarshaller();
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
-		switch (type) {
-		case CLUSTER:
+        switch (type) {
+            case CLUSTER:
                 Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
                 cluster.setName(name);
                 ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
@@ -96,14 +97,14 @@ public class TestLateData {
     }
 
     public void setup() throws Exception {
-		ConfigurationStore store = ConfigurationStore.get();
-		for (EntityType type : EntityType.values()) {
-			for (String name : store.getEntities(type)) {
-				store.remove(type, name);
-			}
-		}
-		storeEntity(EntityType.CLUSTER , "testCluster");
-		storeEntity(EntityType.PROCESS, "sample");
+        ConfigurationStore store = ConfigurationStore.get();
+        for (EntityType type : EntityType.values()) {
+            for (String name : store.getEntities(type)) {
+                store.remove(type, name);
+            }
+        }
+        storeEntity(EntityType.CLUSTER, "testCluster");
+        storeEntity(EntityType.PROCESS, "sample");
         storeEntity(EntityType.FEED, "raw-logs");
         storeEntity(EntityType.FEED, "clicks");
         Unmarshaller unmarshaller = EntityType.PROCESS.getUnmarshaller();
@@ -112,14 +113,14 @@ public class TestLateData {
         store.publish(EntityType.PROCESS, process);
     }
 
-	public String marshallEntity(final Entity entity) throws FalconException,
-			JAXBException {
-		Marshaller marshaller = entity.getEntityType().getMarshaller();
-		StringWriter stringWriter = new StringWriter();
-		marshaller.marshal(entity, stringWriter);
-		return stringWriter.toString();
-	}
-	
+    public String marshallEntity(final Entity entity) throws FalconException,
+                                                             JAXBException {
+        Marshaller marshaller = entity.getEntityType().getMarshaller();
+        StringWriter stringWriter = new StringWriter();
+        marshaller.marshal(entity, stringWriter);
+        return stringWriter.toString();
+    }
+
 //	@Test
 //	private void TestLateWhenInstanceRunning() throws Exception
 //	{
@@ -189,5 +190,5 @@ public class TestLateData {
 //		Assert.assertNotSame(noFilesBefore, noFilesAfterRetry);
 //
 //	}
-	
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
index 0990b57..a13aa02 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
@@ -18,7 +18,6 @@
 package org.apache.falcon.rerun.queue;
 
 import junit.framework.Assert;
-
 import org.apache.activemq.broker.BrokerService;
 import org.apache.falcon.rerun.event.LaterunEvent;
 import org.apache.falcon.rerun.event.RerunEvent;
@@ -27,39 +26,39 @@ import org.testng.annotations.Test;
 
 public class ActiveMQTest {
 
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	private BrokerService broker;
-	private static final String DESTI = "activemq.topic";
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    private BrokerService broker;
+    private static final String DESTI = "activemq.topic";
+
+    @BeforeClass
+    private void setup() throws Exception {
+        broker = new BrokerService();
+        broker.setDataDirectory("target/activemq");
+        broker.addConnector(BROKER_URL);
+        broker.setBrokerName("localhost");
+        broker.setSchedulerSupport(true);
+        broker.start();
+    }
+
+    @Test
+    public void testBrokerStartAndEnqueue() {
+        ActiveMQueue<RerunEvent> activeMQueue = new ActiveMQueue<RerunEvent>(
+                BROKER_URL, DESTI);
+        activeMQueue.init();
+        RerunEvent event = new LaterunEvent("clusterName", "wfId",
+                System.currentTimeMillis(), 60 * 1000, "entityType",
+                "entityName", "instance", 0);
 
-	@BeforeClass
-	private void setup() throws Exception {
-		broker = new BrokerService();
-		broker.setDataDirectory("target/activemq");
-		broker.addConnector(BROKER_URL);
-		broker.setBrokerName("localhost");
-		broker.setSchedulerSupport(true);
-		broker.start();
-	}
+        try {
+            activeMQueue.offer(event);
+            broker.stop();
+            broker.start();
+            activeMQueue.reconnect();
+            activeMQueue.offer(event);
+        } catch (Exception e) {
+            Assert.fail();
+        }
 
-	@Test
-	public void testBrokerStartAndEnqueue() {
-		ActiveMQueue<RerunEvent> activeMQueue = new ActiveMQueue<RerunEvent>(
-				BROKER_URL, DESTI);
-		activeMQueue.init();
-		RerunEvent event = new LaterunEvent("clusterName", "wfId",
-				System.currentTimeMillis(), 60 * 1000, "entityType",
-				"entityName", "instance", 0);
-	
-		try{
-		activeMQueue.offer(event);
-		broker.stop();
-		broker.start();
-		activeMQueue.reconnect();
-		activeMQueue.offer(event);
-		}catch(Exception e){
-			Assert.fail();
-		}
 
-	
-	}
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
index 5bd35e5..d41453b 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
@@ -27,7 +27,7 @@ import java.util.LinkedList;
 
 public class InMemoryQueueTest {
 
-    @Test (timeOut = 10000)
+    @Test(timeOut = 10000)
     public void testDelayedQueue() throws Exception {
         runTest();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/resources/cluster-template.xml
----------------------------------------------------------------------
diff --git a/rerun/src/test/resources/cluster-template.xml b/rerun/src/test/resources/cluster-template.xml
index 46a9805..658365f 100644
--- a/rerun/src/test/resources/cluster-template.xml
+++ b/rerun/src/test/resources/cluster-template.xml
@@ -16,21 +16,22 @@
   limitations under the License.
   -->
 
-<cluster colo="gs" description="" name="corp" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-        <interfaces>
-                <interface type="readonly" endpoint="http://localhost:50070"
-                        version="0.20.2" />
-                <interface type="write" endpoint="hdfs://localhost:8020"                        version="0.20.2" />
-                <interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-                <interface type="workflow" endpoint="http://localhost:11000/oozie/"
-                        version="3.1" />
-                <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-                        version="5.1.6" />
-                <interface type="registry" endpoint="Hcat" version="1" />
-        </interfaces>
-        <locations>
-               <location name="temp" path="/tmp" />
-                <location name="working" path="/projects/falcon/working" />
-                  <location name="staging" path="/projects/falcon/staging" />
-         </locations>
- </cluster>
\ No newline at end of file
+<cluster colo="gs" description="" name="corp" xmlns="uri:falcon:cluster:0.1"
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="http://localhost:50070"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020" version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+        <location name="staging" path="/projects/falcon/staging"/>
+    </locations>
+</cluster>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/resources/feed-template.xml
----------------------------------------------------------------------
diff --git a/rerun/src/test/resources/feed-template.xml b/rerun/src/test/resources/feed-template.xml
index 6a277fe..c3f2834 100644
--- a/rerun/src/test/resources/feed-template.xml
+++ b/rerun/src/test/resources/feed-template.xml
@@ -17,27 +17,28 @@
   -->
 
 <feed description="clicks log" name="raw-logs" xmlns="uri:falcon:feed:0.1"
-	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+        >
 
-	<frequency>minutes(20)</frequency>
+    <frequency>minutes(20)</frequency>
     <timezone>UTC</timezone>
-    
-	<late-arrival cut-off="minutes(6)" />
-	<clusters>
-		<cluster name="corp" type="source">
-			<validity start="2012-05-01T00:00Z" end="2012-12-31T23:59Z"/>
-			<retention limit="months(36)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
-		</cluster>
-	</clusters>
-
-	<locations>
-		<location type="data"
-			path="/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}" />
-		<location type="stats" path="/projects/falcon/clicksStats" />
-		<location type="meta" path="/projects/falcon/clicksMetaData" />
-	</locations>
-
-	<ACL owner="testuser" group="group" permission="0x755" />
-	<schema location="/schema/clicks" provider="protobuf" />
+
+    <late-arrival cut-off="minutes(6)"/>
+    <clusters>
+        <cluster name="corp" type="source">
+            <validity start="2012-05-01T00:00Z" end="2012-12-31T23:59Z"/>
+            <retention limit="months(36)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data"
+                  path="/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
+    </locations>
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/resources/process-template.xml
----------------------------------------------------------------------
diff --git a/rerun/src/test/resources/process-template.xml b/rerun/src/test/resources/process-template.xml
index b7b4eb4..5a66586 100644
--- a/rerun/src/test/resources/process-template.xml
+++ b/rerun/src/test/resources/process-template.xml
@@ -17,10 +17,10 @@
   limitations under the License.
   -->
 
-<process name="sample" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample" xmlns="uri:falcon:process:0.1">
     <clusters>
         <cluster name="testCluster">
-            <validity end="2013-01-01T01:20Z" start="2010-01-01T01:00Z" />
+            <validity end="2013-01-01T01:20Z" start="2010-01-01T01:00Z"/>
         </cluster>
     </clusters>
     <concurrency>1</concurrency>
@@ -29,21 +29,21 @@
     <timezone>UTC</timezone>
 
     <inputs>
-        <input end-instance="today(20,20)" start-instance="today(0,-60)" feed="raw-logs" name="inputData" />
+        <input end-instance="today(20,20)" start-instance="today(0,-60)" feed="raw-logs" name="inputData"/>
     </inputs>
     <outputs>
-        <output instance="now(0,0)" feed="agg-logs" name="outputData" />
+        <output instance="now(0,0)" feed="agg-logs" name="outputData"/>
     </outputs>
     <properties>
-        <property name="queueName" value="default" />
-        <property name="field1" value="value1" />
-        <property name="field3" value="value3" />
-        <property name="field7" value="value7" />
+        <property name="queueName" value="default"/>
+        <property name="field1" value="value1"/>
+        <property name="field3" value="value3"/>
+        <property name="field7" value="value7"/>
     </properties>
-    <workflow path="/examples/apps/aggregator" />
-    <retry policy="exp-backoff" delay="minutes(2)" attempts="2" />
+    <workflow path="/examples/apps/aggregator"/>
+    <retry policy="exp-backoff" delay="minutes(2)" attempts="2"/>
 
     <late-process policy="periodic" delay="minutes(1)">
-        <late-input feed="inputData" workflow-path="hdfs://impression/late/workflow" />
+        <late-input feed="inputData" workflow-path="hdfs://impression/late/workflow"/>
     </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/test/resources/process-template2.xml
----------------------------------------------------------------------
diff --git a/rerun/src/test/resources/process-template2.xml b/rerun/src/test/resources/process-template2.xml
index 014f0e6..bbdcfe0 100644
--- a/rerun/src/test/resources/process-template2.xml
+++ b/rerun/src/test/resources/process-template2.xml
@@ -17,36 +17,36 @@
   limitations under the License.
   -->
 
-<process name="sample2" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample2" xmlns="uri:falcon:process:0.1">
     <clusters>
         <cluster name="testCluster">
-            <validity end="2013-01-01T01:20Z" start="2010-01-01T01:00Z" />
+            <validity end="2013-01-01T01:20Z" start="2010-01-01T01:00Z"/>
         </cluster>
     </clusters>
     <concurrency>1</concurrency>
     <execution>FIFO</execution>
     <frequency>minutes(5)</frequency>
     <timezone>UTC</timezone>
-	<inputs>
-		<input end-instance="yesterday(4,20)" start-instance="yesterday(0,0)"
-			feed="raw-logs" name="inputData" />
-		<input end-instance="yesterday(16,0)" start-instance="yesterday(12,0)"
-			feed="raw-logs" name="inputData2" />
-	</inputs>
-	<outputs>
-		<output instance="now(0,0)" feed="agg-logs" name="outputData" />
-	</outputs>
-	<properties>
-		<property name="queueName" value="default" />
-		<property name="field1" value="value1" />
-		<property name="field3" value="value3" />
-		<property name="field7" value="value7" />
-	</properties>
-	<workflow path="/examples/apps/aggregator" />
-	<retry policy="exp-backoff" delay="minutes(2)"	attempts="2" />
+    <inputs>
+        <input end-instance="yesterday(4,20)" start-instance="yesterday(0,0)"
+               feed="raw-logs" name="inputData"/>
+        <input end-instance="yesterday(16,0)" start-instance="yesterday(12,0)"
+               feed="raw-logs" name="inputData2"/>
+    </inputs>
+    <outputs>
+        <output instance="now(0,0)" feed="agg-logs" name="outputData"/>
+    </outputs>
+    <properties>
+        <property name="queueName" value="default"/>
+        <property name="field1" value="value1"/>
+        <property name="field3" value="value3"/>
+        <property name="field7" value="value7"/>
+    </properties>
+    <workflow path="/examples/apps/aggregator"/>
+    <retry policy="exp-backoff" delay="minutes(2)" attempts="2"/>
 
-	<late-process policy="exp-backoff" delay="hours(1)">
-		<late-input feed="inputData" workflow-path="hdfs://impression/late/workflow" />
-		<late-input feed="inputData2" workflow-path="hdfs://impression/late/workflow" />
-	</late-process>
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input feed="inputData" workflow-path="hdfs://impression/late/workflow"/>
+        <late-input feed="inputData2" workflow-path="hdfs://impression/late/workflow"/>
+    </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 dedb945..14c2c1c 100644
--- a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
+++ b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
@@ -18,33 +18,15 @@
 
 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.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;
 import org.apache.commons.cli.Options;
 import org.apache.commons.el.ExpressionEvaluatorImpl;
+import org.apache.falcon.Pair;
+import org.apache.falcon.entity.common.FeedDataPath;
+import org.apache.falcon.entity.common.FeedDataPath.VARS;
+import org.apache.falcon.expression.ExpressionHelper;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -53,12 +35,18 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.falcon.Pair;
-import org.apache.falcon.entity.common.FeedDataPath.VARS;
-import org.apache.falcon.entity.common.FeedDataPath;
-import org.apache.falcon.expression.ExpressionHelper;
 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.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.
@@ -91,73 +79,73 @@ public class FeedEvictor extends Configured implements Tool {
 
     private FileSystem fs;
     private Map<VARS, String> map = new TreeMap<VARS, String>();
-	private final StringBuffer instancePaths = new StringBuffer("instancePaths=");
-	private final StringBuffer buffer = new StringBuffer();
+    private final StringBuffer instancePaths = new StringBuffer("instancePaths=");
+    private final StringBuffer buffer = new StringBuffer();
 
     @Override
     public int run(String[] args) throws Exception {
-        
-    	CommandLine cmd = getCommand(args);
-        String feedBasePath = cmd.getOptionValue("feedBasePath").replaceAll("\\?\\{", "\\$\\{");        
+
+        CommandLine cmd = getCommand(args);
+        String feedBasePath = cmd.getOptionValue("feedBasePath").replaceAll("\\?\\{", "\\$\\{");
         String retentionType = cmd.getOptionValue("retentionType");
         String retentionLimit = cmd.getOptionValue("retentionLimit");
         String timeZone = cmd.getOptionValue("timeZone");
         String frequency = cmd.getOptionValue("frequency"); //to write out smart path filters
-        String logFile=cmd.getOptionValue("logFile");
+        String logFile = cmd.getOptionValue("logFile");
 
-        String []feedLocs = feedBasePath.split("#");
-        for(String path: feedLocs){
-        	evictor(path, retentionType, retentionLimit, timeZone, frequency, logFile);
+        String[] feedLocs = feedBasePath.split("#");
+        for (String path : feedLocs) {
+            evictor(path, retentionType, retentionLimit, timeZone, frequency, logFile);
         }
 
-		logInstancePaths(new Path(logFile), instancePaths.toString());
-		int len = buffer.length();
-		if (len > 0) {
-			stream.println("instances=" + buffer.substring(0, len - 1));
-		} else {
-			stream.println("instances=NULL");
-		}
+        logInstancePaths(new Path(logFile), instancePaths.toString());
+        int len = buffer.length();
+        if (len > 0) {
+            stream.println("instances=" + buffer.substring(0, len - 1));
+        } else {
+            stream.println("instances=NULL");
+        }
         return 0;
     }
-    
-	private void evictor(String feedBasePath, String retentionType,
-			String retentionLimit, String timeZone, String frequency,
-			String logFile) throws IOException, ELException {
-		Path normalizedPath = new Path(feedBasePath);
-		fs = normalizedPath.getFileSystem(getConf());
-		feedBasePath = normalizedPath.toUri().getPath();
-		LOG.info("Normalized path : " + feedBasePath);
-		Pair<Date, Date> range = getDateRange(retentionLimit);
-		String dateMask = getDateFormatInPath(feedBasePath);
-		List<Path> toBeDeleted = discoverInstanceToDelete(feedBasePath,
-				timeZone, dateMask, range.first);
-
-		LOG.info("Applying retention on " + feedBasePath + " type: "
-				+ retentionType + ", Limit: " + retentionLimit + ", timezone: "
-				+ timeZone + ", frequency: " + frequency);
-
-		DateFormat dateFormat = new SimpleDateFormat(format);
-		dateFormat.setTimeZone(TimeZone.getTimeZone(timeZone));
-		for (Path path : toBeDeleted) {
-			if (deleteInstance(path)) {
-				LOG.info("Deleted instance " + path);
-				Date date = getDate(path, feedBasePath, dateMask, timeZone);
-				buffer.append(dateFormat.format(date)).append(',');
-				instancePaths.append(path).append(",");
-			}
-		}
-
-	}
+
+    private void evictor(String feedBasePath, String retentionType,
+                         String retentionLimit, String timeZone, String frequency,
+                         String logFile) throws IOException, ELException {
+        Path normalizedPath = new Path(feedBasePath);
+        fs = normalizedPath.getFileSystem(getConf());
+        feedBasePath = normalizedPath.toUri().getPath();
+        LOG.info("Normalized path : " + feedBasePath);
+        Pair<Date, Date> range = getDateRange(retentionLimit);
+        String dateMask = getDateFormatInPath(feedBasePath);
+        List<Path> toBeDeleted = discoverInstanceToDelete(feedBasePath,
+                timeZone, dateMask, range.first);
+
+        LOG.info("Applying retention on " + feedBasePath + " type: "
+                + retentionType + ", Limit: " + retentionLimit + ", timezone: "
+                + timeZone + ", frequency: " + frequency);
+
+        DateFormat dateFormat = new SimpleDateFormat(format);
+        dateFormat.setTimeZone(TimeZone.getTimeZone(timeZone));
+        for (Path path : toBeDeleted) {
+            if (deleteInstance(path)) {
+                LOG.info("Deleted instance " + path);
+                Date date = getDate(path, feedBasePath, dateMask, timeZone);
+                buffer.append(dateFormat.format(date)).append(',');
+                instancePaths.append(path).append(",");
+            }
+        }
+
+    }
 
     private void logInstancePaths(Path path, String instancePaths) throws IOException {
-		LOG.info("Writing deleted instances to path " + path);
-		OutputStream out = fs.create(path);
-		out.write(instancePaths.getBytes());
-		out.close();
-		if(LOG.isDebugEnabled()){
-			debug(path, fs);
-		}
-	}
+        LOG.info("Writing deleted instances to path " + path);
+        OutputStream out = fs.create(path);
+        out.write(instancePaths.getBytes());
+        out.close();
+        if (LOG.isDebugEnabled()) {
+            debug(path, fs);
+        }
+    }
 
     private Pair<Date, Date> getDateRange(String period) throws ELException {
         Long duration = (Long) EVALUATOR.evaluate("${" + period + "}",
@@ -279,42 +267,42 @@ public class FeedEvictor extends Configured implements Tool {
     private boolean deleteInstance(Path path) throws IOException {
         return fs.delete(path, true);
     }
-    
-	private void debug(Path outPath, FileSystem fs) throws IOException {
-		ByteArrayOutputStream writer = new ByteArrayOutputStream();
-		InputStream instance = fs.open(outPath);
-		IOUtils.copyBytes(instance, writer, 4096, true);
-		LOG.debug("Instance Paths copied to " + outPath );
-		LOG.debug("Written "+writer);
-	}
-	
-	private CommandLine getCommand(String[] args)
-			throws org.apache.commons.cli.ParseException {
-		Options options = new Options();
-		Option opt;
-		opt = new Option("feedBasePath", true,
-				"base path for feed, ex /data/feed/${YEAR}-${MONTH}");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("retentionType", true,
-				"type of retention policy like delete, archive etc");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("retentionLimit", true,
-				"time limit for retention, ex hours(5), months(2), days(90)");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("timeZone", true, "timezone for feed, ex UTC");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("frequency", true,
-				"frequency of feed,  ex hourly, daily, monthly, minute, weekly, yearly");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("logFile", true, "log file for capturing size of feed");
-		opt.setRequired(true);
-		options.addOption(opt);
-		return new GnuParser().parse(options, args);
-	}
+
+    private void debug(Path outPath, FileSystem fs) throws IOException {
+        ByteArrayOutputStream writer = new ByteArrayOutputStream();
+        InputStream instance = fs.open(outPath);
+        IOUtils.copyBytes(instance, writer, 4096, true);
+        LOG.debug("Instance Paths copied to " + outPath);
+        LOG.debug("Written " + writer);
+    }
+
+    private CommandLine getCommand(String[] args)
+            throws org.apache.commons.cli.ParseException {
+        Options options = new Options();
+        Option opt;
+        opt = new Option("feedBasePath", true,
+                "base path for feed, ex /data/feed/${YEAR}-${MONTH}");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("retentionType", true,
+                "type of retention policy like delete, archive etc");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("retentionLimit", true,
+                "time limit for retention, ex hours(5), months(2), days(90)");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("timeZone", true, "timezone for feed, ex UTC");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("frequency", true,
+                "frequency of feed,  ex hourly, daily, monthly, minute, weekly, yearly");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("logFile", true, "log file for capturing size of feed");
+        opt.setRequired(true);
+        options.addOption(opt);
+        return new GnuParser().parse(options, args);
+    }
 
 }


[39/47] git commit: Fix for failing unit tests, still dependent on updated fix to OOZIE-674

Posted by sr...@apache.org.
Fix for failing unit tests, still dependent on updated fix to OOZIE-674


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

Branch: refs/heads/master
Commit: ca7c16a2fc5bdc1806e32abcb932282edd8795bc
Parents: 5c3dd0e
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Sat Apr 20 04:33:21 2013 +1000
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Sat Apr 20 04:33:21 2013 +1000

----------------------------------------------------------------------
 pom.xml                                            |    1 +
 .../falcon/cluster/util/EmbeddedCluster.java       |    3 ++-
 .../falcon/resource/EntityManagerJerseyTest.java   |    6 ++----
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ca7c16a2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 52d6ffb..ef53e37 100644
--- a/pom.xml
+++ b/pom.xml
@@ -691,6 +691,7 @@
                         <exclude>**/*.iml</exclude>
                         <exclude>**/target/**</exclude>
                         <exclude>**/activemq-data/**</exclude>
+                        <exclude>**/build/**</exclude>
                     </excludes>
                 </configuration>
                 <executions>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ca7c16a2/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 296decb..41f2d95 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
@@ -81,7 +81,8 @@ public class EmbeddedCluster {
             System.setProperty("test.build.data", "webapp/target/" + name + "/data");
         }
         String user = System.getProperty("user.name");
-        cluster.conf.set("hadoop.log.dir", "/tmp");
+        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", "*");
         cluster.conf.set("hadoop.proxyuser.oozie.hosts", "127.0.0.1");
         cluster.conf.set("hadoop.proxyuser.hdfs.groups", "*");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/ca7c16a2/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
index ca2a968..39fa823 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
@@ -432,9 +432,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
 
     private List<Path> createTestData() throws Exception {
         List<Path> list = new ArrayList<Path>();
-        Configuration conf = new Configuration();
-        conf.set("fs.default.name", "hdfs://localhost:8020");
-        FileSystem fs = FileSystem.get(conf);
+        FileSystem fs = cluster.getFileSystem();
         fs.mkdirs(new Path("/user/guest"));
         fs.setOwner(new Path("/user/guest"), REMOTE_USER, "users");
 
@@ -477,7 +475,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         path = new Path("/examples/input-data/rawLogs/" + formatter.format(date) + "/file");
         list.add(path);
         fs.create(path).close();
-        new FsShell(conf).run(new String[]{"-chown", "-R", "guest:users", "/examples/input-data/rawLogs"});
+        new FsShell(cluster.getConf()).run(new String[]{"-chown", "-R", "guest:users", "/examples/input-data/rawLogs"});
         return list;
     }
 


[26/47] git commit: Check style fixes relating to common module

Posted by sr...@apache.org.
Check style fixes relating to common module


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

Branch: refs/heads/master
Commit: bdadf2ed680b338d3ec48cb179300f50db69fee0
Parents: b1fcaac
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 14:59:49 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 14:59:49 2013 +0530

----------------------------------------------------------------------
 .../org/apache/falcon/FalconRuntimException.java   |    2 +-
 .../falcon/cleanup/AbstractCleanupHandler.java     |   28 +-
 .../apache/falcon/cleanup/FeedCleanupHandler.java  |    3 +
 .../falcon/cleanup/ProcessCleanupHandler.java      |    3 +
 .../org/apache/falcon/entity/ClusterHelper.java    |    3 +
 .../apache/falcon/entity/ColoClusterRelation.java  |   27 +-
 .../entity/EntityNotRegisteredException.java       |    3 +
 .../java/org/apache/falcon/entity/EntityUtil.java  |  271 ++++++++-------
 .../java/org/apache/falcon/entity/ExternalId.java  |    3 +
 .../java/org/apache/falcon/entity/FeedHelper.java  |   11 +-
 .../org/apache/falcon/entity/ProcessHelper.java    |    8 +-
 .../apache/falcon/entity/WorkflowNameBuilder.java  |   11 +-
 .../apache/falcon/entity/common/Configuration.java |    3 +
 .../apache/falcon/entity/common/FeedDataPath.java  |   10 +-
 .../falcon/entity/parser/ClusterEntityParser.java  |    3 +
 .../entity/parser/CrossEntityValidations.java      |    8 +-
 .../apache/falcon/entity/parser/EntityParser.java  |   13 +-
 .../falcon/entity/parser/EntityParserFactory.java  |   18 +-
 .../falcon/entity/parser/FeedEntityParser.java     |   21 +-
 .../falcon/entity/parser/ValidationException.java  |    2 +-
 .../falcon/entity/store/ConfigurationStore.java    |   34 +-
 .../entity/store/EntityAlreadyExistsException.java |    3 +
 .../falcon/entity/store/StoreAccessException.java  |    3 +
 .../org/apache/falcon/entity/v0/EntityGraph.java   |   38 ++-
 .../falcon/entity/v0/EntityIntegrityChecker.java   |   19 +-
 .../apache/falcon/expression/ExpressionHelper.java |   47 ++--
 .../java/org/apache/falcon/group/FeedGroupMap.java |   11 +-
 .../org/apache/falcon/security/CurrentUser.java    |   18 +-
 .../apache/falcon/security/FalconLoginModule.java  |    7 +-
 .../apache/falcon/security/FalconPrincipal.java    |    3 +
 .../security/FalconSecurityConfiguration.java      |    3 +
 .../apache/falcon/security/SecurityConstants.java  |    7 +-
 .../service/ConfigurationChangeListener.java       |    3 +
 .../org/apache/falcon/service/FalconService.java   |    3 +
 .../apache/falcon/service/LogCleanupService.java   |    7 +-
 .../apache/falcon/service/ServiceInitializer.java  |    5 +-
 .../java/org/apache/falcon/service/Services.java   |   22 +-
 .../org/apache/falcon/update/UpdateHelper.java     |   87 ++---
 .../apache/falcon/util/ApplicationProperties.java  |   11 +-
 .../org/apache/falcon/util/BuildProperties.java    |   17 +-
 .../apache/falcon/util/DeploymentProperties.java   |   17 +-
 .../org/apache/falcon/util/DeploymentUtil.java     |   29 +-
 .../org/apache/falcon/util/ReflectionUtils.java    |    5 +
 .../org/apache/falcon/util/RuntimeProperties.java  |   23 +-
 .../org/apache/falcon/util/StartupProperties.java  |   16 +-
 .../apache/falcon/workflow/WorkflowBuilder.java    |    6 +-
 .../falcon/workflow/WorkflowEngineFactory.java     |    8 +-
 .../workflow/engine/AbstractWorkflowEngine.java    |   10 +-
 .../engine/WorkflowEngineActionListener.java       |    4 +
 .../falcon/cleanup/LogCleanupServiceTest.java      |   56 ++--
 .../org/apache/falcon/entity/AbstractTestBase.java |   46 ++--
 .../falcon/entity/ColoClusterRelationTest.java     |    3 +
 .../org/apache/falcon/entity/EntityTypeTest.java   |    3 +
 .../org/apache/falcon/entity/EntityUtilTest.java   |    3 +
 .../org/apache/falcon/entity/FeedHelperTest.java   |    3 +
 .../falcon/entity/TestWorkflowNameBuilder.java     |    5 +-
 .../entity/parser/ClusterEntityParserTest.java     |    3 +
 .../falcon/entity/parser/FeedEntityParserTest.java |   21 +-
 .../falcon/entity/parser/FeedUpdateTest.java       |   10 +-
 .../entity/parser/ProcessEntityParserTest.java     |   14 +-
 .../entity/store/ConfigurationStoreTest.java       |    5 +-
 .../apache/falcon/entity/v0/EntityGraphTest.java   |   11 +-
 .../org/apache/falcon/group/FeedGroupMapTest.java  |    3 +
 .../apache/falcon/security/CurrentUserTest.java    |    3 +
 .../org/apache/falcon/update/UpdateHelperTest.java |    9 +-
 .../apache/falcon/util/StartupPropertiesTest.java  |    3 +
 .../org/apache/falcon/logging/LogProvider.java     |    8 +-
 .../apache/falcon/converter/AbstractTestBase.java  |   34 +-
 .../falcon/rerun/handler/LateRerunHandler.java     |    2 +-
 69 files changed, 671 insertions(+), 491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/FalconRuntimException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/FalconRuntimException.java b/common/src/main/java/org/apache/falcon/FalconRuntimException.java
index b603fbf..e465f09 100644
--- a/common/src/main/java/org/apache/falcon/FalconRuntimException.java
+++ b/common/src/main/java/org/apache/falcon/FalconRuntimException.java
@@ -19,7 +19,7 @@
 package org.apache.falcon;
 
 /**
- * Runtime Execption class for Falcon application
+ * Runtime Exception class for Falcon application.
  */
 public class FalconRuntimException extends RuntimeException {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
index 264ea28..49662d1 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
@@ -38,20 +38,25 @@ import javax.servlet.jsp.el.ELException;
 import javax.servlet.jsp.el.ExpressionEvaluator;
 import java.io.IOException;
 
+/**
+ * Falcon cleanup handler for cleaning up work, temp and log files
+ * left behind by falcon.
+ */
 public abstract class AbstractCleanupHandler {
 
     protected static final Logger LOG = Logger
             .getLogger(AbstractCleanupHandler.class);
-    protected final ConfigurationStore STORE = ConfigurationStore.get();
+    protected static final ConfigurationStore STORE = ConfigurationStore.get();
     public static final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
-    public static final ExpressionHelper resolver = ExpressionHelper.get();
+    public static final ExpressionHelper RESOLVER = ExpressionHelper.get();
 
     protected long getRetention(Entity entity, TimeUnit timeUnit)
-            throws FalconException {
+        throws FalconException {
+
         String retention = getRetentionValue(timeUnit);
         try {
             return (Long) EVALUATOR.evaluate("${" + retention + "}",
-                    Long.class, resolver, resolver);
+                    Long.class, RESOLVER, RESOLVER);
         } catch (ELException e) {
             throw new FalconException("Unable to evalue retention limit: "
                     + retention + " for entity: " + entity.getName());
@@ -64,9 +69,9 @@ public abstract class AbstractCleanupHandler {
 
     }
 
-    protected FileStatus[] getAllLogs(
-            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-            throws FalconException {
+    protected FileStatus[] getAllLogs(org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+        throws FalconException {
+
         String stagingPath = ClusterHelper.getLocation(cluster, "staging");
         Path logPath = getLogPath(entity, stagingPath);
         FileSystem fs = getFileSystem(cluster);
@@ -79,9 +84,8 @@ public abstract class AbstractCleanupHandler {
         return paths;
     }
 
-    private FileSystem getFileSystem(
-            org.apache.falcon.entity.v0.cluster.Cluster cluster)
-            throws FalconException {
+    private FileSystem getFileSystem(org.apache.falcon.entity.v0.cluster.Cluster cluster)
+        throws FalconException {
 
         FileSystem fs;
         try {
@@ -94,7 +98,7 @@ public abstract class AbstractCleanupHandler {
     }
 
     protected void delete(Cluster cluster, Entity entity, long retention)
-            throws FalconException {
+        throws FalconException {
 
         FileStatus[] logs = getAllLogs(cluster, entity);
         long now = System.currentTimeMillis();
@@ -104,7 +108,7 @@ public abstract class AbstractCleanupHandler {
                 try {
                     boolean isDeleted = getFileSystem(cluster).delete(
                             log.getPath(), true);
-                    if (isDeleted == false) {
+                    if (!isDeleted) {
                         LOG.error("Unable to delete path: " + log.getPath());
                     } else {
                         LOG.info("Deleted path: " + log.getPath());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
index 37fc653..0c8cf82 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.Collection;
 
+/**
+ * Cleanup files relating to feed management workflows.
+ */
 public class FeedCleanupHandler extends AbstractCleanupHandler {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
index 9b96879..add1237 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.Collection;
 
+/**
+ * Handler to cleanup files left behind by falcon relating to process.
+ */
 public class ProcessCleanupHandler extends AbstractCleanupHandler {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
index 3d77887..e332aba 100644
--- a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
@@ -22,6 +22,9 @@ import org.apache.falcon.entity.v0.cluster.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
+/**
+ * Helper to get end points relating to the cluster.
+ */
 public final class ClusterHelper {
     public static final String DEFAULT_BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java b/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
index 6c5d885..72dd952 100644
--- a/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
+++ b/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
@@ -28,21 +28,24 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class ColoClusterRelation implements ConfigurationChangeListener {
-    private static final ConcurrentHashMap<String, Set<String>> coloClusterMap
-            = new ConcurrentHashMap<String, Set<String>>();
-    private static final ColoClusterRelation instance = new ColoClusterRelation();
+/**
+ * Map of clusters in each colocation/ datacenter.
+ */
+public final class ColoClusterRelation implements ConfigurationChangeListener {
+    private static final ConcurrentHashMap<String, Set<String>> COLO_CLUSTER_MAP =
+        new ConcurrentHashMap<String, Set<String>>();
+    private static final ColoClusterRelation INSTANCE = new ColoClusterRelation();
 
     private ColoClusterRelation() {
     }
 
     public static ColoClusterRelation get() {
-        return instance;
+        return INSTANCE;
     }
 
     public Set<String> getClusters(String colo) {
-        if (coloClusterMap.containsKey(colo)) {
-            return coloClusterMap.get(colo);
+        if (COLO_CLUSTER_MAP.containsKey(colo)) {
+            return COLO_CLUSTER_MAP.get(colo);
         }
         return new HashSet<String>();
     }
@@ -54,8 +57,8 @@ public class ColoClusterRelation implements ConfigurationChangeListener {
         }
 
         Cluster cluster = (Cluster) entity;
-        coloClusterMap.putIfAbsent(cluster.getColo(), new HashSet<String>());
-        coloClusterMap.get(cluster.getColo()).add(cluster.getName());
+        COLO_CLUSTER_MAP.putIfAbsent(cluster.getColo(), new HashSet<String>());
+        COLO_CLUSTER_MAP.get(cluster.getColo()).add(cluster.getName());
     }
 
     @Override
@@ -65,9 +68,9 @@ public class ColoClusterRelation implements ConfigurationChangeListener {
         }
 
         Cluster cluster = (Cluster) entity;
-        coloClusterMap.get(cluster.getColo()).remove(cluster.getName());
-        if (coloClusterMap.get(cluster.getColo()).isEmpty()) {
-            coloClusterMap.remove(cluster.getColo());
+        COLO_CLUSTER_MAP.get(cluster.getColo()).remove(cluster.getName());
+        if (COLO_CLUSTER_MAP.get(cluster.getColo()).isEmpty()) {
+            COLO_CLUSTER_MAP.remove(cluster.getColo());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java b/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
index db922a4..40f83e4 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
@@ -20,6 +20,9 @@ package org.apache.falcon.entity;
 
 import org.apache.falcon.FalconException;
 
+/**
+ * Exception thrown by falcon when entity is not registered already in config store.
+ */
 public class EntityNotRegisteredException extends FalconException {
 
     public EntityNotRegisteredException(String message) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index f0ef515..ba086f8 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -44,12 +44,17 @@ import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.*;
 
-public class EntityUtil {
+/**
+ * Helper to get entity object.
+ */
+public final class EntityUtil {
     private static final long MINUTE_IN_MS = 60000L;
     private static final long HOUR_IN_MS = 3600000L;
     private static final long DAY_IN_MS = 86400000L;
     private static final long MONTH_IN_MS = 2592000000L;
 
+    private EntityUtil() {}
+
     public static <T extends Entity> T getEntity(EntityType type, String entityName) throws FalconException {
         ConfigurationStore configStore = ConfigurationStore.get();
         T entity = configStore.get(type, entityName);
@@ -233,19 +238,19 @@ public class EntityUtil {
 
         int count = 0;
         switch (frequency.getTimeUnit()) {
-            case months:
-                count = (int) ((now.getTime() - startTime.getTime()) / MONTH_IN_MS);
-                break;
-            case days:
-                count = (int) ((now.getTime() - startTime.getTime()) / DAY_IN_MS);
-                break;
-            case hours:
-                count = (int) ((now.getTime() - startTime.getTime()) / HOUR_IN_MS);
-                break;
-            case minutes:
-                count = (int) ((now.getTime() - startTime.getTime()) / MINUTE_IN_MS);
-                break;
-            default:
+        case months:
+            count = (int) ((now.getTime() - startTime.getTime()) / MONTH_IN_MS);
+            break;
+        case days:
+            count = (int) ((now.getTime() - startTime.getTime()) / DAY_IN_MS);
+            break;
+        case hours:
+            count = (int) ((now.getTime() - startTime.getTime()) / HOUR_IN_MS);
+            break;
+        case minutes:
+            count = (int) ((now.getTime() - startTime.getTime()) / MINUTE_IN_MS);
+            break;
+        default:
         }
 
         if (count > 2) {
@@ -268,19 +273,19 @@ public class EntityUtil {
 
         int count = 0;
         switch (frequency.getTimeUnit()) {
-            case months:
-                count = (int) ((instanceTime.getTime() - startTime.getTime()) / MONTH_IN_MS);
-                break;
-            case days:
-                count = (int) ((instanceTime.getTime() - startTime.getTime()) / DAY_IN_MS);
-                break;
-            case hours:
-                count = (int) ((instanceTime.getTime() - startTime.getTime()) / HOUR_IN_MS);
-                break;
-            case minutes:
-                count = (int) ((instanceTime.getTime() - startTime.getTime()) / MINUTE_IN_MS);
-                break;
-            default:
+        case months:
+            count = (int) ((instanceTime.getTime() - startTime.getTime()) / MONTH_IN_MS);
+            break;
+        case days:
+            count = (int) ((instanceTime.getTime() - startTime.getTime()) / DAY_IN_MS);
+            break;
+        case hours:
+            count = (int) ((instanceTime.getTime() - startTime.getTime()) / HOUR_IN_MS);
+            break;
+        case minutes:
+            count = (int) ((instanceTime.getTime() - startTime.getTime()) / MINUTE_IN_MS);
+            break;
+        default:
         }
 
         if (count > 2) {
@@ -340,7 +345,8 @@ public class EntityUtil {
 
     @SuppressWarnings("rawtypes")
     private static void mapToProperties(Object obj, String name, Map<String, String> propMap, String[] filterProps)
-            throws FalconException {
+        throws FalconException {
+
         if (obj == null) {
             return;
         }
@@ -429,36 +435,38 @@ public class EntityUtil {
 
     public static <T extends Entity> T getClusterView(T entity, String clusterName) {
         switch (entity.getEntityType()) {
-            case CLUSTER:
-                return entity;
-
-            case FEED:
-                Feed feed = (Feed) entity.clone();
-                Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
-                Iterator<Cluster> itr = feed.getClusters().getClusters().iterator();
-                while (itr.hasNext()) {
-                    Cluster cluster = itr.next();
-                    //In addition to retaining the required clster, retain the sources clusters if this is the target
-                    // cluster
-                    //1. Retain cluster if cluster n
-                    if (!(cluster.getName().equals(clusterName) ||
-                            (feedCluster.getType() == ClusterType.TARGET && cluster.getType() == ClusterType.SOURCE))) {
-                        itr.remove();
-                    }
+        case CLUSTER:
+            return entity;
+
+        case FEED:
+            Feed feed = (Feed) entity.copy();
+            Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
+            Iterator<Cluster> itr = feed.getClusters().getClusters().iterator();
+            while (itr.hasNext()) {
+                Cluster cluster = itr.next();
+                //In addition to retaining the required clster, retain the sources clusters if this is the target
+                // cluster
+                //1. Retain cluster if cluster n
+                if (!(cluster.getName().equals(clusterName)
+                        || (feedCluster.getType() == ClusterType.TARGET
+                        && cluster.getType() == ClusterType.SOURCE))) {
+                    itr.remove();
                 }
-                return (T) feed;
-
-            case PROCESS:
-                Process process = (Process) entity.clone();
-                Iterator<org.apache.falcon.entity.v0.process.Cluster> procItr
-                        = process.getClusters().getClusters().iterator();
-                while (procItr.hasNext()) {
-                    org.apache.falcon.entity.v0.process.Cluster cluster = procItr.next();
-                    if (!cluster.getName().equals(clusterName)) {
-                        procItr.remove();
-                    }
+            }
+            return (T) feed;
+
+        case PROCESS:
+            Process process = (Process) entity.copy();
+            Iterator<org.apache.falcon.entity.v0.process.Cluster> procItr =
+                process.getClusters().getClusters().iterator();
+            while (procItr.hasNext()) {
+                org.apache.falcon.entity.v0.process.Cluster cluster = procItr.next();
+                if (!cluster.getName().equals(clusterName)) {
+                    procItr.remove();
                 }
-                return (T) process;
+            }
+            return (T) process;
+        default:
         }
         throw new UnsupportedOperationException("Not supported for entity type " + entity.getEntityType());
     }
@@ -466,23 +474,24 @@ public class EntityUtil {
     public static Set<String> getClustersDefined(Entity entity) {
         Set<String> clusters = new HashSet<String>();
         switch (entity.getEntityType()) {
-            case CLUSTER:
-                clusters.add(entity.getName());
-                break;
-
-            case FEED:
-                Feed feed = (Feed) entity;
-                for (Cluster cluster : feed.getClusters().getClusters()) {
-                    clusters.add(cluster.getName());
-                }
-                break;
+        case CLUSTER:
+            clusters.add(entity.getName());
+            break;
+
+        case FEED:
+            Feed feed = (Feed) entity;
+            for (Cluster cluster : feed.getClusters().getClusters()) {
+                clusters.add(cluster.getName());
+            }
+            break;
 
-            case PROCESS:
-                Process process = (Process) entity;
-                for (org.apache.falcon.entity.v0.process.Cluster cluster : process.getClusters().getClusters()) {
-                    clusters.add(cluster.getName());
-                }
-                break;
+        case PROCESS:
+            Process process = (Process) entity;
+            for (org.apache.falcon.entity.v0.process.Cluster cluster : process.getClusters().getClusters()) {
+                clusters.add(cluster.getName());
+            }
+            break;
+        default:
         }
         return clusters;
     }
@@ -503,9 +512,33 @@ public class EntityUtil {
         return applicableClusters;
     }
 
-    public static Path getStagingPath(
-            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-            throws FalconException {
+    public static Retry getRetry(Entity entity) throws FalconException {
+        switch (entity.getEntityType()) {
+        case FEED:
+            if (!RuntimeProperties.get()
+                    .getProperty("feed.retry.allowed", "true")
+                    .equalsIgnoreCase("true")) {
+                return null;
+            }
+            Retry retry = new Retry();
+            retry.setAttempts(Integer.parseInt(RuntimeProperties.get()
+                    .getProperty("feed.retry.attempts", "3")));
+            retry.setDelay(new Frequency(RuntimeProperties.get().getProperty(
+                    "feed.retry.frequency", "minutes(5)")));
+            retry.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
+                    .getProperty("feed.retry.policy", "exp-backoff")));
+            return retry;
+        case PROCESS:
+            Process process = (Process) entity;
+            return process.getRetry();
+        default:
+            throw new FalconException("Cannot create Retry for entity:" + entity.getName());
+        }
+    }
+
+    public static Path getStagingPath(org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+        throws FalconException {
+
         try {
             return new Path(ClusterHelper.getLocation(cluster, "staging"),
                     EntityUtil.getStagingPath(entity));
@@ -514,67 +547,43 @@ public class EntityUtil {
         }
     }
 
-    public static Retry getRetry(Entity entity) throws FalconException {
-        switch (entity.getEntityType()) {
-            case FEED:
-                if (!RuntimeProperties.get()
-                        .getProperty("feed.retry.allowed", "true")
-                        .equalsIgnoreCase("true")) {
-                    return null;
-                }
-                Retry retry = new Retry();
-                retry.setAttempts(Integer.parseInt(RuntimeProperties.get()
-                        .getProperty("feed.retry.attempts", "3")));
-                retry.setDelay(new Frequency(RuntimeProperties.get().getProperty(
-                        "feed.retry.frequency", "minutes(5)")));
-                retry.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
-                        .getProperty("feed.retry.policy", "exp-backoff")));
-                return retry;
-            case PROCESS:
-                Process process = (Process) entity;
-                return process.getRetry();
-            default:
-                throw new FalconException("Cannot create Retry for entity:" + entity.getName());
-        }
-    }
-
     public static LateProcess getLateProcess(Entity entity)
-            throws FalconException {
+        throws FalconException {
+
         switch (entity.getEntityType()) {
-            case FEED:
-                if (!RuntimeProperties.get()
-                        .getProperty("feed.late.allowed", "true")
-                        .equalsIgnoreCase("true")) {
-                    return null;
-                }
-                LateProcess lateProcess = new LateProcess();
-                lateProcess.setDelay(new Frequency(RuntimeProperties.get()
-                        .getProperty("feed.late.frequency", "hours(3)")));
-                lateProcess.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
-                        .getProperty("feed.late.policy", "exp-backoff")));
-                LateInput lateInput = new LateInput();
-                lateInput.setInput(entity.getName());
-                //TODO - Assuming the late workflow is not used
-                lateInput.setWorkflowPath("ignore.xml");
-                lateProcess.getLateInputs().add(lateInput);
-                return lateProcess;
-            case PROCESS:
-                Process process = (Process) entity;
-                return process.getLateProcess();
-            default:
-                throw new FalconException("Cannot create Late Process for entity:" + entity.getName());
-        }
-    }
-
-    public static Path getLogPath(
-            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-            throws FalconException {
-        Path logPath = new Path(ClusterHelper.getLocation(cluster,
+        case FEED:
+            if (!RuntimeProperties.get()
+                    .getProperty("feed.late.allowed", "true")
+                    .equalsIgnoreCase("true")) {
+                return null;
+            }
+            LateProcess lateProcess = new LateProcess();
+            lateProcess.setDelay(new Frequency(RuntimeProperties.get()
+                    .getProperty("feed.late.frequency", "hours(3)")));
+            lateProcess.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
+                    .getProperty("feed.late.policy", "exp-backoff")));
+            LateInput lateInput = new LateInput();
+            lateInput.setInput(entity.getName());
+            //TODO - Assuming the late workflow is not used
+            lateInput.setWorkflowPath("ignore.xml");
+            lateProcess.getLateInputs().add(lateInput);
+            return lateProcess;
+        case PROCESS:
+            Process process = (Process) entity;
+            return process.getLateProcess();
+        default:
+            throw new FalconException("Cannot create Late Process for entity:" + entity.getName());
+        }
+    }
+
+    public static Path getLogPath(org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+        throws FalconException {
+
+        return new Path(ClusterHelper.getLocation(cluster,
                 "staging"), EntityUtil.getStagingPath(entity) + "/../logs");
-        return logPath;
     }
 
-    public static String UTCtoURIDate(String utc) throws FalconException {
+    public static String fromUTCtoURIDate(String utc) throws FalconException {
         DateFormat utcFormat = new SimpleDateFormat(
                 "yyyy'-'MM'-'dd'T'HH':'mm'Z'");
         Date utcDate;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/ExternalId.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ExternalId.java b/common/src/main/java/org/apache/falcon/entity/ExternalId.java
index cb865d2..c53d83a 100644
--- a/common/src/main/java/org/apache/falcon/entity/ExternalId.java
+++ b/common/src/main/java/org/apache/falcon/entity/ExternalId.java
@@ -25,6 +25,9 @@ import org.apache.falcon.entity.v0.SchemaHelper;
 
 import java.util.Date;
 
+/**
+ * External id as represented by workflow engine.
+ */
 public class ExternalId {
     private static final String SEPARATOR = "/";
     private String id;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/FeedHelper.java b/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
index 4a39d8c..c96120d 100644
--- a/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
@@ -28,7 +28,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-public class FeedHelper {
+/**
+ * Feed entity helper methods.
+ */
+public final class FeedHelper {
+
+    private FeedHelper() {}
+
     public static Cluster getCluster(Feed feed, String clusterName) {
         for (Cluster cluster : feed.getClusters().getClusters()) {
             if (cluster.getName().equals(clusterName)) {
@@ -93,7 +99,8 @@ public class FeedHelper {
     }
 
     public static String evaluateClusterExp(org.apache.falcon.entity.v0.cluster.Cluster clusterEntity, String exp)
-            throws FalconException {
+        throws FalconException {
+
         Properties properties = loadClusterProperties(clusterEntity);
         ExpressionHelper expHelp = ExpressionHelper.get();
         expHelp.setPropertiesForVariable(properties);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java b/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
index 598f26e..dc5ae7a 100644
--- a/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
@@ -21,7 +21,13 @@ package org.apache.falcon.entity;
 import org.apache.falcon.entity.v0.process.Cluster;
 import org.apache.falcon.entity.v0.process.Process;
 
-public class ProcessHelper {
+/**
+ * Helper methods for accessing process members.
+ */
+public final class ProcessHelper {
+
+    private ProcessHelper() {}
+
     public static Cluster getCluster(Process process, String clusterName) {
         for (Cluster cluster : process.getClusters().getClusters()) {
             if (cluster.getName().equals(clusterName)) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
index 4a636cd..a1216e7 100644
--- a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
+++ b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
@@ -26,10 +26,14 @@ import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+/**
+ * Convenient builder for workflow name.
+ * @param <T>
+ */
 public class WorkflowNameBuilder<T extends Entity> {
     private static final String PREFIX = "FALCON";
 
-    T entity;
+    private T entity;
     private Tag tag;
     private List<String> suffixes;
 
@@ -37,6 +41,7 @@ public class WorkflowNameBuilder<T extends Entity> {
         this.entity = entity;
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     public void setTag(Tag tag) {
         this.tag = tag;
     }
@@ -44,6 +49,7 @@ public class WorkflowNameBuilder<T extends Entity> {
     public void setSuffixes(List<String> suffixes) {
         this.suffixes = suffixes;
     }
+    //RESUME CHECKSTYLE CHECK
 
     public WorkflowName getWorkflowName() {
         return new WorkflowName(PREFIX, entity.getEntityType().name(),
@@ -61,6 +67,9 @@ public class WorkflowNameBuilder<T extends Entity> {
                 : WorkflowName.getTagAndSuffixes(entity, workflowName).second;
     }
 
+    /**
+     * Workflow name.
+     */
     public static class WorkflowName {
         private static final String SEPARATOR = "_";
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/common/Configuration.java b/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
index 0e7e8f1..2e0e426 100644
--- a/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
+++ b/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
@@ -23,6 +23,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
+/**
+ * Property k/v.
+ */
 public class Configuration implements Iterable<Map.Entry<String, String>>, Cloneable {
 
     private final Map<String, String> properties;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java b/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
index 470c98a..4031e14 100644
--- a/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
+++ b/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
@@ -19,8 +19,16 @@ package org.apache.falcon.entity.common;
 
 import java.util.regex.Pattern;
 
-public class FeedDataPath {
+/**
+ * Helper to map feed path and the time component.
+ */
+public final class FeedDataPath {
+
+    private FeedDataPath() {}
 
+    /**
+     * Standard variables for feed time components.
+     */
     public static enum VARS {
         YEAR("yyyy"), MONTH("MM"), DAY("dd"), HOUR("HH"), MINUTE("mm");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 c6ef988..8011dce 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
@@ -27,6 +27,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+/**
+ * Parser that parses cluster entity definition.
+ */
 public class ClusterEntityParser extends EntityParser<Cluster> {
 
     private static final Logger LOG = Logger.getLogger(ProcessEntityParser.class);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java b/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
index 09aed24..d8d3d78 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
@@ -28,8 +28,13 @@ import org.apache.falcon.expression.ExpressionHelper;
 
 import java.util.Date;
 
+/**
+ * Validation helper functions to validate across process, feed and cluster definitions.
+ */
 public final class CrossEntityValidations {
 
+    private CrossEntityValidations() {}
+
     public static void validateInstanceRange(Process process, Input input, Feed feed) throws FalconException {
 
         try {
@@ -78,7 +83,8 @@ public final class CrossEntityValidations {
     }
 
     public static void validateFeedRetentionPeriod(String startInstance, Feed feed, String clusterName)
-            throws FalconException {
+        throws FalconException {
+
         String feedRetention = FeedHelper.getCluster(feed, clusterName).getRetention().getLimit().toString();
         ExpressionHelper evaluator = ExpressionHelper.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
index 6b06cc4..d24fafc 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
@@ -31,10 +31,10 @@ import java.io.InputStream;
 import java.util.List;
 
 /**
- * Generic Abstract Entity Parser, the concrete FEED, PROCESS and CLUSTER
- * Should extend this parser to implement specific parsing.
+ * Generic Abstract Entity Parser, the concrete FEED, PROCESS and CLUSTER should extend this parser
+ * to implement specific parsing.
  *
- * @param <T>
+ * @param <T> of type Entity
  */
 public abstract class EntityParser<T extends Entity> {
 
@@ -42,11 +42,6 @@ public abstract class EntityParser<T extends Entity> {
 
     private final EntityType entityType;
 
-    /**
-     * Constructor
-     *
-     * @param entityType - can be FEED or PROCESS
-     */
     protected EntityParser(EntityType entityType) {
         this.entityType = entityType;
     }
@@ -69,7 +64,7 @@ public abstract class EntityParser<T extends Entity> {
     }
 
     /**
-     * Parses xml stream
+     * Parses xml stream.
      *
      * @param xmlStream
      * @return entity

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java b/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
index 43542c3..5a33201 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
@@ -33,20 +33,20 @@ public final class EntityParserFactory {
      * unmarshalled easily by concrete classes based on the class type using
      * JAXB.
      *
-     * @param entityType
+     * @param entityType - entity type
      * @return concrete parser based on entity type
      */
     public static EntityParser getParser(final EntityType entityType) {
 
         switch (entityType) {
-            case PROCESS:
-                return new ProcessEntityParser();
-            case FEED:
-                return new FeedEntityParser();
-            case CLUSTER:
-                return new ClusterEntityParser();
-            default:
-                throw new IllegalArgumentException("Unhandled entity type: " + entityType);
+        case PROCESS:
+            return new ProcessEntityParser();
+        case FEED:
+            return new FeedEntityParser();
+        case CLUSTER:
+            return new ClusterEntityParser();
+        default:
+            throw new IllegalArgumentException("Unhandled entity type: " + entityType);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
index 475384e..1c323fd 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
@@ -43,6 +43,9 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.TimeZone;
 
+/**
+ * Parser that parses feed entity definition.
+ */
 public class FeedEntityParser extends EntityParser<Feed> {
 
     private static final Logger LOG = Logger.getLogger(FeedEntityParser.class);
@@ -112,20 +115,16 @@ public class FeedEntityParser extends EntityParser<Feed> {
                             cluster.getName()).getPath()).equals(
                     FeedGroup.getDatePattern(defaultPath))) {
                 throw new ValidationException("Feeds default path pattern: "
-                        + FeedHelper.getLocation(feed, LocationType.DATA)
-                        .getPath()
+                        + FeedHelper.getLocation(feed, LocationType.DATA).getPath()
                         + ", does not match with cluster: "
                         + cluster.getName()
                         + " path pattern: "
-                        + FeedHelper.getLocation(feed, LocationType.DATA,
-                        cluster.getName()).getPath());
+                        + FeedHelper.getLocation(feed, LocationType.DATA, cluster.getName()).getPath());
             }
         }
         for (String groupName : groupNames) {
             FeedGroup group = FeedGroupMap.get().getGroupsMapping().get(groupName);
-            if (group == null || group.canContainFeed(feed)) {
-                continue;
-            } else {
+            if (group != null && !group.canContainFeed(feed)) {
                 throw new ValidationException(
                         "Feed " + feed.getName() + "'s frequency: " + feed.getFrequency().toString()
                                 + ", path pattern: " + FeedHelper.getLocation(feed, LocationType.DATA).getPath()
@@ -242,8 +241,8 @@ public class FeedEntityParser extends EntityParser<Feed> {
                 String part = FeedHelper.normalizePartitionExpression(cluster.getPartition());
                 if (StringUtils.split(part, '/').length == 0) {
                     throw new ValidationException(
-                            "Partition expression has to be specified for cluster " + cluster.getName() +
-                                    " as there are more than one source clusters");
+                            "Partition expression has to be specified for cluster " + cluster.getName()
+                                    + " as there are more than one source clusters");
                 }
                 validateClusterExpDefined(cluster);
 
@@ -256,8 +255,8 @@ public class FeedEntityParser extends EntityParser<Feed> {
                         int numParts = StringUtils.split(part, '/').length;
                         if (numParts > feedParts) {
                             throw new ValidationException(
-                                    "Partition for " + src.getName() + " and " + cluster.getName() +
-                                            "clusters is more than the number of partitions defined in feed");
+                                    "Partition for " + src.getName() + " and " + cluster.getName()
+                                            + "clusters is more than the number of partitions defined in feed");
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java b/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
index e01a378..98f1cb9 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
@@ -21,7 +21,7 @@ package org.apache.falcon.entity.parser;
 import org.apache.falcon.FalconException;
 
 /**
- * ValidationException during parsing
+ * ValidationException during parsing.
  */
 public class ValidationException extends FalconException {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
index 8fd3775..4897c25 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
@@ -37,27 +37,35 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
-import java.util.*;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class ConfigurationStore implements FalconService {
+/**
+ * Persistent store for falcon entities.
+ */
+public final class ConfigurationStore implements FalconService {
 
     private static final Logger LOG = Logger.getLogger(ConfigurationStore.class);
     private static final Logger AUDIT = Logger.getLogger("AUDIT");
     private static final String UTF_8 = "UTF-8";
 
-    private static final ConfigurationStore store = new ConfigurationStore();
+    private static final ConfigurationStore STORE = new ConfigurationStore();
 
     private Set<ConfigurationChangeListener> listeners = new LinkedHashSet<ConfigurationChangeListener>();
 
     private ThreadLocal<Entity> updatesInProgress = new ThreadLocal<Entity>();
 
     public static ConfigurationStore get() {
-        return store;
+        return STORE;
     }
 
     private final Map<EntityType, ConcurrentHashMap<String, Entity>> dictionary
-            = new HashMap<EntityType, ConcurrentHashMap<String, Entity>>();
+        = new HashMap<EntityType, ConcurrentHashMap<String, Entity>>();
 
     private final FileSystem fs;
     private final Path storePath;
@@ -69,15 +77,9 @@ public class ConfigurationStore implements FalconService {
         }
     };
 
-    @SuppressWarnings("unchecked")
     private ConfigurationStore() {
-        Class<? extends Entity>[] entityClasses = new Class[EntityType.values().length];
-
-        int index = 0;
-
         for (EntityType type : EntityType.values()) {
             dictionary.put(type, new ConcurrentHashMap<String, Entity>());
-            entityClasses[index++] = type.getEntityClass();
         }
 
         String uri = StartupProperties.get().getProperty("config.store.uri");
@@ -221,7 +223,7 @@ public class ConfigurationStore implements FalconService {
             T entity = (T) entityMap.get(name);
             if (entity == NULL) { // Object equality being checked
                 try {
-                    entity = this.<T>restore(type, name);
+                    entity = this.restore(type, name);
                 } catch (IOException e) {
                     throw new StoreAccessException(e);
                 }
@@ -240,7 +242,7 @@ public class ConfigurationStore implements FalconService {
     }
 
     /**
-     * Remove an entity which is already stored in the config store
+     * Remove an entity which is already stored in the config store.
      *
      * @param type - Entity type being removed
      * @param name - Name of the entity object being removed
@@ -283,7 +285,7 @@ public class ConfigurationStore implements FalconService {
      * @return - Array of entity types
      */
     public Entity[] search(EntityType type, String... keywords) {
-        return null;// TODO
+        return null;
     }
 
     /**
@@ -310,7 +312,7 @@ public class ConfigurationStore implements FalconService {
     }
 
     /**
-     * Archive removed configuration in the persistent store
+     * Archive removed configuration in the persistent store.
      *
      * @param type - Entity type to archive
      * @param name - name
@@ -335,7 +337,7 @@ public class ConfigurationStore implements FalconService {
      */
     @SuppressWarnings("unchecked")
     private synchronized <T extends Entity> T restore(EntityType type, String name)
-            throws IOException, FalconException {
+        throws IOException, FalconException {
 
         InputStream in = fs.open(new Path(storePath, type + Path.SEPARATOR + URLEncoder.encode(name, UTF_8) + ".xml"));
         try {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/store/EntityAlreadyExistsException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/store/EntityAlreadyExistsException.java b/common/src/main/java/org/apache/falcon/entity/store/EntityAlreadyExistsException.java
index ce550d4..28c5ac0 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/EntityAlreadyExistsException.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/EntityAlreadyExistsException.java
@@ -20,6 +20,9 @@ package org.apache.falcon.entity.store;
 
 import org.apache.falcon.FalconException;
 
+/**
+ * Exception to thrown when entity being sought for addition is already present in config store.
+ */
 public class EntityAlreadyExistsException extends FalconException {
 
     public EntityAlreadyExistsException(Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java b/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
index a231242..318dc2e 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
@@ -20,6 +20,9 @@ package org.apache.falcon.entity.store;
 
 import org.apache.falcon.FalconException;
 
+/**
+ * Exception when there in issue accessing the persistent store.
+ */
 public class StoreAccessException extends FalconException {
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
index f2b66e5..803fa9e 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
@@ -34,9 +34,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
-public class EntityGraph implements ConfigurationChangeListener {
+/**
+ * An in-memory graph of entities and relationship among themselves.
+ */
+public final class EntityGraph implements ConfigurationChangeListener {
 
-    private static Logger LOG = Logger.getLogger(EntityGraph.class);
+    private static final Logger LOG = Logger.getLogger(EntityGraph.class);
 
     private static EntityGraph instance = new EntityGraph();
 
@@ -69,12 +72,13 @@ public class EntityGraph implements ConfigurationChangeListener {
     public void onAdd(Entity entity) throws FalconException {
         Map<Node, Set<Node>> nodeEdges = null;
         switch (entity.getEntityType()) {
-            case PROCESS:
-                nodeEdges = getEdgesFor((Process) entity);
-                break;
-            case FEED:
-                nodeEdges = getEdgesFor((Feed) entity);
-                break;
+        case PROCESS:
+            nodeEdges = getEdgesFor((Process) entity);
+            break;
+        case FEED:
+            nodeEdges = getEdgesFor((Feed) entity);
+            break;
+        default:
         }
         if (nodeEdges == null) {
             return;
@@ -95,12 +99,13 @@ public class EntityGraph implements ConfigurationChangeListener {
     public void onRemove(Entity entity) throws FalconException {
         Map<Node, Set<Node>> nodeEdges = null;
         switch (entity.getEntityType()) {
-            case PROCESS:
-                nodeEdges = getEdgesFor((Process) entity);
-                break;
-            case FEED:
-                nodeEdges = getEdgesFor((Feed) entity);
-                break;
+        case PROCESS:
+            nodeEdges = getEdgesFor((Process) entity);
+            break;
+        case FEED:
+            nodeEdges = getEdgesFor((Feed) entity);
+            break;
+        default:
         }
         if (nodeEdges == null) {
             return;
@@ -178,7 +183,10 @@ public class EntityGraph implements ConfigurationChangeListener {
         return nodeEdges;
     }
 
-    private static class Node {
+    /**
+     * Node element in the graph.
+     */
+    private static final class Node {
 
         private final EntityType type;
         private final String name;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java b/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
index b523c8b..bd32852 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
@@ -26,7 +26,12 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
-public class EntityIntegrityChecker {
+/**
+ * Helper methods to check integrity of entity.
+ */
+public final class EntityIntegrityChecker {
+
+    private EntityIntegrityChecker() {}
 
     public static Pair<String, EntityType>[] referencedBy(Entity entity) throws FalconException {
         Set<Entity> deps = EntityGraph.get().getDependents(entity);
@@ -35,14 +40,14 @@ public class EntityIntegrityChecker {
         }
 
         switch (entity.getEntityType()) {
-            case CLUSTER:
-                return filter(deps, EntityType.FEED, EntityType.PROCESS);
+        case CLUSTER:
+            return filter(deps, EntityType.FEED, EntityType.PROCESS);
 
-            case FEED:
-                return filter(deps, EntityType.PROCESS);
+        case FEED:
+            return filter(deps, EntityType.PROCESS);
 
-            default:
-                return null;
+        default:
+            return null;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java b/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
index f57ef95..5d70933 100644
--- a/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
+++ b/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
@@ -34,19 +34,22 @@ import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+/**
+ * Helper for evaluating expressions.
+ */
 public final class ExpressionHelper implements FunctionMapper, VariableResolver {
 
-    private static final ExpressionHelper instance = new ExpressionHelper();
+    private static final ExpressionHelper INSTANCE = new ExpressionHelper();
 
     private ThreadLocal<Properties> threadVariables = new ThreadLocal<Properties>();
 
-    private static final Pattern sysPropertyPattern = Pattern.compile("\\$\\{[A-Za-z0-9_.]+\\}");
+    private static final Pattern SYS_PROPERTY_PATTERN = Pattern.compile("\\$\\{[A-Za-z0-9_.]+\\}");
 
     private static final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
-    private static final ExpressionHelper resolver = ExpressionHelper.get();
+    private static final ExpressionHelper RESOLVER = ExpressionHelper.get();
 
     public static ExpressionHelper get() {
-        return instance;
+        return INSTANCE;
     }
 
     private ExpressionHelper() {
@@ -59,7 +62,7 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
     @SuppressWarnings("unchecked")
     public <T> T evaluateFullExpression(String expression, Class<T> clazz) throws FalconException {
         try {
-            return (T) EVALUATOR.evaluate(expression, clazz, resolver, resolver);
+            return (T) EVALUATOR.evaluate(expression, clazz, RESOLVER, RESOLVER);
         } catch (ELException e) {
             throw new FalconException("Unable to evaluate " + expression, e);
         }
@@ -94,21 +97,21 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
         Calendar dsInstanceCal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
         dsInstanceCal.setTime(date);
         switch (boundary) {
-            case Calendar.YEAR:
-                dsInstanceCal.set(Calendar.MONTH, 0);
-            case Calendar.MONTH:
-                dsInstanceCal.set(Calendar.DAY_OF_MONTH, 1);
-            case Calendar.DAY_OF_MONTH:
-                dsInstanceCal.set(Calendar.HOUR_OF_DAY, 0);
-            case Calendar.HOUR:
-                dsInstanceCal.set(Calendar.MINUTE, 0);
-                dsInstanceCal.set(Calendar.SECOND, 0);
-                dsInstanceCal.set(Calendar.MILLISECOND, 0);
-                break;
-            case Calendar.SECOND:
-                break;
-            default:
-                throw new IllegalArgumentException("Invalid boundary " + boundary);
+        case Calendar.YEAR:
+            dsInstanceCal.set(Calendar.MONTH, 0);
+        case Calendar.MONTH:
+            dsInstanceCal.set(Calendar.DAY_OF_MONTH, 1);
+        case Calendar.DAY_OF_MONTH:
+            dsInstanceCal.set(Calendar.HOUR_OF_DAY, 0);
+        case Calendar.HOUR:
+            dsInstanceCal.set(Calendar.MINUTE, 0);
+            dsInstanceCal.set(Calendar.SECOND, 0);
+            dsInstanceCal.set(Calendar.MILLISECOND, 0);
+            break;
+        case Calendar.SECOND:
+            break;
+        default:
+            throw new IllegalArgumentException("Invalid boundary " + boundary);
         }
 
         dsInstanceCal.add(Calendar.YEAR, 0);
@@ -182,7 +185,7 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
     }
 
     public static String substitute(String originalValue, Properties properties) {
-        Matcher envVarMatcher = sysPropertyPattern.matcher(originalValue);
+        Matcher envVarMatcher = SYS_PROPERTY_PATTERN.matcher(originalValue);
         while (envVarMatcher.find()) {
             String envVar = originalValue.substring(envVarMatcher.start() + 2,
                     envVarMatcher.end() - 1);
@@ -191,7 +194,7 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
             envVar = "\\$\\{" + envVar + "\\}";
             if (envVal != null) {
                 originalValue = originalValue.replaceAll(envVar, Matcher.quoteReplacement(envVal));
-                envVarMatcher = sysPropertyPattern.matcher(originalValue);
+                envVarMatcher = SYS_PROPERTY_PATTERN.matcher(originalValue);
             }
         }
         return originalValue;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java b/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
index 532392f..ed44b48 100644
--- a/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
+++ b/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
@@ -34,11 +34,11 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 /**
- * Has 2 way mappings from feed to group and group to feed
+ * Has 2 way mappings from feed to group and group to feed.
  */
-public class FeedGroupMap implements ConfigurationChangeListener {
+public final class FeedGroupMap implements ConfigurationChangeListener {
 
-    private static final FeedGroupMap instance = new FeedGroupMap();
+    private static final FeedGroupMap INSTANCE = new FeedGroupMap();
     private Map<String, FeedGroup> groupsMapping = new ConcurrentHashMap<String, FeedGroup>();
 
     private FeedGroupMap() {
@@ -46,7 +46,7 @@ public class FeedGroupMap implements ConfigurationChangeListener {
     }
 
     public static FeedGroupMap get() {
-        return instance;
+        return INSTANCE;
     }
 
     public Map<String, FeedGroup> getGroupsMapping() {
@@ -88,7 +88,8 @@ public class FeedGroupMap implements ConfigurationChangeListener {
 
     @Override
     public void onChange(Entity oldEntity, Entity newEntity)
-            throws FalconException {
+        throws FalconException {
+
         onRemove(oldEntity);
         onAdd(newEntity);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/security/CurrentUser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/CurrentUser.java b/common/src/main/java/org/apache/falcon/security/CurrentUser.java
index 9a3086c..4d2299e 100644
--- a/common/src/main/java/org/apache/falcon/security/CurrentUser.java
+++ b/common/src/main/java/org/apache/falcon/security/CurrentUser.java
@@ -22,14 +22,19 @@ import org.apache.log4j.Logger;
 
 import javax.security.auth.Subject;
 
+/**
+ * Current authenticated user via REST.
+ */
 public final class CurrentUser {
 
-    private static Logger LOG = Logger.getLogger(CurrentUser.class);
+    private static final Logger LOG = Logger.getLogger(CurrentUser.class);
+
+    private static final CurrentUser INSTANCE = new CurrentUser();
 
-    private static final CurrentUser instance = new CurrentUser();
+    private CurrentUser() {}
 
     public static CurrentUser get() {
-        return instance;
+        return INSTANCE;
     }
 
     private final ThreadLocal<Subject> currentSubject =
@@ -37,8 +42,7 @@ public final class CurrentUser {
 
     public static void authenticate(String user) {
         if (user == null || user.isEmpty()) {
-            throw new IllegalStateException
-                    ("Bad user name sent for authentication");
+            throw new IllegalStateException("Bad user name sent for authentication");
         }
         if (user.equals(getUserInternal())) {
             return;
@@ -47,11 +51,11 @@ public final class CurrentUser {
         Subject subject = new Subject();
         subject.getPrincipals().add(new FalconPrincipal(user));
         LOG.info("Logging in " + user);
-        instance.currentSubject.set(subject);
+        INSTANCE.currentSubject.set(subject);
     }
 
     public static Subject getSubject() {
-        return instance.currentSubject.get();
+        return INSTANCE.currentSubject.get();
     }
 
     public static String getUser() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
index a27a342..4f4542c 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
@@ -29,8 +29,11 @@ import javax.security.auth.spi.LoginModule;
 import java.security.Principal;
 import java.util.Map;
 
+/**
+ * Falcon JAAS login module.
+ */
 public class FalconLoginModule implements LoginModule {
-    private static Logger LOG = Logger.getLogger(FalconLoginModule.class);
+    private static final Logger LOG = Logger.getLogger(FalconLoginModule.class);
 
     private Subject subject;
 
@@ -62,11 +65,13 @@ public class FalconLoginModule implements LoginModule {
         throw new LoginException("No such user " + subject);
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     @Override
     public void initialize(Subject subject, CallbackHandler callbackHandler,
                            Map<String, ?> sharedState, Map<String, ?> options) {
         this.subject = subject;
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Override
     public boolean login() throws LoginException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/security/FalconPrincipal.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconPrincipal.java b/common/src/main/java/org/apache/falcon/security/FalconPrincipal.java
index bd27e23..ab93e1a 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconPrincipal.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconPrincipal.java
@@ -20,6 +20,9 @@ package org.apache.falcon.security;
 
 import java.security.Principal;
 
+/**
+ * Falcon JAAS principal object.
+ */
 public class FalconPrincipal implements Principal {
 
     private final String user;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java b/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
index 20ec8df..7689672 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
@@ -22,6 +22,9 @@ import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.login.Configuration;
 import java.util.HashMap;
 
+/**
+ * Falcon JAAS security configuration.
+ */
 public class FalconSecurityConfiguration extends Configuration {
 
     private static final AppConfigurationEntry OS_SPECIFIC_LOGIN =

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/security/SecurityConstants.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/SecurityConstants.java b/common/src/main/java/org/apache/falcon/security/SecurityConstants.java
index b4f060f..8f7ba4a 100644
--- a/common/src/main/java/org/apache/falcon/security/SecurityConstants.java
+++ b/common/src/main/java/org/apache/falcon/security/SecurityConstants.java
@@ -22,7 +22,12 @@ import com.sun.security.auth.UnixPrincipal;
 
 import java.security.Principal;
 
-public class SecurityConstants {
+/**
+ * Constants for the security module.
+ */
+public final class SecurityConstants {
+
+    private SecurityConstants() {}
 
     public static final String OS_LOGIN_MODULE_NAME =
             "com.sun.security.auth.module.UnixLoginModule";

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/service/ConfigurationChangeListener.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/ConfigurationChangeListener.java b/common/src/main/java/org/apache/falcon/service/ConfigurationChangeListener.java
index 65621cb..56953ad 100644
--- a/common/src/main/java/org/apache/falcon/service/ConfigurationChangeListener.java
+++ b/common/src/main/java/org/apache/falcon/service/ConfigurationChangeListener.java
@@ -21,6 +21,9 @@ package org.apache.falcon.service;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 
+/**
+ * Configuration change notification listener.
+ */
 public interface ConfigurationChangeListener {
 
     void onAdd(Entity entity) throws FalconException;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/service/FalconService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/FalconService.java b/common/src/main/java/org/apache/falcon/service/FalconService.java
index 049032b..a1eb8e0 100644
--- a/common/src/main/java/org/apache/falcon/service/FalconService.java
+++ b/common/src/main/java/org/apache/falcon/service/FalconService.java
@@ -20,6 +20,9 @@ package org.apache.falcon.service;
 
 import org.apache.falcon.FalconException;
 
+/**
+ * Falcon service initialized at startup.
+ */
 public interface FalconService {
 
     String getName();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/LogCleanupService.java b/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
index 17d5926..778be11 100644
--- a/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
+++ b/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
@@ -33,10 +33,13 @@ import java.util.Date;
 import java.util.Timer;
 import java.util.TimerTask;
 
+/**
+ * Log cleanup service.
+ */
 public class LogCleanupService implements FalconService {
 
     private static final Logger LOG = Logger.getLogger(LogCleanupService.class);
-    private final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
+    private final ExpressionEvaluator evaluator = new ExpressionEvaluatorImpl();
     private final ExpressionHelper resolver = ExpressionHelper.get();
 
     @Override
@@ -80,7 +83,7 @@ public class LogCleanupService implements FalconService {
         String delay = StartupProperties.get().getProperty(
                 "falcon.cleanup.service.frequency", "days(1)");
         try {
-            return (Long) EVALUATOR.evaluate("${" + delay + "}", Long.class,
+            return (Long) evaluator.evaluate("${" + delay + "}", Long.class,
                     resolver, resolver);
         } catch (ELException e) {
             throw new FalconException("Exception in EL evaluation", e);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java b/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
index 466cb81..5e0256e 100644
--- a/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
+++ b/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
@@ -23,9 +23,12 @@ import org.apache.falcon.util.ReflectionUtils;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.log4j.Logger;
 
+/**
+ * Initializer that Falcon uses at startup to bring up all the falcon startup services.
+ */
 public class ServiceInitializer {
 
-    private static Logger LOG = Logger.getLogger(ServiceInitializer.class);
+    private static final Logger LOG = Logger.getLogger(ServiceInitializer.class);
     private final Services services = Services.get();
 
     public void initialize() throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/service/Services.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/Services.java b/common/src/main/java/org/apache/falcon/service/Services.java
index 955e906..6659ccd 100644
--- a/common/src/main/java/org/apache/falcon/service/Services.java
+++ b/common/src/main/java/org/apache/falcon/service/Services.java
@@ -20,34 +20,34 @@ package org.apache.falcon.service;
 
 import org.apache.falcon.FalconException;
 import org.apache.falcon.util.ReflectionUtils;
-import org.apache.log4j.Logger;
 
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
-
+/**
+ * Repository of services initialized at startup.
+ */
 public final class Services implements Iterable<FalconService> {
-    private static final Logger LOG = Logger.getLogger(Services.class);
 
-    private static Services instance = new Services();
+    private static final Services INSTANCE = new Services();
 
     private Services() {
     }
 
     public static Services get() {
-        return instance;
+        return INSTANCE;
     }
 
     private final Map<String, FalconService> services =
             new LinkedHashMap<String, FalconService>();
 
     public synchronized void register(FalconService service)
-            throws FalconException {
+        throws FalconException {
+
         if (services.containsKey(service.getName())) {
-            throw new FalconException("Service " + service.getName() +
-                    " already registered");
+            throw new FalconException("Service " + service.getName() + " already registered");
         } else {
             services.put(service.getName(), service);
         }
@@ -58,8 +58,7 @@ public final class Services implements Iterable<FalconService> {
         if (services.containsKey(serviceName)) {
             return (T) services.get(serviceName);
         } else {
-            throw new NoSuchElementException("Service " + serviceName +
-                    " not registered with registry");
+            throw new NoSuchElementException("Service " + serviceName + " not registered with registry");
         }
     }
 
@@ -74,8 +73,7 @@ public final class Services implements Iterable<FalconService> {
 
     public FalconService init(String serviceName) throws FalconException {
         if (isRegistered(serviceName)) {
-            throw new FalconException("Service is already initialized " +
-                    serviceName);
+            throw new FalconException("Service is already initialized " + serviceName);
         }
         FalconService service = ReflectionUtils.getInstance(serviceName + ".impl");
         register(service);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/update/UpdateHelper.java b/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
index 4e199da..a9d39de 100644
--- a/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
+++ b/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
@@ -36,37 +36,39 @@ import org.apache.log4j.Logger;
 import java.util.ArrayList;
 import java.util.List;
 
+/**
+ * Helper methods to facilitate entity updates.
+ */
 public final class UpdateHelper {
     private static final Logger LOG = Logger.getLogger(UpdateHelper.class);
+
     private static final String[] FEED_FIELDS = new String[]{"partitions", "groups", "lateArrival.cutOff",
                                                              "schema.location", "schema.provider",
-                                                             "ACL.group", "ACL.owner", "ACL.permission"};
+                                                             "ACL.group", "ACL.owner", "ACL.permission", };
     private static final String[] PROCESS_FIELDS = new String[]{"retry.policy", "retry.delay", "retry.attempts",
                                                                 "lateProcess.policy", "lateProcess.delay",
                                                                 "lateProcess.lateInputs[\\d+].input",
-                                                                "lateProcess.lateInputs[\\d+].workflowPath"};
+                                                                "lateProcess.lateInputs[\\d+].workflowPath", };
+
+    private UpdateHelper() {}
 
     public static boolean shouldUpdate(Entity oldEntity, Entity newEntity, String cluster) throws FalconException {
         Entity oldView = EntityUtil.getClusterView(oldEntity, cluster);
         Entity newView = EntityUtil.getClusterView(newEntity, cluster);
         switch (oldEntity.getEntityType()) {
-            case FEED:
-                if (EntityUtil.equals(oldView, newView, FEED_FIELDS)) {
-                    return false;
-                }
-                return true;
+        case FEED:
+            return !EntityUtil.equals(oldView, newView, FEED_FIELDS);
 
-            case PROCESS:
-                if (EntityUtil.equals(oldView, newView, PROCESS_FIELDS)) {
-                    return false;
-                }
-                return true;
+        case PROCESS:
+            return !EntityUtil.equals(oldView, newView, PROCESS_FIELDS);
+        default:
         }
         throw new IllegalArgumentException("Unhandled entity type " + oldEntity.getEntityType());
     }
 
     public static boolean shouldUpdate(Entity oldEntity, Entity newEntity, Entity affectedEntity)
-            throws FalconException {
+        throws FalconException {
+
         if (oldEntity.getEntityType() == EntityType.FEED && affectedEntity.getEntityType() == EntityType.PROCESS) {
             return shouldUpdate((Feed) oldEntity, (Feed) newEntity, (Process) affectedEntity);
         } else {
@@ -77,26 +79,14 @@ public final class UpdateHelper {
     }
 
     public static boolean shouldUpdate(Feed oldFeed, Feed newFeed, Process affectedProcess) {
-        if (!FeedHelper
-                .getLocation(oldFeed.getLocations(), LocationType.DATA)
-                .getPath()
-                .equals(FeedHelper.getLocation(newFeed.getLocations(),
-                        LocationType.DATA).getPath())
-                || !FeedHelper
-                .getLocation(oldFeed.getLocations(), LocationType.META)
-                .getPath()
-                .equals(FeedHelper.getLocation(newFeed.getLocations(),
-                        LocationType.META).getPath())
-                || !FeedHelper
-                .getLocation(oldFeed.getLocations(), LocationType.STATS)
-                .getPath()
-                .equals(FeedHelper.getLocation(newFeed.getLocations(),
-                        LocationType.STATS).getPath())
-                || !FeedHelper
-                .getLocation(oldFeed.getLocations(), LocationType.TMP)
-                .getPath()
-                .equals(FeedHelper.getLocation(newFeed.getLocations(),
-                        LocationType.TMP).getPath())) {
+        if (!FeedHelper.getLocation(oldFeed.getLocations(), LocationType.DATA)
+            .getPath().equals(FeedHelper.getLocation(newFeed.getLocations(), LocationType.DATA).getPath())
+                || !FeedHelper.getLocation(oldFeed.getLocations(), LocationType.META)
+                    .getPath().equals(FeedHelper.getLocation(newFeed.getLocations(), LocationType.META).getPath())
+                || !FeedHelper.getLocation(oldFeed.getLocations(), LocationType.STATS)
+                    .getPath().equals(FeedHelper.getLocation(newFeed.getLocations(), LocationType.STATS).getPath())
+                || !FeedHelper.getLocation(oldFeed.getLocations(), LocationType.TMP)
+                    .getPath().equals(FeedHelper.getLocation(newFeed.getLocations(), LocationType.TMP).getPath())) {
             return true;
         }
         LOG.debug(oldFeed.toShortString() + ": Location identical. Ignoring...");
@@ -139,27 +129,16 @@ public final class UpdateHelper {
 
         for (Cluster cluster : affectedProcess.getClusters().getClusters()) {
             if (!FeedHelper
-                    .getCluster(oldFeed, cluster.getName())
-                    .getValidity()
-                    .getStart()
-                    .equals(FeedHelper.getCluster(newFeed, cluster.getName())
-                            .getValidity().getStart())
-                    || !FeedHelper.getLocation(oldFeed, LocationType.DATA,
-                    cluster.getName()).getPath().equals(
-                    FeedHelper.getLocation(newFeed, LocationType.DATA,
-                            cluster.getName()).getPath())
-                    || !FeedHelper.getLocation(oldFeed, LocationType.META,
-                    cluster.getName()).getPath().equals(
-                    FeedHelper.getLocation(newFeed, LocationType.META,
-                            cluster.getName()).getPath())
-                    || !FeedHelper.getLocation(oldFeed, LocationType.STATS,
-                    cluster.getName()).getPath().equals(
-                    FeedHelper.getLocation(newFeed, LocationType.STATS,
-                            cluster.getName()).getPath())
-                    || !FeedHelper.getLocation(oldFeed, LocationType.TMP,
-                    cluster.getName()).getPath().equals(
-                    FeedHelper.getLocation(newFeed, LocationType.TMP,
-                            cluster.getName()).getPath())) {
+                    .getCluster(oldFeed, cluster.getName()).getValidity().getStart()
+                    .equals(FeedHelper.getCluster(newFeed, cluster.getName()).getValidity().getStart())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.DATA, cluster.getName()).getPath()
+                    .equals(FeedHelper.getLocation(newFeed, LocationType.DATA, cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.META, cluster.getName()).getPath()
+                    .equals(FeedHelper.getLocation(newFeed, LocationType.META, cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.STATS, cluster.getName()).getPath()
+                    .equals(FeedHelper.getLocation(newFeed, LocationType.STATS, cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.TMP, cluster.getName()).getPath()
+                    .equals(FeedHelper.getLocation(newFeed, LocationType.TMP, cluster.getName()).getPath())) {
                 return true;
             }
             LOG.debug(oldFeed.toShortString() + ": Feed on cluster" + cluster.getName() + " identical. Ignoring...");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java b/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
index f5dbc83..5d670ef 100644
--- a/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
@@ -30,9 +30,12 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
+/**
+ * Base class for reading application properties.
+ */
 public abstract class ApplicationProperties extends Properties {
 
-    private static Logger LOG = Logger.getLogger(ApplicationProperties.class);
+    private static final Logger LOG = Logger.getLogger(ApplicationProperties.class);
 
     protected enum LocationType {
         FILE, HOME, CLASSPATH
@@ -81,12 +84,10 @@ public abstract class ApplicationProperties extends Properties {
         try {
             if (location == LocationType.CLASSPATH) {
                 if (getClass().getResource(propertyFile) != null) {
-                    LOG.info("Property file being loaded from " +
-                            getClass().getResource(propertyFile));
+                    LOG.info("Property file being loaded from " + getClass().getResource(propertyFile));
                     resource = getClass().getResourceAsStream(propertyFile);
                 } else {
-                    LOG.info("Property file being loaded from " +
-                            getClass().getResource("/" + propertyFile));
+                    LOG.info("Property file being loaded from " + getClass().getResource("/" + propertyFile));
                     resource = getClass().getResourceAsStream("/" + propertyFile);
                 }
             } else {


[22/47] git commit: checkstyle fixes for client module

Posted by sr...@apache.org.
checkstyle fixes for client module


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

Branch: refs/heads/master
Commit: e69b220564e9d63c52dbaccfdc1fd0765a639a8b
Parents: a4d79f0
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Tue Apr 16 22:34:38 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Tue Apr 16 22:34:38 2013 +0530

----------------------------------------------------------------------
 client/src/main/java/org/apache/falcon/Pair.java   |    6 +-
 client/src/main/java/org/apache/falcon/Tag.java    |    3 +
 .../main/java/org/apache/falcon/cli/FalconCLI.java |   66 ++++----
 .../org/apache/falcon/client/FalconClient.java     |  138 ++++++++-------
 .../org/apache/falcon/entity/v0/DateValidator.java |   18 +--
 .../java/org/apache/falcon/entity/v0/Entity.java   |    8 +-
 .../org/apache/falcon/entity/v0/EntityType.java    |    2 +-
 .../org/apache/falcon/entity/v0/Frequency.java     |   21 ++-
 .../org/apache/falcon/entity/v0/SchemaHelper.java  |    8 +-
 .../java/org/apache/falcon/resource/APIResult.java |   11 +-
 .../org/apache/falcon/resource/EntityList.java     |    6 +
 .../apache/falcon/resource/InstancesResult.java    |   13 ++
 .../java/org/apache/falcon/cli/TestCLIParser.java  |    3 +
 .../apache/falcon/entity/v0/DateValidatorTest.java |   68 ++++----
 .../org/apache/falcon/entity/v0/TestFrequency.java |    3 +
 15 files changed, 213 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/Pair.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/Pair.java b/client/src/main/java/org/apache/falcon/Pair.java
index 300be27..4c60feb 100644
--- a/client/src/main/java/org/apache/falcon/Pair.java
+++ b/client/src/main/java/org/apache/falcon/Pair.java
@@ -18,7 +18,11 @@
 
 package org.apache.falcon;
 
-
+/**
+ * Simple pair class to hold a pair of object of specific class.
+ * @param <A> - First element in pair.
+ * @param <B> - Second element in pair
+ */
 public class Pair<A, B> {
     public final A first;
     public final B second;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/Tag.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/Tag.java b/client/src/main/java/org/apache/falcon/Tag.java
index fc56632..beeb812 100644
--- a/client/src/main/java/org/apache/falcon/Tag.java
+++ b/client/src/main/java/org/apache/falcon/Tag.java
@@ -20,6 +20,9 @@ package org.apache.falcon;
 
 import org.apache.falcon.entity.v0.EntityType;
 
+/**
+ * Tag to include in the entity type.
+ */
 public enum Tag {
     DEFAULT(EntityType.PROCESS), RETENTION(EntityType.FEED), REPLICATION(EntityType.FEED);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index 575d7a5..7c60328 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -95,11 +95,11 @@ public class FalconCLI {
     }
 
     // TODO help and headers
-    private static final String[] FALCON_HELP = {
-            "the env variable '" + FALCON_URL
-                    + "' is used as default value for the '-" + URL_OPTION + "' option",
-            "custom headers for Falcon web services can be specified using '-D"
-                    + FalconClient.WS_HEADER_PREFIX + "NAME=VALUE'"};
+    private static final String[] FALCON_HELP = { "the env variable '" + FALCON_URL
+                                                          + "' is used as default value for the '-" + URL_OPTION
+                                                          + "' option",
+                                                  "custom headers for Falcon web services can be specified using '-D"
+                                                          + FalconClient.WS_HEADER_PREFIX + "NAME=VALUE'", };
 
     /**
      * Run a CLI programmatically.
@@ -182,8 +182,7 @@ public class FalconCLI {
 
         colo = getColo(colo);
 
-        validateInstanceCommands(optionsList, entity, type, start, end,
-                filePath, colo, clusters, sourceClusters);
+        validateInstanceCommands(optionsList, entity, type, start, colo);
 
         if (optionsList.contains(RUNNING_OPT)) {
             result = client.getRunningInstances(type, entity, colo);
@@ -209,9 +208,9 @@ public class FalconCLI {
         OUT.get().println(result);
     }
 
-    private void validateInstanceCommands(Set<String> optionsList, String entity, String type,
-                                          String start, String end, String filePath, String colo,
-                                          String clusters, String sourceClusters) throws FalconCLIException {
+    private void validateInstanceCommands(Set<String> optionsList,
+                                          String entity, String type,
+                                          String start, String colo) throws FalconCLIException {
 
         if (entity == null || entity.equals("")) {
             throw new FalconCLIException("Missing argument: name");
@@ -249,7 +248,8 @@ public class FalconCLI {
     }
 
     private void entityCommand(CommandLine commandLine)
-            throws FalconCLIException, IOException {
+        throws FalconCLIException, IOException {
+
         String falconUrl = getFalconEndpoint(commandLine);
         FalconClient client = new FalconClient(falconUrl);
 
@@ -265,52 +265,52 @@ public class FalconCLI {
         String colo = commandLine.getOptionValue(COLO_OPT);
 
 
-        validateEntityType(optionsList, entityType);
+        validateEntityType(entityType);
 
         if (optionsList.contains(SUBMIT_OPT)) {
-            validateFilePath(optionsList, filePath);
+            validateFilePath(filePath);
             validateColo(optionsList);
             result = client.submit(entityType, filePath);
         } else if (optionsList.contains(UPDATE_OPT)) {
-            validateFilePath(optionsList, filePath);
+            validateFilePath(filePath);
             validateColo(optionsList);
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             result = client.update(entityType, entityName, filePath);
         } else if (optionsList.contains(SUBMIT_AND_SCHEDULE_OPT)) {
-            validateFilePath(optionsList, filePath);
+            validateFilePath(filePath);
             validateColo(optionsList);
             result = client.submitAndSchedule(entityType, filePath);
         } else if (optionsList.contains(VALIDATE_OPT)) {
-            validateFilePath(optionsList, filePath);
+            validateFilePath(filePath);
             validateColo(optionsList);
             result = client.validate(entityType, filePath);
         } else if (optionsList.contains(SCHEDULE_OPT)) {
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             colo = getColo(colo);
             result = client.schedule(entityType, entityName, colo);
         } else if (optionsList.contains(SUSPEND_OPT)) {
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             colo = getColo(colo);
             result = client.suspend(entityType, entityName, colo);
         } else if (optionsList.contains(RESUME_OPT)) {
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             colo = getColo(colo);
             result = client.resume(entityType, entityName, colo);
         } else if (optionsList.contains(DELETE_OPT)) {
             validateColo(optionsList);
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             result = client.delete(entityType, entityName);
         } else if (optionsList.contains(STATUS_OPT)) {
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             colo = getColo(colo);
             result = client.getStatus(entityType, entityName, colo);
         } else if (optionsList.contains(DEFINITION_OPT)) {
             validateColo(optionsList);
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             result = client.getDefinition(entityType, entityName);
         } else if (optionsList.contains(DEPENDENCY_OPT)) {
             validateColo(optionsList);
-            validateEntityName(optionsList, entityName);
+            validateEntityName(entityName);
             result = client.getDependency(entityType, entityName);
         } else if (optionsList.contains(LIST_OPT)) {
             validateColo(optionsList);
@@ -331,29 +331,33 @@ public class FalconCLI {
         return colo;
     }
 
-    private void validateFilePath(Set<String> optionsList, String filePath)
-            throws FalconCLIException {
+    private void validateFilePath(String filePath)
+        throws FalconCLIException {
+
         if (filePath == null || filePath.equals("")) {
             throw new FalconCLIException("Missing argument: file");
         }
     }
 
     private void validateColo(Set<String> optionsList)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         if (optionsList.contains(COLO_OPT)) {
             throw new FalconCLIException("Invalid argument : " + COLO_OPT);
         }
     }
 
-    private void validateEntityName(Set<String> optionsList, String entityName)
-            throws FalconCLIException {
+    private void validateEntityName(String entityName)
+        throws FalconCLIException {
+
         if (entityName == null || entityName.equals("")) {
             throw new FalconCLIException("Missing argument: name");
         }
     }
 
-    private void validateEntityType(Set<String> optionsList, String entityType)
-            throws FalconCLIException {
+    private void validateEntityType(String entityType)
+        throws FalconCLIException {
+
         if (entityType == null || entityType.equals("")) {
             throw new FalconCLIException("Missing argument: type");
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index bc8efc1..44e9ccd 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -31,7 +31,14 @@ import javax.ws.rs.HttpMethod;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.UriBuilder;
-import java.io.*;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.UnsupportedEncodingException;
 import java.util.Properties;
 
 /**
@@ -40,7 +47,6 @@ import java.util.Properties;
  */
 public class FalconClient {
 
-    private String baseUrl;
     protected static WebResource service;
     public static final String WS_HEADER_PREFIX = "header:";
     private static final String REMOTE_USER = "Remote-User";
@@ -55,9 +61,9 @@ public class FalconClient {
      * @throws IOException
      */
     public FalconClient(String falconUrl) throws IOException {
-        this.baseUrl = notEmpty(falconUrl, "FalconUrl");
-        if (!this.baseUrl.endsWith("/")) {
-            this.baseUrl += "/";
+        String baseUrl = notEmpty(falconUrl, "FalconUrl");
+        if (!baseUrl.endsWith("/")) {
+            baseUrl += "/";
         }
         Client client = Client.create(new DefaultClientConfig());
         setFalconTimeOut(client);
@@ -72,8 +78,8 @@ public class FalconClient {
         Properties prop = new Properties();
         InputStream input = FalconClient.class
                 .getResourceAsStream("/client.properties");
-        int readTimeout = 0;
-        int connectTimeout = 0;
+        int readTimeout;
+        int connectTimeout;
         if (input != null) {
             prop.load(input);
             readTimeout = prop.containsKey("falcon.read.timeout") ? Integer
@@ -172,23 +178,8 @@ public class FalconClient {
         return str;
     }
 
-    /**
-     * Check if the object is not null.
-     *
-     * @param <T>
-     * @param obj
-     * @param name
-     * @return string
-     */
-    public static <T> T notNull(T obj, String name) {
-        if (obj == null) {
-            throw new IllegalArgumentException(name + " cannot be null");
-        }
-        return obj;
-    }
-
     public String schedule(String entityType, String entityName, String colo)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendEntityRequest(Entities.SCHEDULE, entityType, entityName,
                 colo);
@@ -196,63 +187,67 @@ public class FalconClient {
     }
 
     public String suspend(String entityType, String entityName, String colo)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendEntityRequest(Entities.SUSPEND, entityType, entityName, colo);
 
     }
 
     public String resume(String entityType, String entityName, String colo)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendEntityRequest(Entities.RESUME, entityType, entityName, colo);
 
     }
 
     public String delete(String entityType, String entityName)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendEntityRequest(Entities.DELETE, entityType, entityName, null);
 
     }
 
     public String validate(String entityType, String filePath)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         InputStream entityStream = getServletInputStream(filePath);
         return sendEntityRequestWithObject(Entities.VALIDATE, entityType,
                 entityStream, null);
     }
 
     public String submit(String entityType, String filePath)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         InputStream entityStream = getServletInputStream(filePath);
         return sendEntityRequestWithObject(Entities.SUBMIT, entityType,
                 entityStream, null);
     }
 
     public String update(String entityType, String entityName, String filePath)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         InputStream entityStream = getServletInputStream(filePath);
         return sendEntityRequestWithNameAndObject(Entities.UPDATE, entityType,
                 entityName, entityStream);
     }
 
     public String submitAndSchedule(String entityType, String filePath)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         InputStream entityStream = getServletInputStream(filePath);
         return sendEntityRequestWithObject(Entities.SUBMITandSCHEDULE,
                 entityType, entityStream, null);
     }
 
     public String getStatus(String entityType, String entityName, String colo)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendEntityRequest(Entities.STATUS, entityType, entityName, colo);
 
     }
 
     public String getDefinition(String entityType, String entityName)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendDefinitionRequest(Entities.DEFINITION, entityType,
                 entityName);
@@ -260,7 +255,8 @@ public class FalconClient {
     }
 
     public String getDependency(String entityType, String entityName)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         return sendDependencyRequest(Entities.DEPENDENCY, entityType,
                 entityName);
     }
@@ -270,70 +266,78 @@ public class FalconClient {
     }
 
     public String getRunningInstances(String type, String entity, String colo)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return sendInstanceRequest(Instances.RUNNING, type, entity, null, null,
                 null, null, colo);
     }
 
     public String getStatusOfInstances(String type, String entity,
-                                       String start, String end, String runid, String colo)
-            throws FalconCLIException {
+                                       String start, String end,
+                                       String runid, String colo)
+        throws FalconCLIException {
 
         return sendInstanceRequest(Instances.STATUS, type, entity, start, end,
                 null, null, colo);
     }
 
     public String killInstances(String type, String entity, String start,
-                                String end, String colo, String clusters, String sourceClusters)
-            throws FalconCLIException, UnsupportedEncodingException {
+                                String end, String colo, String clusters,
+                                String sourceClusters)
+        throws FalconCLIException, UnsupportedEncodingException {
 
         return sendInstanceRequest(Instances.KILL, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, null), null, colo);
     }
 
     public String suspendInstances(String type, String entity, String start,
-                                   String end, String colo, String clusters, String sourceClusters)
-            throws FalconCLIException, UnsupportedEncodingException {
+                                   String end, String colo, String clusters,
+                                   String sourceClusters)
+        throws FalconCLIException, UnsupportedEncodingException {
 
         return sendInstanceRequest(Instances.SUSPEND, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, null), null, colo);
     }
 
     public String resumeInstances(String type, String entity, String start,
-                                  String end, String colo, String clusters, String sourceClusters)
-            throws FalconCLIException, UnsupportedEncodingException {
+                                  String end, String colo, String clusters,
+                                  String sourceClusters)
+        throws FalconCLIException, UnsupportedEncodingException {
 
         return sendInstanceRequest(Instances.RESUME, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, null), null, colo);
     }
 
     public String rerunInstances(String type, String entity, String start,
-                                 String end, String filePath, String colo, String clusters,
-                                 String sourceClusters) throws FalconCLIException, IOException {
-        StringBuffer sb = new StringBuffer();
+                                 String end, String filePath, String colo,
+                                 String clusters, String sourceClusters)
+        throws FalconCLIException, IOException {
+
+        StringBuilder buffer = new StringBuilder();
         if (filePath != null) {
             BufferedReader in = new BufferedReader(new FileReader(filePath));
             String str;
             while ((str = in.readLine()) != null) {
-                sb.append(str).append("\n");
+                buffer.append(str).append("\n");
             }
             in.close();
         }
-        String temp = (sb.length() == 0) ? null : sb.toString();
+        String temp = (buffer.length() == 0) ? null : buffer.toString();
         return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, temp), null, colo);
     }
 
     public String rerunInstances(String type, String entity, String start,
                                  String end, String colo, String clusters, String sourceClusters)
-            throws FalconCLIException, UnsupportedEncodingException {
+        throws FalconCLIException, UnsupportedEncodingException {
+
         return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, "oozie.wf.rerun.failnodes=true\n"), null, colo);
     }
 
     public String getLogsOfInstances(String type, String entity, String start,
-                                     String end, String colo, String runId) throws FalconCLIException {
+                                     String end, String colo, String runId)
+        throws FalconCLIException {
 
         return sendInstanceRequest(Instances.LOG, type, entity, start, end,
                 null, runId, colo);
@@ -350,44 +354,42 @@ public class FalconClient {
     /**
      * Converts a InputStream into ServletInputStream.
      *
-     * @param filePath
+     * @param filePath - Path of file to stream
      * @return ServletInputStream
      * @throws FalconCLIException
-     * @throws java.io.IOException
      */
     private InputStream getServletInputStream(String filePath)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         if (filePath == null) {
             return null;
         }
-        InputStream stream = null;
+        InputStream stream;
         try {
             stream = new FileInputStream(filePath);
         } catch (FileNotFoundException e) {
             throw new FalconCLIException("File not found:", e);
-        } catch (IOException e) {
-            throw new FalconCLIException("Unable to read file: ", e);
         }
         return stream;
     }
 
     private InputStream getServletInputStream(String clusters,
-                                              String sourceClusters, String properties) throws FalconCLIException,
-                                                                                               UnsupportedEncodingException {
+                                              String sourceClusters, String properties)
+        throws FalconCLIException, UnsupportedEncodingException {
 
-        InputStream stream = null;
-        StringBuffer sb = new StringBuffer();
+        InputStream stream;
+        StringBuilder buffer = new StringBuilder();
         if (clusters != null) {
-            sb.append(FALCON_INSTANCE_ACTION_CLUSTERS + "=" + clusters + "\n");
+            buffer.append(FALCON_INSTANCE_ACTION_CLUSTERS).append('=').append(clusters).append('\n');
         }
         if (sourceClusters != null) {
-            sb.append(FALCON_INSTANCE_SOURCE_CLUSTERS + "=" + sourceClusters + "\n");
+            buffer.append(FALCON_INSTANCE_SOURCE_CLUSTERS).append('=').append(sourceClusters).append('\n');
         }
         if (properties != null) {
-            sb.append(properties);
+            buffer.append(properties);
         }
-        stream = new ByteArrayInputStream(sb.toString().getBytes("UTF-8"));
-        return (sb.length() == 0) ? null : stream;
+        stream = new ByteArrayInputStream(buffer.toString().getBytes("UTF-8"));
+        return (buffer.length() == 0) ? null : stream;
     }
     // private ServletInputStream getServletInputStream(final InputStream
     // stream)
@@ -538,7 +540,7 @@ public class FalconClient {
     }
 
     private String sendAdminRequest(AdminOperations job)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         ClientResponse clientResponse = service.path(job.path)
                 .header(REMOTE_USER, USER).accept(job.mimeType)
@@ -547,14 +549,14 @@ public class FalconClient {
     }
 
     private String parseAPIResult(ClientResponse clientResponse)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         APIResult result = clientResponse.getEntity(APIResult.class);
         return result.getMessage();
     }
 
     private String parseEntityList(ClientResponse clientResponse)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         EntityList result = clientResponse.getEntity(EntityList.class);
         if (result == null || result.getElements() == null) {
@@ -565,7 +567,7 @@ public class FalconClient {
     }
 
     private String parseStringResult(ClientResponse clientResponse)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         return clientResponse.getEntity(String.class);
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java b/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
index cde7792..e9cdd87 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
@@ -23,11 +23,11 @@ import java.util.regex.Pattern;
 /**
  * Date utility class.
  */
-public class DateValidator {
+public final class DateValidator {
 
     private static final String DATE_PATTERN =
             "(2\\d\\d\\d|19\\d\\d)-(0[1-9]|1[012])-(0[1-9]|1[0-9]|2[0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9])Z";
-    private static final Pattern pattern = Pattern.compile(DATE_PATTERN);
+    private static final Pattern PATTERN = Pattern.compile(DATE_PATTERN);
 
     private DateValidator() {
     }
@@ -40,7 +40,7 @@ public class DateValidator {
      */
     public static boolean validate(final String date) {
 
-        Matcher matcher = pattern.matcher(date);
+        Matcher matcher = PATTERN.matcher(date);
 
         if (matcher.matches()) {
 
@@ -60,17 +60,9 @@ public class DateValidator {
                 } else if (month.equals("2") || month.equals("02")) {
                     // leap year
                     if (year % 4 == 0) {
-                        if (day.equals("30") || day.equals("31")) {
-                            return false;
-                        } else {
-                            return true;
-                        }
+                        return !(day.equals("30") || day.equals("31"));
                     } else {
-                        if (day.equals("29") || day.equals("30") || day.equals("31")) {
-                            return false;
-                        } else {
-                            return true;
-                        }
+                        return !(day.equals("29") || day.equals("30") || day.equals("31"));
                     }
                 } else {
                     return true;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
index 9fb926d..8759976 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
@@ -23,7 +23,10 @@ import javax.xml.bind.Unmarshaller;
 import java.io.StringReader;
 import java.io.StringWriter;
 
-public abstract class Entity {
+/**
+ * Base class that all entity jaxb object will extend.
+ */
+public abstract class Entity implements Cloneable{
     public abstract String getName();
 
     public EntityType getEntityType() {
@@ -86,7 +89,8 @@ public abstract class Entity {
     }
 
     @Override
-    public Entity clone() {
+    public Entity clone() throws CloneNotSupportedException {
+        Object ignore = super.clone();
         return fromString(getEntityType(), toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java b/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
index d33bdf0..758308e 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
@@ -27,7 +27,7 @@ import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
 /**
- * Enum for types of entities in Falcon Process, Feed and Cluster
+ * Enum for types of entities in Falcon Process, Feed and Cluster.
  */
 public enum EntityType {
     FEED(Feed.class, "/feed-0.1.xsd", "name"),

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java b/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
index 0d1be26..911073a 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
@@ -22,10 +22,15 @@ import java.util.Calendar;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-
+/**
+ * Frequency as supported in the xsd definitions.
+ */
 public class Frequency {
     private static final Pattern PATTERN = Pattern.compile("(minutes|hours|days|months)\\((\\d+)\\)");
 
+    /**
+     * TimeUnit corresponding to the frequency.
+     */
     public static enum TimeUnit {
         minutes(Calendar.MINUTE), hours(Calendar.HOUR), days(Calendar.DATE), months(Calendar.MONTH);
 
@@ -90,11 +95,15 @@ public class Frequency {
         }
 
         Frequency freq = (Frequency) obj;
-        if (this == freq) {
-            return true;
-        }
+        return this == freq || this.getFrequency() == freq.getFrequency()
+                && this.getTimeUnit() == freq.getTimeUnit();
 
-        return this.getFrequency() == freq.getFrequency() &&
-                this.getTimeUnit() == freq.getTimeUnit();
+    }
+
+    @Override
+    public int hashCode() {
+        int result = timeUnit.hashCode();
+        result = 31 * result + frequency;
+        return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java b/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
index 9baf827..c6de668 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
@@ -24,7 +24,13 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.TimeZone;
 
-public class SchemaHelper {
+/**
+ * Support function to parse and format date in xsd string.
+ */
+public final class SchemaHelper {
+
+    private SchemaHelper() {}
+
     public static String getTimeZoneId(TimeZone tz) {
         return tz.getID();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/resource/APIResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/APIResult.java b/client/src/main/java/org/apache/falcon/resource/APIResult.java
index 7b138bc..2f1420d 100644
--- a/client/src/main/java/org/apache/falcon/resource/APIResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/APIResult.java
@@ -31,7 +31,7 @@ import java.util.UUID;
 
 /**
  * APIResult is the output returned by all the APIs; status-SUCCEEDED or FAILED
- * message- detailed message
+ * message- detailed message.
  */
 @XmlRootElement(name = "result")
 @XmlAccessorType(XmlAccessType.FIELD)
@@ -43,16 +43,19 @@ public class APIResult {
 
     private String requestId;
 
-    private static final JAXBContext jc;
+    private static final JAXBContext JAXB_CONTEXT;
 
     static {
         try {
-            jc = JAXBContext.newInstance(APIResult.class);
+            JAXB_CONTEXT = JAXBContext.newInstance(APIResult.class);
         } catch (JAXBException e) {
             throw new RuntimeException(e);
         }
     }
 
+    /**
+     * API Result status.
+     */
     public static enum Status {
         SUCCEEDED, PARTIAL, FAILED
     }
@@ -93,7 +96,7 @@ public class APIResult {
     public String toString() {
         try {
             StringWriter stringWriter = new StringWriter();
-            Marshaller marshaller = jc.createMarshaller();
+            Marshaller marshaller = JAXB_CONTEXT.createMarshaller();
             marshaller.marshal(this, stringWriter);
             return stringWriter.toString();
         } catch (JAXBException e) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/resource/EntityList.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntityList.java b/client/src/main/java/org/apache/falcon/resource/EntityList.java
index 24771cc..61110b2 100644
--- a/client/src/main/java/org/apache/falcon/resource/EntityList.java
+++ b/client/src/main/java/org/apache/falcon/resource/EntityList.java
@@ -25,6 +25,9 @@ import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
+/**
+ * Entity list used for marshalling / unmarshalling with REST calls.
+ */
 @XmlRootElement(name = "entities")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class EntityList {
@@ -32,6 +35,9 @@ public class EntityList {
     @XmlElement(name = "entity")
     private EntityElement[] elements;
 
+    /**
+     * Element within an entity.
+     */
     public static class EntityElement {
         @XmlElement
         public String type;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
index f790df1..cc737a5 100644
--- a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
@@ -22,8 +22,15 @@ import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 import java.util.Date;
 
+/**
+ * Pojo for JAXB marshalling / unmarshalling.
+ */
 @XmlRootElement
 public class InstancesResult extends APIResult {
+
+    /**
+     * Workflow status as being set in result object.
+     */
     public static enum WorkflowStatus {
         WAITING, RUNNING, SUSPENDED, KILLED, FAILED, SUCCEEDED, ERROR
     }
@@ -58,6 +65,9 @@ public class InstancesResult extends APIResult {
         this.instances = instances;
     }
 
+    /**
+     * A single instance object inside instance result.
+     */
     @XmlRootElement(name = "instance")
     public static class Instance {
         @XmlElement
@@ -147,6 +157,9 @@ public class InstancesResult extends APIResult {
         }
     }
 
+    /**
+     * Instance action inside an instance object.
+     */
     @XmlRootElement(name = "actions")
     public static class InstanceAction {
         @XmlElement

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java b/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
index a52da32..6ac3e9f 100644
--- a/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
+++ b/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
@@ -24,6 +24,9 @@ import org.apache.commons.cli.ParseException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Command parser for CLI.
+ */
 public class TestCLIParser {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java b/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
index fc81896..e288b39 100644
--- a/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
+++ b/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
@@ -23,58 +23,58 @@ import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
 /**
- * Date format yyyy/mm/dd validator Testing
+ * Date format yyyy/mm/dd validator Testing.
  */
 public class DateValidatorTest {
 
     @DataProvider
-    public Object[][] ValidDateProvider() {
+    public Object[][] validDateProvider() {
         return new Object[][]{
-                new Object[]{"2011-11-01T00:00Z"}, new Object[]{"2020-01-01T00:00Z"},
-                new Object[]{"2010-01-31T00:59Z"}, new Object[]{"2020-01-31T00:00Z"},
-                new Object[]{"2008-02-29T01:00Z"}, new Object[]{"2008-02-29T00:00Z"},
-                new Object[]{"2009-02-28T01:01Z"}, new Object[]{"2009-02-28T00:00Z"},
-                new Object[]{"2010-03-31T23:00Z"}, new Object[]{"2010-03-31T00:00Z"},
-                new Object[]{"2010-04-30T23:59Z"}, new Object[]{"2010-04-30T00:00Z"},
-                new Object[]{"2010-05-31T23:23Z"}, new Object[]{"2010-05-31T00:00Z"},
-                new Object[]{"2010-06-30T00:00Z"}, new Object[]{"2010-06-30T00:00Z"},
-                new Object[]{"2010-07-31T00:00Z"}, new Object[]{"2010-07-31T00:00Z"},
-                new Object[]{"2010-08-31T00:00Z"}, new Object[]{"2010-08-31T00:00Z"},
-                new Object[]{"2010-09-30T00:00Z"}, new Object[]{"2010-09-30T00:00Z"},
-                new Object[]{"2010-10-31T00:00Z"}, new Object[]{"2010-10-31T00:00Z"},
-                new Object[]{"2010-11-30T00:00Z"}, new Object[]{"2010-11-30T00:00Z"},
-                new Object[]{"2010-12-31T00:00Z"}, new Object[]{"2010-12-31T00:00Z"},
-                new Object[]{"1999-01-30T01:00Z"}, new Object[]{"2999-12-31T00:00Z"}
+            new Object[]{"2011-11-01T00:00Z", }, new Object[]{"2020-01-01T00:00Z", },
+            new Object[]{"2010-01-31T00:59Z", }, new Object[]{"2020-01-31T00:00Z", },
+            new Object[]{"2008-02-29T01:00Z", }, new Object[]{"2008-02-29T00:00Z", },
+            new Object[]{"2009-02-28T01:01Z", }, new Object[]{"2009-02-28T00:00Z", },
+            new Object[]{"2010-03-31T23:00Z", }, new Object[]{"2010-03-31T00:00Z", },
+            new Object[]{"2010-04-30T23:59Z", }, new Object[]{"2010-04-30T00:00Z", },
+            new Object[]{"2010-05-31T23:23Z", }, new Object[]{"2010-05-31T00:00Z", },
+            new Object[]{"2010-06-30T00:00Z", }, new Object[]{"2010-06-30T00:00Z", },
+            new Object[]{"2010-07-31T00:00Z", }, new Object[]{"2010-07-31T00:00Z", },
+            new Object[]{"2010-08-31T00:00Z", }, new Object[]{"2010-08-31T00:00Z", },
+            new Object[]{"2010-09-30T00:00Z", }, new Object[]{"2010-09-30T00:00Z", },
+            new Object[]{"2010-10-31T00:00Z", }, new Object[]{"2010-10-31T00:00Z", },
+            new Object[]{"2010-11-30T00:00Z", }, new Object[]{"2010-11-30T00:00Z", },
+            new Object[]{"2010-12-31T00:00Z", }, new Object[]{"2010-12-31T00:00Z", },
+            new Object[]{"1999-01-30T01:00Z", }, new Object[]{"2999-12-31T00:00Z", },
         };
     }
 
     @DataProvider
-    public Object[][] InvalidDateProvider() {
+    public Object[][] invalidDateProvider() {
         return new Object[][]{
-                new Object[]{"2010-12-31T00:60Z"}, new Object[]{"2010-12-31T24:00Z"},
-                new Object[]{"2010-01-32T00:00Z"}, new Object[]{"2020-01-32T00:00Z"},
-                new Object[]{"2010-13-1T00:00Z"}, new Object[]{"1820-01-01T00:00Z"},
-                new Object[]{"2007-2-29T00:00Z"}, new Object[]{"2007-02-29T00:00Z"},
-                new Object[]{"2008-2-30T00:00Z"}, new Object[]{"2008-02-31T00:00Z"},
-                new Object[]{"2008-a-29T00:00Z"}, new Object[]{"2008-02aT00:00Z"},
-                new Object[]{"2008-2-333T00:00Z"}, new Object[]{"200a-02-29T00:00Z"},
-                new Object[]{"2010-4-31T00:00Z"}, new Object[]{"2010-04-31T00:00Z"},
-                new Object[]{"2010-6-31T00:00Z"}, new Object[]{"2010-06-31T00:00Z"},
-                new Object[]{"2010-9-31T00:00Z"}, new Object[]{"2010-09-31T00:00Z"},
-                new Object[]{"2010-11-31T00:00Z"}, new Object[]{"1999-04-31T01:00Z"},
+            new Object[]{"2010-12-31T00:60Z", }, new Object[]{"2010-12-31T24:00Z", },
+            new Object[]{"2010-01-32T00:00Z", }, new Object[]{"2020-01-32T00:00Z", },
+            new Object[]{"2010-13-1T00:00Z", }, new Object[]{"1820-01-01T00:00Z", },
+            new Object[]{"2007-2-29T00:00Z", }, new Object[]{"2007-02-29T00:00Z", },
+            new Object[]{"2008-2-30T00:00Z", }, new Object[]{"2008-02-31T00:00Z", },
+            new Object[]{"2008-a-29T00:00Z", }, new Object[]{"2008-02aT00:00Z", },
+            new Object[]{"2008-2-333T00:00Z", }, new Object[]{"200a-02-29T00:00Z", },
+            new Object[]{"2010-4-31T00:00Z", }, new Object[]{"2010-04-31T00:00Z", },
+            new Object[]{"2010-6-31T00:00Z", }, new Object[]{"2010-06-31T00:00Z", },
+            new Object[]{"2010-9-31T00:00Z", }, new Object[]{"2010-09-31T00:00Z", },
+            new Object[]{"2010-11-31T00:00Z", }, new Object[]{"1999-04-31T01:00Z", },
         };
     }
 
-    @Test(dataProvider = "ValidDateProvider")
-    public void ValidDateTest(String date) {
+    @Test(dataProvider = "validDateProvider")
+    public void validDateTest(String date) {
         boolean valid = DateValidator.validate(date);
         System.out.println("Date is valid : " + date + " , " + valid);
         Assert.assertEquals(valid, true);
     }
 
-    @Test(dataProvider = "InvalidDateProvider",
-            dependsOnMethods = "ValidDateTest")
-    public void InValidDateTest(String date) {
+    @Test(dataProvider = "invalidDateProvider",
+            dependsOnMethods = "validDateTest")
+    public void invalidDateTest(String date) {
         boolean valid = DateValidator.validate(date);
         System.out.println("Date is valid : " + date + " , " + valid);
         Assert.assertEquals(valid, false);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e69b2205/client/src/test/java/org/apache/falcon/entity/v0/TestFrequency.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/falcon/entity/v0/TestFrequency.java b/client/src/test/java/org/apache/falcon/entity/v0/TestFrequency.java
index 05e9352..e5eb3f1 100644
--- a/client/src/test/java/org/apache/falcon/entity/v0/TestFrequency.java
+++ b/client/src/test/java/org/apache/falcon/entity/v0/TestFrequency.java
@@ -21,6 +21,9 @@ package org.apache.falcon.entity.v0;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Test frequency.
+ */
 @Test
 public class TestFrequency {
 


[34/47] git commit: More check style fixes relating to metrics module

Posted by sr...@apache.org.
More check style fixes relating to metrics module


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

Branch: refs/heads/master
Commit: 223d8f02190e08a39c23da62f8f97528f6c44a5d
Parents: 2a2fa50
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:46:49 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:46:49 2013 +0530

----------------------------------------------------------------------
 .../org/apache/falcon/aspect/GenericAlert.java     |    4 ++--
 1 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/223d8f02/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
index 8099760..275a725 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
@@ -31,7 +31,7 @@ public final class GenericAlert {
 
     private GenericAlert() {}
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK ParameterNumberCheck
     @Monitored(event = "retry-instance-failed")
     public static String alertRetryFailed(
             @Dimension(value = "entity-type") String entityType,
@@ -88,7 +88,7 @@ public final class GenericAlert {
 
         return "IGNORE";
     }
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK ParameterNumberCheck
 
     @Monitored(event = "rerun-queue-failed")
     public static String alertRerunConsumerFailed(


[13/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java b/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
index 14d35aa..1f3f256 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
@@ -18,15 +18,15 @@
 
 package org.apache.falcon.aspect;
 
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.falcon.util.ResourcesReflectionUtil;
 import org.apache.log4j.Logger;
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
 
+import java.util.HashMap;
+import java.util.Map;
+
 /**
  * Abstract Falcon Aspect, which intercept methods annotated with Monitored and
  * publishes messages. Subclasses should override publishMessage Method.
@@ -34,65 +34,65 @@ import org.aspectj.lang.annotation.Aspect;
 @Aspect
 public abstract class AbstractFalconAspect {
 
-	private static final Logger LOG = Logger
-			.getLogger(AbstractFalconAspect.class);
+    private static final Logger LOG = Logger
+            .getLogger(AbstractFalconAspect.class);
 
-	@Around("@annotation(org.apache.falcon.monitors.Monitored)")
-	public Object logAround(ProceedingJoinPoint joinPoint) throws Throwable {
+    @Around("@annotation(org.apache.falcon.monitors.Monitored)")
+    public Object logAround(ProceedingJoinPoint joinPoint) throws Throwable {
 
-		String methodName = joinPoint.getSignature().getName();
-		Object[] args = joinPoint.getArgs();
-		Object result = null;
-		ResourceMessage.Status status;
+        String methodName = joinPoint.getSignature().getName();
+        Object[] args = joinPoint.getArgs();
+        Object result = null;
+        ResourceMessage.Status status;
 
-		long startTime = System.nanoTime();
-		long endTime;
-		try {
-			result = joinPoint.proceed();
-		} catch (Exception e) {
-			endTime = System.nanoTime();
-			status = ResourceMessage.Status.FAILED;
-			publishMessage(getResourceMessage(joinPoint.getSignature()
-					.getDeclaringType().getSimpleName()
-					+ "." + methodName, args, status, endTime - startTime));
-			throw e;
-		}
-		endTime = System.nanoTime();
-		status = ResourceMessage.Status.SUCCEEDED;
-		publishMessage(getResourceMessage(joinPoint.getSignature()
-				.getDeclaringType().getSimpleName()
-				+ "." + methodName, args, status, endTime - startTime));
-		return result;
-	}
+        long startTime = System.nanoTime();
+        long endTime;
+        try {
+            result = joinPoint.proceed();
+        } catch (Exception e) {
+            endTime = System.nanoTime();
+            status = ResourceMessage.Status.FAILED;
+            publishMessage(getResourceMessage(joinPoint.getSignature()
+                    .getDeclaringType().getSimpleName()
+                    + "." + methodName, args, status, endTime - startTime));
+            throw e;
+        }
+        endTime = System.nanoTime();
+        status = ResourceMessage.Status.SUCCEEDED;
+        publishMessage(getResourceMessage(joinPoint.getSignature()
+                .getDeclaringType().getSimpleName()
+                + "." + methodName, args, status, endTime - startTime));
+        return result;
+    }
 
-	private ResourceMessage getResourceMessage(String methodName,
-			Object[] args, ResourceMessage.Status status, long executionTime) {
-		String action = ResourcesReflectionUtil
-				.getResourceMonitorName(methodName);
+    private ResourceMessage getResourceMessage(String methodName,
+                                               Object[] args, ResourceMessage.Status status, long executionTime) {
+        String action = ResourcesReflectionUtil
+                .getResourceMonitorName(methodName);
 
-		assert action != null : "Method :" + methodName
-				+ " not parsed by reflection util";
-		Map<String, String> dimensions = new HashMap<String, String>();
+        assert action != null : "Method :" + methodName
+                + " not parsed by reflection util";
+        Map<String, String> dimensions = new HashMap<String, String>();
 
-		if (ResourcesReflectionUtil.getResourceDimensionsName(methodName) == null) {
-			LOG.warn("Class for method name: " + methodName
-					+ " is not added to ResourcesReflectionUtil");
-		} else {
-			for (Map.Entry<Integer, String> param : ResourcesReflectionUtil
-					.getResourceDimensionsName(methodName).entrySet()) {
-				dimensions.put(
-						param.getValue(),
-						args[param.getKey()] == null ? "NULL" : args[param
-								.getKey()].toString());
-			}
-		}
-		Integer timeTakenArg = ResourcesReflectionUtil
-				.getResourceTimeTakenName(methodName);
-		return timeTakenArg == null ? new ResourceMessage(action, dimensions,
-				status, executionTime) : new ResourceMessage(action,
-				dimensions, status, Long.valueOf(args[timeTakenArg]
-						.toString()));
-	}
+        if (ResourcesReflectionUtil.getResourceDimensionsName(methodName) == null) {
+            LOG.warn("Class for method name: " + methodName
+                    + " is not added to ResourcesReflectionUtil");
+        } else {
+            for (Map.Entry<Integer, String> param : ResourcesReflectionUtil
+                    .getResourceDimensionsName(methodName).entrySet()) {
+                dimensions.put(
+                        param.getValue(),
+                        args[param.getKey()] == null ? "NULL" : args[param
+                                .getKey()].toString());
+            }
+        }
+        Integer timeTakenArg = ResourcesReflectionUtil
+                .getResourceTimeTakenName(methodName);
+        return timeTakenArg == null ? new ResourceMessage(action, dimensions,
+                status, executionTime) : new ResourceMessage(action,
+                dimensions, status, Long.valueOf(args[timeTakenArg]
+                .toString()));
+    }
 
-	abstract public void publishMessage(ResourceMessage message);
+    abstract public void publishMessage(ResourceMessage message);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
index 9705f9e..b792434 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
@@ -29,83 +29,83 @@ import org.aspectj.lang.annotation.Aspect;
 @Aspect
 public class GenericAlert {
 
-	@Monitored(event = "retry-instance-failed")
-	public static String alertRetryFailed(
-			@Dimension(value = "entity-type") String entityType,
-			@Dimension(value = "entity-name") String entityName,
-			@Dimension(value = "nominal-name") String nominalTime,
-			@Dimension(value = "wf-id") String wfId,
-			@Dimension(value = "run-id") String runId,
-			@Dimension(value = "error-message") String message) {
-		return "IGNORE";
-	}
-	
-	@Monitored(event = "late-rerun-failed")
-	public static String alertLateRerunFailed(
-			@Dimension(value = "entity-type") String entityType,
-			@Dimension(value = "entity-name") String entityName,
-			@Dimension(value = "nominal-name") String nominalTime,
-			@Dimension(value = "wf-id") String wfId,
-			@Dimension(value = "run-id") String runId,
-			@Dimension(value = "error-message") String message) {
-		return "IGNORE";
+    @Monitored(event = "retry-instance-failed")
+    public static String alertRetryFailed(
+            @Dimension(value = "entity-type") String entityType,
+            @Dimension(value = "entity-name") String entityName,
+            @Dimension(value = "nominal-name") String nominalTime,
+            @Dimension(value = "wf-id") String wfId,
+            @Dimension(value = "run-id") String runId,
+            @Dimension(value = "error-message") String message) {
+        return "IGNORE";
+    }
+
+    @Monitored(event = "late-rerun-failed")
+    public static String alertLateRerunFailed(
+            @Dimension(value = "entity-type") String entityType,
+            @Dimension(value = "entity-name") String entityName,
+            @Dimension(value = "nominal-name") String nominalTime,
+            @Dimension(value = "wf-id") String wfId,
+            @Dimension(value = "run-id") String runId,
+            @Dimension(value = "error-message") String message) {
+        return "IGNORE";
 
-	}
+    }
 
-	@Monitored(event = "wf-instance-failed")
-	public static String instrumentFailedInstance(
-			@Dimension(value = "cluster") String cluster,
-			@Dimension(value = "entity-type") String entityType,
-			@Dimension(value = "entity-name") String entityName,
-			@Dimension(value = "nominal-time") String nominalTime,
-			@Dimension(value = "wf-id") String workflowId,
-			@Dimension(value = "run-id") String runId,
-			@Dimension(value = "operation") String operation,
+    @Monitored(event = "wf-instance-failed")
+    public static String instrumentFailedInstance(
+            @Dimension(value = "cluster") String cluster,
+            @Dimension(value = "entity-type") String entityType,
+            @Dimension(value = "entity-name") String entityName,
+            @Dimension(value = "nominal-time") String nominalTime,
+            @Dimension(value = "wf-id") String workflowId,
+            @Dimension(value = "run-id") String runId,
+            @Dimension(value = "operation") String operation,
             @Dimension(value = "start-time") String startTime,
             @Dimension(value = "error-message") String errorMessage,
             @Dimension(value = "message") String message,
-			@TimeTaken long timeTaken)
-			throws Exception {
-		return "IGNORE";
-	}
-	
-	@Monitored(event = "wf-instance-succeeded")
-	public static String instrumentSucceededInstance(
-			@Dimension(value = "cluster") String cluster,
-			@Dimension(value = "entity-type") String entityType,
-			@Dimension(value = "entity-name") String entityName,
-			@Dimension(value = "nominal-time") String nominalTime,
-			@Dimension(value = "wf-id") String workflowId,
-			@Dimension(value = "run-id") String runId,
-			@Dimension(value = "operation") String operation,
+            @TimeTaken long timeTaken)
+            throws Exception {
+        return "IGNORE";
+    }
+
+    @Monitored(event = "wf-instance-succeeded")
+    public static String instrumentSucceededInstance(
+            @Dimension(value = "cluster") String cluster,
+            @Dimension(value = "entity-type") String entityType,
+            @Dimension(value = "entity-name") String entityName,
+            @Dimension(value = "nominal-time") String nominalTime,
+            @Dimension(value = "wf-id") String workflowId,
+            @Dimension(value = "run-id") String runId,
+            @Dimension(value = "operation") String operation,
             @Dimension(value = "start-time") String startTime,
-			@TimeTaken long timeTaken)
-			throws Exception {
-		return "IGNORE";
-	}
-	
-	@Monitored(event = "rerun-queue-failed")
-	public static String alertRerunConsumerFailed(
-			@Dimension(value = "message") String message,
-			@Dimension(value = "exception") Exception exception) {
-		return "IGNORE";
+            @TimeTaken long timeTaken)
+            throws Exception {
+        return "IGNORE";
+    }
 
-	}
+    @Monitored(event = "rerun-queue-failed")
+    public static String alertRerunConsumerFailed(
+            @Dimension(value = "message") String message,
+            @Dimension(value = "exception") Exception exception) {
+        return "IGNORE";
+
+    }
 
-	@Monitored(event = "sla-miss")
-	public static String alertOnLikelySLAMiss(
+    @Monitored(event = "sla-miss")
+    public static String alertOnLikelySLAMiss(
             @Dimension(value = "cluster") String cluster,
             @Dimension(value = "entity-type") String entityType,
             @Dimension(value = "entity-name") String entityName,
             @Dimension(value = "nominal-time") String nominalTime) {
-		return "IGNORE";
-	}
-	
-	@Monitored(event = "log-cleanup-service-failed")
-	public static String alertLogCleanupServiceFailed(
-			@Dimension(value = "message") String message,
-			@Dimension(value = "exception") Throwable throwable) {
-		return "IGNORE";
+        return "IGNORE";
+    }
+
+    @Monitored(event = "log-cleanup-service-failed")
+    public static String alertLogCleanupServiceFailed(
+            @Dimension(value = "message") String message,
+            @Dimension(value = "exception") Throwable throwable) {
+        return "IGNORE";
 
-	}
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java b/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
index b00cbca..c7ad8e7 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
@@ -23,43 +23,44 @@ import java.util.Map;
 //Message to be sent to logging system
 public class ResourceMessage {
 
-	private String action;
-	private Map<String, String> dimensions;
-	private Status status;
-	private long executionTime;
-	
-	public enum Status{
-		SUCCEEDED, FAILED
-	}
+    private String action;
+    private Map<String, String> dimensions;
+    private Status status;
+    private long executionTime;
 
-	public ResourceMessage(String action, Map<String, String> dimensions,
-			Status status, long executionTime) {
-		this.action = action;
-		this.dimensions = dimensions;
-		this.status = status;
-		this.executionTime = executionTime;
-	}
-	
-	public String getAction() {
-		return action;
-	}
+    public enum Status {
+        SUCCEEDED, FAILED
+    }
 
-	public Map<String, String> getDimensions() {
-		return dimensions;
-	}
+    public ResourceMessage(String action, Map<String, String> dimensions,
+                           Status status, long executionTime) {
+        this.action = action;
+        this.dimensions = dimensions;
+        this.status = status;
+        this.executionTime = executionTime;
+    }
 
-	public Status getStatus() {
-		return status;
-	}
+    public String getAction() {
+        return action;
+    }
 
-	public long getExecutionTime() {
-		return executionTime;
-	}
+    public Map<String, String> getDimensions() {
+        return dimensions;
+    }
 
-	@Override
-	public String toString() {
-		return "{Action:"+action+", Dimensions:"+dimensions+", Status: "+status.name()+", Time-taken:"+executionTime+" ns}";
-	}
+    public Status getStatus() {
+        return status;
+    }
+
+    public long getExecutionTime() {
+        return executionTime;
+    }
+
+    @Override
+    public String toString() {
+        return "{Action:" + action + ", Dimensions:" + dimensions + ", Status: " + status.name() + ", Time-taken:"
+                + executionTime + " ns}";
+    }
 
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java b/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
index 2f8dd6d..119a7f7 100644
--- a/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
+++ b/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
@@ -21,7 +21,7 @@ package org.apache.falcon.util;
 import org.apache.falcon.monitors.Monitored;
 import org.apache.falcon.monitors.TimeTaken;
 
-import java.awt.Dimension;
+import java.awt.*;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.util.Collections;
@@ -31,55 +31,54 @@ import java.util.Map;
 /**
  * Builds a cached of methods annotated with Monitored and params of methods
  * annotated with Dimension.
- * 
  */
 public class ResourcesReflectionUtil {
 
-	private static final Map<String, MethodAnnotation> methods = new HashMap<String, MethodAnnotation>();
-	
-	private ResourcesReflectionUtil(){
-		
-	}
-
-	static {
-		//TODO load these classes from properties file
-		buildAnnotationsMapForClass("org.apache.falcon.resource.proxy.SchedulableEntityManagerProxy");
-		buildAnnotationsMapForClass("org.apache.falcon.resource.proxy.InstanceManagerProxy");
-		buildAnnotationsMapForClass("org.apache.falcon.resource.AbstractInstanceManager");
-		buildAnnotationsMapForClass("org.apache.falcon.service.FalconTopicSubscriber");
-		buildAnnotationsMapForClass("org.apache.falcon.aspect.GenericAlert");
-	}
-
-	public static Map<Integer, String> getResourceDimensionsName(String methodName) {
-		return methods.get(methodName)!=null?Collections.unmodifiableMap(methods.get(methodName).params):null;
-	}
-
-	public static String getResourceMonitorName(String methodName) {
-		return methods.get(methodName)!=null?methods.get(methodName).monitoredName:null;
-	}	
-	
-	public static Integer getResourceTimeTakenName(String methodName) {
-		return methods.get(methodName) != null ? methods.get(methodName).timeTakenArgIndex
-				: null;
-	}
-
-	public static class MethodAnnotation {
-		private String monitoredName;
-		// argument <index,DimensionValue>
-		private Map<Integer, String> params = new HashMap<Integer, String>();
-		
-		//to override time taken by an api
-		private Integer timeTakenArgIndex;
-		
-		@Override
-		public String toString() {
-			return "{" + monitoredName + "[" + params.toString() + "]" + "}";
-		}
-
-	}
-
-	private static void buildAnnotationsMapForClass(String className) {
-		Class clazz;
+    private static final Map<String, MethodAnnotation> methods = new HashMap<String, MethodAnnotation>();
+
+    private ResourcesReflectionUtil() {
+
+    }
+
+    static {
+        //TODO load these classes from properties file
+        buildAnnotationsMapForClass("org.apache.falcon.resource.proxy.SchedulableEntityManagerProxy");
+        buildAnnotationsMapForClass("org.apache.falcon.resource.proxy.InstanceManagerProxy");
+        buildAnnotationsMapForClass("org.apache.falcon.resource.AbstractInstanceManager");
+        buildAnnotationsMapForClass("org.apache.falcon.service.FalconTopicSubscriber");
+        buildAnnotationsMapForClass("org.apache.falcon.aspect.GenericAlert");
+    }
+
+    public static Map<Integer, String> getResourceDimensionsName(String methodName) {
+        return methods.get(methodName) != null ? Collections.unmodifiableMap(methods.get(methodName).params) : null;
+    }
+
+    public static String getResourceMonitorName(String methodName) {
+        return methods.get(methodName) != null ? methods.get(methodName).monitoredName : null;
+    }
+
+    public static Integer getResourceTimeTakenName(String methodName) {
+        return methods.get(methodName) != null ? methods.get(methodName).timeTakenArgIndex
+                : null;
+    }
+
+    public static class MethodAnnotation {
+        private String monitoredName;
+        // argument <index,DimensionValue>
+        private Map<Integer, String> params = new HashMap<Integer, String>();
+
+        //to override time taken by an api
+        private Integer timeTakenArgIndex;
+
+        @Override
+        public String toString() {
+            return "{" + monitoredName + "[" + params.toString() + "]" + "}";
+        }
+
+    }
+
+    private static void buildAnnotationsMapForClass(String className) {
+        Class clazz;
         try {
             clazz = ResourcesReflectionUtil.class.
                     getClassLoader().loadClass(className);
@@ -88,7 +87,7 @@ public class ResourcesReflectionUtil {
         }
         Method[] declMethods = clazz.getMethods();
 
-		// scan every method
+        // scan every method
         for (Method declMethod : declMethods) {
             Annotation[] methodAnnots = declMethod.getDeclaredAnnotations();
             // scan every annotation on method
@@ -109,41 +108,41 @@ public class ResourcesReflectionUtil {
 
             }
         }
-	}
-
-	private static Map<Integer, String> getDeclaredParamAnnots(
-			Annotation[][] paramAnnots, MethodAnnotation annotation) {
-		Map<Integer, String> params = new HashMap<Integer, String>();
-		for (int i = 0; i < paramAnnots.length; i++) {
-			for (int j = 0; j < paramAnnots[i].length; j++) {
-				if (paramAnnots[i][j].annotationType().getSimpleName()
-						.equals(Dimension.class.getSimpleName())) {
-					params.put(i, getAnnotationValue(paramAnnots[i][j], "value"));
-				}
-				if (paramAnnots[i][j].annotationType().getSimpleName()
-						.equals(TimeTaken.class.getSimpleName())) {
-					annotation.timeTakenArgIndex = i;
-				}
-			}
-		}
-		return params;
-
-	}
-
-	private static String getAnnotationValue(Annotation annotation,
-			String attributeName) {
-		String value = null;
-
-		if (annotation != null) {
-			try {
-				value = (String) annotation.annotationType()
-						.getMethod(attributeName).invoke(annotation);
-			} catch (Exception ignore) {
-				ignore.printStackTrace();
-			}
-		}
-
-		return value;
-	}
+    }
+
+    private static Map<Integer, String> getDeclaredParamAnnots(
+            Annotation[][] paramAnnots, MethodAnnotation annotation) {
+        Map<Integer, String> params = new HashMap<Integer, String>();
+        for (int i = 0; i < paramAnnots.length; i++) {
+            for (int j = 0; j < paramAnnots[i].length; j++) {
+                if (paramAnnots[i][j].annotationType().getSimpleName()
+                        .equals(Dimension.class.getSimpleName())) {
+                    params.put(i, getAnnotationValue(paramAnnots[i][j], "value"));
+                }
+                if (paramAnnots[i][j].annotationType().getSimpleName()
+                        .equals(TimeTaken.class.getSimpleName())) {
+                    annotation.timeTakenArgIndex = i;
+                }
+            }
+        }
+        return params;
+
+    }
+
+    private static String getAnnotationValue(Annotation annotation,
+                                             String attributeName) {
+        String value = null;
+
+        if (annotation != null) {
+            try {
+                value = (String) annotation.annotationType()
+                        .getMethod(attributeName).invoke(annotation);
+            } catch (Exception ignore) {
+                ignore.printStackTrace();
+            }
+        }
+
+        return value;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java b/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
index 915a69c..ac3e76e 100644
--- a/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
+++ b/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
@@ -18,24 +18,7 @@
 
 package org.apache.falcon.converter;
 
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.StringWriter;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.Tag;
@@ -54,9 +37,21 @@ import org.apache.falcon.oozie.workflow.WORKFLOWAPP;
 import org.apache.falcon.service.FalconPathFilter;
 import org.apache.falcon.service.SharedLibraryHostingService;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.OozieClient;
 
+import javax.xml.bind.*;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
 public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     private static Logger LOG = Logger.getLogger(AbstractOozieEntityMapper.class);
@@ -65,8 +60,8 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected static final String ACTUAL_TIME_EL = "${coord:formatTime(coord:actualTime(), 'yyyy-MM-dd-HH-mm')}";
     protected static final Long DEFAULT_BROKER_MSG_TTL = 3 * 24 * 60L;
-    protected static final String MR_QUEUE_NAME="queueName";
-    protected static final String MR_JOB_PRIORITY="jobPriority";
+    protected static final String MR_QUEUE_NAME = "queueName";
+    protected static final String MR_JOB_PRIORITY = "jobPriority";
 
     protected static final JAXBContext workflowJaxbContext;
     protected static final JAXBContext coordJaxbContext;
@@ -75,16 +70,18 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
     protected static final FalconPathFilter falconJarFilter = new FalconPathFilter() {
         @Override
         public boolean accept(Path path) {
-            if (path.getName().startsWith("falcon"))
+            if (path.getName().startsWith("falcon")) {
                 return true;
+            }
             return false;
         }
 
         @Override
         public String getJarName(Path path) {
             String name = path.getName();
-            if(name.endsWith(".jar"))
+            if (name.endsWith(".jar")) {
                 name = name.substring(0, name.indexOf(".jar"));
+            }
             return name;
         }
     };
@@ -127,7 +124,8 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
         }
         for (COORDINATORAPP coordinatorapp : coordinators) {
             Path coordPath = getCoordPath(bundlePath, coordinatorapp.getName());
-            String coordXmlName = marshal(cluster, coordinatorapp, coordPath, EntityUtil.getWorkflowNameSuffix(coordinatorapp.getName(), entity));
+            String coordXmlName = marshal(cluster, coordinatorapp, coordPath,
+                    EntityUtil.getWorkflowNameSuffix(coordinatorapp.getName(), entity));
             createTempDir(cluster, coordPath);
             COORDINATOR bundleCoord = new COORDINATOR();
             bundleCoord.setName(coordinatorapp.getName());
@@ -145,23 +143,26 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
         try {
             Path libPath = new Path(coordPath, "lib");
             FileSystem fs = FileSystem.get(ClusterHelper.getConfiguration(cluster));
-            if (!fs.exists(libPath))
+            if (!fs.exists(libPath)) {
                 fs.mkdirs(libPath);
+            }
 
             SharedLibraryHostingService.pushLibsToHDFS(libPath.toString(), cluster, falconJarFilter);
         } catch (IOException e) {
             LOG.error("Failed to copy shared libs on cluster " + cluster.getName(), e);
-            throw new FalconException("Failed to copy shared libs on cluster " + cluster.getName(),e);
+            throw new FalconException("Failed to copy shared libs on cluster " + cluster.getName(), e);
         }
     }
 
     protected abstract List<COORDINATORAPP> getCoordinators(Cluster cluster, Path bundlePath) throws FalconException;
 
     protected org.apache.falcon.oozie.coordinator.CONFIGURATION getCoordConfig(Map<String, String> propMap) {
-        org.apache.falcon.oozie.coordinator.CONFIGURATION conf = new org.apache.falcon.oozie.coordinator.CONFIGURATION();
+        org.apache.falcon.oozie.coordinator.CONFIGURATION conf
+                = new org.apache.falcon.oozie.coordinator.CONFIGURATION();
         List<org.apache.falcon.oozie.coordinator.CONFIGURATION.Property> props = conf.getProperty();
-        for (Entry<String, String> prop : propMap.entrySet())
+        for (Entry<String, String> prop : propMap.entrySet()) {
             props.add(createCoordProperty(prop.getKey(), prop.getValue()));
+        }
         return conf;
     }
 
@@ -172,37 +173,42 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
         props.put(ARG.timeStamp.getPropName(), ACTUAL_TIME_EL);
         props.put("userBrokerUrl", ClusterHelper.getMessageBrokerUrl(cluster));
         props.put("userBrokerImplClass", ClusterHelper.getMessageBrokerImplClass(cluster));
-        String falconBrokerUrl = StartupProperties.get().getProperty(ARG.brokerUrl.getPropName(), "tcp://localhost:61616?daemon=true");
+        String falconBrokerUrl = StartupProperties.get().getProperty(ARG.brokerUrl.getPropName(),
+                "tcp://localhost:61616?daemon=true");
         props.put(ARG.brokerUrl.getPropName(), falconBrokerUrl);
         String falconBrokerImplClass = StartupProperties.get().getProperty(ARG.brokerImplClass.getPropName(),
                 ClusterHelper.DEFAULT_BROKER_IMPL_CLASS);
         props.put(ARG.brokerImplClass.getPropName(), falconBrokerImplClass);
-        String jmsMessageTTL = StartupProperties.get().getProperty("broker.ttlInMins", DEFAULT_BROKER_MSG_TTL.toString());
+        String jmsMessageTTL = StartupProperties.get().getProperty("broker.ttlInMins",
+                DEFAULT_BROKER_MSG_TTL.toString());
         props.put(ARG.brokerTTL.getPropName(), jmsMessageTTL);
         props.put(ARG.entityType.getPropName(), entity.getEntityType().name());
         props.put("logDir", getStoragePath(new Path(coordPath, "../../logs")));
-        props.put(OozieClient.EXTERNAL_ID, new ExternalId(entity.getName(), EntityUtil.getWorkflowNameTag(coordName, entity),
-                "${coord:nominalTime()}").getId());
+        props.put(OozieClient.EXTERNAL_ID,
+                new ExternalId(entity.getName(), EntityUtil.getWorkflowNameTag(coordName, entity),
+                        "${coord:nominalTime()}").getId());
         props.put("workflowEngineUrl", ClusterHelper.getOozieUrl(cluster));
-		try {
-			if (EntityUtil.getLateProcess(entity) == null
-					|| EntityUtil.getLateProcess(entity).getLateInputs() == null
-					|| EntityUtil.getLateProcess(entity).getLateInputs().size() == 0) {
-				props.put("shouldRecord", "false");
-			} else {
-				props.put("shouldRecord", "true");
-			}
-		} catch (FalconException e) {
-			LOG.error("Unable to get Late Process for entity:" + entity, e);
-			throw new FalconRuntimException(e);
-		}
+        try {
+            if (EntityUtil.getLateProcess(entity) == null
+                    || EntityUtil.getLateProcess(entity).getLateInputs() == null
+                    || EntityUtil.getLateProcess(entity).getLateInputs().size() == 0) {
+                props.put("shouldRecord", "false");
+            } else {
+                props.put("shouldRecord", "true");
+            }
+        } catch (FalconException e) {
+            LOG.error("Unable to get Late Process for entity:" + entity, e);
+            throw new FalconRuntimException(e);
+        }
         props.put("entityName", entity.getName());
         props.put("entityType", entity.getEntityType().name().toLowerCase());
         props.put(ARG.cluster.getPropName(), cluster.getName());
-        if(cluster.getProperties() != null)
-            for(Property prop:cluster.getProperties().getProperties())
+        if (cluster.getProperties() != null) {
+            for (Property prop : cluster.getProperties().getProperties()) {
                 props.put(prop.getName(), prop.getValue());
-        
+            }
+        }
+
         props.put(MR_QUEUE_NAME, "default");
         props.put(MR_JOB_PRIORITY, "NORMAL");
         //props in entity override the set props.
@@ -210,21 +216,25 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
         return props;
     }
 
-    protected org.apache.falcon.oozie.coordinator.CONFIGURATION.Property createCoordProperty(String name, String value) {
-        org.apache.falcon.oozie.coordinator.CONFIGURATION.Property prop = new org.apache.falcon.oozie.coordinator.CONFIGURATION.Property();
+    protected org.apache.falcon.oozie.coordinator.CONFIGURATION.Property createCoordProperty(String name,
+                                                                                             String value) {
+        org.apache.falcon.oozie.coordinator.CONFIGURATION.Property prop
+                = new org.apache.falcon.oozie.coordinator.CONFIGURATION.Property();
         prop.setName(name);
         prop.setValue(value);
         return prop;
     }
 
     protected org.apache.falcon.oozie.bundle.CONFIGURATION.Property createBundleProperty(String name, String value) {
-        org.apache.falcon.oozie.bundle.CONFIGURATION.Property prop = new org.apache.falcon.oozie.bundle.CONFIGURATION.Property();
+        org.apache.falcon.oozie.bundle.CONFIGURATION.Property prop
+                = new org.apache.falcon.oozie.bundle.CONFIGURATION.Property();
         prop.setName(name);
         prop.setValue(value);
         return prop;
     }
 
-    protected void marshal(Cluster cluster, JAXBElement<?> jaxbElement, JAXBContext jaxbContext, Path outPath) throws FalconException {
+    protected void marshal(Cluster cluster, JAXBElement<?> jaxbElement, JAXBContext jaxbContext, Path outPath)
+            throws FalconException {
         try {
             Marshaller marshaller = jaxbContext.createMarshaller();
             marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
@@ -260,8 +270,9 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
     }
 
     protected String marshal(Cluster cluster, COORDINATORAPP coord, Path outPath, String name) throws FalconException {
-        if(StringUtils.isEmpty(name))
+        if (StringUtils.isEmpty(name)) {
             name = "coordinator";
+        }
         name = name + ".xml";
         marshal(cluster, new ObjectFactory().createCoordinatorApp(coord), coordJaxbContext, new Path(outPath, name));
         return name;
@@ -269,26 +280,30 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected void marshal(Cluster cluster, BUNDLEAPP bundle, Path outPath) throws FalconException {
 
-        marshal(cluster, new org.apache.falcon.oozie.bundle.ObjectFactory().createBundleApp(bundle), bundleJaxbContext, new Path(
-                outPath, "bundle.xml"));
+        marshal(cluster, new org.apache.falcon.oozie.bundle.ObjectFactory().createBundleApp(bundle), bundleJaxbContext,
+                new Path(
+                        outPath, "bundle.xml"));
     }
 
     protected void marshal(Cluster cluster, WORKFLOWAPP workflow, Path outPath) throws FalconException {
 
-        marshal(cluster, new org.apache.falcon.oozie.workflow.ObjectFactory().createWorkflowApp(workflow), workflowJaxbContext,
+        marshal(cluster, new org.apache.falcon.oozie.workflow.ObjectFactory().createWorkflowApp(workflow),
+                workflowJaxbContext,
                 new Path(outPath, "workflow.xml"));
     }
 
     protected String getStoragePath(Path path) {
-        if (path != null)
+        if (path != null) {
             return getStoragePath(path.toString());
+        }
         return null;
     }
 
     protected String getStoragePath(String path) {
         if (StringUtils.isNotEmpty(path)) {
-            if (new Path(path).toUri().getScheme()==null)
+            if (new Path(path).toUri().getScheme() == null) {
                 path = "${nameNode}" + path;
+            }
         }
         return path;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/logging/LogMover.java b/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
index 4d28ded..261fc53 100644
--- a/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
+++ b/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
@@ -17,190 +17,182 @@
  */
 package org.apache.falcon.logging;
 
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URL;
-import java.net.URLConnection;
-import java.util.List;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.*;
+import org.apache.falcon.entity.v0.EntityType;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobID;
-import org.apache.hadoop.mapred.RunningJob;
-import org.apache.hadoop.mapred.TaskCompletionEvent;
+import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.falcon.entity.v0.EntityType;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.OozieClientException;
 import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.client.WorkflowJob;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.List;
+
 public class LogMover extends Configured implements Tool {
 
-	private static final Logger LOG = Logger.getLogger(LogMover.class);
-
-	private static class ARGS {
-		String oozieUrl;
-		String subflowId;
-		String runId;
-		String logDir;
-		String status;
-		String entityType;
-	}
-
-	public static void main(String[] args) throws Exception {
-		ToolRunner.run(new LogMover(), args);
-	}
-
-	@Override
-	public int run(String[] arguments) throws Exception {
-		try {
-			ARGS args = new ARGS();
-			setupArgs(arguments, args);
-			OozieClient client = new OozieClient(args.oozieUrl);
-			WorkflowJob jobInfo = null;
-			try {
-				jobInfo = client.getJobInfo(args.subflowId);
-			} catch (OozieClientException e) {
-				LOG.error("Error getting jobinfo for: " + args.subflowId, e);
-				return 0;
-			}
-			Path path = new Path(args.logDir + "/"
-					+ String.format("%03d", Integer.parseInt(args.runId)));
-
-			FileSystem fs = path.getFileSystem(getConf());
-
-			if (args.entityType.equalsIgnoreCase(EntityType.FEED.name())) {
-				// if replication wf 
-				copyOozieLog(client, fs, path, jobInfo.getId());
-				copyTTlogs(args, fs, path, jobInfo.getActions().get(2));
-			} else {
-				// if process wf
-				String subflowId = jobInfo.getExternalId();
-				copyOozieLog(client, fs, path, subflowId);
-				WorkflowJob subflowInfo = client.getJobInfo(subflowId);
-				List<WorkflowAction> actions = subflowInfo.getActions();
-				for (WorkflowAction action : actions) {
-					if (action.getType().equals("pig")
-							|| action.getType().equals("java")) {
-						copyTTlogs(args, fs, path, action);
-					} else {
-						LOG.info("Ignoring hadoop TT log for non-pig and non-java action:"
-								+ action.getName());
-					}
-				}
-
-			}
-
-		} catch (Exception e) {
-			LOG.error("Exception in log mover:", e);
-		}
-		return 0;
-	}
-
-	private void copyOozieLog(OozieClient client, FileSystem fs, Path path,
-			String id) throws OozieClientException, IOException {
-		InputStream in = new ByteArrayInputStream(client.getJobLog(id).getBytes());
-		OutputStream out = fs.create(new Path(path, "oozie.log"));
-		IOUtils.copyBytes(in, out, 4096, true);
-		LOG.info("Copied oozie log to " + path);
-	}
-
-	private void copyTTlogs(ARGS args, FileSystem fs, Path path,
-			WorkflowAction action) throws Exception {
-		String ttLogURL = getTTlogURL(action.getExternalId());
-		if (ttLogURL != null) {
-			LOG.info("Fetching log for action: " + action.getExternalId()
-					+ " from url: " + ttLogURL);
-			InputStream in = getURLinputStream(new URL(ttLogURL));
-			OutputStream out = fs.create(new Path(path, action.getName() + "_"
-					+ getMappedStatus(action.getStatus()) + ".log"));
-			IOUtils.copyBytes(in, out, 4096, true);
-			LOG.info("Copied log to " + path);
-		}
-	}
-
-	private String getMappedStatus(WorkflowAction.Status status) {
-		if (status == WorkflowAction.Status.FAILED
-				|| status == WorkflowAction.Status.KILLED
-				|| status == WorkflowAction.Status.ERROR) {
-			return "FAILED";
-		} else {
-			return "SUCCEEDED";
-		}
-	}
-
-	private void setupArgs(String[] arguments, ARGS args) throws ParseException {
-		Options options = new Options();
-		Option opt;
-		opt = new Option("workflowEngineUrl", true,
-				"url of workflow engine, ex:oozie");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("subflowId", true, "external id of userworkflow");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("runId", true, "current workflow's runid");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("logDir", true, "log dir where job logs are stored");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("status", true, "user workflow status");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("entityType", true, "entity type feed or process");
-		opt.setRequired(true);
-		options.addOption(opt);
-
-		CommandLine cmd = new GnuParser().parse(options, arguments);
-
-		args.oozieUrl = cmd.getOptionValue("workflowEngineUrl");
-		args.subflowId = cmd.getOptionValue("subflowId");
-		args.runId = cmd.getOptionValue("runId");
-		args.logDir = cmd.getOptionValue("logDir");
-		args.status = cmd.getOptionValue("status");
-		args.entityType = cmd.getOptionValue("entityType");
-
-	}
-
-	private String getTTlogURL(String jobId) throws Exception {
-		JobConf jobConf = new JobConf(getConf());
-		JobClient jobClient = new JobClient(jobConf);
-		RunningJob job = jobClient.getJob(JobID.forName(jobId));
-		if (job == null) {
-			LOG.warn("No running job for job id: " + jobId);
-			return null;
-		}
-		TaskCompletionEvent[] tasks = job.getTaskCompletionEvents(0);
-		// 0th even is setup, 1 event is launcher, 2 event is cleanup
-		if (tasks != null && tasks.length == 3 && tasks[1] != null) {
-			return tasks[1].getTaskTrackerHttp() + "/tasklog?attemptid="
-					+ tasks[1].getTaskAttemptId() + "&all=true";
-		} else {
-			LOG.warn("No running task for job: " + jobId);
-		}
-		return null;
-	}
-
-	private InputStream getURLinputStream(URL url) throws IOException {
-		URLConnection connection = url.openConnection();
-		connection.setDoOutput(true);
-		connection.connect();
-		return connection.getInputStream();
-	}
+    private static final Logger LOG = Logger.getLogger(LogMover.class);
+
+    private static class ARGS {
+        String oozieUrl;
+        String subflowId;
+        String runId;
+        String logDir;
+        String status;
+        String entityType;
+    }
+
+    public static void main(String[] args) throws Exception {
+        ToolRunner.run(new LogMover(), args);
+    }
+
+    @Override
+    public int run(String[] arguments) throws Exception {
+        try {
+            ARGS args = new ARGS();
+            setupArgs(arguments, args);
+            OozieClient client = new OozieClient(args.oozieUrl);
+            WorkflowJob jobInfo = null;
+            try {
+                jobInfo = client.getJobInfo(args.subflowId);
+            } catch (OozieClientException e) {
+                LOG.error("Error getting jobinfo for: " + args.subflowId, e);
+                return 0;
+            }
+            Path path = new Path(args.logDir + "/"
+                    + String.format("%03d", Integer.parseInt(args.runId)));
+
+            FileSystem fs = path.getFileSystem(getConf());
+
+            if (args.entityType.equalsIgnoreCase(EntityType.FEED.name())) {
+                // if replication wf
+                copyOozieLog(client, fs, path, jobInfo.getId());
+                copyTTlogs(args, fs, path, jobInfo.getActions().get(2));
+            } else {
+                // if process wf
+                String subflowId = jobInfo.getExternalId();
+                copyOozieLog(client, fs, path, subflowId);
+                WorkflowJob subflowInfo = client.getJobInfo(subflowId);
+                List<WorkflowAction> actions = subflowInfo.getActions();
+                for (WorkflowAction action : actions) {
+                    if (action.getType().equals("pig")
+                            || action.getType().equals("java")) {
+                        copyTTlogs(args, fs, path, action);
+                    } else {
+                        LOG.info("Ignoring hadoop TT log for non-pig and non-java action:"
+                                + action.getName());
+                    }
+                }
+
+            }
+
+        } catch (Exception e) {
+            LOG.error("Exception in log mover:", e);
+        }
+        return 0;
+    }
+
+    private void copyOozieLog(OozieClient client, FileSystem fs, Path path,
+                              String id) throws OozieClientException, IOException {
+        InputStream in = new ByteArrayInputStream(client.getJobLog(id).getBytes());
+        OutputStream out = fs.create(new Path(path, "oozie.log"));
+        IOUtils.copyBytes(in, out, 4096, true);
+        LOG.info("Copied oozie log to " + path);
+    }
+
+    private void copyTTlogs(ARGS args, FileSystem fs, Path path,
+                            WorkflowAction action) throws Exception {
+        String ttLogURL = getTTlogURL(action.getExternalId());
+        if (ttLogURL != null) {
+            LOG.info("Fetching log for action: " + action.getExternalId()
+                    + " from url: " + ttLogURL);
+            InputStream in = getURLinputStream(new URL(ttLogURL));
+            OutputStream out = fs.create(new Path(path, action.getName() + "_"
+                    + getMappedStatus(action.getStatus()) + ".log"));
+            IOUtils.copyBytes(in, out, 4096, true);
+            LOG.info("Copied log to " + path);
+        }
+    }
+
+    private String getMappedStatus(WorkflowAction.Status status) {
+        if (status == WorkflowAction.Status.FAILED
+                || status == WorkflowAction.Status.KILLED
+                || status == WorkflowAction.Status.ERROR) {
+            return "FAILED";
+        } else {
+            return "SUCCEEDED";
+        }
+    }
+
+    private void setupArgs(String[] arguments, ARGS args) throws ParseException {
+        Options options = new Options();
+        Option opt;
+        opt = new Option("workflowEngineUrl", true,
+                "url of workflow engine, ex:oozie");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("subflowId", true, "external id of userworkflow");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("runId", true, "current workflow's runid");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("logDir", true, "log dir where job logs are stored");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("status", true, "user workflow status");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("entityType", true, "entity type feed or process");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        CommandLine cmd = new GnuParser().parse(options, arguments);
+
+        args.oozieUrl = cmd.getOptionValue("workflowEngineUrl");
+        args.subflowId = cmd.getOptionValue("subflowId");
+        args.runId = cmd.getOptionValue("runId");
+        args.logDir = cmd.getOptionValue("logDir");
+        args.status = cmd.getOptionValue("status");
+        args.entityType = cmd.getOptionValue("entityType");
+
+    }
+
+    private String getTTlogURL(String jobId) throws Exception {
+        JobConf jobConf = new JobConf(getConf());
+        JobClient jobClient = new JobClient(jobConf);
+        RunningJob job = jobClient.getJob(JobID.forName(jobId));
+        if (job == null) {
+            LOG.warn("No running job for job id: " + jobId);
+            return null;
+        }
+        TaskCompletionEvent[] tasks = job.getTaskCompletionEvents(0);
+        // 0th even is setup, 1 event is launcher, 2 event is cleanup
+        if (tasks != null && tasks.length == 3 && tasks[1] != null) {
+            return tasks[1].getTaskTrackerHttp() + "/tasklog?attemptid="
+                    + tasks[1].getTaskAttemptId() + "&all=true";
+        } else {
+            LOG.warn("No running task for job: " + jobId);
+        }
+        return null;
+    }
+
+    private InputStream getURLinputStream(URL url) throws IOException {
+        URLConnection connection = url.openConnection();
+        connection.setDoOutput(true);
+        connection.connect();
+        return connection.getInputStream();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
index 86b9b71..11eeadd 100644
--- a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
+++ b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
@@ -17,13 +17,7 @@
  */
 package org.apache.falcon.logging;
 
-import java.io.IOException;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
@@ -33,127 +27,133 @@ import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.resource.InstancesResult.Instance;
 import org.apache.falcon.resource.InstancesResult.InstanceAction;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.OozieClientException;
 import org.mortbay.log.Log;
 
+import java.io.IOException;
+
 public final class LogProvider {
-	private static final Logger LOG = Logger.getLogger(LogProvider.class);
-
-	public Instance populateLogUrls(Entity entity, Instance instance,
-			String runId) throws FalconException {
-
-		Cluster clusterObj = (Cluster) ConfigurationStore.get().get(
-				EntityType.CLUSTER, instance.cluster);
-		String resolvedRunId = "-";
-		try {
-			FileSystem fs = FileSystem.get(
-					new Path(ClusterHelper.getStorageUrl(clusterObj)).toUri(),
-					new Configuration());
-			resolvedRunId = getResolvedRunId(fs, clusterObj, entity, instance,
-					runId);
-			// if runId param is not resolved, i.e job is killed or not started
-			// or running
-			if (resolvedRunId.equals("-")
-					&& StringUtils.isEmpty(instance.logFile)) {
-				instance.logFile = "-";
-				return instance;
-			}
-			return populateActionLogUrls(fs, clusterObj, entity, instance,
-					resolvedRunId);
-		} catch (IOException e) {
-			LOG.warn("Exception while getting FS in LogProvider", e);
-		} catch (Exception e) {
-			LOG.warn("Exception in LogProvider while getting resolving run id",
-					e);
-		}
-		return instance;
-	}
-
-	public String getResolvedRunId(FileSystem fs, Cluster cluster,
-			Entity entity, Instance instance, String runId)
-			throws FalconException, IOException {
-		if (StringUtils.isEmpty(runId)) {
-			Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
-					EntityUtil.getLogPath(cluster, entity) + "/job-"
-							+ EntityUtil.UTCtoURIDate(instance.instance) + "/*");
-
-			FileStatus[] runs = fs.globStatus(jobPath);
-			if (runs.length > 0) {
-				// this is the latest run, dirs are sorted in increasing
-				// order of runs
-				return runs[runs.length - 1].getPath().getName();
-			} else {
-				LOG.warn("No run dirs are available in logs dir:" + jobPath);
-				return "-";
-			}
-		} else {
-			Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
-					EntityUtil.getLogPath(cluster, entity) + "/job-"
-							+ EntityUtil.UTCtoURIDate(instance.instance) + "/"
-							+ getFormatedRunId(runId));
-			if (fs.exists(jobPath)) {
-				return getFormatedRunId(runId);
-			} else {
-				Log.warn("No run dirs are available in logs dir:" + jobPath);
-				return "-";
-			}
-		}
-
-	}
-
-	private Instance populateActionLogUrls(FileSystem fs, Cluster cluster,
-			Entity entity, Instance instance, String formatedRunId)
-			throws FalconException, OozieClientException, IOException {
-
-		Path actionPaths = new Path(ClusterHelper.getStorageUrl(cluster),
-				EntityUtil.getLogPath(cluster, entity) + "/job-"
-						+ EntityUtil.UTCtoURIDate(instance.instance) + "/"
-						+ formatedRunId + "/*");
-		FileStatus[] actions = fs.globStatus(actionPaths);
-		InstanceAction[] instanceActions = new InstanceAction[actions.length - 1];
-		instance.actions = instanceActions;
-		int i = 0;
-		for (FileStatus file : actions) {
-			Path filePath = file.getPath();
-			String dfsBrowserUrl = getDFSbrowserUrl(
-					ClusterHelper.getStorageUrl(cluster),
-					EntityUtil.getLogPath(cluster, entity) + "/job-"
-							+ EntityUtil.UTCtoURIDate(instance.instance) + "/"
-							+ formatedRunId, file.getPath().getName());
-			if (filePath.getName().equals("oozie.log")) {
-				instance.logFile = dfsBrowserUrl;
-				continue;
-			}
-
-			InstanceAction instanceAction = new InstanceAction(
-					getActionName(filePath.getName()),
-					getActionStatus(filePath.getName()), dfsBrowserUrl);
-			instanceActions[i++] = instanceAction;
-		}
-
-		return instance;
-
-	}
-
-	private String getActionName(String fileName) {
-		return fileName.replaceAll("_SUCCEEDED.log", "").replaceAll(
-				"_FAILED.log", "");
-	}
-
-	private String getActionStatus(String fileName) {
-		return fileName.contains("_SUCCEEDED.log") ? "SUCCEEDED" : "FAILED";
-	}
-
-	private String getDFSbrowserUrl(String hdfsPath, String logPath,
-			String fileName) throws FalconException {
-		String scheme = new Path(hdfsPath).toUri().getScheme();
-		String httpUrl = hdfsPath.replaceAll(scheme+"://", "http://").replaceAll(
-				":[0-9]+", ":50070");
-		return new Path(httpUrl, "/data/"+logPath + "/" + fileName).toString();
-	}
-
-	private String getFormatedRunId(String runId) {
-		return String.format("%03d", Integer.parseInt(runId));
-	}
+    private static final Logger LOG = Logger.getLogger(LogProvider.class);
+
+    public Instance populateLogUrls(Entity entity, Instance instance,
+                                    String runId) throws FalconException {
+
+        Cluster clusterObj = (Cluster) ConfigurationStore.get().get(
+                EntityType.CLUSTER, instance.cluster);
+        String resolvedRunId = "-";
+        try {
+            FileSystem fs = FileSystem.get(
+                    new Path(ClusterHelper.getStorageUrl(clusterObj)).toUri(),
+                    new Configuration());
+            resolvedRunId = getResolvedRunId(fs, clusterObj, entity, instance,
+                    runId);
+            // if runId param is not resolved, i.e job is killed or not started
+            // or running
+            if (resolvedRunId.equals("-")
+                    && StringUtils.isEmpty(instance.logFile)) {
+                instance.logFile = "-";
+                return instance;
+            }
+            return populateActionLogUrls(fs, clusterObj, entity, instance,
+                    resolvedRunId);
+        } catch (IOException e) {
+            LOG.warn("Exception while getting FS in LogProvider", e);
+        } catch (Exception e) {
+            LOG.warn("Exception in LogProvider while getting resolving run id",
+                    e);
+        }
+        return instance;
+    }
+
+    public String getResolvedRunId(FileSystem fs, Cluster cluster,
+                                   Entity entity, Instance instance, String runId)
+            throws FalconException, IOException {
+        if (StringUtils.isEmpty(runId)) {
+            Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
+                    EntityUtil.getLogPath(cluster, entity) + "/job-"
+                            + EntityUtil.UTCtoURIDate(instance.instance) + "/*");
+
+            FileStatus[] runs = fs.globStatus(jobPath);
+            if (runs.length > 0) {
+                // this is the latest run, dirs are sorted in increasing
+                // order of runs
+                return runs[runs.length - 1].getPath().getName();
+            } else {
+                LOG.warn("No run dirs are available in logs dir:" + jobPath);
+                return "-";
+            }
+        } else {
+            Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
+                    EntityUtil.getLogPath(cluster, entity) + "/job-"
+                            + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                            + getFormatedRunId(runId));
+            if (fs.exists(jobPath)) {
+                return getFormatedRunId(runId);
+            } else {
+                Log.warn("No run dirs are available in logs dir:" + jobPath);
+                return "-";
+            }
+        }
+
+    }
+
+    private Instance populateActionLogUrls(FileSystem fs, Cluster cluster,
+                                           Entity entity, Instance instance, String formatedRunId)
+            throws FalconException, OozieClientException, IOException {
+
+        Path actionPaths = new Path(ClusterHelper.getStorageUrl(cluster),
+                EntityUtil.getLogPath(cluster, entity) + "/job-"
+                        + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                        + formatedRunId + "/*");
+        FileStatus[] actions = fs.globStatus(actionPaths);
+        InstanceAction[] instanceActions = new InstanceAction[actions.length - 1];
+        instance.actions = instanceActions;
+        int i = 0;
+        for (FileStatus file : actions) {
+            Path filePath = file.getPath();
+            String dfsBrowserUrl = getDFSbrowserUrl(
+                    ClusterHelper.getStorageUrl(cluster),
+                    EntityUtil.getLogPath(cluster, entity) + "/job-"
+                            + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                            + formatedRunId, file.getPath().getName());
+            if (filePath.getName().equals("oozie.log")) {
+                instance.logFile = dfsBrowserUrl;
+                continue;
+            }
+
+            InstanceAction instanceAction = new InstanceAction(
+                    getActionName(filePath.getName()),
+                    getActionStatus(filePath.getName()), dfsBrowserUrl);
+            instanceActions[i++] = instanceAction;
+        }
+
+        return instance;
+
+    }
+
+    private String getActionName(String fileName) {
+        return fileName.replaceAll("_SUCCEEDED.log", "").replaceAll(
+                "_FAILED.log", "");
+    }
+
+    private String getActionStatus(String fileName) {
+        return fileName.contains("_SUCCEEDED.log") ? "SUCCEEDED" : "FAILED";
+    }
+
+    private String getDFSbrowserUrl(String hdfsPath, String logPath,
+                                    String fileName) throws FalconException {
+        String scheme = new Path(hdfsPath).toUri().getScheme();
+        String httpUrl = hdfsPath.replaceAll(scheme + "://", "http://").replaceAll(
+                ":[0-9]+", ":50070");
+        return new Path(httpUrl, "/data/" + logPath + "/" + fileName).toString();
+    }
+
+    private String getFormatedRunId(String runId) {
+        return String.format("%03d", Integer.parseInt(runId));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java b/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
index 398369b..03bcea6 100644
--- a/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
+++ b/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
@@ -21,7 +21,7 @@ package org.apache.falcon.service;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 
-public interface FalconPathFilter extends PathFilter{
-    
+public interface FalconPathFilter extends PathFilter {
+
     String getJarName(Path path);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java b/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
index de006f1..841177a 100644
--- a/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
+++ b/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
@@ -18,13 +18,6 @@
 
 package org.apache.falcon.service;
 
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.v0.Entity;
@@ -33,26 +26,37 @@ import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.cluster.Interfacetype;
 import org.apache.falcon.util.DeploymentUtil;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+import java.io.File;
+import java.io.IOException;
+
 public class SharedLibraryHostingService implements ConfigurationChangeListener {
     private static Logger LOG = Logger.getLogger(SharedLibraryHostingService.class);
 
     private static final String[] LIBS = StartupProperties.get().getProperty("shared.libs").split(",");
-    
+
     private static final FalconPathFilter nonFalconJarFilter = new FalconPathFilter() {
         @Override
         public boolean accept(Path path) {
-            for(String jarName:LIBS) {
-                if(path.getName().startsWith(jarName))  return true;
+            for (String jarName : LIBS) {
+                if (path.getName().startsWith(jarName)) {
+                    return true;
+                }
             }
             return false;
         }
 
         @Override
         public String getJarName(Path path) {
-            for(String jarName:LIBS) {
-                if(path.getName().startsWith(jarName))  return jarName;
+            for (String jarName : LIBS) {
+                if (path.getName().startsWith(jarName)) {
+                    return jarName;
+                }
             }
             throw new IllegalArgumentException(path + " is not accepted!");
         }
@@ -67,36 +71,41 @@ public class SharedLibraryHostingService implements ConfigurationChangeListener
         }
     }
 
-    public static void pushLibsToHDFS(String path, Cluster cluster, FalconPathFilter pathFilter) throws IOException, FalconException {
+    public static void pushLibsToHDFS(String path, Cluster cluster, FalconPathFilter pathFilter)
+            throws IOException, FalconException {
         String localPaths = StartupProperties.get().getProperty("system.lib.location");
         assert localPaths != null && !localPaths.isEmpty() : "Invalid value for system.lib.location";
         if (!new File(localPaths).isDirectory()) {
-            throw new FalconException(localPaths + " configured for system.lib.location doesn't contain any valid libs");
+            throw new FalconException(
+                    localPaths + " configured for system.lib.location doesn't contain any valid libs");
         }
-        
+
         Configuration conf = ClusterHelper.getConfiguration(cluster);
         conf.setInt("ipc.client.connect.max.retries", 10);
-		FileSystem fs = null;
-		try {
-			fs = FileSystem.get(conf);
-		} catch (Exception e) {
-			throw new FalconException("Unable to connect to HDFS: "
-					+ ClusterHelper.getStorageUrl(cluster));
-		}
+        FileSystem fs = null;
+        try {
+            fs = FileSystem.get(conf);
+        } catch (Exception e) {
+            throw new FalconException("Unable to connect to HDFS: "
+                    + ClusterHelper.getStorageUrl(cluster));
+        }
         Path clusterPath = new Path(path);
-        if(!fs.exists(clusterPath))
+        if (!fs.exists(clusterPath)) {
             fs.mkdirs(clusterPath);
-            
+        }
+
         for (File localFile : new File(localPaths).listFiles()) {
             Path localPath = new Path(localFile.getAbsolutePath());
-            if (!pathFilter.accept(localPath))
+            if (!pathFilter.accept(localPath)) {
                 continue;
+            }
 
             Path clusterFile = new Path(path, pathFilter.getJarName(localPath) + ".jar");
             if (fs.exists(clusterFile)) {
                 FileStatus fstat = fs.getFileStatus(clusterFile);
-                if (fstat.getLen() == localFile.length())
+                if (fstat.getLen() == localFile.length()) {
                     continue;
+                }
             }
             fs.copyFromLocalFile(false, true, new Path(localFile.getAbsolutePath()), clusterFile);
             LOG.info("Copied " + localFile.getAbsolutePath() + " to " + path + " in " + fs.getUri());
@@ -105,13 +114,15 @@ public class SharedLibraryHostingService implements ConfigurationChangeListener
 
     @Override
     public void onAdd(Entity entity) throws FalconException {
-        if (entity.getEntityType() != EntityType.CLUSTER)
+        if (entity.getEntityType() != EntityType.CLUSTER) {
             return;
+        }
 
         Cluster cluster = (Cluster) entity;
         String currentColo = DeploymentUtil.getCurrentColo();
-        if (DeploymentUtil.isEmbeddedMode() || currentColo.equals(cluster.getColo()))
+        if (DeploymentUtil.isEmbeddedMode() || currentColo.equals(cluster.getColo())) {
             addLibsTo(cluster);
+        }
     }
 
     @Override
@@ -121,14 +132,15 @@ public class SharedLibraryHostingService implements ConfigurationChangeListener
 
     @Override
     public void onChange(Entity oldEntity, Entity newEntity) throws FalconException {
-        if (oldEntity.getEntityType() != EntityType.CLUSTER)
+        if (oldEntity.getEntityType() != EntityType.CLUSTER) {
             return;
+        }
         Cluster oldCluster = (Cluster) oldEntity;
         Cluster newCluster = (Cluster) newEntity;
         if (!ClusterHelper.getInterface(oldCluster, Interfacetype.WRITE).getEndpoint()
                 .equals(ClusterHelper.getInterface(newCluster, Interfacetype.WRITE).getEndpoint())
                 || !ClusterHelper.getInterface(oldCluster, Interfacetype.WORKFLOW).getEndpoint()
-                        .equals(ClusterHelper.getInterface(newCluster, Interfacetype.WORKFLOW).getEndpoint())) {
+                .equals(ClusterHelper.getInterface(newCluster, Interfacetype.WORKFLOW).getEndpoint())) {
             addLibsTo(newCluster);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java b/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
index 62afde1..20ef6bf 100644
--- a/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
+++ b/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
@@ -17,12 +17,12 @@
  */
 package org.apache.falcon.util;
 
+import org.apache.hadoop.conf.Configuration;
+
 import java.io.ByteArrayInputStream;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.conf.Configuration;
-
 public final class OozieUtils {
 
     public static Properties toProperties(String properties) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
index fe41861..4c99bfc 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
@@ -17,192 +17,189 @@
  */
 package org.apache.falcon.workflow;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.*;
+import org.apache.falcon.logging.LogMover;
+import org.apache.falcon.messaging.MessageProducer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.falcon.logging.LogMover;
-import org.apache.falcon.messaging.MessageProducer;
 import org.apache.log4j.Logger;
-import org.mortbay.log.Log;
-
-public class FalconPostProcessing extends Configured implements Tool{
-	private static final Logger LOG = Logger.getLogger(FalconPostProcessing.class);
-
-	public enum Arg{
-		CLUSTER("cluster","name of the current cluster"),
-		ENTITY_TYPE("entityType","type of the entity"),
-		ENTITY_NAME("entityName","name of the entity"),
-		NOMINAL_TIME("nominalTime","instance time"),
-		OPERATION("operation","operation like generate, delete, replicate"),
-		WORKFLOW_ID("workflowId","current workflow-id of the instance"),
-		RUN_ID("runId","current run-id of the instance"),
-		STATUS("status","status of the user workflow isnstance"),
-		TIMESTAMP("timeStamp","current timestamp"),
-		TOPIC_NAME("topicName","name of the topic to be used to send JMS message"),
-		BRKR_IMPL_CLASS("brokerImplClass","falcon message broker Implementation class"),
-		BRKR_URL("brokerUrl","falcon message broker url"),
-		USER_BRKR_IMPL_CLASS("userBrokerImplClass","user broker Impl class"),
-		USER_BRKR_URL("userBrokerUrl","user broker url"),
-		BRKR_TTL("brokerTTL", "time to live for broker message in sec"),
-		FEED_NAMES("feedNames", "name of the feeds which are generated/replicated/deleted"),
-		FEED_INSTANCE_PATHS("feedInstancePaths","comma seperated feed instance paths"),
-		LOG_FILE("logFile","log file path where feeds to be deleted are recorded"),
-		WF_ENGINE_URL("workflowEngineUrl","url of workflow engine server, ex:oozie"),
-		USER_SUBFLOW_ID("subflowId","external id of user workflow"),
-		LOG_DIR("logDir","log dir where job logs are copied");		
-
-		private String name;
-		private String description;
-		Arg(String name, String description){
-			this.name=name;
-			this.description=description;
-		}
-
-		public Option getOption() {
-			return new Option(this.name, true, this.description);
-		}
-
-		public String getOptionName(){
-			return this.name;
-		}
-
-		public String getOptionValue(CommandLine cmd){
-			return cmd.getOptionValue(this.name);
-		}
-
-	}
-
-	public static void main(String[] args) throws Exception {
-		ToolRunner.run(new Configuration(), new FalconPostProcessing(), args);
-	}
-
-	@Override
-	public int run(String[] args) throws Exception {
-
-		CommandLine cmd = getCommand(args);
-
-		LOG.info("Sending user message "+cmd);
-		invokeUserMessageProducer(cmd);
-		//LogMover doesnt throw exception, a failed logmover will not fail the user workflow
-		LOG.info("Moving logs "+cmd);
-		invokeLogProducer(cmd);
-		LOG.info("Sending falcon message "+cmd);
-		invokeFalconMessageProducer(cmd);
-
-		return 0;
-	}
-
-	private void invokeUserMessageProducer(CommandLine cmd) throws Exception {
-		List<String> args = new ArrayList<String>();
-		addArg(args,cmd, Arg.CLUSTER);
-		addArg(args,cmd, Arg.ENTITY_TYPE);
-		addArg(args,cmd,Arg.ENTITY_NAME);
-		addArg(args,cmd,Arg.NOMINAL_TIME);
-		addArg(args,cmd,Arg.OPERATION);
-		addArg(args,cmd,Arg.WORKFLOW_ID);
-		addArg(args,cmd,Arg.RUN_ID);
-		addArg(args,cmd,Arg.STATUS);
-		addArg(args,cmd,Arg.TIMESTAMP);
-		//special args for user JMS message producer
-		args.add("-"+Arg.TOPIC_NAME.getOptionName()); //user topic
-		args.add("FALCON."+Arg.ENTITY_NAME.getOptionValue(cmd));
-		//note, the user broker impl class arg name to MessageProducer is brokerImplClass 
-		args.add("-"+Arg.BRKR_IMPL_CLASS.getOptionName());
-		args.add(Arg.USER_BRKR_IMPL_CLASS.getOptionValue(cmd));
-		args.add("-"+Arg.BRKR_URL.getOptionName());
-		args.add(Arg.USER_BRKR_URL.getOptionValue(cmd));
-		addArg(args,cmd,Arg.BRKR_TTL);
-		addArg(args,cmd,Arg.FEED_NAMES);
-		addArg(args,cmd,Arg.FEED_INSTANCE_PATHS);
-		addArg(args,cmd,Arg.LOG_FILE);
-
-		MessageProducer.main(args.toArray(new String[0]));
-	}
-
-	private void invokeFalconMessageProducer(CommandLine cmd) throws Exception {
-		List<String> args = new ArrayList<String>();
-		addArg(args,cmd, Arg.CLUSTER);
-		addArg(args,cmd, Arg.ENTITY_TYPE);
-		addArg(args,cmd,Arg.ENTITY_NAME);
-		addArg(args,cmd,Arg.NOMINAL_TIME);
-		addArg(args,cmd,Arg.OPERATION);
-		addArg(args,cmd,Arg.WORKFLOW_ID);
-		addArg(args,cmd,Arg.RUN_ID);
-		addArg(args,cmd,Arg.STATUS);
-		addArg(args,cmd,Arg.TIMESTAMP);
-		//special args Falcon JMS message producer
-		args.add("-"+Arg.TOPIC_NAME.getOptionName());
-		args.add("FALCON.ENTITY.TOPIC");
-		args.add("-"+Arg.BRKR_IMPL_CLASS.getOptionName());
-		args.add(Arg.BRKR_IMPL_CLASS.getOptionValue(cmd));
-		args.add("-"+Arg.BRKR_URL.getOptionName());
-		args.add(Arg.BRKR_URL.getOptionValue(cmd));
-		addArg(args,cmd,Arg.BRKR_TTL);
-		addArg(args,cmd,Arg.FEED_NAMES);
-		addArg(args,cmd,Arg.FEED_INSTANCE_PATHS);
-		addArg(args,cmd,Arg.LOG_FILE);
-
-		MessageProducer.main(args.toArray(new String[0]));
-
-	}
-	private void invokeLogProducer(CommandLine cmd) throws Exception{
-		List<String> args = new ArrayList<String>();
-		addArg(args,cmd,Arg.WF_ENGINE_URL);
-		addArg(args,cmd,Arg.ENTITY_TYPE);
-		addArg(args,cmd,Arg.USER_SUBFLOW_ID);
-		addArg(args,cmd,Arg.RUN_ID);
-		addArg(args,cmd,Arg.LOG_DIR);
-		addArg(args,cmd,Arg.STATUS);
-
-		LogMover.main(args.toArray(new String[0]));
-
-	}
-
-	private void addArg(List<String> args, CommandLine cmd, Arg arg) {
-		args.add("-"+arg.getOptionName());
-		args.add(arg.getOptionValue(cmd));
-	}
-
-	private static CommandLine getCommand(String[] arguments)
-			throws ParseException {
-		Options options = new Options();
-		addOption(options, Arg.CLUSTER);
-		addOption(options, Arg.ENTITY_TYPE);
-		addOption(options, Arg.ENTITY_NAME);
-		addOption(options, Arg.NOMINAL_TIME);
-		addOption(options, Arg.OPERATION);
-		addOption(options, Arg.WORKFLOW_ID);
-		addOption(options, Arg.RUN_ID);
-		addOption(options, Arg.STATUS);
-		addOption(options, Arg.TIMESTAMP);
-		addOption(options, Arg.BRKR_IMPL_CLASS);
-		addOption(options, Arg.BRKR_URL);
-		addOption(options, Arg.USER_BRKR_IMPL_CLASS);
-		addOption(options, Arg.USER_BRKR_URL);
-		addOption(options, Arg.BRKR_TTL);
-		addOption(options, Arg.FEED_NAMES);
-		addOption(options, Arg.FEED_INSTANCE_PATHS);
-		addOption(options, Arg.LOG_FILE);
-		addOption(options,Arg.WF_ENGINE_URL);
-		addOption(options, Arg.USER_SUBFLOW_ID);
-		addOption(options, Arg.LOG_DIR);
-		return new GnuParser().parse(options, arguments);
-	}
-
-	private static void addOption(Options options, Arg arg) {
-		Option option = arg.getOption();
-		option.setRequired(true);
-		options.addOption(option);
-	}
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class FalconPostProcessing extends Configured implements Tool {
+    private static final Logger LOG = Logger.getLogger(FalconPostProcessing.class);
+
+    public enum Arg {
+        CLUSTER("cluster", "name of the current cluster"),
+        ENTITY_TYPE("entityType", "type of the entity"),
+        ENTITY_NAME("entityName", "name of the entity"),
+        NOMINAL_TIME("nominalTime", "instance time"),
+        OPERATION("operation", "operation like generate, delete, replicate"),
+        WORKFLOW_ID("workflowId", "current workflow-id of the instance"),
+        RUN_ID("runId", "current run-id of the instance"),
+        STATUS("status", "status of the user workflow isnstance"),
+        TIMESTAMP("timeStamp", "current timestamp"),
+        TOPIC_NAME("topicName", "name of the topic to be used to send JMS message"),
+        BRKR_IMPL_CLASS("brokerImplClass", "falcon message broker Implementation class"),
+        BRKR_URL("brokerUrl", "falcon message broker url"),
+        USER_BRKR_IMPL_CLASS("userBrokerImplClass", "user broker Impl class"),
+        USER_BRKR_URL("userBrokerUrl", "user broker url"),
+        BRKR_TTL("brokerTTL", "time to live for broker message in sec"),
+        FEED_NAMES("feedNames", "name of the feeds which are generated/replicated/deleted"),
+        FEED_INSTANCE_PATHS("feedInstancePaths", "comma seperated feed instance paths"),
+        LOG_FILE("logFile", "log file path where feeds to be deleted are recorded"),
+        WF_ENGINE_URL("workflowEngineUrl", "url of workflow engine server, ex:oozie"),
+        USER_SUBFLOW_ID("subflowId", "external id of user workflow"),
+        LOG_DIR("logDir", "log dir where job logs are copied");
+
+        private String name;
+        private String description;
+
+        Arg(String name, String description) {
+            this.name = name;
+            this.description = description;
+        }
+
+        public Option getOption() {
+            return new Option(this.name, true, this.description);
+        }
+
+        public String getOptionName() {
+            return this.name;
+        }
+
+        public String getOptionValue(CommandLine cmd) {
+            return cmd.getOptionValue(this.name);
+        }
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        ToolRunner.run(new Configuration(), new FalconPostProcessing(), args);
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+        CommandLine cmd = getCommand(args);
+
+        LOG.info("Sending user message " + cmd);
+        invokeUserMessageProducer(cmd);
+        //LogMover doesnt throw exception, a failed logmover will not fail the user workflow
+        LOG.info("Moving logs " + cmd);
+        invokeLogProducer(cmd);
+        LOG.info("Sending falcon message " + cmd);
+        invokeFalconMessageProducer(cmd);
+
+        return 0;
+    }
+
+    private void invokeUserMessageProducer(CommandLine cmd) throws Exception {
+        List<String> args = new ArrayList<String>();
+        addArg(args, cmd, Arg.CLUSTER);
+        addArg(args, cmd, Arg.ENTITY_TYPE);
+        addArg(args, cmd, Arg.ENTITY_NAME);
+        addArg(args, cmd, Arg.NOMINAL_TIME);
+        addArg(args, cmd, Arg.OPERATION);
+        addArg(args, cmd, Arg.WORKFLOW_ID);
+        addArg(args, cmd, Arg.RUN_ID);
+        addArg(args, cmd, Arg.STATUS);
+        addArg(args, cmd, Arg.TIMESTAMP);
+        //special args for user JMS message producer
+        args.add("-" + Arg.TOPIC_NAME.getOptionName()); //user topic
+        args.add("FALCON." + Arg.ENTITY_NAME.getOptionValue(cmd));
+        //note, the user broker impl class arg name to MessageProducer is brokerImplClass
+        args.add("-" + Arg.BRKR_IMPL_CLASS.getOptionName());
+        args.add(Arg.USER_BRKR_IMPL_CLASS.getOptionValue(cmd));
+        args.add("-" + Arg.BRKR_URL.getOptionName());
+        args.add(Arg.USER_BRKR_URL.getOptionValue(cmd));
+        addArg(args, cmd, Arg.BRKR_TTL);
+        addArg(args, cmd, Arg.FEED_NAMES);
+        addArg(args, cmd, Arg.FEED_INSTANCE_PATHS);
+        addArg(args, cmd, Arg.LOG_FILE);
+
+        MessageProducer.main(args.toArray(new String[0]));
+    }
+
+    private void invokeFalconMessageProducer(CommandLine cmd) throws Exception {
+        List<String> args = new ArrayList<String>();
+        addArg(args, cmd, Arg.CLUSTER);
+        addArg(args, cmd, Arg.ENTITY_TYPE);
+        addArg(args, cmd, Arg.ENTITY_NAME);
+        addArg(args, cmd, Arg.NOMINAL_TIME);
+        addArg(args, cmd, Arg.OPERATION);
+        addArg(args, cmd, Arg.WORKFLOW_ID);
+        addArg(args, cmd, Arg.RUN_ID);
+        addArg(args, cmd, Arg.STATUS);
+        addArg(args, cmd, Arg.TIMESTAMP);
+        //special args Falcon JMS message producer
+        args.add("-" + Arg.TOPIC_NAME.getOptionName());
+        args.add("FALCON.ENTITY.TOPIC");
+        args.add("-" + Arg.BRKR_IMPL_CLASS.getOptionName());
+        args.add(Arg.BRKR_IMPL_CLASS.getOptionValue(cmd));
+        args.add("-" + Arg.BRKR_URL.getOptionName());
+        args.add(Arg.BRKR_URL.getOptionValue(cmd));
+        addArg(args, cmd, Arg.BRKR_TTL);
+        addArg(args, cmd, Arg.FEED_NAMES);
+        addArg(args, cmd, Arg.FEED_INSTANCE_PATHS);
+        addArg(args, cmd, Arg.LOG_FILE);
+
+        MessageProducer.main(args.toArray(new String[0]));
+
+    }
+
+    private void invokeLogProducer(CommandLine cmd) throws Exception {
+        List<String> args = new ArrayList<String>();
+        addArg(args, cmd, Arg.WF_ENGINE_URL);
+        addArg(args, cmd, Arg.ENTITY_TYPE);
+        addArg(args, cmd, Arg.USER_SUBFLOW_ID);
+        addArg(args, cmd, Arg.RUN_ID);
+        addArg(args, cmd, Arg.LOG_DIR);
+        addArg(args, cmd, Arg.STATUS);
+
+        LogMover.main(args.toArray(new String[0]));
+
+    }
+
+    private void addArg(List<String> args, CommandLine cmd, Arg arg) {
+        args.add("-" + arg.getOptionName());
+        args.add(arg.getOptionValue(cmd));
+    }
+
+    private static CommandLine getCommand(String[] arguments)
+            throws ParseException {
+        Options options = new Options();
+        addOption(options, Arg.CLUSTER);
+        addOption(options, Arg.ENTITY_TYPE);
+        addOption(options, Arg.ENTITY_NAME);
+        addOption(options, Arg.NOMINAL_TIME);
+        addOption(options, Arg.OPERATION);
+        addOption(options, Arg.WORKFLOW_ID);
+        addOption(options, Arg.RUN_ID);
+        addOption(options, Arg.STATUS);
+        addOption(options, Arg.TIMESTAMP);
+        addOption(options, Arg.BRKR_IMPL_CLASS);
+        addOption(options, Arg.BRKR_URL);
+        addOption(options, Arg.USER_BRKR_IMPL_CLASS);
+        addOption(options, Arg.USER_BRKR_URL);
+        addOption(options, Arg.BRKR_TTL);
+        addOption(options, Arg.FEED_NAMES);
+        addOption(options, Arg.FEED_INSTANCE_PATHS);
+        addOption(options, Arg.LOG_FILE);
+        addOption(options, Arg.WF_ENGINE_URL);
+        addOption(options, Arg.USER_SUBFLOW_ID);
+        addOption(options, Arg.LOG_DIR);
+        return new GnuParser().parse(options, arguments);
+    }
+
+    private static void addOption(Options options, Arg arg) {
+        Option option = arg.getOption();
+        option.setRequired(true);
+        options.addOption(option);
+    }
 
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java b/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
index 85c8697..8d67a9b 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
@@ -18,11 +18,6 @@
 
 package org.apache.falcon.workflow;
 
-import java.util.Date;
-import java.util.List;
-import java.util.Properties;
-
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
@@ -32,22 +27,27 @@ import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.cluster.Property;
 import org.apache.falcon.workflow.engine.OozieWorkflowEngine;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.OozieClient;
 
+import java.util.Date;
+import java.util.List;
+import java.util.Properties;
+
 public abstract class OozieWorkflowBuilder<T extends Entity> extends WorkflowBuilder<T> {
-    
+
     private static Logger LOG = Logger.getLogger(OozieWorkflowBuilder.class);
     protected static final ConfigurationStore configStore = ConfigurationStore.get();
-    
+
     protected Properties createAppProperties(String clusterName, Path bundlePath, String user) throws FalconException {
 
         Cluster cluster = EntityUtil.getEntity(EntityType.CLUSTER, clusterName);
         Properties properties = new Properties();
-		if (cluster.getProperties() != null) {
-			addClusterProperties(properties, cluster.getProperties()
-					.getProperties());
-		}
+        if (cluster.getProperties() != null) {
+            addClusterProperties(properties, cluster.getProperties()
+                    .getProperties());
+        }
         properties.setProperty(OozieWorkflowEngine.NAME_NODE,
                 ClusterHelper.getStorageUrl(cluster));
         properties.setProperty(OozieWorkflowEngine.JOB_TRACKER,
@@ -55,20 +55,20 @@ public abstract class OozieWorkflowBuilder<T extends Entity> extends WorkflowBui
         properties.setProperty(OozieClient.BUNDLE_APP_PATH,
                 "${" + OozieWorkflowEngine.NAME_NODE + "}" + bundlePath.toString());
         properties.setProperty("colo.name", cluster.getColo());
-        
+
         properties.setProperty(OozieClient.USER_NAME, user);
         properties.setProperty(OozieClient.USE_SYSTEM_LIBPATH, "true");
         properties.setProperty("falcon.libpath", ClusterHelper.getLocation(cluster, "working") + "/lib");
         LOG.info("Cluster: " + cluster.getName() + ", PROPS: " + properties);
         return properties;
     }
-    
-	private void addClusterProperties(Properties properties,
-			List<Property> clusterProperties) {
-		for (Property prop : clusterProperties) {
-			properties.setProperty(prop.getName(), prop.getValue());
-		}
-	}
 
-	public abstract Date getNextStartTime(T entity, String cluster, Date now) throws FalconException;
+    private void addClusterProperties(Properties properties,
+                                      List<Property> clusterProperties) {
+        for (Property prop : clusterProperties) {
+            properties.setProperty(prop.getName(), prop.getValue());
+        }
+    }
+
+    public abstract Date getNextStartTime(T entity, String cluster, Date now) throws FalconException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
index 985b297..f2c9ac3 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
@@ -18,13 +18,13 @@
 
 package org.apache.falcon.workflow.engine;
 
-import java.util.Date;
-import java.util.List;
-
 import org.apache.oozie.client.BundleJob;
 import org.apache.oozie.client.CoordinatorJob;
 
-public class NullBundleJob implements BundleJob{
+import java.util.Date;
+import java.util.List;
+
+public class NullBundleJob implements BundleJob {
 
     @Override
     public String getAppPath() {


[36/47] git commit: Impacted changes due to checkstyle fixes in oozie module

Posted by sr...@apache.org.
Impacted changes due to checkstyle fixes in oozie module


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

Branch: refs/heads/master
Commit: 8dbbd5caffb4fe4886cf796be8c765c04141c1c5
Parents: 9b4d845
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:47:37 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:47:37 2013 +0530

----------------------------------------------------------------------
 .../workflow/OozieProcessWorkflowBuilder.java      |    2 +-
 1 files changed, 1 insertions(+), 1 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/8dbbd5ca/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
index 76b5cb4..e422ec4 100644
--- a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
+++ b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
@@ -105,7 +105,7 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
             return null;
         }
 
-        Cluster cluster = configStore.get(EntityType.CLUSTER, processCluster.getName());
+        Cluster cluster = CONFIG_STORE.get(EntityType.CLUSTER, processCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(process));
         Process processClone = (Process) process.copy();
         EntityUtil.setStartDate(processClone, clusterName, startDate);


[31/47] git commit: More check style fixes relating to client module

Posted by sr...@apache.org.
More check style fixes relating to client module


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

Branch: refs/heads/master
Commit: 968f197dff0053fc91505432d356fff334901198
Parents: 6119bed
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:46:10 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:46:10 2013 +0530

----------------------------------------------------------------------
 client/src/main/java/org/apache/falcon/Pair.java   |    4 ++--
 .../org/apache/falcon/client/FalconClient.java     |    8 ++++----
 .../org/apache/falcon/resource/EntityList.java     |    4 ++--
 .../apache/falcon/resource/InstancesResult.java    |    4 ++--
 4 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/968f197d/client/src/main/java/org/apache/falcon/Pair.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/Pair.java b/client/src/main/java/org/apache/falcon/Pair.java
index e6b44e9..e1f8bc3 100644
--- a/client/src/main/java/org/apache/falcon/Pair.java
+++ b/client/src/main/java/org/apache/falcon/Pair.java
@@ -25,10 +25,10 @@ package org.apache.falcon;
  */
 public class Pair<A, B> {
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     public final A first;
     public final B second;
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     public Pair(A fst, B snd) {
         this.first = fst;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/968f197d/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index 30c0ec6..cf92b84 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -308,7 +308,7 @@ public class FalconClient {
                 getServletInputStream(clusters, sourceClusters, null), null, colo);
     }
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     public String rerunInstances(String type, String entity, String start,
                                  String end, String filePath, String colo,
                                  String clusters, String sourceClusters)
@@ -327,7 +327,7 @@ public class FalconClient {
         return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, temp), null, colo);
     }
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     public String rerunInstances(String type, String entity, String start,
                                  String end, String colo, String clusters, String sourceClusters)
@@ -503,7 +503,7 @@ public class FalconClient {
                 .method(instances.method, InstancesResult.class);
     }
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     private String sendInstanceRequest(Instances instances, String type,
                                        String entity, String start, String end, InputStream props,
                                        String runid, String colo) throws FalconCLIException {
@@ -541,7 +541,7 @@ public class FalconClient {
         }
 
     }
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     private String sendAdminRequest(AdminOperations job)
         throws FalconCLIException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/968f197d/client/src/main/java/org/apache/falcon/resource/EntityList.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntityList.java b/client/src/main/java/org/apache/falcon/resource/EntityList.java
index 933a737..0344f2e 100644
--- a/client/src/main/java/org/apache/falcon/resource/EntityList.java
+++ b/client/src/main/java/org/apache/falcon/resource/EntityList.java
@@ -39,12 +39,12 @@ public class EntityList {
      * Element within an entity.
      */
     public static class EntityElement {
-        //SUSPEND CHECKSTYLE CHECK
+        //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
         @XmlElement
         public String type;
         @XmlElement
         public String name;
-        //RESUME CHECKSTYLE CHECK
+        //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
         public EntityElement() {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/968f197d/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
index 65a8b65..1822273 100644
--- a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
@@ -25,7 +25,7 @@ import java.util.Date;
 /**
  * Pojo for JAXB marshalling / unmarshalling.
  */
-//SUSPEND CHECKSTYLE CHECK
+//SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
 @XmlRootElement
 public class InstancesResult extends APIResult {
 
@@ -199,4 +199,4 @@ public class InstancesResult extends APIResult {
         }
     }
 }
-//RESUME CHECKSTYLE CHECK
+//RESUME CHECKSTYLE CHECK VisibilityModifierCheck


[11/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
index 739d655..9a41d33 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
@@ -18,11 +18,6 @@
 
 package org.apache.falcon.resource;
 
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.Response;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconWebException;
 import org.apache.falcon.entity.EntityUtil;
@@ -32,6 +27,11 @@ import org.apache.falcon.entity.v0.UnschedulableEntityException;
 import org.apache.falcon.monitors.Dimension;
 import org.apache.log4j.Logger;
 
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+
 /**
  * REST resource of allowed actions on Schedulable Entities Only Process and
  * Feed can have schedulable actions
@@ -42,13 +42,15 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
 
     /**
      * Schedules an submitted entity immediately
-     * 
+     *
      * @param type
      * @param entity
      * @return APIResult
      */
-    public APIResult schedule(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
-            @Dimension("entityName") @PathParam("entity") String entity, @Dimension("colo") @PathParam("colo") String colo) {
+    public APIResult schedule(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
+            @Dimension("entityName") @PathParam("entity") String entity,
+            @Dimension("colo") @PathParam("colo") String colo) {
         checkColo(colo);
         try {
             audit(request, entity, type, "SCHEDULED");
@@ -68,11 +70,12 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
 
     /**
      * Submits a new entity and schedules it immediately
-     * 
+     *
      * @param type
      * @return
      */
-    public APIResult submitAndSchedule(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
+    public APIResult submitAndSchedule(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
             @Dimension("colo") @PathParam("colo") String colo) {
         checkColo(colo);
         try {
@@ -80,7 +83,8 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
             audit(request, "STREAMED_DATA", type, "SUBMIT_AND_SCHEDULE");
             Entity entity = submitInternal(request, type);
             scheduleInternal(type, entity.getName());
-            return new APIResult(APIResult.Status.SUCCEEDED, entity.getName() + "(" + type + ") scheduled successfully");
+            return new APIResult(APIResult.Status.SUCCEEDED,
+                    entity.getName() + "(" + type + ") scheduled successfully");
         } catch (Throwable e) {
             LOG.error("Unable to submit and schedule ", e);
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
@@ -89,22 +93,25 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
 
     /**
      * Suspends a running entity
-     * 
+     *
      * @param type
      * @param entity
      * @return APIResult
      */
-    public APIResult suspend(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
-            @Dimension("entityName") @PathParam("entity") String entity, @Dimension("entityName") @PathParam("entity") String colo) {
+    public APIResult suspend(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
+            @Dimension("entityName") @PathParam("entity") String entity,
+            @Dimension("entityName") @PathParam("entity") String colo) {
         checkColo(colo);
         try {
             checkSchedulableEntity(type);
             audit(request, entity, type, "SUSPEND");
             Entity entityObj = EntityUtil.getEntity(type, entity);
-            if (getWorkflowEngine().isActive(entityObj))
+            if (getWorkflowEngine().isActive(entityObj)) {
                 getWorkflowEngine().suspend(entityObj);
-            else
+            } else {
                 throw new FalconException(entity + "(" + type + ") is not scheduled");
+            }
             return new APIResult(APIResult.Status.SUCCEEDED, entity + "(" + type + ") suspended successfully");
         } catch (Throwable e) {
             LOG.error("Unable to suspend entity", e);
@@ -114,23 +121,26 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
 
     /**
      * Resumes a suspended entity
-     * 
+     *
      * @param type
      * @param entity
      * @return APIResult
      */
-    public APIResult resume(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
-            @Dimension("entityName") @PathParam("entity") String entity, @Dimension("colo") @PathParam("colo") String colo) {
+    public APIResult resume(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
+            @Dimension("entityName") @PathParam("entity") String entity,
+            @Dimension("colo") @PathParam("colo") String colo) {
 
         checkColo(colo);
         try {
             checkSchedulableEntity(type);
             audit(request, entity, type, "RESUME");
             Entity entityObj = EntityUtil.getEntity(type, entity);
-            if (getWorkflowEngine().isActive(entityObj))
+            if (getWorkflowEngine().isActive(entityObj)) {
                 getWorkflowEngine().resume(entityObj);
-            else
+            } else {
                 throw new FalconException(entity + "(" + type + ") is not scheduled");
+            }
             return new APIResult(APIResult.Status.SUCCEEDED, entity + "(" + type + ") resumed successfully");
         } catch (Throwable e) {
             LOG.error("Unable to resume entity", e);
@@ -141,7 +151,8 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
     private void checkSchedulableEntity(String type) throws UnschedulableEntityException {
         EntityType entityType = EntityType.valueOf(type.toUpperCase());
         if (!entityType.isSchedulable()) {
-            throw new UnschedulableEntityException("Entity type (" + type + ") " + " cannot be Scheduled/Suspended/Resumed");
+            throw new UnschedulableEntityException(
+                    "Entity type (" + type + ") " + " cannot be Scheduled/Suspended/Resumed");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
index 47c70ed..7b15b96 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
@@ -30,7 +30,7 @@ public abstract class AbstractChannel implements Channel {
 
     protected Method getMethod(Class service, String methodName, Object... args)
             throws FalconException {
-        MethodKey methodKey = new MethodKey(methodName,  args);
+        MethodKey methodKey = new MethodKey(methodName, args);
         Method method = methods.get(methodKey);
         if (method == null) {
             for (Method item : service.getDeclaredMethods()) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
index 64d39ba..9873a96 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
@@ -18,25 +18,9 @@
 
 package org.apache.falcon.resource.channel;
 
-import java.lang.annotation.Annotation;
-import java.lang.reflect.Method;
-import java.util.Properties;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.Response.Status.Family;
-import javax.ws.rs.core.UriBuilder;
-
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.resource.proxy.BufferedRequest;
 import org.apache.falcon.security.CurrentUser;
@@ -44,9 +28,15 @@ import org.apache.falcon.util.DeploymentProperties;
 import org.apache.falcon.util.RuntimeProperties;
 import org.apache.log4j.Logger;
 
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.*;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status.Family;
+import javax.ws.rs.core.UriBuilder;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.Properties;
 
 public class HTTPChannel extends AbstractChannel {
     private static final Logger LOG = Logger.getLogger(HTTPChannel.class);
@@ -121,8 +111,9 @@ public class HTTPChannel extends AbstractChannel {
     }
 
     private boolean isPost(String httpMethod) {
-        if(httpMethod.equals("POST") || httpMethod.equals("PUT"))
+        if (httpMethod.equals("POST") || httpMethod.equals("PUT")) {
             return true;
+        }
         return false;
     }
 
@@ -181,13 +172,19 @@ public class HTTPChannel extends AbstractChannel {
 
     private String getHttpMethod(Method method) {
         PUT put = method.getAnnotation(PUT.class);
-        if (put != null) return HttpMethod.PUT;
+        if (put != null) {
+            return HttpMethod.PUT;
+        }
 
         POST post = method.getAnnotation(POST.class);
-        if (post != null) return HttpMethod.POST;
+        if (post != null) {
+            return HttpMethod.POST;
+        }
 
         DELETE delete = method.getAnnotation(DELETE.class);
-        if (delete != null) return HttpMethod.DELETE;
+        if (delete != null) {
+            return HttpMethod.DELETE;
+        }
 
         return HttpMethod.GET;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
index 9691040..b4d88f9 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
@@ -18,8 +18,6 @@
 
 package org.apache.falcon.resource.channel;
 
-import java.lang.reflect.Method;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.FalconWebException;
@@ -27,6 +25,8 @@ import org.apache.falcon.resource.AbstractEntityManager;
 import org.apache.falcon.util.ReflectionUtils;
 import org.apache.log4j.Logger;
 
+import java.lang.reflect.Method;
+
 public class IPCChannel extends AbstractChannel {
     private static final Logger LOG = Logger.getLogger(IPCChannel.class);
     private AbstractEntityManager service;
@@ -46,10 +46,16 @@ public class IPCChannel extends AbstractChannel {
             return (T) method.invoke(service, args);
         } catch (Exception e) {
             Throwable cause = e.getCause();
-            if (cause != null)  {
-                if (cause instanceof FalconWebException) throw (FalconWebException) cause;
-                if (cause instanceof FalconRuntimException) throw (FalconRuntimException) cause;
-                if (cause instanceof FalconException) throw (FalconException) cause;
+            if (cause != null) {
+                if (cause instanceof FalconWebException) {
+                    throw (FalconWebException) cause;
+                }
+                if (cause instanceof FalconRuntimException) {
+                    throw (FalconRuntimException) cause;
+                }
+                if (cause instanceof FalconException) {
+                    throw (FalconException) cause;
+                }
             }
             throw new FalconException("Unable to invoke on the channel " + methodName +
                     " on service : " + service.getClass().getName() + cause);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java b/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
index ceb99e6..e4fddc2 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
@@ -42,19 +42,25 @@ public class MethodKey {
 
     @Override
     public boolean equals(Object methodRHS) {
-        if (this == methodRHS) return true;
+        if (this == methodRHS) {
+            return true;
+        }
         if (methodRHS == null ||
-                getClass() != methodRHS.getClass()) return false;
+                getClass() != methodRHS.getClass()) {
+            return false;
+        }
 
         MethodKey methodKey = (MethodKey) methodRHS;
 
         if (name != null ? !name.equals(methodKey.name) :
-                methodKey.name != null) return false;
+                methodKey.name != null) {
+            return false;
+        }
         boolean matching = true;
         for (int index = 0; index < argClasses.length; index++) {
             if (argClasses[index] != null && methodKey.argClasses[index] != null &&
                     !methodKey.argClasses[index].isAssignableFrom(argClasses[index])) {
-                 matching = false;
+                matching = false;
             }
         }
         return matching;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java b/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
index 5e6564e..11d0882 100644
--- a/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
+++ b/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
@@ -18,6 +18,11 @@
 
 package org.apache.falcon.resource.provider;
 
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import org.apache.falcon.resource.APIResult;
+import org.apache.falcon.resource.InstancesResult;
+
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.ext.ContextResolver;
@@ -25,17 +30,13 @@ import javax.ws.rs.ext.Provider;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.api.json.JSONJAXBContext;
-import org.apache.falcon.resource.APIResult;
-import org.apache.falcon.resource.InstancesResult;
-
 @Provider
 @Produces(MediaType.APPLICATION_JSON)
-public class JAXBContextResolver implements ContextResolver<JAXBContext>{
+public class JAXBContextResolver implements ContextResolver<JAXBContext> {
     private static JAXBContext context;
-    private static Class<?>[] types = { InstancesResult.class, APIResult.class, InstancesResult.Instance.class,
-            InstancesResult.WorkflowStatus.class };
+    private static Class<?>[] types = {InstancesResult.class, APIResult.class, InstancesResult.Instance.class,
+                                       InstancesResult.WorkflowStatus.class};
+
     static {
         try {
             context = new JSONJAXBContext(JSONConfiguration.natural().build(), types);
@@ -44,8 +45,9 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext>{
         }
     }
 
-    public JAXBContextResolver() { }
-    
+    public JAXBContextResolver() {
+    }
+
     public JAXBContext getContext(Class<?> objectType) {
         for (Class<?> type : types) {
             if (type == objectType) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java b/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
index 4d1f315..e797fb6 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
@@ -18,8 +18,8 @@
 
 package org.apache.falcon.resource.proxy;
 
-import org.apache.hadoop.io.IOUtils;
 import org.apache.falcon.FalconWebException;
+import org.apache.hadoop.io.IOUtils;
 
 import javax.servlet.RequestDispatcher;
 import javax.servlet.ServletInputStream;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
index 80a0cad..c637984 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
@@ -18,23 +18,6 @@
 
 package org.apache.falcon.resource.proxy;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.FalconWebException;
@@ -47,6 +30,13 @@ import org.apache.falcon.resource.InstancesResult.Instance;
 import org.apache.falcon.resource.channel.Channel;
 import org.apache.falcon.resource.channel.ChannelFactory;
 
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.*;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.*;
+
 @Path("instance")
 public class InstanceManagerProxy extends AbstractInstanceManager {
 
@@ -78,11 +68,11 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @GET
     @Path("running/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="running")
+    @Monitored(event = "running")
     @Override
     public InstancesResult getRunningInstances(@Dimension("entityType") @PathParam("type") final String type,
-                                                      @Dimension("entityName") @PathParam("entity") final String entity,
-                                                      @Dimension("colo") @QueryParam("colo") String colo) {
+                                               @Dimension("entityName") @PathParam("entity") final String entity,
+                                               @Dimension("colo") @QueryParam("colo") String colo) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -95,13 +85,13 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @GET
     @Path("status/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="instance-status")
+    @Monitored(event = "instance-status")
     @Override
     public InstancesResult getStatus(@Dimension("entityType") @PathParam("type") final String type,
-                                            @Dimension("entityName") @PathParam("entity") final String entity,
-                                            @Dimension("start-time") @QueryParam("start") final String startStr,
-                                            @Dimension("end-time") @QueryParam("end") final String endStr,
-                                            @Dimension("colo") @QueryParam("colo") final String colo) {
+                                     @Dimension("entityName") @PathParam("entity") final String entity,
+                                     @Dimension("start-time") @QueryParam("start") final String startStr,
+                                     @Dimension("end-time") @QueryParam("end") final String endStr,
+                                     @Dimension("colo") @QueryParam("colo") final String colo) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -110,19 +100,19 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             }
         }.execute(colo, type, entity);
     }
-    
-	@GET
-	@Path("logs/{type}/{entity}")
-	@Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event = "instance-logs")
-	@Override
-	public InstancesResult getLogs(
-			@Dimension("type") @PathParam("type") final String type,
-			@Dimension("entity") @PathParam("entity") final String entity,
-			@Dimension("start-time") @QueryParam("start") final String startStr,
-			@Dimension("end-time") @QueryParam("end") final String endStr,
-			@Dimension("colo") @QueryParam("colo") final String colo,
-			@Dimension("run-id") @QueryParam("runid") final String runId) {
+
+    @GET
+    @Path("logs/{type}/{entity}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Monitored(event = "instance-logs")
+    @Override
+    public InstancesResult getLogs(
+            @Dimension("type") @PathParam("type") final String type,
+            @Dimension("entity") @PathParam("entity") final String entity,
+            @Dimension("start-time") @QueryParam("start") final String startStr,
+            @Dimension("end-time") @QueryParam("end") final String endStr,
+            @Dimension("colo") @QueryParam("colo") final String colo,
+            @Dimension("run-id") @QueryParam("runid") final String runId) {
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
@@ -135,7 +125,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @POST
     @Path("kill/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="kill-instance")
+    @Monitored(event = "kill-instance")
     @Override
     public InstancesResult killInstance(@Context HttpServletRequest request,
                                         @Dimension("entityType") @PathParam("type") final String type,
@@ -149,7 +139,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("killInstance",
-                    bufferedRequest, type, entity, startStr, endStr, colo);
+                        bufferedRequest, type, entity, startStr, endStr, colo);
             }
         }.execute(colo, type, entity);
     }
@@ -157,7 +147,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @POST
     @Path("suspend/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="suspend-instance")
+    @Monitored(event = "suspend-instance")
     @Override
     public InstancesResult suspendInstance(@Context HttpServletRequest request,
                                            @Dimension("entityType") @PathParam("type") final String type,
@@ -170,7 +160,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("suspendInstance",
-                    bufferedRequest, type, entity, startStr, endStr, colo);
+                        bufferedRequest, type, entity, startStr, endStr, colo);
             }
         }.execute(colo, type, entity);
     }
@@ -178,7 +168,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @POST
     @Path("resume/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="resume-instance")
+    @Monitored(event = "resume-instance")
     @Override
     public InstancesResult resumeInstance(@Context HttpServletRequest request,
                                           @Dimension("entityType") @PathParam("type") final String type,
@@ -192,7 +182,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("resumeInstance",
-                    bufferedRequest, type, entity, startStr, endStr, colo);
+                        bufferedRequest, type, entity, startStr, endStr, colo);
             }
         }.execute(colo, type, entity);
     }
@@ -200,21 +190,21 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
     @POST
     @Path("rerun/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="re-run-instance")
+    @Monitored(event = "re-run-instance")
     @Override
     public InstancesResult reRunInstance(@Dimension("entityType") @PathParam("type") final String type,
-                                                @Dimension("entityName") @PathParam("entity") final String entity,
-                                                @Dimension("start-time") @QueryParam("start") final String startStr,
-                                                @Dimension("end-time") @QueryParam("end") final String endStr,
-                                                @Context HttpServletRequest request,
-                                                @Dimension("colo") @QueryParam("colo") String colo) {
+                                         @Dimension("entityName") @PathParam("entity") final String entity,
+                                         @Dimension("start-time") @QueryParam("start") final String startStr,
+                                         @Dimension("end-time") @QueryParam("end") final String endStr,
+                                         @Context HttpServletRequest request,
+                                         @Dimension("colo") @QueryParam("colo") String colo) {
 
         final HttpServletRequest bufferedRequest = new BufferedRequest(request);
         return new InstanceProxy() {
             @Override
             protected InstancesResult doExecute(String colo) throws FalconException {
                 return getInstanceManager(colo).invoke("reRunInstance",
-                    type, entity, startStr, endStr, bufferedRequest, colo);
+                        type, entity, startStr, endStr, bufferedRequest, colo);
             }
         }.execute(colo, type, entity);
     }
@@ -225,7 +215,7 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
             Set<String> colos = getColosFromExpression(coloExpr, type, name);
 
             Map<String, InstancesResult> results = new HashMap<String, InstancesResult>();
-            for (String colo:colos) {
+            for (String colo : colos) {
                 try {
                     APIResult resultHolder = doExecute(colo);
                     if (resultHolder instanceof InstancesResult) {
@@ -249,26 +239,29 @@ public class InstanceManagerProxy extends AbstractInstanceManager {
 
         protected abstract InstancesResult doExecute(String colo) throws FalconException;
     }
-    
+
     private InstancesResult consolidateInstanceResult(Map<String, InstancesResult> results) {
-        if (results == null || results.isEmpty())
+        if (results == null || results.isEmpty()) {
             return null;
+        }
 
         StringBuilder message = new StringBuilder();
         StringBuilder requestIds = new StringBuilder();
         List<Instance> instances = new ArrayList<Instance>();
         int statusCount = 0;
-        for (String colo:results.keySet()) {
+        for (String colo : results.keySet()) {
             InstancesResult result = results.get(colo);
             message.append(colo).append('/').append(result.getMessage()).append('\n');
             requestIds.append(colo).append('/').append(result.getRequestId()).append('\n');
             statusCount += result.getStatus().ordinal();
 
-            if (result.getInstances() == null) continue;
+            if (result.getInstances() == null) {
+                continue;
+            }
 
             for (Instance instance : result.getInstances()) {
-            	instance.instance = instance.getInstance();
-            	instances.add(instance);
+                instance.instance = instance.getInstance();
+                instances.add(instance);
             }
         }
         Instance[] arrInstances = new Instance[instances.size()];

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
index 418b7b7..aa1ef6d 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
@@ -18,25 +18,6 @@
 
 package org.apache.falcon.resource.proxy;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.GET;
-import javax.ws.rs.POST;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.FalconWebException;
@@ -53,6 +34,17 @@ import org.apache.falcon.resource.channel.Channel;
 import org.apache.falcon.resource.channel.ChannelFactory;
 import org.apache.falcon.util.DeploymentUtil;
 
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.*;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 @Path("entities")
 public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityManager {
     private static final String PRISM_TAG = "prism";
@@ -69,7 +61,7 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
             for (String colo : colos) {
                 initializeFor(colo);
             }
-            
+
             DeploymentUtil.setPrismMode();
         } catch (FalconException e) {
             throw new FalconRuntimException("Unable to initialize channels", e);
@@ -96,18 +88,20 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
     }
 
     private BufferedRequest getBufferedRequest(HttpServletRequest request) {
-        if (request instanceof BufferedRequest)
+        if (request instanceof BufferedRequest) {
             return (BufferedRequest) request;
+        }
         return new BufferedRequest(request);
     }
 
     @POST
     @Path("submit/{type}")
-    @Consumes({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Consumes({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "submit")
     @Override
-    public APIResult submit(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
+    public APIResult submit(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
             @Dimension("colo") @QueryParam("colo") final String ignore) {
 
         final HttpServletRequest bufferedRequest = getBufferedRequest(request);
@@ -137,8 +131,8 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @POST
     @Path("validate/{type}")
-    @Consumes({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Consumes({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Override
     public APIResult validate(@Context HttpServletRequest request, @PathParam("type") String type) {
         return super.validate(request, type);
@@ -146,15 +140,17 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @DELETE
     @Path("delete/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "delete")
     @Override
-    public APIResult delete(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
-            @Dimension("entityName") @PathParam("entity") final String entity, @Dimension("colo") @QueryParam("colo") String ignore) {
+    public APIResult delete(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
+            @Dimension("entityName") @PathParam("entity") final String entity,
+            @Dimension("colo") @QueryParam("colo") String ignore) {
 
         final HttpServletRequest bufferedRequest = new BufferedRequest(request);
         Map<String, APIResult> results = new HashMap<String, APIResult>();
-        
+
         results.put("falcon", new EntityProxy(type, entity) {
             @Override
             public APIResult execute() {
@@ -167,7 +163,7 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
                     throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
                 }
             }
-            
+
             @Override
             protected APIResult doExecute(String colo) throws FalconException {
                 return getConfigSyncChannel(colo).invoke("delete", bufferedRequest, type, entity, colo);
@@ -175,17 +171,18 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
         }.execute());
 
         if (!embeddedMode) {
-            results.put(PRISM_TAG,  super.delete(bufferedRequest, type, entity, currentColo));
+            results.put(PRISM_TAG, super.delete(bufferedRequest, type, entity, currentColo));
         }
         return consolidateResult(results);
     }
 
     @POST
     @Path("update/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "update")
     @Override
-    public APIResult update(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
+    public APIResult update(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
             @Dimension("entityName") @PathParam("entity") final String entityName,
             @Dimension("colo") @QueryParam("colo") String ignore) {
 
@@ -199,13 +196,13 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
         oldColos.removeAll(mergedColos);   //Old colos where delete should be called
 
         Map<String, APIResult> results = new HashMap<String, APIResult>();
-        if(!oldColos.isEmpty()) {
+        if (!oldColos.isEmpty()) {
             results.put("delete", new EntityProxy(type, entityName) {
                 @Override
                 protected Set<String> getColosToApply() {
                     return oldColos;
                 }
-    
+
                 @Override
                 protected APIResult doExecute(String colo) throws FalconException {
                     return getConfigSyncChannel(colo).invoke("delete", bufferedRequest, type, entityName, colo);
@@ -213,13 +210,13 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
             }.execute());
         }
 
-        if(!mergedColos.isEmpty()) {
+        if (!mergedColos.isEmpty()) {
             results.put("update", new EntityProxy(type, entityName) {
                 @Override
                 protected Set<String> getColosToApply() {
                     return mergedColos;
                 }
-    
+
                 @Override
                 protected APIResult doExecute(String colo) throws FalconException {
                     return getConfigSyncChannel(colo).invoke("update", bufferedRequest, type, entityName, colo);
@@ -227,35 +224,35 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
             }.execute());
         }
 
-        if(!newColos.isEmpty()) {
+        if (!newColos.isEmpty()) {
             results.put("submit", new EntityProxy(type, entityName) {
                 @Override
                 protected Set<String> getColosToApply() {
                     return newColos;
                 }
-    
+
                 @Override
                 protected APIResult doExecute(String colo) throws FalconException {
                     return getConfigSyncChannel(colo).invoke("submit", bufferedRequest, type, colo);
                 }
             }.execute());
         }
-        
+
         if (!embeddedMode) {
             results.put(PRISM_TAG, super.update(bufferedRequest, type, entityName, currentColo));
         }
-        
+
         return consolidateResult(results);
     }
 
     @GET
     @Path("status/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "status")
     @Override
     public APIResult getStatus(@Dimension("entityType") @PathParam("type") final String type,
-            @Dimension("entityName") @PathParam("entity") final String entity,
-            @Dimension("colo") @QueryParam("colo") final String coloExpr) throws FalconWebException {
+                               @Dimension("entityName") @PathParam("entity") final String entity,
+                               @Dimension("colo") @QueryParam("colo") final String coloExpr) throws FalconWebException {
         return new EntityProxy(type, entity) {
             @Override
             protected Set<String> getColosToApply() {
@@ -275,7 +272,7 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
     @Monitored(event = "dependencies")
     @Override
     public EntityList getDependencies(@Dimension("entityType") @PathParam("type") String type,
-            @Dimension("entityName") @PathParam("entity") String entity) {
+                                      @Dimension("entityName") @PathParam("entity") String entity) {
         return super.getDependencies(type, entity);
     }
 
@@ -289,7 +286,7 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @GET
     @Path("definition/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Override
     public String getEntityDefinition(@PathParam("type") String type, @PathParam("entity") String entityName) {
         return super.getEntityDefinition(type, entityName);
@@ -297,13 +294,13 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @POST
     @Path("schedule/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "schedule")
     @Override
     public APIResult schedule(@Context final HttpServletRequest request,
-            @Dimension("entityType") @PathParam("type") final String type,
-            @Dimension("entityName") @PathParam("entity") final String entity,
-            @Dimension("colo") @QueryParam("colo") final String coloExpr) {
+                              @Dimension("entityType") @PathParam("type") final String type,
+                              @Dimension("entityName") @PathParam("entity") final String entity,
+                              @Dimension("colo") @QueryParam("colo") final String coloExpr) {
 
         final HttpServletRequest bufferedRequest = getBufferedRequest(request);
         return new EntityProxy(type, entity) {
@@ -321,11 +318,12 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @POST
     @Path("submitAndSchedule/{type}")
-    @Consumes({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Consumes({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "submitAndSchedule")
     @Override
-    public APIResult submitAndSchedule(@Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
+    public APIResult submitAndSchedule(
+            @Context HttpServletRequest request, @Dimension("entityType") @PathParam("type") String type,
             @Dimension("colo") @QueryParam("colo") String coloExpr) {
         BufferedRequest bufferedRequest = new BufferedRequest(request);
         String entity = getEntity(bufferedRequest, type).getName();
@@ -337,13 +335,13 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @POST
     @Path("suspend/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "suspend")
     @Override
     public APIResult suspend(@Context final HttpServletRequest request,
-            @Dimension("entityType") @PathParam("type") final String type,
-            @Dimension("entityName") @PathParam("entity") final String entity,
-            @Dimension("colo") @QueryParam("colo") final String coloExpr) {
+                             @Dimension("entityType") @PathParam("type") final String type,
+                             @Dimension("entityName") @PathParam("entity") final String entity,
+                             @Dimension("colo") @QueryParam("colo") final String coloExpr) {
 
         final HttpServletRequest bufferedRequest = new BufferedRequest(request);
         return new EntityProxy(type, entity) {
@@ -361,10 +359,11 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
     @POST
     @Path("resume/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "resume")
     @Override
-    public APIResult resume(@Context final HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
+    public APIResult resume(
+            @Context final HttpServletRequest request, @Dimension("entityType") @PathParam("type") final String type,
             @Dimension("entityName") @PathParam("entity") final String entity,
             @Dimension("colo") @QueryParam("colo") final String coloExpr) {
 
@@ -400,7 +399,8 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
                 try {
                     results.put(colo, doExecute(colo));
                 } catch (FalconException e) {
-                    results.put(colo, new APIResult(APIResult.Status.FAILED, e.getClass().getName() + "::" + e.getMessage()));
+                    results.put(colo,
+                            new APIResult(APIResult.Status.FAILED, e.getClass().getName() + "::" + e.getMessage()));
                 }
             }
             APIResult finalResult = consolidateResult(results);
@@ -417,10 +417,11 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
 
         protected abstract APIResult doExecute(String colo) throws FalconException;
     }
-    
+
     private APIResult consolidateResult(Map<String, APIResult> results) {
-        if (results == null || results.size() == 0)
+        if (results == null || results.size() == 0) {
             return null;
+        }
 
         StringBuilder buffer = new StringBuilder();
         StringBuilder requestIds = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java b/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
index 1ef12c0..e3aedb1 100644
--- a/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
+++ b/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
@@ -37,9 +37,9 @@ public class BasicAuthFilter implements Filter {
     private static final Logger LOG = Logger.getLogger(BasicAuthFilter.class);
 
     private static final String GUEST = "guest";
-    
-	private static final Set<String> BLACK_LISTED_USER = new HashSet<String>(
-			Arrays.asList(new String[] { "hdfs", "mapred", "oozie", "falcon" }));
+
+    private static final Set<String> BLACK_LISTED_USER = new HashSet<String>(
+            Arrays.asList(new String[]{"hdfs", "mapred", "oozie", "falcon"}));
 
     private boolean secure;
 
@@ -69,7 +69,7 @@ public class BasicAuthFilter implements Filter {
 
         String user;
         String requestId = UUID.randomUUID().toString();
-        
+
         if (!secure) {
             user = GUEST;
         } else {
@@ -77,13 +77,12 @@ public class BasicAuthFilter implements Filter {
         }
 
         if (user == null || user.isEmpty()) {
-        	httpResponse.sendError(Response.Status.BAD_REQUEST.getStatusCode(),
-        			"Remote user header can't be empty");
-        } else if(BLACK_LISTED_USER.contains(user)){
-        	httpResponse.sendError(Response.Status.BAD_REQUEST.getStatusCode(),
-        			"Remote user header can't be superusers:"+BLACK_LISTED_USER);
-        }
-        else {
+            httpResponse.sendError(Response.Status.BAD_REQUEST.getStatusCode(),
+                    "Remote user header can't be empty");
+        } else if (BLACK_LISTED_USER.contains(user)) {
+            httpResponse.sendError(Response.Status.BAD_REQUEST.getStatusCode(),
+                    "Remote user header can't be superusers:" + BLACK_LISTED_USER);
+        } else {
             CurrentUser.authenticate(user);
             try {
                 NDC.push(user + ":" + httpRequest.getMethod() + "/" + httpRequest.getPathInfo());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java b/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
index 45561d7..4e48488 100644
--- a/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
+++ b/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
@@ -35,99 +35,99 @@ import java.lang.reflect.InvocationTargetException;
 import java.util.Date;
 
 public class FalconTopicSubscriber implements MessageListener, ExceptionListener {
-	private static final Logger LOG = Logger
-			.getLogger(FalconTopicSubscriber.class);
-
-	private TopicSubscriber subscriber;
-	private String implementation;
-	private String userName;
-	private String password;
-	private String url;
-	private String topicName;
-	private Connection connection;
-
-	private AbstractRerunHandler retryHandler = RerunHandlerFactory
-			.getRerunHandler(RerunType.RETRY);
-	private AbstractRerunHandler latedataHandler = RerunHandlerFactory
-			.getRerunHandler(RerunType.LATE);
-
-	public FalconTopicSubscriber(String implementation, String userName,
-			String password, String url, String topicName) {
-		this.implementation = implementation;
-		this.userName = userName;
-		this.password = password;
-		this.url = url;
-		this.topicName = topicName;
-	}
-
-	public void startSubscriber() throws FalconException {
-		try {
-			connection = createAndGetConnection(implementation, userName,
-					password, url);
-			TopicSession session = (TopicSession) connection.createSession(
-					false, Session.AUTO_ACKNOWLEDGE);
-			Topic destination = session.createTopic(topicName);
-			subscriber = session.createSubscriber(destination);
-			subscriber.setMessageListener(this);
-			connection.setExceptionListener(this);
-			connection.start();
-		} catch (Exception e) {
-			LOG.error("Error starting subscriber of topic: " + this.toString(),
-					e);
-			throw new FalconException(e);
-		}
-	}
-
-	@Override
-	public void onMessage(Message message) {
-		MapMessage mapMessage = (MapMessage) message;
-		try {
-			debug(mapMessage);
-			String cluster = mapMessage.getString(ARG.cluster.getArgName());
-			String entityName = mapMessage.getString(ARG.entityName
-					.getArgName());
-			String entityType = mapMessage.getString(ARG.entityType
-					.getArgName());
-			String workflowId = mapMessage.getString(ARG.workflowId
-					.getArgName());
-			String runId = mapMessage.getString(ARG.runId.getArgName());
-			String nominalTime = mapMessage.getString(ARG.nominalTime
-					.getArgName());
-			String status = mapMessage.getString(ARG.status.getArgName());
-			String operation = mapMessage.getString(ARG.operation.getArgName());
-
-			AbstractWorkflowEngine wfEngine = WorkflowEngineFactory.getWorkflowEngine();
-			InstancesResult result = wfEngine
-					.getJobDetails(cluster, workflowId);
+    private static final Logger LOG = Logger
+            .getLogger(FalconTopicSubscriber.class);
+
+    private TopicSubscriber subscriber;
+    private String implementation;
+    private String userName;
+    private String password;
+    private String url;
+    private String topicName;
+    private Connection connection;
+
+    private AbstractRerunHandler retryHandler = RerunHandlerFactory
+            .getRerunHandler(RerunType.RETRY);
+    private AbstractRerunHandler latedataHandler = RerunHandlerFactory
+            .getRerunHandler(RerunType.LATE);
+
+    public FalconTopicSubscriber(String implementation, String userName,
+                                 String password, String url, String topicName) {
+        this.implementation = implementation;
+        this.userName = userName;
+        this.password = password;
+        this.url = url;
+        this.topicName = topicName;
+    }
+
+    public void startSubscriber() throws FalconException {
+        try {
+            connection = createAndGetConnection(implementation, userName,
+                    password, url);
+            TopicSession session = (TopicSession) connection.createSession(
+                    false, Session.AUTO_ACKNOWLEDGE);
+            Topic destination = session.createTopic(topicName);
+            subscriber = session.createSubscriber(destination);
+            subscriber.setMessageListener(this);
+            connection.setExceptionListener(this);
+            connection.start();
+        } catch (Exception e) {
+            LOG.error("Error starting subscriber of topic: " + this.toString(),
+                    e);
+            throw new FalconException(e);
+        }
+    }
+
+    @Override
+    public void onMessage(Message message) {
+        MapMessage mapMessage = (MapMessage) message;
+        try {
+            debug(mapMessage);
+            String cluster = mapMessage.getString(ARG.cluster.getArgName());
+            String entityName = mapMessage.getString(ARG.entityName
+                    .getArgName());
+            String entityType = mapMessage.getString(ARG.entityType
+                    .getArgName());
+            String workflowId = mapMessage.getString(ARG.workflowId
+                    .getArgName());
+            String runId = mapMessage.getString(ARG.runId.getArgName());
+            String nominalTime = mapMessage.getString(ARG.nominalTime
+                    .getArgName());
+            String status = mapMessage.getString(ARG.status.getArgName());
+            String operation = mapMessage.getString(ARG.operation.getArgName());
+
+            AbstractWorkflowEngine wfEngine = WorkflowEngineFactory.getWorkflowEngine();
+            InstancesResult result = wfEngine
+                    .getJobDetails(cluster, workflowId);
             Date startTime = result.getInstances()[0].startTime;
             Date endTime = result.getInstances()[0].endTime;
             Long duration = (endTime.getTime() - startTime.getTime()) * 1000000;
-			if (status.equalsIgnoreCase("FAILED")) {
-				retryHandler.handleRerun(cluster, entityType, entityName,
-						nominalTime, runId, workflowId,
-						System.currentTimeMillis());
-				GenericAlert.instrumentFailedInstance(cluster, entityType,
-						entityName, nominalTime, workflowId, runId, operation,
+            if (status.equalsIgnoreCase("FAILED")) {
+                retryHandler.handleRerun(cluster, entityType, entityName,
+                        nominalTime, runId, workflowId,
+                        System.currentTimeMillis());
+                GenericAlert.instrumentFailedInstance(cluster, entityType,
+                        entityName, nominalTime, workflowId, runId, operation,
                         SchemaHelper.formatDateUTC(startTime),
                         "", "", duration);
-			} else if (status.equalsIgnoreCase("SUCCEEDED")) {
-				latedataHandler.handleRerun(cluster, entityType, entityName,
-						nominalTime, runId, workflowId,
-						System.currentTimeMillis());
-				GenericAlert.instrumentSucceededInstance(cluster, entityType,
+            } else if (status.equalsIgnoreCase("SUCCEEDED")) {
+                latedataHandler.handleRerun(cluster, entityType, entityName,
+                        nominalTime, runId, workflowId,
+                        System.currentTimeMillis());
+                GenericAlert.instrumentSucceededInstance(cluster, entityType,
                         entityName, nominalTime, workflowId, runId, operation,
                         SchemaHelper.formatDateUTC(startTime),
                         duration);
                 notifySLAService(cluster, entityName, entityType, nominalTime, duration);
             }
 
-		} catch (Exception ignore) {
-			LOG.info(
-					"Error in onMessage for subscriber of topic: "
-							+ this.toString(), ignore);
-		}
+        } catch (Exception ignore) {
+            LOG.info(
+                    "Error in onMessage for subscriber of topic: "
+                            + this.toString(), ignore);
+        }
 
-	}
+    }
 
     private void notifySLAService(String cluster, String entityName,
                                   String entityType, String nominalTime, Long duration) {
@@ -157,45 +157,49 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
         }
     }
 
-	@Override
-	public void onException(JMSException ignore) {
-		LOG.info(
-				"Error in onException for subscriber of topic: "
-						+ this.toString(), ignore);
-	}
-
-	public void closeSubscriber() throws FalconException {
-		try {
-			LOG.info("Closing subscriber on topic : " + this.topicName);
-			subscriber.close();
-			connection.close();
-		} catch (JMSException e) {
-			LOG.error("Error closing subscriber of topic: " + this.toString(),
-					e);
-			throw new FalconException(e);
-		}
-	}
-
-	private static Connection createAndGetConnection(String implementation,
-			String userName, String password, String url) throws JMSException,
-			ClassNotFoundException, IllegalArgumentException,
-			SecurityException, InstantiationException, IllegalAccessException,
-			InvocationTargetException, NoSuchMethodException {
-
-		@SuppressWarnings("unchecked")
-		Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) FalconTopicSubscriber.class
-				.getClassLoader().loadClass(implementation);
-
-		ConnectionFactory connectionFactory = clazz.getConstructor(
-				String.class, String.class, String.class).newInstance(userName,
-				password, url);
+    @Override
+    public void onException(JMSException ignore) {
+        LOG.info(
+                "Error in onException for subscriber of topic: "
+                        + this.toString(), ignore);
+    }
+
+    public void closeSubscriber() throws FalconException {
+        try {
+            LOG.info("Closing subscriber on topic : " + this.topicName);
+            subscriber.close();
+            connection.close();
+        } catch (JMSException e) {
+            LOG.error("Error closing subscriber of topic: " + this.toString(),
+                    e);
+            throw new FalconException(e);
+        }
+    }
+
+    private static Connection createAndGetConnection(String implementation,
+                                                     String userName, String password, String url) throws JMSException,
+                                                                                                          ClassNotFoundException,
+                                                                                                          IllegalArgumentException,
+                                                                                                          SecurityException,
+                                                                                                          InstantiationException,
+                                                                                                          IllegalAccessException,
+                                                                                                          InvocationTargetException,
+                                                                                                          NoSuchMethodException {
+
+        @SuppressWarnings("unchecked")
+        Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) FalconTopicSubscriber.class
+                .getClassLoader().loadClass(implementation);
+
+        ConnectionFactory connectionFactory = clazz.getConstructor(
+                String.class, String.class, String.class).newInstance(userName,
+                password, url);
 
         return connectionFactory.createConnection();
-	}
+    }
 
-	@Override
-	public String toString() {
-		return topicName;
-	}
+    @Override
+    public String toString() {
+        return topicName;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java b/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
index 5f7541e..9d5e78f 100644
--- a/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
+++ b/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
@@ -24,44 +24,44 @@ public class ProcessSubscriberService implements FalconService {
 
     private FalconTopicSubscriber subscriber;
 
-	private static enum JMSprops {
-		FalconBrokerImplClass("broker.impl.class", "org.apache.activemq.ActiveMQConnectionFactory"),
-		FalconBrokerUrl("broker.url", "tcp://localhost:61616?daemon=true"),
-		FalconEntityTopic("entity.topic", "FALCON.ENTITY.TOPIC");
+    private static enum JMSprops {
+        FalconBrokerImplClass("broker.impl.class", "org.apache.activemq.ActiveMQConnectionFactory"),
+        FalconBrokerUrl("broker.url", "tcp://localhost:61616?daemon=true"),
+        FalconEntityTopic("entity.topic", "FALCON.ENTITY.TOPIC");
 
-		private String propName;
-		private String defaultPropValue;
+        private String propName;
+        private String defaultPropValue;
 
-		private JMSprops(String propName, String defaultPropValue) {
-			this.propName = propName;
-			this.defaultPropValue = defaultPropValue;
-		}
+        private JMSprops(String propName, String defaultPropValue) {
+            this.propName = propName;
+            this.defaultPropValue = defaultPropValue;
+        }
 
-	}
+    }
 
-	@Override
-	public String getName() {
-		return ProcessSubscriberService.class.getSimpleName();
-	}
+    @Override
+    public String getName() {
+        return ProcessSubscriberService.class.getSimpleName();
+    }
 
-	@Override
-	public void init() throws FalconException {
+    @Override
+    public void init() throws FalconException {
         String falconBrokerImplClass = getPropertyValue(JMSprops.FalconBrokerImplClass);
         String falconBrokerUrl = getPropertyValue(JMSprops.FalconBrokerUrl);
         String falconEntityTopic = getPropertyValue(JMSprops.FalconEntityTopic);
 
-		subscriber = new FalconTopicSubscriber(falconBrokerImplClass, "", "",
+        subscriber = new FalconTopicSubscriber(falconBrokerImplClass, "", "",
                 falconBrokerUrl, falconEntityTopic);
-		subscriber.startSubscriber();
-	}
+        subscriber.startSubscriber();
+    }
 
-	private String getPropertyValue(JMSprops prop) {
-		return StartupProperties.get().getProperty(prop.propName,
-				prop.defaultPropValue);
-	}
+    private String getPropertyValue(JMSprops prop) {
+        return StartupProperties.get().getProperty(prop.propName,
+                prop.defaultPropValue);
+    }
 
-	@Override
-	public void destroy() throws FalconException {
-		subscriber.closeSubscriber();
-	}
+    @Override
+    public void destroy() throws FalconException {
+        subscriber.closeSubscriber();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java b/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
index 71a6c3f..da7887e 100644
--- a/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
+++ b/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
@@ -118,7 +118,7 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
     private void removeFromPendingList(Entity entity, String cluster, Date nominalTime) {
         ConcurrentMap<Date, Date> pendingInstances = pendingJobs.get(getKey(entity, cluster));
         if (pendingInstances != null) {
-            LOG.debug("Removing from pending jobs: " + getKey(entity,  cluster) + " ---> " +
+            LOG.debug("Removing from pending jobs: " + getKey(entity, cluster) + " ---> " +
                     SchemaHelper.formatDateUTC(nominalTime));
             pendingInstances.remove(nominalTime);
         }
@@ -129,7 +129,9 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
         @Override
         public void run() {
             try {
-                if (monitoredEntities.isEmpty()) return;
+                if (monitoredEntities.isEmpty()) {
+                    return;
+                }
                 Set<String> keys = new HashSet<String>(monitoredEntities.keySet());
                 checkSLAMissOnPendingEntities(keys);
                 addNewPendingEntities(keys);
@@ -142,7 +144,9 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
             Date now = new Date();
             for (String key : keys) {
                 ConcurrentMap<Date, Date> pendingInstances = pendingJobs.get(key);
-                if (pendingInstances == null) continue;
+                if (pendingInstances == null) {
+                    continue;
+                }
                 ConcurrentMap<Date, Date> interim =
                         new ConcurrentHashMap<Date, Date>(pendingInstances);
                 for (Map.Entry<Date, Date> entry : interim.entrySet()) {
@@ -174,7 +178,9 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
                 Frequency frequency = EntityUtil.getFrequency(entity);
                 TimeZone timeZone = EntityUtil.getTimeZone(entity);
                 Date nextStart = EntityUtil.getNextStartTime(startTime, frequency, timeZone, now);
-                if (nextStart.after(windowEndTime)) continue;
+                if (nextStart.after(windowEndTime)) {
+                    continue;
+                }
                 ConcurrentMap<Date, Date> pendingInstances = pendingJobs.get(key);
                 while (!nextStart.after(windowEndTime)) {
                     if (pendingInstances == null) {
@@ -182,7 +188,9 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
                         pendingInstances = pendingJobs.get(key);
                     }
                     Long latency = monitoredEntities.get(key);
-                    if (latency == null) break;
+                    if (latency == null) {
+                        break;
+                    }
                     pendingInstances.putIfAbsent(nextStart, new Date(nextStart.getTime() +
                             latency * 1500));  //1.5 times latency is when it is supposed to have breached
                     LOG.debug("Adding to pending jobs: " + key + " ---> " +

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
index 34a7f9f..6baeb13 100644
--- a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
+++ b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
@@ -25,24 +25,24 @@ import org.mortbay.jetty.bio.SocketConnector;
 import org.mortbay.jetty.webapp.WebAppContext;
 
 public class EmbeddedServer {
-  private Server server = new Server();
-  private Connector connector = new SocketConnector();
+    private Server server = new Server();
+    private Connector connector = new SocketConnector();
 
-  public EmbeddedServer(int port, String path) {
-    connector.setPort(port);
-    connector.setHost("0.0.0.0");
-    server.addConnector(connector);
+    public EmbeddedServer(int port, String path) {
+        connector.setPort(port);
+        connector.setHost("0.0.0.0");
+        server.addConnector(connector);
 
-    WebAppContext application = new WebAppContext(path, "/");
-    server.setHandler(application);
-  }
+        WebAppContext application = new WebAppContext(path, "/");
+        server.setHandler(application);
+    }
 
-  public void start() throws Exception {
-	Services.get().reset();
-    server.start();
-  }
+    public void start() throws Exception {
+        Services.get().reset();
+        server.start();
+    }
 
-  public void stop() throws Exception {
-    server.stop();
-  }
+    public void stop() throws Exception {
+        server.stop();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/prism/src/main/resources/log4j.xml b/prism/src/main/resources/log4j.xml
index 299143b..ea08108 100644
--- a/prism/src/main/resources/log4j.xml
+++ b/prism/src/main/resources/log4j.xml
@@ -20,72 +20,72 @@
 <!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
 
 <log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-  <appender name="console" class="org.apache.log4j.ConsoleAppender">
-    <param name="Target" value="System.out"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="console" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
 
-  <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/prism.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/prism.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/prism-audit.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
+
+    <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/prsim-tranlog.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
 
-  <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/prism-audit.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
-  <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/prsim-tranlog.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
     <appender name="METRIC" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/prism-metric.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %m%n"/>
-    </layout>
-  </appender>
+        <param name="File" value="/var/log/falcon/prism-metric.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %m%n"/>
+        </layout>
+    </appender>
+
+    <logger name="org.apache.falcon">
+        <level value="debug"/>
+        <appender-ref ref="FILE"/>
+    </logger>
+
+    <logger name="AUDIT">
+        <level value="info"/>
+        <appender-ref ref="AUDIT"/>
+    </logger>
 
-  <logger name="org.apache.falcon">
-    <level value="debug"/>
-    <appender-ref ref="FILE" />
-  </logger>
+    <logger name="TRANSACTIONLOG">
+        <level value="info"/>
+        <appender-ref ref="TRANSACTIONLOG"/>
+    </logger>
 
-  <logger name="AUDIT">
-    <level value="info"/>
-    <appender-ref ref="AUDIT" />
-  </logger>
-  
-  <logger name="TRANSACTIONLOG">
-    <level value="info"/>
-    <appender-ref ref="TRANSACTIONLOG" />
-  </logger>
-  
     <logger name="METRIC">
-    <level value="info"/>
-    <appender-ref ref="METRIC" />
-  </logger>
+        <level value="info"/>
+        <appender-ref ref="METRIC"/>
+    </logger>
 
-  <root>
-    <priority value ="info" />
-    <appender-ref ref="console" />
-  </root>
+    <root>
+        <priority value="info"/>
+        <appender-ref ref="console"/>
+    </root>
 
 </log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/prism/src/main/webapp/WEB-INF/web.xml b/prism/src/main/webapp/WEB-INF/web.xml
index 8a9406a..00a6c42 100644
--- a/prism/src/main/webapp/WEB-INF/web.xml
+++ b/prism/src/main/webapp/WEB-INF/web.xml
@@ -25,37 +25,39 @@
     <display-name>Apache Falcon Prism</display-name>
     <description>Apache Falcon Prism</description>
 
-  <filter>
-    <filter-name>auth</filter-name>
-    <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
-  </filter>
-
-  <filter-mapping>
-    <filter-name>auth</filter-name>
-    <servlet-name>FalconProxyAPI</servlet-name>
-  </filter-mapping>
-
-  <listener>
-    <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
-  </listener>
-
-  <servlet>
-      <servlet-name>FalconProxyAPI</servlet-name>
-      <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
-          <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
-      </init-param>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.packages</param-name>
-          <param-value>org.apache.falcon.resource.admin,org.apache.falcon.resource.provider,org.apache.falcon.resource.proxy</param-value>
-      </init-param>
-      <load-on-startup>1</load-on-startup>
-  </servlet>
-
-  <servlet-mapping>
-      <servlet-name>FalconProxyAPI</servlet-name>
-      <url-pattern>/api/*</url-pattern>
-  </servlet-mapping>
+    <filter>
+        <filter-name>auth</filter-name>
+        <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
+    </filter>
+
+    <filter-mapping>
+        <filter-name>auth</filter-name>
+        <servlet-name>FalconProxyAPI</servlet-name>
+    </filter-mapping>
+
+    <listener>
+        <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
+    </listener>
+
+    <servlet>
+        <servlet-name>FalconProxyAPI</servlet-name>
+        <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
+            <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
+        </init-param>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.packages</param-name>
+            <param-value>
+                org.apache.falcon.resource.admin,org.apache.falcon.resource.provider,org.apache.falcon.resource.proxy
+            </param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
+    <servlet-mapping>
+        <servlet-name>FalconProxyAPI</servlet-name>
+        <url-pattern>/api/*</url-pattern>
+    </servlet-mapping>
 
 </web-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java b/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
index b2436af..979c349 100644
--- a/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
+++ b/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
@@ -22,11 +22,11 @@ import org.testng.annotations.Test;
 
 @Aspect
 public class GenericAlertTest {
-	
-	@Test
-	public void testWfInstanceFailedAlert() throws Exception{
-		GenericAlert.instrumentFailedInstance("cluster", "process", "agg-coord", "120:df", "ef-id", "1",
+
+    @Test
+    public void testWfInstanceFailedAlert() throws Exception {
+        GenericAlert.instrumentFailedInstance("cluster", "process", "agg-coord", "120:df", "ef-id", "1",
                 "DELETE", "now", "error", "none", 1242);
-	}
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java b/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
index 2c2a8ed..2e04904 100644
--- a/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
+++ b/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
@@ -18,67 +18,66 @@
 
 package org.apache.falcon.aspect;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import junit.framework.Assert;
-
 import org.apache.falcon.FalconWebException;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.AbstractEntityManager;
 import org.testng.annotations.Test;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public class LoggingAspectTest {
 
-	
-	private AbstractEntityManager em = new AbstractEntityManager() {
+
+    private AbstractEntityManager em = new AbstractEntityManager() {
     };
 
-	private volatile Exception threadException;
+    private volatile Exception threadException;
+
+    @Test(expectedExceptions = FalconWebException.class)
+    public void testBeanLoading() {
 
-	@Test(expectedExceptions=FalconWebException.class)
-	public void testBeanLoading() {
+        APIResult result = em.getStatus("type", "entity", "colo");
+    }
 
-		APIResult result = em.getStatus("type", "entity", "colo");
-	}
+    @Test
+    public void testExceptionBeanLoading() {
+        try {
+            em.getStatus("type", "entity", "colo");
+            System.out.println();
+
+        } catch (Exception e) {
+            return;
+        }
+        Assert.fail("Exepected excpetion");
+    }
 
-	@Test
-	public void testExceptionBeanLoading() {
-		try {
-			em.getStatus("type", "entity", "colo");
-			System.out.println();
-			
-		} catch (Exception e) {
-			return;
-		}
-		Assert.fail("Exepected excpetion");
-	}
-	
-	@Test(expectedExceptions=FalconWebException.class)
-	public void testConcurrentRequests() throws Exception{
+    @Test(expectedExceptions = FalconWebException.class)
+    public void testConcurrentRequests() throws Exception {
         List<Thread> threadList = new ArrayList<Thread>();
         for (int i = 0; i < 5; i++) {
             threadList.add(new Thread() {
                 public void run() {
                     try {
-                    	testBeanLoading();
+                        testBeanLoading();
                     } catch (Exception e) {
-                    	e.printStackTrace();
-                    	threadException =e;
+                        e.printStackTrace();
+                        threadException = e;
                         throw new RuntimeException(e);
                     }
                 }
             });
         }
-        
-        for(Thread thread:threadList) {
+
+        for (Thread thread : threadList) {
             thread.start();
             thread.join();
         }
-		
-		if (threadException != null) {
-			throw threadException;
-		}
-	}
+
+        if (threadException != null) {
+            throw threadException;
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
index 5954815..24e5a8a 100644
--- a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
@@ -54,7 +54,7 @@ public class EntityManagerTest extends AbstractEntityManager {
     @DataProvider(name = "validXMLServletStreamProvider")
     private Object[][] servletStreamProvider() {
         ServletInputStream validProcessXML = getServletInputStream(SAMPLE_PROCESS_XML);
-        return new Object[][] { { EntityType.PROCESS, validProcessXML },
+        return new Object[][]{{EntityType.PROCESS, validProcessXML},
         };
 
     }
@@ -97,7 +97,8 @@ public class EntityManagerTest extends AbstractEntityManager {
             validate(mockHttpServletRequest,
                     "InvalidEntityType");
             Assert.fail("Invalid entity type was accepted by the system");
-        } catch (FalconWebException ignore) {}
+        } catch (FalconWebException ignore) {
+        }
     }
 
     /**


[02/47] git commit: Adding missing license header

Posted by sr...@apache.org.
Adding missing license header


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

Branch: refs/heads/master
Commit: b09c5ee0e947b96777548058b16dc135c5988145
Parents: ce2589e
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Fri Apr 12 13:49:41 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Fri Apr 12 13:49:41 2013 +0530

----------------------------------------------------------------------
 .../falcon/rerun/queue/InMemoryQueueTest.java      |   17 +++++++++++++++
 1 files changed, 17 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b09c5ee0/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
index 09779db..5bd35e5 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.falcon.rerun.queue;
 
 import org.apache.falcon.FalconException;


[27/47] git commit: Check style fixes relating to feed module

Posted by sr...@apache.org.
Check style fixes relating to feed module


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

Branch: refs/heads/master
Commit: afbcea089140baf9c7400d8e31fa99748c9f8d2e
Parents: bdadf2e
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 15:06:46 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 15:06:46 2013 +0530

----------------------------------------------------------------------
 .../apache/falcon/converter/OozieFeedMapper.java   |   36 +++++++++------
 .../falcon/workflow/OozieFeedWorkflowBuilder.java  |   12 +++--
 .../falcon/converter/OozieFeedMapperTest.java      |   11 +++--
 3 files changed, 35 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/afbcea08/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java b/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
index d02f6c1..a409e44 100644
--- a/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
+++ b/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
@@ -47,9 +47,13 @@ import org.apache.log4j.Logger;
 import java.io.IOException;
 import java.util.*;
 
+/**
+ * Mapper which maps feed definition to oozie workflow definitions for
+ * replication & retention.
+ */
 public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
 
-    private static Logger LOG = Logger.getLogger(OozieFeedMapper.class);
+    private static final Logger LOG = Logger.getLogger(OozieFeedMapper.class);
 
     private static final int THIRTY_MINUTES = 30 * 60 * 1000;
 
@@ -150,7 +154,8 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
     }
 
     private List<COORDINATORAPP> getReplicationCoordinators(Cluster targetCluster, Path bundlePath)
-            throws FalconException {
+        throws FalconException {
+
         Feed feed = getEntity();
         List<COORDINATORAPP> replicationCoords = new ArrayList<COORDINATORAPP>();
 
@@ -176,7 +181,8 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
     }
 
     private COORDINATORAPP createAndGetCoord(Feed feed, Cluster srcCluster, Cluster trgCluster, Path bundlePath)
-            throws FalconException {
+        throws FalconException {
+
         COORDINATORAPP replicationCoord;
         String coordName;
         try {
@@ -186,14 +192,14 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
             replicationCoord.setName(coordName);
             replicationCoord.setFrequency("${coord:" + feed.getFrequency().toString() + "}");
 
-            long frequency_ms = ExpressionHelper.get().
+            long frequencyInMillis = ExpressionHelper.get().
                     evaluate(feed.getFrequency().toString(), Long.class);
-            long timeout_ms = frequency_ms * 6;
-            if (timeout_ms < THIRTY_MINUTES) {
-                timeout_ms = THIRTY_MINUTES;
+            long timeoutInMillis = frequencyInMillis * 6;
+            if (timeoutInMillis < THIRTY_MINUTES) {
+                timeoutInMillis = THIRTY_MINUTES;
             }
-            replicationCoord.getControls().setTimeout(String.valueOf(timeout_ms / (1000 * 60)));
-            replicationCoord.getControls().setThrottle(String.valueOf(timeout_ms / frequency_ms * 2));
+            replicationCoord.getControls().setTimeout(String.valueOf(timeoutInMillis / (1000 * 60)));
+            replicationCoord.getControls().setThrottle(String.valueOf(timeoutInMillis / frequencyInMillis * 2));
 
             Date srcStartDate = FeedHelper.getCluster(feed, srcCluster.getName()).getValidity().getStart();
             Date srcEndDate = FeedHelper.getCluster(feed, srcCluster.getName()).getValidity().getEnd();
@@ -218,11 +224,10 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
             SYNCDATASET inputDataset = (SYNCDATASET) replicationCoord.getDatasets().getDatasetOrAsyncDataset().get(0);
             SYNCDATASET outputDataset = (SYNCDATASET) replicationCoord.getDatasets().getDatasetOrAsyncDataset().get(1);
 
-            inputDataset.setUriTemplate(new Path(ClusterHelper
-                    .getStorageUrl(srcCluster), FeedHelper.getLocation(feed,
-                    LocationType.DATA, srcCluster.getName()).getPath()).toString());
-            outputDataset.setUriTemplate(getStoragePath(FeedHelper.getLocation(
-                    feed, LocationType.DATA, trgCluster.getName()).getPath()));
+            inputDataset.setUriTemplate(new Path(ClusterHelper.getStorageUrl(srcCluster),
+                FeedHelper.getLocation(feed, LocationType.DATA, srcCluster.getName()).getPath()).toString());
+            outputDataset.setUriTemplate(getStoragePath(
+                FeedHelper.getLocation(feed, LocationType.DATA, trgCluster.getName()).getPath()));
             setDatasetValues(inputDataset, feed, srcCluster);
             setDatasetValues(outputDataset, feed, srcCluster);
             if (feed.getAvailabilityFlag() == null) {
@@ -248,7 +253,8 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
     }
 
     private ACTION getReplicationWorkflowAction(Cluster srcCluster, Cluster trgCluster, Path wfPath, String wfName)
-            throws FalconException {
+        throws FalconException {
+
         ACTION replicationAction = new ACTION();
         WORKFLOW replicationWF = new WORKFLOW();
         try {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/afbcea08/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
index 14f39e9..0cbdf77 100644
--- a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
+++ b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.*;
 
+/**
+ * Workflow definition builder for feed replication & retention.
+ */
 public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
 
     @Override
@@ -53,11 +56,10 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
 
     @Override
     public Properties newWorkflowSchedule(Feed feed, Date startDate, String clusterName, String user)
-            throws FalconException {
+        throws FalconException {
+
         org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
-        if (!startDate.before(feedCluster.getValidity().getEnd()))
-        // start time >= end time
-        {
+        if (!startDate.before(feedCluster.getValidity().getEnd())) {
             return null;
         }
 
@@ -84,6 +86,6 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
     public String[] getWorkflowNames(Feed entity) {
         return new String[]{
                 EntityUtil.getWorkflowName(Tag.RETENTION, entity).toString(),
-                EntityUtil.getWorkflowName(Tag.REPLICATION, entity).toString()};
+                EntityUtil.getWorkflowName(Tag.REPLICATION, entity).toString(), };
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/afbcea08/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 3a7f4ae..d3d8e91 100644
--- a/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
+++ b/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
@@ -41,13 +41,16 @@ 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;
-    ConfigurationStore store = ConfigurationStore.get();
-    Cluster srcCluster;
-    Cluster trgCluster;
-    Feed feed;
+    private ConfigurationStore store = ConfigurationStore.get();
+    private Cluster srcCluster;
+    private Cluster trgCluster;
+    private Feed feed;
 
     private static final String SRC_CLUSTER_PATH = "/src-cluster.xml";
     private static final String TRG_CLUSTER_PATH = "/trg-cluster.xml";


[15/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java b/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
index 15e6e32..e58184e 100644
--- a/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
@@ -18,12 +18,12 @@
 
 package org.apache.falcon.entity.store;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 import org.testng.Assert;
 import org.testng.annotations.AfterSuite;
@@ -34,49 +34,49 @@ import java.io.IOException;
 
 public class ConfigurationStoreTest {
 
-  private static Logger LOG = Logger.getLogger(ConfigurationStoreTest.class);
+    private static Logger LOG = Logger.getLogger(ConfigurationStoreTest.class);
 
-  private ConfigurationStore store = ConfigurationStore.get();
+    private ConfigurationStore store = ConfigurationStore.get();
 
-  @Test
-  public void testPublish() throws Exception {
-    Process process = new Process();
-    process.setName("hello");
-    store.publish(EntityType.PROCESS, process);
-    Process p = store.get(EntityType.PROCESS, "hello");
-    Assert.assertEquals(p, process);
-  }
+    @Test
+    public void testPublish() throws Exception {
+        Process process = new Process();
+        process.setName("hello");
+        store.publish(EntityType.PROCESS, process);
+        Process p = store.get(EntityType.PROCESS, "hello");
+        Assert.assertEquals(p, process);
+    }
 
-  @Test
-  public void testGet() throws Exception {
-    Process p = store.get(EntityType.PROCESS, "notfound");
-    Assert.assertNull(p);
-  }
+    @Test
+    public void testGet() throws Exception {
+        Process p = store.get(EntityType.PROCESS, "notfound");
+        Assert.assertNull(p);
+    }
 
-  @Test
-  public void testRemove() throws Exception {
-    Process process = new Process();
-    process.setName("remove");
-    store.publish(EntityType.PROCESS, process);
-    Process p = store.get(EntityType.PROCESS, "remove");
-    Assert.assertEquals(p, process);
-    store.remove(EntityType.PROCESS, "remove");
-    p = store.get(EntityType.PROCESS, "remove");
-    Assert.assertNull(p);
-  }
+    @Test
+    public void testRemove() throws Exception {
+        Process process = new Process();
+        process.setName("remove");
+        store.publish(EntityType.PROCESS, process);
+        Process p = store.get(EntityType.PROCESS, "remove");
+        Assert.assertEquals(p, process);
+        store.remove(EntityType.PROCESS, "remove");
+        p = store.get(EntityType.PROCESS, "remove");
+        Assert.assertNull(p);
+    }
 
-  @Test
-  public void testSearch() throws Exception {
-    //TODO
-  }
+    @Test
+    public void testSearch() throws Exception {
+        //TODO
+    }
 
-  @BeforeSuite
-  @AfterSuite
-  public void cleanup() throws IOException {
-    Path path = new Path(StartupProperties.get().
-        getProperty("config.store.uri"));
-    FileSystem fs = FileSystem.get(path.toUri(), new Configuration());
-    fs.delete(path, true);
-    LOG.info("Cleaned up " + path);
-  }
+    @BeforeSuite
+    @AfterSuite
+    public void cleanup() throws IOException {
+        Path path = new Path(StartupProperties.get().
+                getProperty("config.store.uri"));
+        FileSystem fs = FileSystem.get(path.toUri(), new Configuration());
+        fs.delete(path, true);
+        LOG.info("Cleaned up " + path);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
index 6078e01..9b4bffc 100644
--- a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
@@ -18,22 +18,19 @@
 
 package org.apache.falcon.entity.v0;
 
-import java.util.Set;
-
 import org.apache.falcon.entity.AbstractTestBase;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Clusters;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.Input;
-import org.apache.falcon.entity.v0.process.Inputs;
-import org.apache.falcon.entity.v0.process.Output;
-import org.apache.falcon.entity.v0.process.Outputs;
+import org.apache.falcon.entity.v0.process.*;
 import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class EntityGraphTest extends AbstractTestBase{
+import java.util.Set;
+
+public class EntityGraphTest extends AbstractTestBase {
 
     private ConfigurationStore store = ConfigurationStore.get();
 
@@ -101,7 +98,9 @@ public class EntityGraphTest extends AbstractTestBase{
     }
 
     private Feed addInput(Process process, String feed, Cluster cluster) {
-        if (process.getInputs() == null) process.setInputs(new Inputs());
+        if (process.getInputs() == null) {
+            process.setInputs(new Inputs());
+        }
         Inputs inputs = process.getInputs();
         Input input = new Input();
         input.setFeed(feed);
@@ -114,11 +113,13 @@ public class EntityGraphTest extends AbstractTestBase{
                 new org.apache.falcon.entity.v0.feed.Cluster();
         feedCluster.setName(cluster.getName());
         clusters.getClusters().add(feedCluster);
-        return f1; 
+        return f1;
     }
 
     private void attachInput(Process process, Feed feed) {
-        if (process.getInputs() == null) process.setInputs(new Inputs());
+        if (process.getInputs() == null) {
+            process.setInputs(new Inputs());
+        }
         Inputs inputs = process.getInputs();
         Input input = new Input();
         input.setFeed(feed.getName());
@@ -126,7 +127,9 @@ public class EntityGraphTest extends AbstractTestBase{
     }
 
     private Feed addOutput(Process process, String feed, Cluster cluster) {
-        if (process.getOutputs() == null) process.setOutputs(new Outputs());
+        if (process.getOutputs() == null) {
+            process.setOutputs(new Outputs());
+        }
         Outputs Outputs = process.getOutputs();
         Output Output = new Output();
         Output.setFeed(feed);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java b/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
index bd5fedb..f7812b9 100644
--- a/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
+++ b/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
@@ -17,12 +17,7 @@
  */
 package org.apache.falcon.group;
 
-import java.util.Map;
-
-import javax.xml.bind.JAXBException;
-
 import junit.framework.Assert;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.AbstractTestBase;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -36,166 +31,169 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-public class FeedGroupMapTest extends AbstractTestBase{
-	private ConfigurationStore store = ConfigurationStore.get();
-	private static Cluster cluster;
-
-	@BeforeClass
-	public void setUp() throws Exception {
-		cluster = (Cluster) EntityType.CLUSTER
-				.getUnmarshaller()
-				.unmarshal(
-						FeedGroupMapTest.class
-								.getResourceAsStream("/config/cluster/cluster-0.1.xml"));
-	}
-
-	@BeforeMethod
-	public void cleanup() throws Exception {
-	    cleanupStore();
-	}
-	
-	@Test
-	public void testOnAdd() throws FalconException, JAXBException {
-		store.publish(EntityType.CLUSTER, cluster);
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("f1");
-		feed1.setGroups("group1,group2,group3");
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
-		location.setType(LocationType.DATA);
-		feed1.setLocations(new Locations());
-		feed1.getLocations().getLocations().add(location);
-		store.publish(EntityType.FEED, feed1);
-		Map<String, FeedGroup> groupMapping = FeedGroupMap.get()
-				.getGroupsMapping();
-
-		FeedGroup group = groupMapping.get("group1");
-		Assert.assertEquals(group.getName(), "group1");
-		Assert.assertEquals(group.getFeeds().size(), 1);
-		assertFields(group, feed1);
-
-		group = groupMapping.get("group2");
-		Assert.assertEquals(group.getName(), "group2");
-		Assert.assertEquals(group.getFeeds().size(), 1);
-		assertFields(group, feed1);
-
-		group = groupMapping.get("group3");
-		Assert.assertEquals(group.getName(), "group3");
-		Assert.assertEquals(group.getFeeds().size(), 1);
-		assertFields(group, feed1);
-
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-
-		feed2.setName("f2");
-		feed2.setGroups("group1,group5,group3");
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/${MONTH}/${DAY}/ad2");
+import javax.xml.bind.JAXBException;
+import java.util.Map;
+
+public class FeedGroupMapTest extends AbstractTestBase {
+    private ConfigurationStore store = ConfigurationStore.get();
+    private static Cluster cluster;
+
+    @BeforeClass
+    public void setUp() throws Exception {
+        cluster = (Cluster) EntityType.CLUSTER
+                .getUnmarshaller()
+                .unmarshal(
+                        FeedGroupMapTest.class
+                                .getResourceAsStream("/config/cluster/cluster-0.1.xml"));
+    }
+
+    @BeforeMethod
+    public void cleanup() throws Exception {
+        cleanupStore();
+    }
+
+    @Test
+    public void testOnAdd() throws FalconException, JAXBException {
+        store.publish(EntityType.CLUSTER, cluster);
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("f1");
+        feed1.setGroups("group1,group2,group3");
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        location.setType(LocationType.DATA);
+        feed1.setLocations(new Locations());
+        feed1.getLocations().getLocations().add(location);
+        store.publish(EntityType.FEED, feed1);
+        Map<String, FeedGroup> groupMapping = FeedGroupMap.get()
+                .getGroupsMapping();
+
+        FeedGroup group = groupMapping.get("group1");
+        Assert.assertEquals(group.getName(), "group1");
+        Assert.assertEquals(group.getFeeds().size(), 1);
+        assertFields(group, feed1);
+
+        group = groupMapping.get("group2");
+        Assert.assertEquals(group.getName(), "group2");
+        Assert.assertEquals(group.getFeeds().size(), 1);
+        assertFields(group, feed1);
+
+        group = groupMapping.get("group3");
+        Assert.assertEquals(group.getName(), "group3");
+        Assert.assertEquals(group.getFeeds().size(), 1);
+        assertFields(group, feed1);
+
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+
+        feed2.setName("f2");
+        feed2.setGroups("group1,group5,group3");
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/${MONTH}/${DAY}/ad2");
         location.setType(LocationType.DATA);
         feed2.setLocations(new Locations());
         feed2.getLocations().getLocations().add(location);
-		store.publish(EntityType.FEED, feed2);
-		groupMapping = FeedGroupMap.get().getGroupsMapping();
-
-		group = groupMapping.get("group1");
-		Assert.assertEquals(group.getName(), "group1");
-		Assert.assertEquals(group.getFeeds().size(), 2);
-		assertFields(group, feed2);
-
-		group = groupMapping.get("group2");
-		Assert.assertEquals(group.getName(), "group2");
-		Assert.assertEquals(group.getFeeds().size(), 1);
-		assertFields(group, feed2);
-
-		group = groupMapping.get("group3");
-		Assert.assertEquals(group.getName(), "group3");
-		Assert.assertEquals(group.getFeeds().size(), 2);
-		assertFields(group, feed2);
-
-		group = groupMapping.get("group5");
-		Assert.assertEquals(group.getName(), "group5");
-		Assert.assertEquals(group.getFeeds().size(), 1);
-		assertFields(group, feed2);
-
-	}
-
-	@Test
-	public void testOnRemove() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("f1");
-		store.publish(EntityType.CLUSTER, cluster);
-		feed1.setGroups("group7,group8,group9");
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        store.publish(EntityType.FEED, feed2);
+        groupMapping = FeedGroupMap.get().getGroupsMapping();
+
+        group = groupMapping.get("group1");
+        Assert.assertEquals(group.getName(), "group1");
+        Assert.assertEquals(group.getFeeds().size(), 2);
+        assertFields(group, feed2);
+
+        group = groupMapping.get("group2");
+        Assert.assertEquals(group.getName(), "group2");
+        Assert.assertEquals(group.getFeeds().size(), 1);
+        assertFields(group, feed2);
+
+        group = groupMapping.get("group3");
+        Assert.assertEquals(group.getName(), "group3");
+        Assert.assertEquals(group.getFeeds().size(), 2);
+        assertFields(group, feed2);
+
+        group = groupMapping.get("group5");
+        Assert.assertEquals(group.getName(), "group5");
+        Assert.assertEquals(group.getFeeds().size(), 1);
+        assertFields(group, feed2);
+
+    }
+
+    @Test
+    public void testOnRemove() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("f1");
+        store.publish(EntityType.CLUSTER, cluster);
+        feed1.setGroups("group7,group8,group9");
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
         location.setType(LocationType.DATA);
         feed1.setLocations(new Locations());
         feed1.getLocations().getLocations().add(location);
-		store.publish(EntityType.FEED, feed1);
-
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed2.setName("f2");
-		feed2.setGroups("group7,group8,group10");
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/${MONTH}/${DAY}/ad2");
+        store.publish(EntityType.FEED, feed1);
+
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed2.setName("f2");
+        feed2.setGroups("group7,group8,group10");
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/${MONTH}/${DAY}/ad2");
         location.setType(LocationType.DATA);
         feed2.setLocations(new Locations());
         feed2.getLocations().getLocations().add(location);
-		store.publish(EntityType.FEED, feed2);
+        store.publish(EntityType.FEED, feed2);
 
-		Map<String, FeedGroup> groupMapping = FeedGroupMap.get()
-				.getGroupsMapping();
+        Map<String, FeedGroup> groupMapping = FeedGroupMap.get()
+                .getGroupsMapping();
 
-		store.remove(EntityType.FEED, "f2");
+        store.remove(EntityType.FEED, "f2");
 
-		FeedGroup group = groupMapping.get("group7");
-		Assert.assertEquals(group.getName(), "group7");
-		Assert.assertEquals(group.getFeeds().size(), 1);
+        FeedGroup group = groupMapping.get("group7");
+        Assert.assertEquals(group.getName(), "group7");
+        Assert.assertEquals(group.getFeeds().size(), 1);
 
-		group = groupMapping.get("group8");
-		Assert.assertEquals(group.getName(), "group8");
-		Assert.assertEquals(group.getFeeds().size(), 1);
+        group = groupMapping.get("group8");
+        Assert.assertEquals(group.getName(), "group8");
+        Assert.assertEquals(group.getFeeds().size(), 1);
 
-		group = groupMapping.get("group10");
-		Assert.assertEquals(null, group);
+        group = groupMapping.get("group10");
+        Assert.assertEquals(null, group);
 
-		store.remove(EntityType.FEED, "f1");
+        store.remove(EntityType.FEED, "f1");
 
-		group = groupMapping.get("group7");
-		Assert.assertEquals(null, group);
+        group = groupMapping.get("group7");
+        Assert.assertEquals(null, group);
 
-		group = groupMapping.get("group8");
-		Assert.assertEquals(null, group);
+        group = groupMapping.get("group8");
+        Assert.assertEquals(null, group);
 
-		group = groupMapping.get("group9");
-		Assert.assertEquals(null, group);
+        group = groupMapping.get("group9");
+        Assert.assertEquals(null, group);
 
-	}
+    }
 
-	@Test
-	public void testNullGroup() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("f5" + System.currentTimeMillis());
-		store.publish(EntityType.CLUSTER, cluster);
-		feed1.setGroups(null);
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+    @Test
+    public void testNullGroup() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("f5" + System.currentTimeMillis());
+        store.publish(EntityType.CLUSTER, cluster);
+        feed1.setGroups(null);
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
         location.setType(LocationType.DATA);
         feed1.setLocations(new Locations());
         feed1.getLocations().getLocations().add(location);
-		store.publish(EntityType.FEED, feed1);
+        store.publish(EntityType.FEED, feed1);
 
-	}
+    }
 
-	private void assertFields(FeedGroup group, Feed feed) {
-		Assert.assertEquals(group.getFrequency(), feed.getFrequency());
-		Assert.assertEquals(group.getDatePattern(),
-				"[${DAY}, ${MONTH}, ${YEAR}]");
-	}
+    private void assertFields(FeedGroup group, Feed feed) {
+        Assert.assertEquals(group.getFrequency(), feed.getFrequency());
+        Assert.assertEquals(group.getDatePattern(),
+                "[${DAY}, ${MONTH}, ${YEAR}]");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java b/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
index 3ba095f..b31e3cc 100644
--- a/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
+++ b/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
@@ -23,7 +23,7 @@ import org.testng.annotations.Test;
 
 public class CurrentUserTest {
 
-    @Test(threadPoolSize = 10, invocationCount = 10,  timeOut = 10000)
+    @Test(threadPoolSize = 10, invocationCount = 10, timeOut = 10000)
     public void testGetUser() throws Exception {
         String id = Long.toString(System.nanoTime());
         CurrentUser.authenticate(id);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 9777088..11e5d06 100644
--- a/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
+++ b/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
@@ -27,11 +27,7 @@ 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.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.feed.*;
 import org.apache.falcon.entity.v0.process.PolicyType;
 import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
@@ -53,40 +49,40 @@ public class UpdateHelperTest extends AbstractTestBase {
         setup();
     }
 
-	@AfterClass
-	public void tearDown() {
-		this.dfsCluster.shutdown();
-	}
+    @AfterClass
+    public void tearDown() {
+        this.dfsCluster.shutdown();
+    }
 
-	@BeforeMethod
-	public void setUp() throws Exception {
+    @BeforeMethod
+    public void setUp() throws Exception {
         storeEntity(EntityType.CLUSTER, "testCluster");
         storeEntity(EntityType.CLUSTER, "backupCluster");
         storeEntity(EntityType.FEED, "clicksFeed");
         storeEntity(EntityType.FEED, "impressionFeed");
         storeEntity(EntityType.FEED, "imp-click-join1");
         storeEntity(EntityType.FEED, "imp-click-join2");
-	}
-	
-	@Test
-	public void testShouldUpdate2() throws Exception {
+    }
+
+    @Test
+    public void testShouldUpdate2() throws Exception {
         Feed oldFeed = parser.parseAndValidate(this.getClass()
                 .getResourceAsStream(FEED_XML));
         String cluster = "testCluster";
-        Feed newFeed = (Feed)oldFeed.clone();
+        Feed newFeed = (Feed) oldFeed.clone();
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, cluster));
-        
+
         newFeed.setGroups("newgroups");
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, cluster));
         newFeed.getLateArrival().setCutOff(Frequency.fromString("hours(8)"));
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, cluster));
         newFeed.setFrequency(Frequency.fromString("days(1)"));
         Assert.assertTrue(UpdateHelper.shouldUpdate(oldFeed, newFeed, cluster));
-        
+
         Process oldProcess = processParser.parseAndValidate(this.getClass().
                 getResourceAsStream(PROCESS_XML));
         Process newProcess = (Process) oldProcess.clone();
-        
+
         newProcess.getRetry().setPolicy(PolicyType.FINAL);
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldProcess, newProcess, cluster));
         newProcess.getLateProcess().getLateInputs().remove(1);
@@ -94,15 +90,15 @@ public class UpdateHelperTest extends AbstractTestBase {
         newProcess.getLateProcess().setPolicy(PolicyType.PERIODIC);
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldProcess, newProcess, cluster));
         newProcess.setFrequency(Frequency.fromString("days(1)"));
-        Assert.assertTrue(UpdateHelper.shouldUpdate(oldProcess, newProcess, cluster));        
-	}
-	
+        Assert.assertTrue(UpdateHelper.shouldUpdate(oldProcess, newProcess, cluster));
+    }
+
     @Test
     public void testShouldUpdate() throws Exception {
         Feed oldFeed = parser.parseAndValidate(this.getClass()
                 .getResourceAsStream(FEED_XML));
 
-        Feed newFeed = (Feed)oldFeed.clone();
+        Feed newFeed = (Feed) oldFeed.clone();
         Process process = processParser.parseAndValidate(this.getClass().
                 getResourceAsStream(PROCESS_XML));
 
@@ -142,11 +138,13 @@ public class UpdateHelperTest extends AbstractTestBase {
         newFeed.getProperties().getProperties().remove(0);
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, process));
 
-        FeedHelper.getCluster(newFeed, process.getClusters().getClusters().get(0).getName()).getValidity().setStart(SchemaHelper.parseDateUTC("2012-11-01T00:00Z"));
+        FeedHelper.getCluster(newFeed, process.getClusters().getClusters().get(0).getName()).getValidity().setStart(
+                SchemaHelper.parseDateUTC("2012-11-01T00:00Z"));
         Assert.assertTrue(UpdateHelper.shouldUpdate(oldFeed, newFeed, process));
 
         FeedHelper.getCluster(newFeed, process.getClusters().getClusters().get(0).getName()).getValidity().
-                setStart(FeedHelper.getCluster(oldFeed, process.getClusters().getClusters().get(0).getName()).getValidity().getStart());
+                setStart(FeedHelper.getCluster(oldFeed,
+                        process.getClusters().getClusters().get(0).getName()).getValidity().getStart());
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, process));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java b/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
index 6e6a782..fc3d604 100644
--- a/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
+++ b/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
@@ -18,17 +18,18 @@
 
 package org.apache.falcon.util;
 
-import static org.testng.AssertJUnit.assertEquals;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import static org.testng.AssertJUnit.assertEquals;
+
 @Test
 public class StartupPropertiesTest {
     @BeforeClass
     public void setUp() {
         StartupProperties.get();
     }
-    
+
     public void testDomain() {
         StartupProperties props = (StartupProperties) StartupProperties.get();
         assertEquals("debug", props.getDomain());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/cluster/cluster-0.1.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/cluster/cluster-0.1.xml b/common/src/test/resources/config/cluster/cluster-0.1.xml
index ee59616..cf53398 100644
--- a/common/src/test/resources/config/cluster/cluster-0.1.xml
+++ b/common/src/test/resources/config/cluster/cluster-0.1.xml
@@ -17,26 +17,27 @@
   limitations under the License.
   -->
 
-<cluster colo="default" description="" name="testCluster" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="hftp://localhost:50010"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="staging" path="/projects/falcon/staging" />
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/falcon/working" />
-	</locations>
-	<properties>
-		<property name="field1" value="value1" />
-		<property name="field2" value="value2" />
-	</properties>
+<cluster colo="default" description="" name="testCluster" xmlns="uri:falcon:cluster:0.1"
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:50010"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="staging" path="/projects/falcon/staging"/>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+    </locations>
+    <properties>
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/feed/feed-0.1.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/feed/feed-0.1.xml b/common/src/test/resources/config/feed/feed-0.1.xml
index ee04499..0269fa0 100644
--- a/common/src/test/resources/config/feed/feed-0.1.xml
+++ b/common/src/test/resources/config/feed/feed-0.1.xml
@@ -16,10 +16,11 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="good" />
+<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="good"/>
     </partitions>
 
     <groups>online,bi</groups>
@@ -27,31 +28,33 @@
 
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    
-    <late-arrival cut-off="hours(6)" />
+
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="testCluster" type="source">
             <validity start="2011-11-01T00:00Z" end="2011-12-31T00:00Z"/>
-            <retention limit="hours(48)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
-       		<locations>
-        		<location type="data" path="/projects/falcon/clicks" />
-        		<location type="stats" path="/projects/falcon/clicksStats" />
-        		<location type="meta" path="/projects/falcon/clicksMetaData" />
-    		</locations>
+            <retention limit="hours(48)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <locations>
+                <location type="data" path="/projects/falcon/clicks"/>
+                <location type="stats" path="/projects/falcon/clicksStats"/>
+                <location type="meta" path="/projects/falcon/clicksMetaData"/>
+            </locations>
         </cluster>
         <cluster name="backupCluster" type="target">
             <validity start="2011-11-01T00:00Z" end="2011-12-31T00:00Z"/>
-            <retention limit="hours(6)" action="archive" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="archive"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/feed/feed-0.2.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/feed/feed-0.2.xml b/common/src/test/resources/config/feed/feed-0.2.xml
index 1f15cdf..a3f4f4d 100644
--- a/common/src/test/resources/config/feed/feed-0.2.xml
+++ b/common/src/test/resources/config/feed/feed-0.2.xml
@@ -16,35 +16,38 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="good" />
+<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="good"/>
     </partitions>
 
     <groups>online,bi</groups>
 
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    <late-arrival cut-off="hours(6)" />
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="testCluster" type="source" partition="*/${cluster.colo}">
             <validity start="2021-11-01T00:00Z" end="2021-12-31T00:00Z"/>
-            <retention limit="hours(48)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(48)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
         <cluster name="backupCluster" type="target">
             <validity start="2011-11-01T00:00Z" end="2011-12-31T00:00Z"/>
-            <retention limit="hours(6)" action="archive" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="archive"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
 
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/process/process-0.1.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/process/process-0.1.xml b/common/src/test/resources/config/process/process-0.1.xml
index 196f2cb..bb5cd35 100644
--- a/common/src/test/resources/config/process/process-0.1.xml
+++ b/common/src/test/resources/config/process/process-0.1.xml
@@ -16,10 +16,10 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<process name="sample" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample" xmlns="uri:falcon:process:0.1">
     <clusters>
         <cluster name="testCluster">
-            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z" />
+            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z"/>
         </cluster>
     </clusters>
     <parallel>1</parallel>
@@ -28,27 +28,27 @@
 
     <!-- what -->
     <inputs>
-        <input name="impression" feed="impressionFeed" start="today(0,0)" end="today(2,0)" partition="*/US" />
-        <input name="clicks" feed="clicksFeed" start="yesterday(0,0)" end="yesterday(20,0)" />
+        <input name="impression" feed="impressionFeed" start="today(0,0)" end="today(2,0)" partition="*/US"/>
+        <input name="clicks" feed="clicksFeed" start="yesterday(0,0)" end="yesterday(20,0)"/>
     </inputs>
 
     <outputs>
-        <output name="impOutput" feed="imp-click-join1" instance="today(0,0)" />
-        <output name="clicksOutput" feed="imp-click-join2" instance="today(0,0)" />
+        <output name="impOutput" feed="imp-click-join1" instance="today(0,0)"/>
+        <output name="clicksOutput" feed="imp-click-join2" instance="today(0,0)"/>
     </outputs>
 
     <!-- how -->
     <properties>
-        <property name="name1" value="value1" />
-        <property name="name2" value="value2" />
+        <property name="name1" value="value1"/>
+        <property name="name2" value="value2"/>
     </properties>
 
-    <workflow engine="oozie" path="/path/to/workflow" />
+    <workflow engine="oozie" path="/path/to/workflow"/>
 
-    <retry policy="periodic" delay="minutes(10)" attempts="3" />
+    <retry policy="periodic" delay="minutes(10)" attempts="3"/>
 
     <late-process policy="exp-backoff" delay="hours(1)">
-        <late-input input="impression" workflow-path="himpression/late/workflow" />
-        <late-input input="clicks" workflow-path="hdfs://clicks/late/workflow" />
+        <late-input input="impression" workflow-path="himpression/late/workflow"/>
+        <late-input input="clicks" workflow-path="hdfs://clicks/late/workflow"/>
     </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/process/process-0.2.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/process/process-0.2.xml b/common/src/test/resources/config/process/process-0.2.xml
index 7e817e3..c4cd83e 100644
--- a/common/src/test/resources/config/process/process-0.2.xml
+++ b/common/src/test/resources/config/process/process-0.2.xml
@@ -16,43 +16,43 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<process name="sample" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample" xmlns="uri:falcon:process:0.1">
     <clusters>
         <cluster name="testCluster">
-            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z" />
+            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z"/>
         </cluster>
     </clusters>
-	<!-- when -->
-	<parallel>1</parallel>
-	<order>LIFO</order>
-	<frequency>hours(1)</frequency>
+    <!-- when -->
+    <parallel>1</parallel>
+    <order>LIFO</order>
+    <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
 
-	<!-- what -->
-	<inputs>
-		<input name="impression" feed="impressionFeed" start="today(0,0)"
-			end="today(2,0)" partition="*/US" />
-		<input name="clicks" feed="clicks" start="yesterday(0,0)"
-			end="yesterday(20,0)" />
-	</inputs>
-
-	<outputs>
-		<output name="impOutput" feed="imp-click-join1" instance="today(0,0)" />
-		<output name="clicksOutput" feed="imp-click-join2" instance="today(0,0)" />
-	</outputs>
-
-	<!-- how -->
-	<properties>
-		<property name="name1" value="value1" />
-		<property name="name2" value="value2" />
-	</properties>
-
-	<workflow engine="oozie" path="/path/to/workflow" />
-
-	<retry policy="periodic" delay="minutes(10)" attempts="3" />
-
-	<late-process policy="exp-backoff" delay="hours(1)">
-		<late-input input="impression" workflow-path="himpression/late/workflow" />
-		<late-input input="clicks" workflow-path="hdfs://clicks/late/workflow" />
-	</late-process>
+    <!-- what -->
+    <inputs>
+        <input name="impression" feed="impressionFeed" start="today(0,0)"
+               end="today(2,0)" partition="*/US"/>
+        <input name="clicks" feed="clicks" start="yesterday(0,0)"
+               end="yesterday(20,0)"/>
+    </inputs>
+
+    <outputs>
+        <output name="impOutput" feed="imp-click-join1" instance="today(0,0)"/>
+        <output name="clicksOutput" feed="imp-click-join2" instance="today(0,0)"/>
+    </outputs>
+
+    <!-- how -->
+    <properties>
+        <property name="name1" value="value1"/>
+        <property name="name2" value="value2"/>
+    </properties>
+
+    <workflow engine="oozie" path="/path/to/workflow"/>
+
+    <retry policy="periodic" delay="minutes(10)" attempts="3"/>
+
+    <late-process policy="exp-backoff" delay="hours(1)">
+        <late-input input="impression" workflow-path="himpression/late/workflow"/>
+        <late-input input="clicks" workflow-path="hdfs://clicks/late/workflow"/>
+    </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/resources/config/process/process-invalid.xml
----------------------------------------------------------------------
diff --git a/common/src/test/resources/config/process/process-invalid.xml b/common/src/test/resources/config/process/process-invalid.xml
index edd9311..d98a6e9 100644
--- a/common/src/test/resources/config/process/process-invalid.xml
+++ b/common/src/test/resources/config/process/process-invalid.xml
@@ -16,23 +16,23 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<Process name="" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-  <clusters>
-    <cluster name="">cluster</cluster>
-  </clusters>
-  <frequency>frequency</frequency>
-  <periodicity>periodicity</periodicity>
-  <validity end="" start="">validity</validity>
-  <inputs>
-    <somenode>somenode</somenode>
-    <input end="" feed="" start="">input</input>
-  </inputs>
-  <outputs>
-    <output feed="" instance="">output</output>
-  </outputs>
-  <workflow engine="" path="">workflow</workflow>
-  <retry attempts="" delay="" delayUnit="" policy="">retry</retry>
-  <late-process delay="" delayUnit="" policy="">
-    <late-input input="" workflow-path="">late-input</late-input>
-  </late-process>
+<Process name="">
+    <clusters>
+        <cluster name="">cluster</cluster>
+    </clusters>
+    <frequency>frequency</frequency>
+    <periodicity>periodicity</periodicity>
+    <validity end="" start="">validity</validity>
+    <inputs>
+        <somenode>somenode</somenode>
+        <input end="" feed="" start="">input</input>
+    </inputs>
+    <outputs>
+        <output feed="" instance="">output</output>
+    </outputs>
+    <workflow engine="" path="">workflow</workflow>
+    <retry attempts="" delay="" delayUnit="" policy="">retry</retry>
+    <late-process delay="" delayUnit="" policy="">
+        <late-input input="" workflow-path="">late-input</late-input>
+    </late-process>
 </Process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index 8e8a444..a206267 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -16,7 +16,8 @@
   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">
+<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>
         <groupId>org.apache.falcon</groupId>
@@ -37,7 +38,6 @@
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-site-plugin</artifactId>
-                <version>3.0</version>
                 <dependencies>
                     <dependency>
                         <groupId>org.apache.maven.doxia</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/docs/src/site/site.xml
----------------------------------------------------------------------
diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml
index cbc230b..4238dee 100644
--- a/docs/src/site/site.xml
+++ b/docs/src/site/site.xml
@@ -21,11 +21,11 @@
         <name>Falcon</name>
     </bannerLeft>
 
-  <skin>
-    <groupId>org.apache.maven.skins</groupId>
-    <artifactId>maven-classic-skin</artifactId>
-    <version>1.0</version>
-  </skin>
+    <skin>
+        <groupId>org.apache.maven.skins</groupId>
+        <artifactId>maven-classic-skin</artifactId>
+        <version>1.0</version>
+    </skin>
 
     <body/>
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java b/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
index 73ce42b..d02f6c1 100644
--- a/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
+++ b/feed/src/main/java/org/apache/falcon/converter/OozieFeedMapper.java
@@ -18,16 +18,7 @@
 
 package org.apache.falcon.converter;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.ClusterHelper;
@@ -50,8 +41,12 @@ import org.apache.falcon.oozie.coordinator.COORDINATORAPP;
 import org.apache.falcon.oozie.coordinator.SYNCDATASET;
 import org.apache.falcon.oozie.coordinator.WORKFLOW;
 import org.apache.falcon.oozie.workflow.WORKFLOWAPP;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+import java.io.IOException;
+import java.util.*;
+
 public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
 
     private static Logger LOG = Logger.getLogger(OozieFeedMapper.class);
@@ -61,8 +56,8 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
     private static final String RETENTION_WF_TEMPLATE = "/config/workflow/retention-workflow.xml";
     private static final String REPLICATION_COORD_TEMPLATE = "/config/coordinator/replication-coordinator.xml";
     private static final String REPLICATION_WF_TEMPLATE = "/config/workflow/replication-workflow.xml";
-    
-    private static final String FEED_PATH_SEP="#";
+
+    private static final String FEED_PATH_SEP = "#";
 
     public OozieFeedMapper(Feed feed) {
         super(feed);
@@ -86,7 +81,8 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
         org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, cluster.getName());
 
         if (feedCluster.getValidity().getEnd().before(new Date())) {
-            LOG.warn("Feed Retention is not applicable as Feed's end time for cluster " + cluster.getName() + " is not in the future");
+            LOG.warn("Feed Retention is not applicable as Feed's end time for cluster " + cluster.getName()
+                    + " is not in the future");
             return null;
         }
         COORDINATORAPP retentionApp = new COORDINATORAPP();
@@ -121,20 +117,20 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
             Map<String, String> props = createCoordDefaultConfiguration(cluster, wfPath, wfName);
 
             org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, cluster.getName());
-            String feedPathMask = getLocationURI(cluster, feed,LocationType.DATA);
-			String metaPathMask = getLocationURI(cluster, feed, LocationType.META);
+            String feedPathMask = getLocationURI(cluster, feed, LocationType.DATA);
+            String metaPathMask = getLocationURI(cluster, feed, LocationType.META);
             String statsPathMask = getLocationURI(cluster, feed, LocationType.STATS);
             String tmpPathMask = getLocationURI(cluster, feed, LocationType.TMP);
 
             StringBuilder feedBasePaths = new StringBuilder(feedPathMask);
-            if(metaPathMask!=null){
-            	feedBasePaths.append(FEED_PATH_SEP).append(metaPathMask);
+            if (metaPathMask != null) {
+                feedBasePaths.append(FEED_PATH_SEP).append(metaPathMask);
             }
-            if(statsPathMask!=null){
-            	feedBasePaths.append(FEED_PATH_SEP).append(statsPathMask);
+            if (statsPathMask != null) {
+                feedBasePaths.append(FEED_PATH_SEP).append(statsPathMask);
             }
-            if(tmpPathMask!=null){
-            	feedBasePaths.append(FEED_PATH_SEP).append(tmpPathMask);
+            if (tmpPathMask != null) {
+                feedBasePaths.append(FEED_PATH_SEP).append(tmpPathMask);
             }
 
             props.put("feedDataPath", feedBasePaths.toString().replaceAll("\\$\\{", "\\?\\{"));
@@ -153,23 +149,25 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
         }
     }
 
-    private List<COORDINATORAPP> getReplicationCoordinators(Cluster targetCluster, Path bundlePath) throws FalconException {
+    private List<COORDINATORAPP> getReplicationCoordinators(Cluster targetCluster, Path bundlePath)
+            throws FalconException {
         Feed feed = getEntity();
         List<COORDINATORAPP> replicationCoords = new ArrayList<COORDINATORAPP>();
-        
+
         if (FeedHelper.getCluster(feed, targetCluster.getName()).getType() == ClusterType.TARGET) {
             String coordName = EntityUtil.getWorkflowName(Tag.REPLICATION, feed).toString();
             Path basePath = getCoordPath(bundlePath, coordName);
             createReplicatonWorkflow(targetCluster, basePath, coordName);
-            
+
             for (org.apache.falcon.entity.v0.feed.Cluster feedCluster : feed.getClusters().getClusters()) {
                 if (feedCluster.getType() == ClusterType.SOURCE) {
                     COORDINATORAPP coord = createAndGetCoord(feed,
-                            (Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, feedCluster.getName()), targetCluster,
+                            (Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, feedCluster.getName()),
+                            targetCluster,
                             bundlePath);
-					if (coord != null) {
-						replicationCoords.add(coord);
-					}
+                    if (coord != null) {
+                        replicationCoords.add(coord);
+                    }
                 }
             }
 
@@ -183,14 +181,17 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
         String coordName;
         try {
             replicationCoord = getCoordinatorTemplate(REPLICATION_COORD_TEMPLATE);
-            coordName = EntityUtil.getWorkflowName(Tag.REPLICATION, Arrays.asList(srcCluster.getName()), feed).toString();
+            coordName = EntityUtil.getWorkflowName(Tag.REPLICATION, Arrays.asList(srcCluster.getName()),
+                    feed).toString();
             replicationCoord.setName(coordName);
             replicationCoord.setFrequency("${coord:" + feed.getFrequency().toString() + "}");
 
             long frequency_ms = ExpressionHelper.get().
                     evaluate(feed.getFrequency().toString(), Long.class);
             long timeout_ms = frequency_ms * 6;
-            if (timeout_ms < THIRTY_MINUTES) timeout_ms = THIRTY_MINUTES;
+            if (timeout_ms < THIRTY_MINUTES) {
+                timeout_ms = THIRTY_MINUTES;
+            }
             replicationCoord.getControls().setTimeout(String.valueOf(timeout_ms / (1000 * 60)));
             replicationCoord.getControls().setThrottle(String.valueOf(timeout_ms / frequency_ms * 2));
 
@@ -198,28 +199,30 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
             Date srcEndDate = FeedHelper.getCluster(feed, srcCluster.getName()).getValidity().getEnd();
             Date trgStartDate = FeedHelper.getCluster(feed, trgCluster.getName()).getValidity().getStart();
             Date trgEndDate = FeedHelper.getCluster(feed, trgCluster.getName()).getValidity().getEnd();
-			if (srcStartDate.after(trgEndDate)
-					|| trgStartDate.after(srcEndDate)) {
-				LOG.warn("Not creating replication coordinator, as the source cluster:"
-						+ srcCluster.getName()
-						+ " and target cluster: "
-						+ trgCluster.getName()
-						+ " do not have overlapping dates");
-				return null;
-			}
-            replicationCoord.setStart(srcStartDate.after(trgStartDate) ? SchemaHelper.formatDateUTC(srcStartDate) : SchemaHelper
-                    .formatDateUTC(trgStartDate));
-            replicationCoord.setEnd(srcEndDate.before(trgEndDate) ? SchemaHelper.formatDateUTC(srcEndDate) : SchemaHelper
-                    .formatDateUTC(trgEndDate));
+            if (srcStartDate.after(trgEndDate)
+                    || trgStartDate.after(srcEndDate)) {
+                LOG.warn("Not creating replication coordinator, as the source cluster:"
+                        + srcCluster.getName()
+                        + " and target cluster: "
+                        + trgCluster.getName()
+                        + " do not have overlapping dates");
+                return null;
+            }
+            replicationCoord.setStart(
+                    srcStartDate.after(trgStartDate) ? SchemaHelper.formatDateUTC(srcStartDate) : SchemaHelper
+                            .formatDateUTC(trgStartDate));
+            replicationCoord.setEnd(
+                    srcEndDate.before(trgEndDate) ? SchemaHelper.formatDateUTC(srcEndDate) : SchemaHelper
+                            .formatDateUTC(trgEndDate));
             replicationCoord.setTimezone(feed.getTimezone().getID());
             SYNCDATASET inputDataset = (SYNCDATASET) replicationCoord.getDatasets().getDatasetOrAsyncDataset().get(0);
             SYNCDATASET outputDataset = (SYNCDATASET) replicationCoord.getDatasets().getDatasetOrAsyncDataset().get(1);
 
-			inputDataset.setUriTemplate(new Path(ClusterHelper
-					.getStorageUrl(srcCluster), FeedHelper.getLocation(feed,
-					LocationType.DATA,srcCluster.getName()).getPath()).toString());
-			outputDataset.setUriTemplate(getStoragePath(FeedHelper.getLocation(
-					feed, LocationType.DATA, trgCluster.getName()).getPath()));
+            inputDataset.setUriTemplate(new Path(ClusterHelper
+                    .getStorageUrl(srcCluster), FeedHelper.getLocation(feed,
+                    LocationType.DATA, srcCluster.getName()).getPath()).toString());
+            outputDataset.setUriTemplate(getStoragePath(FeedHelper.getLocation(
+                    feed, LocationType.DATA, trgCluster.getName()).getPath()));
             setDatasetValues(inputDataset, feed, srcCluster);
             setDatasetValues(outputDataset, feed, srcCluster);
             if (feed.getAvailabilityFlag() == null) {
@@ -238,25 +241,30 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
     }
 
     private void setDatasetValues(SYNCDATASET dataset, Feed feed, Cluster cluster) {
-        dataset.setInitialInstance(SchemaHelper.formatDateUTC(FeedHelper.getCluster(feed, cluster.getName()).getValidity().getStart()));
+        dataset.setInitialInstance(
+                SchemaHelper.formatDateUTC(FeedHelper.getCluster(feed, cluster.getName()).getValidity().getStart()));
         dataset.setTimezone(feed.getTimezone().getID());
         dataset.setFrequency("${coord:" + feed.getFrequency().toString() + "}");
     }
 
-    private ACTION getReplicationWorkflowAction(Cluster srcCluster, Cluster trgCluster, Path wfPath, String wfName) throws FalconException {
+    private ACTION getReplicationWorkflowAction(Cluster srcCluster, Cluster trgCluster, Path wfPath, String wfName)
+            throws FalconException {
         ACTION replicationAction = new ACTION();
         WORKFLOW replicationWF = new WORKFLOW();
         try {
             replicationWF.setAppPath(getStoragePath(wfPath.toString()));
             Feed feed = getEntity();
 
-            String srcPart = FeedHelper.normalizePartitionExpression(FeedHelper.getCluster(feed, srcCluster.getName()).getPartition());
+            String srcPart = FeedHelper.normalizePartitionExpression(
+                    FeedHelper.getCluster(feed, srcCluster.getName()).getPartition());
             srcPart = FeedHelper.evaluateClusterExp(srcCluster, srcPart);
-            String targetPart = FeedHelper.normalizePartitionExpression(FeedHelper.getCluster(feed, trgCluster.getName()).getPartition());
+            String targetPart = FeedHelper.normalizePartitionExpression(
+                    FeedHelper.getCluster(feed, trgCluster.getName()).getPartition());
             targetPart = FeedHelper.evaluateClusterExp(trgCluster, targetPart);
-            
+
             StringBuilder pathsWithPartitions = new StringBuilder();
-            pathsWithPartitions.append("${coord:dataIn('input')}/").append(FeedHelper.normalizePartitionExpression(srcPart, targetPart));
+            pathsWithPartitions.append("${coord:dataIn('input')}/").append(
+                    FeedHelper.normalizePartitionExpression(srcPart, targetPart));
 
             Map<String, String> props = createCoordDefaultConfiguration(trgCluster, wfPath, wfName);
             props.put("srcClusterName", srcCluster.getName());
@@ -294,26 +302,27 @@ public class OozieFeedMapper extends AbstractOozieEntityMapper<Feed> {
         Feed feed = getEntity();
         Map<String, String> props = new HashMap<String, String>();
         if (feed.getProperties() != null) {
-            for (Property prop : feed.getProperties().getProperties())
+            for (Property prop : feed.getProperties().getProperties()) {
                 props.put(prop.getName(), prop.getValue());
+            }
         }
         return props;
     }
-    
-	private String getLocationURI(Cluster cluster, Feed feed, LocationType type) {
-		String path = FeedHelper.getLocation(feed, type, cluster.getName())
-				.getPath();
-
-		if (!path.equals("/tmp")) {
-			if (new Path(path).toUri().getScheme() == null){
-				return  new Path(ClusterHelper.getStorageUrl(cluster), path)
-						.toString();}
-			else{
-				return  path;
-			}
-		}
-		return null;
 
-}
+    private String getLocationURI(Cluster cluster, Feed feed, LocationType type) {
+        String path = FeedHelper.getLocation(feed, type, cluster.getName())
+                .getPath();
+
+        if (!path.equals("/tmp")) {
+            if (new Path(path).toUri().getScheme() == null) {
+                return new Path(ClusterHelper.getStorageUrl(cluster), path)
+                        .toString();
+            } else {
+                return path;
+            }
+        }
+        return null;
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
index 3aa9eca..02cb740 100644
--- a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
+++ b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
@@ -18,13 +18,6 @@
 
 package org.apache.falcon.workflow;
 
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
 import org.apache.falcon.converter.AbstractOozieEntityMapper;
@@ -36,29 +29,37 @@ import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.security.CurrentUser;
+import org.apache.hadoop.fs.Path;
+
+import java.util.*;
 
 public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
 
     @Override
     public Map<String, Properties> newWorkflowSchedule(Feed feed, List<String> clusters) throws FalconException {
         Map<String, Properties> propertiesMap = new HashMap<String, Properties>();
-        
-        for (String clusterName: clusters) {
+
+        for (String clusterName : clusters) {
             org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
-            Properties properties = newWorkflowSchedule(feed, feedCluster.getValidity().getStart(), clusterName, 
+            Properties properties = newWorkflowSchedule(feed, feedCluster.getValidity().getStart(), clusterName,
                     CurrentUser.getUser());
-            if (properties == null) continue;
+            if (properties == null) {
+                continue;
+            }
             propertiesMap.put(clusterName, properties);
         }
         return propertiesMap;
     }
 
     @Override
-    public Properties newWorkflowSchedule(Feed feed, Date startDate, String clusterName, String user) throws FalconException {
+    public Properties newWorkflowSchedule(Feed feed, Date startDate, String clusterName, String user)
+            throws FalconException {
         org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
         if (!startDate.before(feedCluster.getValidity().getEnd()))
-            // start time >= end time
+        // start time >= end time
+        {
             return null;
+        }
 
         Cluster cluster = configStore.get(EntityType.CLUSTER, feedCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(feed));
@@ -66,7 +67,7 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
         EntityUtil.setStartDate(feedClone, clusterName, startDate);
 
         AbstractOozieEntityMapper<Feed> mapper = new OozieFeedMapper(feedClone);
-        if(!mapper.map(cluster, bundlePath)){
+        if (!mapper.map(cluster, bundlePath)) {
             return null;
         }
         return createAppProperties(clusterName, bundlePath, user);
@@ -79,10 +80,10 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
                 feed.getFrequency(), feed.getTimezone(), now);
     }
 
-	@Override
-	public String[] getWorkflowNames(Feed entity) {
-		return new String[] {
-				EntityUtil.getWorkflowName(Tag.RETENTION, entity).toString(),
-				EntityUtil.getWorkflowName(Tag.REPLICATION, entity).toString() };
-	}
+    @Override
+    public String[] getWorkflowNames(Feed entity) {
+        return new String[]{
+                EntityUtil.getWorkflowName(Tag.RETENTION, entity).toString(),
+                EntityUtil.getWorkflowName(Tag.REPLICATION, entity).toString()};
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/main/resources/config/coordinator/replication-coordinator.xml
----------------------------------------------------------------------
diff --git a/feed/src/main/resources/config/coordinator/replication-coordinator.xml b/feed/src/main/resources/config/coordinator/replication-coordinator.xml
index ac1267a..693b0bd 100644
--- a/feed/src/main/resources/config/coordinator/replication-coordinator.xml
+++ b/feed/src/main/resources/config/coordinator/replication-coordinator.xml
@@ -16,36 +16,36 @@
   limitations under the License.
   -->
 <coordinator-app name="#NAME#" frequency="#FREQUENCY#"
-	start="#START_TIME#" end="#END_TIME" timezone="#TIMEZONE#"
-	xmlns="uri:oozie:coordinator:0.3">
-	<controls>
-		<concurrency>1</concurrency>
-		<execution>FIFO</execution>
-	</controls>
-	<datasets>
-		<dataset name="input-dataset" frequency="#FEED_FREQ#"
-			initial-instance="#START_TIME#" timezone="#TIMEZONE#">
-			<uri-template>#FEED_PATH#</uri-template>
-		</dataset>
-		<dataset name="output-dataset" frequency="#FEED_FREQ#"
-			initial-instance="#START_TIME#" timezone="#TIMEZONE#">
-			<uri-template>#FEED_PATH#</uri-template>
-		</dataset>
-	</datasets>
-	<input-events>
-		<data-in name="input" dataset="input-dataset">
-			<instance>${coord:current(0)}</instance>
-		</data-in>
-	</input-events>
-	<output-events>
-		<data-out name="output" dataset="output-dataset">
-			<instance>${coord:current(0)}</instance>
-		</data-out>
-	</output-events>
-	<action>
-		<workflow>
-			<app-path>#WF_PATH#</app-path>
-			<configuration />
-		</workflow>
-	</action>
+                 start="#START_TIME#" end="#END_TIME" timezone="#TIMEZONE#"
+                 xmlns="uri:oozie:coordinator:0.3">
+    <controls>
+        <concurrency>1</concurrency>
+        <execution>FIFO</execution>
+    </controls>
+    <datasets>
+        <dataset name="input-dataset" frequency="#FEED_FREQ#"
+                 initial-instance="#START_TIME#" timezone="#TIMEZONE#">
+            <uri-template>#FEED_PATH#</uri-template>
+        </dataset>
+        <dataset name="output-dataset" frequency="#FEED_FREQ#"
+                 initial-instance="#START_TIME#" timezone="#TIMEZONE#">
+            <uri-template>#FEED_PATH#</uri-template>
+        </dataset>
+    </datasets>
+    <input-events>
+        <data-in name="input" dataset="input-dataset">
+            <instance>${coord:current(0)}</instance>
+        </data-in>
+    </input-events>
+    <output-events>
+        <data-out name="output" dataset="output-dataset">
+            <instance>${coord:current(0)}</instance>
+        </data-out>
+    </output-events>
+    <action>
+        <workflow>
+            <app-path>#WF_PATH#</app-path>
+            <configuration/>
+        </workflow>
+    </action>
 </coordinator-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/main/resources/config/workflow/replication-workflow.xml
----------------------------------------------------------------------
diff --git a/feed/src/main/resources/config/workflow/replication-workflow.xml b/feed/src/main/resources/config/workflow/replication-workflow.xml
index 5bb34e8..7aab158 100644
--- a/feed/src/main/resources/config/workflow/replication-workflow.xml
+++ b/feed/src/main/resources/config/workflow/replication-workflow.xml
@@ -16,99 +16,125 @@
   limitations under the License.
   -->
 <workflow-app xmlns='uri:oozie:workflow:0.3' name='falcon-feed-parent-workflow'>
-	<start to='should-record' />
-	<decision name='should-record'>
+    <start to='should-record'/>
+    <decision name='should-record'>
         <switch>
-            <case to="recordsize"> 
-              ${shouldRecord=="true"}
+            <case to="recordsize">
+                ${shouldRecord=="true"}
             </case>
             <default to="replication"/>
         </switch>
     </decision>
-	<action name='recordsize'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.latedata.LateDataHandler</main-class>
-			<arg>-out</arg><arg>${logDir}/latedata/${nominalTime}/${srcClusterName}</arg>
-			<arg>-paths</arg><arg>${falconInPaths}</arg>
-			<arg>-falconInputFeeds</arg><arg>${falconInputFeeds}</arg>
-			<capture-output />
-		</java>
-		<ok to="replication" />
-		<error to="fail" />
-	</action>
+    <action name='recordsize'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.latedata.LateDataHandler</main-class>
+            <arg>-out</arg>
+            <arg>${logDir}/latedata/${nominalTime}/${srcClusterName}</arg>
+            <arg>-paths</arg>
+            <arg>${falconInPaths}</arg>
+            <arg>-falconInputFeeds</arg>
+            <arg>${falconInputFeeds}</arg>
+            <capture-output/>
+        </java>
+        <ok to="replication"/>
+        <error to="fail"/>
+    </action>
     <action name="replication">
         <java>
             <job-tracker>${jobTracker}</job-tracker>
             <name-node>${nameNode}</name-node>
             <configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
             <main-class>org.apache.falcon.replication.FeedReplicator</main-class>
             <arg>-Dfalcon.include.path=${sourceRelativePaths}</arg>
             <arg>-Dmapred.job.queue.name=${queueName}</arg>
             <arg>-Dmapred.job.priority=${jobPriority}</arg>
-			<arg>-maxMaps</arg><arg>5</arg>
-			<arg>-sourcePaths</arg><arg>${distcpSourcePaths}</arg>
-			<arg>-targetPath</arg><arg>${distcpTargetPaths}</arg>
+            <arg>-maxMaps</arg>
+            <arg>5</arg>
+            <arg>-sourcePaths</arg>
+            <arg>${distcpSourcePaths}</arg>
+            <arg>-targetPath</arg>
+            <arg>${distcpTargetPaths}</arg>
             <file>${wf:conf("falcon.libpath")}/hadoop-distcp.jar</file>
         </java>
         <ok to="succeeded-post-processing"/>
         <error to="failed-post-processing"/>
     </action>
-   <action name='succeeded-post-processing'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
-			<arg>-cluster</arg><arg>${cluster}</arg>
-			<arg>-entityType</arg><arg>${entityType}</arg>
-			<arg>-entityName</arg><arg>${entityName}</arg>
-			<arg>-nominalTime</arg><arg>${nominalTime}</arg>
-			<arg>-operation</arg><arg>REPLICATE</arg>
-			<arg>-workflowId</arg><arg>${wf:id()}</arg>
-			<arg>-runId</arg><arg>${wf:run()}</arg>
-			<arg>-status</arg><arg>SUCCEEDED</arg>
-			<arg>-timeStamp</arg><arg>${timeStamp}</arg>			
-			<arg>-brokerImplClass</arg><arg>${wf:conf("broker.impl.class")}</arg>
-			<arg>-brokerUrl</arg><arg>${wf:conf("broker.url")}</arg>
-			<arg>-userBrokerImplClass</arg><arg>${userBrokerImplClass}</arg>
-			<arg>-userBrokerUrl</arg><arg>${userBrokerUrl}</arg>
-			<arg>-brokerTTL</arg><arg>${wf:conf("broker.ttlInMins")}</arg>
-			<arg>-feedNames</arg><arg>${feedNames}</arg>
-			<arg>-feedInstancePaths</arg><arg>${feedInstancePaths}</arg>			
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}-${srcClusterName}.csv</arg>
-			<arg>-workflowEngineUrl</arg> <arg>${workflowEngineUrl}</arg>
-			<arg>-subflowId</arg> <arg>${wf:id()}</arg>
-			<arg>-logDir</arg> <arg>${logDir}/job-${nominalTime}/${srcClusterName}/</arg>
+    <action name='succeeded-post-processing'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
+            <arg>-cluster</arg>
+            <arg>${cluster}</arg>
+            <arg>-entityType</arg>
+            <arg>${entityType}</arg>
+            <arg>-entityName</arg>
+            <arg>${entityName}</arg>
+            <arg>-nominalTime</arg>
+            <arg>${nominalTime}</arg>
+            <arg>-operation</arg>
+            <arg>REPLICATE</arg>
+            <arg>-workflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-runId</arg>
+            <arg>${wf:run()}</arg>
+            <arg>-status</arg>
+            <arg>SUCCEEDED</arg>
+            <arg>-timeStamp</arg>
+            <arg>${timeStamp}</arg>
+            <arg>-brokerImplClass</arg>
+            <arg>${wf:conf("broker.impl.class")}</arg>
+            <arg>-brokerUrl</arg>
+            <arg>${wf:conf("broker.url")}</arg>
+            <arg>-userBrokerImplClass</arg>
+            <arg>${userBrokerImplClass}</arg>
+            <arg>-userBrokerUrl</arg>
+            <arg>${userBrokerUrl}</arg>
+            <arg>-brokerTTL</arg>
+            <arg>${wf:conf("broker.ttlInMins")}</arg>
+            <arg>-feedNames</arg>
+            <arg>${feedNames}</arg>
+            <arg>-feedInstancePaths</arg>
+            <arg>${feedInstancePaths}</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}-${srcClusterName}.csv</arg>
+            <arg>-workflowEngineUrl</arg>
+            <arg>${workflowEngineUrl}</arg>
+            <arg>-subflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-logDir</arg>
+            <arg>${logDir}/job-${nominalTime}/${srcClusterName}/</arg>
             <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
             <file>${wf:conf("falcon.libpath")}/ant.jar</file>
             <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>
@@ -116,45 +142,65 @@
             <file>${wf:conf("falcon.libpath")}/json-simple.jar</file>
             <file>${wf:conf("falcon.libpath")}/oozie-client.jar</file>
             <file>${wf:conf("falcon.libpath")}/spring-jms.jar</file>
-		</java>
-		<ok to="end" />
-		<error to="fail" />
-	</action>
-		<action name='failed-post-processing'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
-			<arg>-cluster</arg><arg>${cluster}</arg>
-			<arg>-entityType</arg><arg>${entityType}</arg>
-			<arg>-entityName</arg><arg>${entityName}</arg>
-			<arg>-nominalTime</arg><arg>${nominalTime}</arg>
-			<arg>-operation</arg><arg>REPLICATE</arg>
-			<arg>-workflowId</arg><arg>${wf:id()}</arg>
-			<arg>-runId</arg><arg>${wf:run()}</arg>
-			<arg>-status</arg><arg>FAILED</arg>
-			<arg>-timeStamp</arg><arg>${timeStamp}</arg>			
-			<arg>-brokerImplClass</arg><arg>${wf:conf("broker.impl.class")}</arg>
-			<arg>-brokerUrl</arg><arg>${wf:conf("broker.url")}</arg>
-			<arg>-userBrokerImplClass</arg><arg>${userBrokerImplClass}</arg>
-			<arg>-userBrokerUrl</arg><arg>${userBrokerUrl}</arg>
-			<arg>-brokerTTL</arg><arg>${wf:conf("broker.ttlInMins")}</arg>
-			<arg>-feedNames</arg><arg>${feedNames}</arg>
-			<arg>-feedInstancePaths</arg><arg>${feedInstancePaths}</arg>			
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}-${srcClusterName}.csv</arg>
-			<arg>-workflowEngineUrl</arg> <arg>${workflowEngineUrl}</arg>
-			<arg>-subflowId</arg> <arg>${wf:id()}</arg>
-			<arg>-logDir</arg> <arg>${logDir}/job-${nominalTime}/${srcClusterName}/</arg>
+        </java>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    <action name='failed-post-processing'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
+            <arg>-cluster</arg>
+            <arg>${cluster}</arg>
+            <arg>-entityType</arg>
+            <arg>${entityType}</arg>
+            <arg>-entityName</arg>
+            <arg>${entityName}</arg>
+            <arg>-nominalTime</arg>
+            <arg>${nominalTime}</arg>
+            <arg>-operation</arg>
+            <arg>REPLICATE</arg>
+            <arg>-workflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-runId</arg>
+            <arg>${wf:run()}</arg>
+            <arg>-status</arg>
+            <arg>FAILED</arg>
+            <arg>-timeStamp</arg>
+            <arg>${timeStamp}</arg>
+            <arg>-brokerImplClass</arg>
+            <arg>${wf:conf("broker.impl.class")}</arg>
+            <arg>-brokerUrl</arg>
+            <arg>${wf:conf("broker.url")}</arg>
+            <arg>-userBrokerImplClass</arg>
+            <arg>${userBrokerImplClass}</arg>
+            <arg>-userBrokerUrl</arg>
+            <arg>${userBrokerUrl}</arg>
+            <arg>-brokerTTL</arg>
+            <arg>${wf:conf("broker.ttlInMins")}</arg>
+            <arg>-feedNames</arg>
+            <arg>${feedNames}</arg>
+            <arg>-feedInstancePaths</arg>
+            <arg>${feedInstancePaths}</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}-${srcClusterName}.csv</arg>
+            <arg>-workflowEngineUrl</arg>
+            <arg>${workflowEngineUrl}</arg>
+            <arg>-subflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-logDir</arg>
+            <arg>${logDir}/job-${nominalTime}/${srcClusterName}/</arg>
             <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
             <file>${wf:conf("falcon.libpath")}/ant.jar</file>
             <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>
@@ -162,14 +208,14 @@
             <file>${wf:conf("falcon.libpath")}/json-simple.jar</file>
             <file>${wf:conf("falcon.libpath")}/oozie-client.jar</file>
             <file>${wf:conf("falcon.libpath")}/spring-jms.jar</file>
-		</java>
-		<ok to="fail" />
-		<error to="fail" />
-	</action>
-	<kill name="fail">
-		<message>Workflow failed, error
-			message[${wf:errorMessage(wf:lastErrorNode())}]
-		</message>
-	</kill>
-	<end name='end' />
+        </java>
+        <ok to="fail"/>
+        <error to="fail"/>
+    </action>
+    <kill name="fail">
+        <message>Workflow failed, error
+            message[${wf:errorMessage(wf:lastErrorNode())}]
+        </message>
+    </kill>
+    <end name='end'/>
 </workflow-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/main/resources/config/workflow/retention-workflow.xml
----------------------------------------------------------------------
diff --git a/feed/src/main/resources/config/workflow/retention-workflow.xml b/feed/src/main/resources/config/workflow/retention-workflow.xml
index 73f5d36..422209e 100644
--- a/feed/src/main/resources/config/workflow/retention-workflow.xml
+++ b/feed/src/main/resources/config/workflow/retention-workflow.xml
@@ -16,64 +16,86 @@
   limitations under the License.
   -->
 <workflow-app xmlns='uri:oozie:workflow:0.3' name='falcon-feed-parent-workflow'>
-	<start to='eviction' />
+    <start to='eviction'/>
     <action name="eviction">
         <java>
             <job-tracker>${jobTracker}</job-tracker>
             <name-node>${nameNode}</name-node>
             <configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
             <main-class>org.apache.falcon.retention.FeedEvictor</main-class>
-            <arg>-feedBasePath</arg><arg>${feedDataPath}</arg>
-            <arg>-retentionType</arg><arg>instance</arg>
-            <arg>-retentionLimit</arg><arg>${limit}</arg>
-            <arg>-timeZone</arg><arg>${timeZone}</arg>
-            <arg>-frequency</arg><arg>${frequency}</arg>
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
+            <arg>-feedBasePath</arg>
+            <arg>${feedDataPath}</arg>
+            <arg>-retentionType</arg>
+            <arg>instance</arg>
+            <arg>-retentionLimit</arg>
+            <arg>${limit}</arg>
+            <arg>-timeZone</arg>
+            <arg>${timeZone}</arg>
+            <arg>-frequency</arg>
+            <arg>${frequency}</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
         </java>
         <ok to="jms-messaging"/>
         <error to="fail"/>
     </action>
-	
-	<action name='jms-messaging'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.messaging.MessageProducer</main-class>
-			<arg>-entityName</arg><arg>${entityName}</arg>
-			<arg>-feedNames</arg><arg>${feedNames}</arg>
-			<arg>-feedInstancePaths</arg><arg>${feedInstancePaths}</arg>
-			<arg>-workflowId</arg><arg>${wf:id()}</arg>
-			<arg>-runId</arg><arg>${wf:run()}</arg>
-			<arg>-nominalTime</arg><arg>${nominalTime}</arg>
-			<arg>-timeStamp</arg><arg>${timeStamp}</arg>
-			<arg>-brokerUrl</arg><arg>${userBrokerUrl}</arg>
-			<arg>-brokerImplClass</arg><arg>${userBrokerImplClass}</arg>
-			<arg>-entityType</arg><arg>${entityType}</arg>
-			<arg>-operation</arg><arg>DELETE</arg>
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
-			<arg>-topicName</arg><arg>FALCON.${entityName}</arg>
-			<arg>-status</arg><arg>SUCCEEDED</arg>
-			<arg>-brokerTTL</arg><arg>${wf:conf("broker.ttlInMins")}</arg>
-			<arg>-cluster</arg><arg>${cluster}</arg>
+
+    <action name='jms-messaging'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.messaging.MessageProducer</main-class>
+            <arg>-entityName</arg>
+            <arg>${entityName}</arg>
+            <arg>-feedNames</arg>
+            <arg>${feedNames}</arg>
+            <arg>-feedInstancePaths</arg>
+            <arg>${feedInstancePaths}</arg>
+            <arg>-workflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-runId</arg>
+            <arg>${wf:run()}</arg>
+            <arg>-nominalTime</arg>
+            <arg>${nominalTime}</arg>
+            <arg>-timeStamp</arg>
+            <arg>${timeStamp}</arg>
+            <arg>-brokerUrl</arg>
+            <arg>${userBrokerUrl}</arg>
+            <arg>-brokerImplClass</arg>
+            <arg>${userBrokerImplClass}</arg>
+            <arg>-entityType</arg>
+            <arg>${entityType}</arg>
+            <arg>-operation</arg>
+            <arg>DELETE</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
+            <arg>-topicName</arg>
+            <arg>FALCON.${entityName}</arg>
+            <arg>-status</arg>
+            <arg>SUCCEEDED</arg>
+            <arg>-brokerTTL</arg>
+            <arg>${wf:conf("broker.ttlInMins")}</arg>
+            <arg>-cluster</arg>
+            <arg>${cluster}</arg>
             <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
             <file>${wf:conf("falcon.libpath")}/ant.jar</file>
             <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>
@@ -81,15 +103,15 @@
             <file>${wf:conf("falcon.libpath")}/json-simple.jar</file>
             <file>${wf:conf("falcon.libpath")}/oozie-client.jar</file>
             <file>${wf:conf("falcon.libpath")}/spring-jms.jar</file>
-		</java>
-		<ok to="end" />
-		<error to="fail" />
-	</action>	
+        </java>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
 
-	<kill name="fail">
-		<message>Workflow failed, error
-			message[${wf:errorMessage(wf:lastErrorNode())}]
-		</message>
-	</kill>
-	<end name='end' />
+    <kill name="fail">
+        <message>Workflow failed, error
+            message[${wf:errorMessage(wf:lastErrorNode())}]
+        </message>
+    </kill>
+    <end name='end'/>
 </workflow-app>


[20/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index c4f5748..bc8efc1 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -18,30 +18,21 @@
 
 package org.apache.falcon.client;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.UnsupportedEncodingException;
-import java.util.Properties;
-
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriBuilder;
-
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.EntityList;
 import org.apache.falcon.resource.InstancesResult;
 
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriBuilder;
+import java.io.*;
+import java.util.Properties;
 
 /**
  * Client API to submit and manage Falcon Entities (Cluster, Feed, Process) jobs
@@ -49,639 +40,637 @@ import com.sun.jersey.api.client.config.DefaultClientConfig;
  */
 public class FalconClient {
 
-	private String baseUrl;
-	protected static WebResource service;
-	public static final String WS_HEADER_PREFIX = "header:";
-	private static final String REMOTE_USER = "Remote-User";
-	private static final String USER = System.getProperty("user.name");
-	private static final String FALCON_INSTANCE_ACTION_CLUSTERS = "falcon.instance.action.clusters";
-	private static final String FALCON_INSTANCE_SOURCE_CLUSTERS = "falcon.instance.source.clusters";
-	/**
-	 * Create a Falcon client instance.
-	 * 
-	 * @param falconUrl
-	 *            of the server to which client interacts
-	 * @throws IOException
-	 */
-	public FalconClient(String falconUrl) throws IOException {
-		this.baseUrl = notEmpty(falconUrl, "FalconUrl");
-		if (!this.baseUrl.endsWith("/")) {
-			this.baseUrl += "/";
-		}
-		Client client = Client.create(new DefaultClientConfig());
-		setFalconTimeOut(client);
-		FalconClient.service = client.resource(UriBuilder.fromUri(baseUrl)
-				.build());
-		client.resource(UriBuilder.fromUri(baseUrl).build());
-
-		// addHeaders();
-	}
-
-	private void setFalconTimeOut(Client client) throws IOException {
-		Properties prop = new Properties();
-		InputStream input = FalconClient.class
-				.getResourceAsStream("/client.properties");
-		int readTimeout = 0;
-		int connectTimeout = 0;
-		if (input != null) {
-			prop.load(input);
-			readTimeout = prop.containsKey("falcon.read.timeout") ? Integer
-					.parseInt(prop.getProperty("falcon.read.timeout")) : 180000;
-			connectTimeout = prop.containsKey("falcon.connect.timeout") ? Integer
-					.parseInt(prop.getProperty("falcon.connect.timeout"))
-					: 180000;
-		} else {
-			readTimeout = 180000;
-			connectTimeout = 180000;
-		}
-		client.setConnectTimeout(connectTimeout);
-		client.setReadTimeout(readTimeout);
-	}
-
-	/**
-	 * Methods allowed on Entity Resources
-	 */
-	protected static enum Entities {
-		VALIDATE("api/entities/validate/", HttpMethod.POST, MediaType.TEXT_XML), SUBMIT(
-				"api/entities/submit/", HttpMethod.POST, MediaType.TEXT_XML), UPDATE(
-				"api/entities/update/", HttpMethod.POST, MediaType.TEXT_XML), SUBMITandSCHEDULE(
-				"api/entities/submitAndSchedule/", HttpMethod.POST,
-				MediaType.TEXT_XML), SCHEDULE("api/entities/schedule/",
-				HttpMethod.POST, MediaType.TEXT_XML), SUSPEND(
-				"api/entities/suspend/", HttpMethod.POST, MediaType.TEXT_XML), RESUME(
-				"api/entities/resume/", HttpMethod.POST, MediaType.TEXT_XML), DELETE(
-				"api/entities/delete/", HttpMethod.DELETE, MediaType.TEXT_XML), STATUS(
-				"api/entities/status/", HttpMethod.GET, MediaType.TEXT_XML), DEFINITION(
-				"api/entities/definition/", HttpMethod.GET, MediaType.TEXT_XML), LIST(
-				"api/entities/list/", HttpMethod.GET, MediaType.TEXT_XML), DEPENDENCY(
-				"api/entities/dependencies/", HttpMethod.GET,
-				MediaType.TEXT_XML);
-
-		private String path;
-		private String method;
-		private String mimeType;
-
-		Entities(String path, String method, String mimeType) {
-			this.path = path;
-			this.method = method;
-			this.mimeType = mimeType;
-		}
-	}
-
-	/**
-	 * Methods allowed on Process Instance Resources
-	 */
-	protected static enum Instances {
-		RUNNING("api/instance/running/", HttpMethod.GET,
-				MediaType.APPLICATION_JSON), STATUS("api/instance/status/",
-				HttpMethod.GET, MediaType.APPLICATION_JSON), KILL(
-				"api/instance/kill/", HttpMethod.POST,
-				MediaType.APPLICATION_JSON), SUSPEND("api/instance/suspend/",
-				HttpMethod.POST, MediaType.APPLICATION_JSON), RESUME(
-				"api/instance/resume/", HttpMethod.POST,
-				MediaType.APPLICATION_JSON), RERUN("api/instance/rerun/",
-				HttpMethod.POST, MediaType.APPLICATION_JSON) ,LOG("api/instance/logs/",
-				HttpMethod.GET, MediaType.APPLICATION_JSON);
-		private String path;
-		private String method;
-		private String mimeType;
-
-		Instances(String path, String method, String mimeType) {
-			this.path = path;
-			this.method = method;
-			this.mimeType = mimeType;
-		}
-	}
-	
-	protected static enum AdminOperations {
-		
-		STACK("api/admin/stack", HttpMethod.GET,
-				MediaType.TEXT_PLAIN), 
-		VERSION("api/admin/version", HttpMethod.GET,
-				MediaType.TEXT_PLAIN);
-		private String path;
-		private String method;
-		private String mimeType;
-		
-		AdminOperations(String path, String method, String mimeType) {
-			this.path = path;
-			this.method = method;
-			this.mimeType = mimeType;
-		}
-	}
-
-	public String notEmpty(String str, String name) {
-		if (str == null) {
-
-			throw new IllegalArgumentException(name + " cannot be null");
-		}
-		if (str.length() == 0) {
-			throw new IllegalArgumentException(name + " cannot be empty");
-		}
-		return str;
-	}
-
-	/**
-	 * Check if the object is not null.
-	 * 
-	 * @param <T>
-	 * @param obj
-	 * @param name
-	 * @return string
-	 */
-	public static <T> T notNull(T obj, String name) {
-		if (obj == null) {
-			throw new IllegalArgumentException(name + " cannot be null");
-		}
-		return obj;
-	}
-
-	public String schedule(String entityType, String entityName, String colo)
-			throws FalconCLIException {
-
-		return sendEntityRequest(Entities.SCHEDULE, entityType, entityName,
-				colo);
-
-	}
-
-	public String suspend(String entityType, String entityName, String colo)
-			throws FalconCLIException {
-
-		return sendEntityRequest(Entities.SUSPEND, entityType, entityName, colo);
-
-	}
-
-	public String resume(String entityType, String entityName, String colo)
-			throws FalconCLIException {
-
-		return sendEntityRequest(Entities.RESUME, entityType, entityName, colo);
-
-	}
-
-	public String delete(String entityType, String entityName)
-			throws FalconCLIException {
-
-		return sendEntityRequest(Entities.DELETE, entityType, entityName, null);
-
-	}
-
-	public String validate(String entityType, String filePath)
-			throws FalconCLIException {
-		InputStream entityStream = getServletInputStream(filePath);
-		return sendEntityRequestWithObject(Entities.VALIDATE, entityType,
-				entityStream, null);
-	}
-
-	public String submit(String entityType, String filePath)
-			throws FalconCLIException {
-		InputStream entityStream = getServletInputStream(filePath);
-		return sendEntityRequestWithObject(Entities.SUBMIT, entityType,
-				entityStream, null);
-	}
-
-	public String update(String entityType, String entityName, String filePath) 
-			throws FalconCLIException {
-		InputStream entityStream = getServletInputStream(filePath);
-		return sendEntityRequestWithNameAndObject(Entities.UPDATE, entityType,
-				entityName, entityStream);
-	}
-
-	public String submitAndSchedule(String entityType, String filePath) 
-			throws FalconCLIException {
-		InputStream entityStream = getServletInputStream(filePath);
-		return sendEntityRequestWithObject(Entities.SUBMITandSCHEDULE,
-				entityType, entityStream, null);
-	}
-
-	public String getStatus(String entityType, String entityName, String colo)
-			throws FalconCLIException {
-
-		return sendEntityRequest(Entities.STATUS, entityType, entityName, colo);
-
-	}
-
-	public String getDefinition(String entityType, String entityName)
-			throws FalconCLIException {
-
-		return sendDefinitionRequest(Entities.DEFINITION, entityType,
-				entityName);
-
-	}
-
-	public String getDependency(String entityType, String entityName)
-			throws FalconCLIException {
-		return sendDependencyRequest(Entities.DEPENDENCY, entityType,
-				entityName);
-	}
-
-	public String getEntityList(String entityType) throws FalconCLIException {
-		return sendListRequest(Entities.LIST, entityType);
-	}
-
-	public String getRunningInstances(String type, String entity, String colo)
-			throws FalconCLIException {
-
-		return sendInstanceRequest(Instances.RUNNING, type, entity, null, null,
-				null, null, colo);
-	}
-
-	public String getStatusOfInstances(String type, String entity,
-			String start, String end, String runid, String colo)
-			throws FalconCLIException {
-
-		return sendInstanceRequest(Instances.STATUS, type, entity, start, end,
-				null, null, colo);
-	}
-
-	public String killInstances(String type, String entity, String start,
-			String end, String colo, String clusters, String sourceClusters)
-			throws FalconCLIException, UnsupportedEncodingException {
-
-		return sendInstanceRequest(Instances.KILL, type, entity, start, end,
-				getServletInputStream(clusters, sourceClusters, null), null, colo);
-	}
-
-	public String suspendInstances(String type, String entity, String start,
-			String end, String colo, String clusters, String sourceClusters)
-			throws FalconCLIException, UnsupportedEncodingException {
-
-		return sendInstanceRequest(Instances.SUSPEND, type, entity, start, end,
-				getServletInputStream(clusters, sourceClusters, null), null, colo);
-	}
-
-	public String resumeInstances(String type, String entity, String start,
-			String end, String colo, String clusters, String sourceClusters)
-			throws FalconCLIException, UnsupportedEncodingException {
-
-		return sendInstanceRequest(Instances.RESUME, type, entity, start, end,
-				getServletInputStream(clusters, sourceClusters, null), null, colo);
-	}
-
-	public String rerunInstances(String type, String entity, String start,
-			String end, String filePath, String colo, String clusters,
-			String sourceClusters) throws FalconCLIException, IOException {
-		StringBuffer sb = new StringBuffer();
-		if(filePath != null)	
-		{
-			BufferedReader in = new BufferedReader(new FileReader(filePath));
-		    String str;
-		    while ((str = in.readLine()) != null) {
-		    	sb.append(str).append("\n");
-		    }
-		    in.close();
-		}
-		String temp = (sb.length() == 0) ? null : sb.toString();
-		return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
-				getServletInputStream(clusters, sourceClusters, temp), null, colo);
-	}
-
-	public String rerunInstances(String type, String entity, String start,
-			String end, String colo, String clusters, String sourceClusters)
-			throws FalconCLIException, UnsupportedEncodingException {
-		return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
-				getServletInputStream(clusters, sourceClusters, "oozie.wf.rerun.failnodes=true\n"), null, colo);
-	}
-	
-	public String getLogsOfInstances(String type, String entity, String start,
-			String end, String colo, String runId) throws FalconCLIException {
-
-		return sendInstanceRequest(Instances.LOG, type, entity, start, end,
-				null, runId, colo);
-	}
-	
-	public String getThreadDump() throws FalconCLIException{
-		return sendAdminRequest(AdminOperations.STACK);
-	}
-	
-	public String getVersion() throws FalconCLIException{
-		return sendAdminRequest(AdminOperations.VERSION);
-	}
-
-	/**
-	 * Converts a InputStream into ServletInputStream
-	 * 
-	 * @param filePath
-	 * @return ServletInputStream
-	 * @throws FalconCLIException
-	 * @throws java.io.IOException
-	 */
-	private InputStream getServletInputStream(String filePath)
-			throws FalconCLIException {
-		if (filePath == null) {
-			return null;
-		}
-		InputStream stream = null;
-		try {
-			stream = new FileInputStream(filePath);
-		} catch (FileNotFoundException e) {
-			throw new FalconCLIException("File not found:", e);
-		} catch (IOException e) {
-			throw new FalconCLIException("Unable to read file: ", e);
-		}
-		return stream;
-	}
-	
-	private InputStream getServletInputStream(String clusters,
-			String sourceClusters, String properties) throws FalconCLIException,
-			UnsupportedEncodingException {
-		
-		InputStream stream = null;
-		StringBuffer sb = new StringBuffer();
-		if (clusters != null) {
-			sb.append(FALCON_INSTANCE_ACTION_CLUSTERS + "=" + clusters + "\n");
-		}
-		if (sourceClusters != null) {
-			sb.append(FALCON_INSTANCE_SOURCE_CLUSTERS + "=" + sourceClusters + "\n");
-		} 
-		if(properties != null) {
-			sb.append(properties);
-		}
-		stream = new ByteArrayInputStream(sb.toString().getBytes("UTF-8"));
-		return (sb.length() == 0) ? null : stream;
-	}
-	// private ServletInputStream getServletInputStream(final InputStream
-	// stream)
-	// throws IOException {
-	// return new ServletInputStream() {
-	//
-	// @Override
-	// public int read() throws IOException {
-	// return stream.read();
-	// }
-	// };
-	// }
-
-	private String sendEntityRequest(Entities entities, String entityType,
-			String entityName, String colo) throws FalconCLIException {
-		
-		WebResource resource = service.path(entities.path)
-				.path(entityType).path(entityName);
-		if (colo != null) {
-			resource = resource.queryParam("colo", colo);
-		}
-		ClientResponse clientResponse = resource.header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class);
-
-		checkIfSuccessfull(clientResponse);
-
-		return parseAPIResult(clientResponse);
-	}
-
-	private String sendDefinitionRequest(Entities entities, String entityType,
-			String entityName) throws FalconCLIException {
-
-		ClientResponse clientResponse = service.path(entities.path)
-				.path(entityType).path(entityName).header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class);
-
-		checkIfSuccessfull(clientResponse);
-		return clientResponse.getEntity(String.class);
-	}
-
-	private String sendDependencyRequest(Entities entities, String entityType,
-			String entityName) throws FalconCLIException {
-
-		ClientResponse clientResponse = service.path(entities.path)
-				.path(entityType).path(entityName).header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class);
-
-		checkIfSuccessfull(clientResponse);
-
-		return parseEntityList(clientResponse);
-
-	}
-
-	private String sendListRequest(Entities entities, String entityType)
-			throws FalconCLIException {
-
-		ClientResponse clientResponse = service.path(entities.path)
-				.path(entityType).header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class);
-
-		checkIfSuccessfull(clientResponse);
-
-		return parseEntityList(clientResponse);
-
-	}
-
-	private String sendEntityRequestWithObject(Entities entities,
-			String entityType, Object requestObject, String colo) throws FalconCLIException {
-		
-		WebResource resource = service.path(entities.path)
-				.path(entityType);
-		if (colo != null) {
-			resource = resource.queryParam("colo", colo);
-		}
-		ClientResponse clientResponse = resource.header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class, requestObject);
-
-		checkIfSuccessfull(clientResponse);
-
-		return parseAPIResult(clientResponse);
-
-	}
-
-	private String sendEntityRequestWithNameAndObject(Entities entities,
-			String entityType, String entityName, Object requestObject)
-			throws FalconCLIException {
-		
-		ClientResponse clientResponse = service.path(entities.path)
-				.path(entityType).path(entityName).header(REMOTE_USER, USER)
-				.accept(entities.mimeType).type(MediaType.TEXT_XML)
-				.method(entities.method, ClientResponse.class, requestObject);
-
-		checkIfSuccessfull(clientResponse);
-
-		return parseAPIResult(clientResponse);
-
-	}
-
-	public InstancesResult instanceCmd(Instances instances, String type, String name, String start, String end, String colo) {
+    private String baseUrl;
+    protected static WebResource service;
+    public static final String WS_HEADER_PREFIX = "header:";
+    private static final String REMOTE_USER = "Remote-User";
+    private static final String USER = System.getProperty("user.name");
+    private static final String FALCON_INSTANCE_ACTION_CLUSTERS = "falcon.instance.action.clusters";
+    private static final String FALCON_INSTANCE_SOURCE_CLUSTERS = "falcon.instance.source.clusters";
+
+    /**
+     * Create a Falcon client instance.
+     *
+     * @param falconUrl of the server to which client interacts
+     * @throws IOException
+     */
+    public FalconClient(String falconUrl) throws IOException {
+        this.baseUrl = notEmpty(falconUrl, "FalconUrl");
+        if (!this.baseUrl.endsWith("/")) {
+            this.baseUrl += "/";
+        }
+        Client client = Client.create(new DefaultClientConfig());
+        setFalconTimeOut(client);
+        FalconClient.service = client.resource(UriBuilder.fromUri(baseUrl)
+                .build());
+        client.resource(UriBuilder.fromUri(baseUrl).build());
+
+        // addHeaders();
+    }
+
+    private void setFalconTimeOut(Client client) throws IOException {
+        Properties prop = new Properties();
+        InputStream input = FalconClient.class
+                .getResourceAsStream("/client.properties");
+        int readTimeout = 0;
+        int connectTimeout = 0;
+        if (input != null) {
+            prop.load(input);
+            readTimeout = prop.containsKey("falcon.read.timeout") ? Integer
+                    .parseInt(prop.getProperty("falcon.read.timeout")) : 180000;
+            connectTimeout = prop.containsKey("falcon.connect.timeout") ? Integer
+                    .parseInt(prop.getProperty("falcon.connect.timeout"))
+                    : 180000;
+        } else {
+            readTimeout = 180000;
+            connectTimeout = 180000;
+        }
+        client.setConnectTimeout(connectTimeout);
+        client.setReadTimeout(readTimeout);
+    }
+
+    /**
+     * Methods allowed on Entity Resources.
+     */
+    protected static enum Entities {
+        VALIDATE("api/entities/validate/", HttpMethod.POST, MediaType.TEXT_XML), SUBMIT(
+                "api/entities/submit/", HttpMethod.POST, MediaType.TEXT_XML), UPDATE(
+                "api/entities/update/", HttpMethod.POST, MediaType.TEXT_XML), SUBMITandSCHEDULE(
+                "api/entities/submitAndSchedule/", HttpMethod.POST,
+                MediaType.TEXT_XML), SCHEDULE("api/entities/schedule/",
+                HttpMethod.POST, MediaType.TEXT_XML), SUSPEND(
+                "api/entities/suspend/", HttpMethod.POST, MediaType.TEXT_XML), RESUME(
+                "api/entities/resume/", HttpMethod.POST, MediaType.TEXT_XML), DELETE(
+                "api/entities/delete/", HttpMethod.DELETE, MediaType.TEXT_XML), STATUS(
+                "api/entities/status/", HttpMethod.GET, MediaType.TEXT_XML), DEFINITION(
+                "api/entities/definition/", HttpMethod.GET, MediaType.TEXT_XML), LIST(
+                "api/entities/list/", HttpMethod.GET, MediaType.TEXT_XML), DEPENDENCY(
+                "api/entities/dependencies/", HttpMethod.GET,
+                MediaType.TEXT_XML);
+
+        private String path;
+        private String method;
+        private String mimeType;
+
+        Entities(String path, String method, String mimeType) {
+            this.path = path;
+            this.method = method;
+            this.mimeType = mimeType;
+        }
+    }
+
+    /**
+     * Methods allowed on Process Instance Resources.
+     */
+    protected static enum Instances {
+        RUNNING("api/instance/running/", HttpMethod.GET,
+                MediaType.APPLICATION_JSON), STATUS("api/instance/status/",
+                HttpMethod.GET, MediaType.APPLICATION_JSON), KILL(
+                "api/instance/kill/", HttpMethod.POST,
+                MediaType.APPLICATION_JSON), SUSPEND("api/instance/suspend/",
+                HttpMethod.POST, MediaType.APPLICATION_JSON), RESUME(
+                "api/instance/resume/", HttpMethod.POST,
+                MediaType.APPLICATION_JSON), RERUN("api/instance/rerun/",
+                HttpMethod.POST, MediaType.APPLICATION_JSON), LOG("api/instance/logs/",
+                HttpMethod.GET, MediaType.APPLICATION_JSON);
+        private String path;
+        private String method;
+        private String mimeType;
+
+        Instances(String path, String method, String mimeType) {
+            this.path = path;
+            this.method = method;
+            this.mimeType = mimeType;
+        }
+    }
+
+    protected static enum AdminOperations {
+
+        STACK("api/admin/stack", HttpMethod.GET,
+                MediaType.TEXT_PLAIN),
+        VERSION("api/admin/version", HttpMethod.GET,
+                MediaType.TEXT_PLAIN);
+        private String path;
+        private String method;
+        private String mimeType;
+
+        AdminOperations(String path, String method, String mimeType) {
+            this.path = path;
+            this.method = method;
+            this.mimeType = mimeType;
+        }
+    }
+
+    public String notEmpty(String str, String name) {
+        if (str == null) {
+
+            throw new IllegalArgumentException(name + " cannot be null");
+        }
+        if (str.length() == 0) {
+            throw new IllegalArgumentException(name + " cannot be empty");
+        }
+        return str;
+    }
+
+    /**
+     * Check if the object is not null.
+     *
+     * @param <T>
+     * @param obj
+     * @param name
+     * @return string
+     */
+    public static <T> T notNull(T obj, String name) {
+        if (obj == null) {
+            throw new IllegalArgumentException(name + " cannot be null");
+        }
+        return obj;
+    }
+
+    public String schedule(String entityType, String entityName, String colo)
+            throws FalconCLIException {
+
+        return sendEntityRequest(Entities.SCHEDULE, entityType, entityName,
+                colo);
+
+    }
+
+    public String suspend(String entityType, String entityName, String colo)
+            throws FalconCLIException {
+
+        return sendEntityRequest(Entities.SUSPEND, entityType, entityName, colo);
+
+    }
+
+    public String resume(String entityType, String entityName, String colo)
+            throws FalconCLIException {
+
+        return sendEntityRequest(Entities.RESUME, entityType, entityName, colo);
+
+    }
+
+    public String delete(String entityType, String entityName)
+            throws FalconCLIException {
+
+        return sendEntityRequest(Entities.DELETE, entityType, entityName, null);
+
+    }
+
+    public String validate(String entityType, String filePath)
+            throws FalconCLIException {
+        InputStream entityStream = getServletInputStream(filePath);
+        return sendEntityRequestWithObject(Entities.VALIDATE, entityType,
+                entityStream, null);
+    }
+
+    public String submit(String entityType, String filePath)
+            throws FalconCLIException {
+        InputStream entityStream = getServletInputStream(filePath);
+        return sendEntityRequestWithObject(Entities.SUBMIT, entityType,
+                entityStream, null);
+    }
+
+    public String update(String entityType, String entityName, String filePath)
+            throws FalconCLIException {
+        InputStream entityStream = getServletInputStream(filePath);
+        return sendEntityRequestWithNameAndObject(Entities.UPDATE, entityType,
+                entityName, entityStream);
+    }
+
+    public String submitAndSchedule(String entityType, String filePath)
+            throws FalconCLIException {
+        InputStream entityStream = getServletInputStream(filePath);
+        return sendEntityRequestWithObject(Entities.SUBMITandSCHEDULE,
+                entityType, entityStream, null);
+    }
+
+    public String getStatus(String entityType, String entityName, String colo)
+            throws FalconCLIException {
+
+        return sendEntityRequest(Entities.STATUS, entityType, entityName, colo);
+
+    }
+
+    public String getDefinition(String entityType, String entityName)
+            throws FalconCLIException {
+
+        return sendDefinitionRequest(Entities.DEFINITION, entityType,
+                entityName);
+
+    }
+
+    public String getDependency(String entityType, String entityName)
+            throws FalconCLIException {
+        return sendDependencyRequest(Entities.DEPENDENCY, entityType,
+                entityName);
+    }
+
+    public String getEntityList(String entityType) throws FalconCLIException {
+        return sendListRequest(Entities.LIST, entityType);
+    }
+
+    public String getRunningInstances(String type, String entity, String colo)
+            throws FalconCLIException {
+
+        return sendInstanceRequest(Instances.RUNNING, type, entity, null, null,
+                null, null, colo);
+    }
+
+    public String getStatusOfInstances(String type, String entity,
+                                       String start, String end, String runid, String colo)
+            throws FalconCLIException {
+
+        return sendInstanceRequest(Instances.STATUS, type, entity, start, end,
+                null, null, colo);
+    }
+
+    public String killInstances(String type, String entity, String start,
+                                String end, String colo, String clusters, String sourceClusters)
+            throws FalconCLIException, UnsupportedEncodingException {
+
+        return sendInstanceRequest(Instances.KILL, type, entity, start, end,
+                getServletInputStream(clusters, sourceClusters, null), null, colo);
+    }
+
+    public String suspendInstances(String type, String entity, String start,
+                                   String end, String colo, String clusters, String sourceClusters)
+            throws FalconCLIException, UnsupportedEncodingException {
+
+        return sendInstanceRequest(Instances.SUSPEND, type, entity, start, end,
+                getServletInputStream(clusters, sourceClusters, null), null, colo);
+    }
+
+    public String resumeInstances(String type, String entity, String start,
+                                  String end, String colo, String clusters, String sourceClusters)
+            throws FalconCLIException, UnsupportedEncodingException {
+
+        return sendInstanceRequest(Instances.RESUME, type, entity, start, end,
+                getServletInputStream(clusters, sourceClusters, null), null, colo);
+    }
+
+    public String rerunInstances(String type, String entity, String start,
+                                 String end, String filePath, String colo, String clusters,
+                                 String sourceClusters) throws FalconCLIException, IOException {
+        StringBuffer sb = new StringBuffer();
+        if (filePath != null) {
+            BufferedReader in = new BufferedReader(new FileReader(filePath));
+            String str;
+            while ((str = in.readLine()) != null) {
+                sb.append(str).append("\n");
+            }
+            in.close();
+        }
+        String temp = (sb.length() == 0) ? null : sb.toString();
+        return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
+                getServletInputStream(clusters, sourceClusters, temp), null, colo);
+    }
+
+    public String rerunInstances(String type, String entity, String start,
+                                 String end, String colo, String clusters, String sourceClusters)
+            throws FalconCLIException, UnsupportedEncodingException {
+        return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
+                getServletInputStream(clusters, sourceClusters, "oozie.wf.rerun.failnodes=true\n"), null, colo);
+    }
+
+    public String getLogsOfInstances(String type, String entity, String start,
+                                     String end, String colo, String runId) throws FalconCLIException {
+
+        return sendInstanceRequest(Instances.LOG, type, entity, start, end,
+                null, runId, colo);
+    }
+
+    public String getThreadDump() throws FalconCLIException {
+        return sendAdminRequest(AdminOperations.STACK);
+    }
+
+    public String getVersion() throws FalconCLIException {
+        return sendAdminRequest(AdminOperations.VERSION);
+    }
+
+    /**
+     * Converts a InputStream into ServletInputStream.
+     *
+     * @param filePath
+     * @return ServletInputStream
+     * @throws FalconCLIException
+     * @throws java.io.IOException
+     */
+    private InputStream getServletInputStream(String filePath)
+            throws FalconCLIException {
+        if (filePath == null) {
+            return null;
+        }
+        InputStream stream = null;
+        try {
+            stream = new FileInputStream(filePath);
+        } catch (FileNotFoundException e) {
+            throw new FalconCLIException("File not found:", e);
+        } catch (IOException e) {
+            throw new FalconCLIException("Unable to read file: ", e);
+        }
+        return stream;
+    }
+
+    private InputStream getServletInputStream(String clusters,
+                                              String sourceClusters, String properties) throws FalconCLIException,
+                                                                                               UnsupportedEncodingException {
+
+        InputStream stream = null;
+        StringBuffer sb = new StringBuffer();
+        if (clusters != null) {
+            sb.append(FALCON_INSTANCE_ACTION_CLUSTERS + "=" + clusters + "\n");
+        }
+        if (sourceClusters != null) {
+            sb.append(FALCON_INSTANCE_SOURCE_CLUSTERS + "=" + sourceClusters + "\n");
+        }
+        if (properties != null) {
+            sb.append(properties);
+        }
+        stream = new ByteArrayInputStream(sb.toString().getBytes("UTF-8"));
+        return (sb.length() == 0) ? null : stream;
+    }
+    // private ServletInputStream getServletInputStream(final InputStream
+    // stream)
+    // throws IOException {
+    // return new ServletInputStream() {
+    //
+    // @Override
+    // public int read() throws IOException {
+    // return stream.read();
+    // }
+    // };
+    // }
+
+    private String sendEntityRequest(Entities entities, String entityType,
+                                     String entityName, String colo) throws FalconCLIException {
+
+        WebResource resource = service.path(entities.path)
+                .path(entityType).path(entityName);
+        if (colo != null) {
+            resource = resource.queryParam("colo", colo);
+        }
+        ClientResponse clientResponse = resource.header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class);
+
+        checkIfSuccessfull(clientResponse);
+
+        return parseAPIResult(clientResponse);
+    }
+
+    private String sendDefinitionRequest(Entities entities, String entityType,
+                                         String entityName) throws FalconCLIException {
+
+        ClientResponse clientResponse = service.path(entities.path)
+                .path(entityType).path(entityName).header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class);
+
+        checkIfSuccessfull(clientResponse);
+        return clientResponse.getEntity(String.class);
+    }
+
+    private String sendDependencyRequest(Entities entities, String entityType,
+                                         String entityName) throws FalconCLIException {
+
+        ClientResponse clientResponse = service.path(entities.path)
+                .path(entityType).path(entityName).header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class);
+
+        checkIfSuccessfull(clientResponse);
+
+        return parseEntityList(clientResponse);
+    }
+
+    private String sendListRequest(Entities entities, String entityType)
+            throws FalconCLIException {
+
+        ClientResponse clientResponse = service.path(entities.path)
+                .path(entityType).header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class);
+
+        checkIfSuccessfull(clientResponse);
+
+        return parseEntityList(clientResponse);
+    }
+
+    private String sendEntityRequestWithObject(Entities entities, String entityType,
+                                               Object requestObject, String colo) throws FalconCLIException {
+        WebResource resource = service.path(entities.path)
+                .path(entityType);
+        if (colo != null) {
+            resource = resource.queryParam("colo", colo);
+        }
+        ClientResponse clientResponse = resource.header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class, requestObject);
+
+        checkIfSuccessfull(clientResponse);
+
+        return parseAPIResult(clientResponse);
+    }
+
+    private String sendEntityRequestWithNameAndObject(Entities entities, String entityType, String entityName,
+                                                      Object requestObject) throws FalconCLIException {
+
+        ClientResponse clientResponse = service.path(entities.path)
+                .path(entityType).path(entityName).header(REMOTE_USER, USER)
+                .accept(entities.mimeType).type(MediaType.TEXT_XML)
+                .method(entities.method, ClientResponse.class, requestObject);
+
+        checkIfSuccessfull(clientResponse);
+
+        return parseAPIResult(clientResponse);
+    }
+
+    public InstancesResult instanceCmd(Instances instances, String type, String name,
+                                       String start, String end, String colo) {
         WebResource resource = service.path(instances.path).path(type).path(name);
         resource = resource.queryParam("start", start);
-        if(end != null)
+        if (end != null) {
             resource = resource.queryParam("end", end);
+        }
         resource = resource.queryParam("colo", colo);
-        
-        return resource.header(REMOTE_USER, USER).accept(instances.mimeType).method(instances.method, InstancesResult.class);
-	}
-	
-	private String sendInstanceRequest(Instances instances, String type,
-			String entity, String start, String end, InputStream props,
-			String runid, String colo) throws FalconCLIException {
-		WebResource resource = service.path(instances.path).path(type)
-				.path(entity);
-		if (start != null) {
-			resource = resource.queryParam("start", start);
-		}
-		if (end != null) {
-			resource = resource.queryParam("end", end);
-		}
-		if (runid != null) {
-			resource = resource.queryParam("runid", runid);
-		}
-		if (colo != null) {
-			resource = resource.queryParam("colo", colo);
-		}
-		
-
-		ClientResponse clientResponse = null;
-		if (props == null) {
-			clientResponse = resource.header(REMOTE_USER, USER)
-					.accept(instances.mimeType)
-					.method(instances.method, ClientResponse.class);
-		} else {
-			clientResponse = resource.header(REMOTE_USER, USER)
-					.accept(instances.mimeType)
-					.method(instances.method, ClientResponse.class, props);
-		}
-		checkIfSuccessfull(clientResponse);
-		
-		if(instances.name().equals("LOG"))
-			return parseProcessInstanceResultLogs(clientResponse, runid);
-		else
-			return parseProcessInstanceResult(clientResponse);
-
-	}
-	
-	private String sendAdminRequest(AdminOperations job) 
-			throws FalconCLIException {
-		
-		ClientResponse clientResponse = service.path(job.path)
-				.header(REMOTE_USER, USER).accept(job.mimeType)
-				.type(MediaType.TEXT_PLAIN).method(job.method, ClientResponse.class);
-		return parseStringResult(clientResponse);	
-	}
-	
-	private String parseAPIResult(ClientResponse clientResponse)
-			throws FalconCLIException {
-
-		APIResult result = clientResponse.getEntity(APIResult.class);
-		return result.getMessage();
-
-	}
-
-	private String parseEntityList(ClientResponse clientResponse)
-			throws FalconCLIException {
-
-		EntityList result = clientResponse.getEntity(EntityList.class);
-		if (result == null || result.getElements() == null) {
-			return "";
-		}
-		return result.toString();
-
-	}
-
-	private String parseStringResult(ClientResponse clientResponse)
-			throws FalconCLIException {
-
-		return clientResponse.getEntity(String.class);
-	}
-
-	private String parseProcessInstanceResult(ClientResponse clientResponse) {
-		InstancesResult result = clientResponse
-				.getEntity(InstancesResult.class);
-		StringBuffer sb = new StringBuffer();
-		String toAppend = null;
-		
-		sb.append("Consolidated Status: " + result.getStatus() + "\n");
-		
-		sb.append("\nInstances:\n");
-		sb.append("Instance\t\tCluster\t\tSourceCluster\t\tStatus\t\tStart\t\tEnd\t\tDetails\t\t\t\t\tLog\n");
-		sb.append("----------------------------------------------------------------------------------------------------------------------------------------\n");
-		if(result.getInstances() != null){
-			for (InstancesResult.Instance instance : result.getInstances()) {
-				
-				toAppend = instance.getInstance() != null  ? instance.getInstance() : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = instance.getCluster() != null ? instance.getCluster() : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = instance.getSourceCluster() != null ? instance.getSourceCluster() : "-";
-				sb.append(toAppend + "\t");	
-				
-				toAppend =  (instance.getStatus() != null ? instance.getStatus().toString() : "-");
-				sb.append(toAppend + "\t");	
-				
-				toAppend = instance.getStartTime() != null ? SchemaHelper.formatDateUTC(instance.getStartTime()) : "-";
-				sb.append(toAppend + "\t");	
-				
-				toAppend = instance.getEndTime() != null ? SchemaHelper.formatDateUTC(instance.getEndTime()) : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = (instance.getDetails() != null && !instance.getDetails().equals("")) ? instance.getDetails() : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = instance.getLogFile() != null ? instance.getLogFile() : "-";
-				sb.append(toAppend + "\n");	
-				
-			}
-		}
-		sb.append("\nAdditional Information:\n");
-		sb.append("Response: " + result.getMessage());
-		sb.append("Request Id: " + result.getRequestId() );
-		return sb.toString();
-	}
-	
-	private String parseProcessInstanceResultLogs(ClientResponse clientResponse, String runid) {
-		InstancesResult result = clientResponse
-				.getEntity(InstancesResult.class);
-		StringBuffer sb = new StringBuffer();
-		String toAppend = null;
-		
-		sb.append("Consolidated Status: " + result.getStatus() + "\n");
-		
-		sb.append("\nInstances:\n");
-		sb.append("Instance\t\tCluster\t\tSourceCluster\t\tStatus\t\tRunID\t\t\tLog\n");
-		sb.append("----------------------------------------------------------------------------------------------------\n");
-		if(result.getInstances() != null){
-			for (InstancesResult.Instance instance : result.getInstances()) {
-				
-				toAppend = (instance.getInstance() != null ) ? instance.getInstance() : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = instance.getCluster() != null ? instance.getCluster() : "-";
-				sb.append(toAppend + "\t");
-				
-				toAppend = instance.getSourceCluster() != null ? instance.getSourceCluster() : "-";
-				sb.append(toAppend + "\t");	
-				
-				toAppend =  (instance.getStatus() != null ? instance.getStatus().toString() : "-");
-				sb.append(toAppend + "\t");	
-				
-				toAppend =  (runid != null ? runid : "latest");
-				sb.append(toAppend + "\t");	
-				
-				toAppend = instance.getLogFile() != null ? instance.getLogFile() : "-";
-				sb.append(toAppend + "\n");	
-				
-				
-				if (instance.actions != null) {
-					sb.append("actions:\n");
-					for (InstancesResult.InstanceAction action : instance.actions) {
-						sb.append("    ").append(action.getAction()+ "\t" + action.getStatus() + "\t" + action.getLogFile()).append("\n");
-					}
-				}
-			}
-		}
-		sb.append("\nAdditional Information:\n");
-		sb.append("Response: " + result.getMessage());
-		sb.append("Request Id: " + result.getRequestId() );
-		return sb.toString();
-	}
-	private void checkIfSuccessfull(ClientResponse clientResponse)
-			throws FalconCLIException {
-		if (clientResponse.getStatus() == Response.Status.BAD_REQUEST
-				.getStatusCode()) {
-			throw FalconCLIException.fromReponse(clientResponse);
-		}
-
-	}
 
+        return resource.header(REMOTE_USER, USER)
+                .accept(instances.mimeType)
+                .method(instances.method, InstancesResult.class);
+    }
+
+    private String sendInstanceRequest(Instances instances, String type,
+                                       String entity, String start, String end, InputStream props,
+                                       String runid, String colo) throws FalconCLIException {
+        WebResource resource = service.path(instances.path).path(type)
+                .path(entity);
+        if (start != null) {
+            resource = resource.queryParam("start", start);
+        }
+        if (end != null) {
+            resource = resource.queryParam("end", end);
+        }
+        if (runid != null) {
+            resource = resource.queryParam("runid", runid);
+        }
+        if (colo != null) {
+            resource = resource.queryParam("colo", colo);
+        }
+
+        ClientResponse clientResponse;
+        if (props == null) {
+            clientResponse = resource.header(REMOTE_USER, USER)
+                    .accept(instances.mimeType)
+                    .method(instances.method, ClientResponse.class);
+        } else {
+            clientResponse = resource.header(REMOTE_USER, USER)
+                    .accept(instances.mimeType)
+                    .method(instances.method, ClientResponse.class, props);
+        }
+        checkIfSuccessfull(clientResponse);
+
+        if (instances.name().equals("LOG")) {
+            return parseProcessInstanceResultLogs(clientResponse, runid);
+        } else {
+            return parseProcessInstanceResult(clientResponse);
+        }
+
+    }
+
+    private String sendAdminRequest(AdminOperations job)
+            throws FalconCLIException {
+
+        ClientResponse clientResponse = service.path(job.path)
+                .header(REMOTE_USER, USER).accept(job.mimeType)
+                .type(MediaType.TEXT_PLAIN).method(job.method, ClientResponse.class);
+        return parseStringResult(clientResponse);
+    }
+
+    private String parseAPIResult(ClientResponse clientResponse)
+            throws FalconCLIException {
+
+        APIResult result = clientResponse.getEntity(APIResult.class);
+        return result.getMessage();
+    }
+
+    private String parseEntityList(ClientResponse clientResponse)
+            throws FalconCLIException {
+
+        EntityList result = clientResponse.getEntity(EntityList.class);
+        if (result == null || result.getElements() == null) {
+            return "";
+        }
+        return result.toString();
+
+    }
+
+    private String parseStringResult(ClientResponse clientResponse)
+            throws FalconCLIException {
+
+        return clientResponse.getEntity(String.class);
+    }
+
+    private String parseProcessInstanceResult(ClientResponse clientResponse) {
+        InstancesResult result = clientResponse
+                .getEntity(InstancesResult.class);
+        StringBuilder sb = new StringBuilder();
+        String toAppend;
+
+        sb.append("Consolidated Status: ").append(result.getStatus()).append("\n");
+
+        sb.append("\nInstances:\n");
+        sb.append("Instance\t\tCluster\t\tSourceCluster\t\tStatus\t\tStart\t\tEnd\t\tDetails\t\t\t\t\tLog\n");
+        sb.append("-----------------------------------------------------------------------------------------------\n");
+        if (result.getInstances() != null) {
+            for (InstancesResult.Instance instance : result.getInstances()) {
+
+                toAppend = instance.getInstance() != null ? instance.getInstance() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getCluster() != null ? instance.getCluster() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getSourceCluster() != null ? instance.getSourceCluster() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = (instance.getStatus() != null ? instance.getStatus().toString() : "-");
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getStartTime() != null
+                        ? SchemaHelper.formatDateUTC(instance.getStartTime()) : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getEndTime() != null
+                        ? SchemaHelper.formatDateUTC(instance.getEndTime()) : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = (instance.getDetails() != null && !instance.getDetails().equals(""))
+                        ? instance.getDetails() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getLogFile() != null ? instance.getLogFile() : "-";
+                sb.append(toAppend).append("\n");
+
+            }
+        }
+        sb.append("\nAdditional Information:\n");
+        sb.append("Response: ").append(result.getMessage());
+        sb.append("Request Id: ").append(result.getRequestId());
+        return sb.toString();
+    }
+
+    private String parseProcessInstanceResultLogs(ClientResponse clientResponse, String runid) {
+        InstancesResult result = clientResponse
+                .getEntity(InstancesResult.class);
+        StringBuilder sb = new StringBuilder();
+        String toAppend;
+
+        sb.append("Consolidated Status: ").append(result.getStatus()).append("\n");
+
+        sb.append("\nInstances:\n");
+        sb.append("Instance\t\tCluster\t\tSourceCluster\t\tStatus\t\tRunID\t\t\tLog\n");
+        sb.append("-----------------------------------------------------------------------------------------------\n");
+        if (result.getInstances() != null) {
+            for (InstancesResult.Instance instance : result.getInstances()) {
+
+                toAppend = (instance.getInstance() != null) ? instance.getInstance() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getCluster() != null ? instance.getCluster() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getSourceCluster() != null ? instance.getSourceCluster() : "-";
+                sb.append(toAppend).append("\t");
+
+                toAppend = (instance.getStatus() != null ? instance.getStatus().toString() : "-");
+                sb.append(toAppend).append("\t");
+
+                toAppend = (runid != null ? runid : "latest");
+                sb.append(toAppend).append("\t");
+
+                toAppend = instance.getLogFile() != null ? instance.getLogFile() : "-";
+                sb.append(toAppend).append("\n");
+
+                if (instance.actions != null) {
+                    sb.append("actions:\n");
+                    for (InstancesResult.InstanceAction action : instance.actions) {
+                        sb.append("    ").append(action.getAction()).append("\t");
+                        sb.append(action.getStatus()).append("\t").append(action.getLogFile()).append("\n");
+                    }
+                }
+            }
+        }
+        sb.append("\nAdditional Information:\n");
+        sb.append("Response: ").append(result.getMessage());
+        sb.append("Request Id: ").append(result.getRequestId());
+        return sb.toString();
+    }
+
+    private void checkIfSuccessfull(ClientResponse clientResponse)
+            throws FalconCLIException {
+        if (clientResponse.getStatus() == Response.Status.BAD_REQUEST
+                .getStatusCode()) {
+            throw FalconCLIException.fromReponse(clientResponse);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java b/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
index 8bf8341..cde7792 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/DateValidator.java
@@ -20,62 +20,66 @@ package org.apache.falcon.entity.v0;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+/**
+ * Date utility class.
+ */
 public class DateValidator {
 
-    private static final String DATE_PATTERN = "(2\\d\\d\\d|19\\d\\d)-(0[1-9]|1[012])-(0[1-9]|1[0-9]|2[0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9])Z";
-	private static final Pattern pattern = Pattern.compile(DATE_PATTERN);
+    private static final String DATE_PATTERN =
+            "(2\\d\\d\\d|19\\d\\d)-(0[1-9]|1[012])-(0[1-9]|1[0-9]|2[0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9])Z";
+    private static final Pattern pattern = Pattern.compile(DATE_PATTERN);
+
+    private DateValidator() {
+    }
 
-	/**
-	 * Validate date format with regular expression
-	 * 
-	 * @param date
-	 *            date address for validation
-	 * @return true valid date fromat, false invalid date format
-	 */
-	public static boolean validate(final String date) {
+    /**
+     * Validate date format with regular expression.
+     *
+     * @param date date address for validation
+     * @return true valid date fromat, false invalid date format
+     */
+    public static boolean validate(final String date) {
 
-		Matcher matcher = pattern.matcher(date);
+        Matcher matcher = pattern.matcher(date);
 
-		if (matcher.matches()) {
+        if (matcher.matches()) {
 
-			matcher.reset();
+            matcher.reset();
 
-			if (matcher.find()) {
+            if (matcher.find()) {
 
-				int year = Integer.parseInt(matcher.group(1));
-				String month = matcher.group(2);
-				String day = matcher.group(3);
+                int year = Integer.parseInt(matcher.group(1));
+                String month = matcher.group(2);
+                String day = matcher.group(3);
 
-				if (day.equals("31")
-						&& (month.equals("4") || month.equals("6")
-								|| month.equals("9") || month.equals("11")
-								|| month.equals("04") || month.equals("06") || month
-									.equals("09"))) {
-					return false; // only 1,3,5,7,8,10,12 has 31 days
-				} else if (month.equals("2") || month.equals("02")) {
-					// leap year
-					if (year % 4 == 0) {
-						if (day.equals("30") || day.equals("31")) {
-							return false;
-						} else {
-							return true;
-						}
-					} else {
-						if (day.equals("29") || day.equals("30")
-								|| day.equals("31")) {
-							return false;
-						} else {
-							return true;
-						}
-					}
-				} else {
-					return true;
-				}
-			} else {
-				return false;
-			}
-		} else {
-			return false;
-		}
-	}
+                if (day.equals("31")
+                        && (month.equals("4") || month.equals("6")
+                        || month.equals("9") || month.equals("11")
+                        || month.equals("04") || month.equals("06") || month.equals("09"))) {
+                    return false; // only 1,3,5,7,8,10,12 has 31 days
+                } else if (month.equals("2") || month.equals("02")) {
+                    // leap year
+                    if (year % 4 == 0) {
+                        if (day.equals("30") || day.equals("31")) {
+                            return false;
+                        } else {
+                            return true;
+                        }
+                    } else {
+                        if (day.equals("29") || day.equals("30") || day.equals("31")) {
+                            return false;
+                        } else {
+                            return true;
+                        }
+                    }
+                } else {
+                    return true;
+                }
+            } else {
+                return false;
+            }
+        } else {
+            return false;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
index 3947e72..9fb926d 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
@@ -18,11 +18,10 @@
 
 package org.apache.falcon.entity.v0;
 
-import java.io.StringReader;
-import java.io.StringWriter;
-
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
+import java.io.StringReader;
+import java.io.StringWriter;
 
 public abstract class Entity {
     public abstract String getName();
@@ -38,18 +37,17 @@ public abstract class Entity {
 
     @Override
     public boolean equals(Object o) {
-        if (this == o)
+        if (this == o) {
             return true;
-        if (o==null || !o.getClass().equals(this.getClass()))
+        }
+        if (o == null || !o.getClass().equals(this.getClass())) {
             return false;
+        }
 
         Entity entity = (Entity) o;
 
         String name = getName();
-        if (name != null ? !name.equals(entity.getName()) : entity.getName() != null)
-            return false;
-
-        return true;
+        return !(name != null ? !name.equals(entity.getName()) : entity.getName() != null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java b/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
index 752a92d..d33bdf0 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/EntityType.java
@@ -18,25 +18,20 @@
 
 package org.apache.falcon.entity.v0;
 
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.ValidationEvent;
-import javax.xml.bind.ValidationEventHandler;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
 
+import javax.xml.bind.*;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
 /**
  * Enum for types of entities in Falcon Process, Feed and Cluster
  */
 public enum EntityType {
-    FEED(Feed.class, "/feed-0.1.xsd", "name"), 
-    PROCESS(Process.class, "/process-0.1.xsd", "name"), 
+    FEED(Feed.class, "/feed-0.1.xsd", "name"),
+    PROCESS(Process.class, "/process-0.1.xsd", "name"),
     CLUSTER(Cluster.class, "/cluster-0.1.xsd", "name");
 
     //Fail unmarshalling of whole xml if unmarshalling of any element fails
@@ -46,7 +41,7 @@ public enum EntityType {
             return false;
         }
     }
-    
+
     private static final String NS = "http://www.w3.org/2001/XMLSchema";
 
     private final Class<? extends Entity> clazz;
@@ -62,7 +57,7 @@ public enum EntityType {
         this.schemaFile = schemaFile;
         try {
             jaxbContext = JAXBContext.newInstance(typeClass);
-            synchronized(this) {
+            synchronized (this) {
                 SchemaFactory schemaFactory = SchemaFactory.newInstance(NS);
                 schema = schemaFactory.newSchema(getClass().getResource(schemaFile));
             }
@@ -78,24 +73,24 @@ public enum EntityType {
     public String getSchemaFile() {
         return schemaFile;
     }
-    
+
     public Marshaller getMarshaller() throws JAXBException {
         Marshaller marshaller = jaxbContext.createMarshaller();
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
         return marshaller;
     }
-    
+
     public Unmarshaller getUnmarshaller() throws JAXBException {
         Unmarshaller unmarshaller = jaxbContext.createUnmarshaller();
         unmarshaller.setSchema(schema);
         unmarshaller.setEventHandler(new EventHandler());
         return unmarshaller;
     }
-    
+
     public boolean isSchedulable() {
         return this != EntityType.CLUSTER;
     }
-    
+
     public String[] getImmutableProperties() {
         return immutableProperties;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java b/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
index 300de7b..0d1be26 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Frequency.java
@@ -25,7 +25,7 @@ import java.util.regex.Pattern;
 
 public class Frequency {
     private static final Pattern PATTERN = Pattern.compile("(minutes|hours|days|months)\\((\\d+)\\)");
-    
+
     public static enum TimeUnit {
         minutes(Calendar.MINUTE), hours(Calendar.HOUR), days(Calendar.DATE), months(Calendar.MONTH);
 
@@ -39,7 +39,7 @@ public class Frequency {
             return calendarUnit;
         }
     }
-    
+
     private TimeUnit timeUnit;
     private int frequency;
 
@@ -47,29 +47,30 @@ public class Frequency {
         this.frequency = freq;
         this.timeUnit = timeUnit;
     }
-    
+
     public Frequency(String strValue) {
         Matcher matcher = PATTERN.matcher(strValue);
-        if(!matcher.matches())
+        if (!matcher.matches()) {
             throw new IllegalArgumentException("Invalid frequency: " + strValue);
-        
+        }
+
         timeUnit = TimeUnit.valueOf(matcher.group(1));
         frequency = Integer.valueOf(matcher.group(2));
     }
-    
+
     public static Frequency fromString(String strValue) {
         return new Frequency(strValue);
     }
-    
+
     public static String toString(Frequency freq) {
         return freq.toString();
     }
-    
+
     @Override
     public String toString() {
         return timeUnit.name() + "(" + frequency + ")";
     }
-    
+
     public TimeUnit getTimeUnit() {
         return timeUnit;
     }
@@ -77,20 +78,23 @@ public class Frequency {
     public int getFrequency() {
         return frequency;
     }
-    
+
     @Override
     public boolean equals(Object obj) {
-        if(obj == null)
+        if (obj == null) {
             return false;
-        
-        if(!(obj instanceof Frequency))
+        }
+
+        if (!(obj instanceof Frequency)) {
             return false;
-        
+        }
+
         Frequency freq = (Frequency) obj;
-        if(this == freq)
+        if (this == freq) {
             return true;
-        
-        return this.getFrequency() == freq.getFrequency() && 
+        }
+
+        return this.getFrequency() == freq.getFrequency() &&
                 this.getTimeUnit() == freq.getTimeUnit();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java b/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
index d4e57f9..9baf827 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/SchemaHelper.java
@@ -40,8 +40,9 @@ public class SchemaHelper {
     }
 
     public static Date parseDateUTC(String dateStr) {
-        if(!DateValidator.validate(dateStr))
+        if (!DateValidator.validate(dateStr)) {
             throw new IllegalArgumentException(dateStr + " is not a valid UTC string");
+        }
         try {
             return getDateFormat().parse(dateStr);
         } catch (ParseException e) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/resource/APIResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/APIResult.java b/client/src/main/java/org/apache/falcon/resource/APIResult.java
index bf4cad2..7b138bc 100644
--- a/client/src/main/java/org/apache/falcon/resource/APIResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/APIResult.java
@@ -18,8 +18,7 @@
 
 package org.apache.falcon.resource;
 
-import java.io.StringWriter;
-import java.util.UUID;
+import org.apache.log4j.NDC;
 
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
@@ -27,11 +26,10 @@ import javax.xml.bind.Marshaller;
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.log4j.NDC;
+import java.io.StringWriter;
+import java.util.UUID;
 
 /**
- * 
  * APIResult is the output returned by all the APIs; status-SUCCEEDED or FAILED
  * message- detailed message
  */

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/resource/EntityList.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntityList.java b/client/src/main/java/org/apache/falcon/resource/EntityList.java
index 79247e3..24771cc 100644
--- a/client/src/main/java/org/apache/falcon/resource/EntityList.java
+++ b/client/src/main/java/org/apache/falcon/resource/EntityList.java
@@ -29,7 +29,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class EntityList {
 
-    @XmlElement (name = "entity")
+    @XmlElement(name = "entity")
     private EntityElement[] elements;
 
     public static class EntityElement {
@@ -54,7 +54,8 @@ public class EntityList {
     }
 
     //For JAXB
-    public EntityList() {}
+    public EntityList() {
+    }
 
     public EntityList(Entity[] elements) {
         EntityElement[] items = new EntityElement[elements.length];

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
index 7d64111..f790df1 100644
--- a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
@@ -18,36 +18,35 @@
 
 package org.apache.falcon.resource;
 
-import java.util.Date;
-
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
+import java.util.Date;
 
 @XmlRootElement
 public class InstancesResult extends APIResult {
-	public static enum WorkflowStatus {
-		WAITING, RUNNING, SUSPENDED, KILLED, FAILED, SUCCEEDED, ERROR
-	}
+    public static enum WorkflowStatus {
+        WAITING, RUNNING, SUSPENDED, KILLED, FAILED, SUCCEEDED, ERROR
+    }
 
-	@XmlElement
+    @XmlElement
     private Instance[] instances;
 
     private InstancesResult() { // for jaxb
         super();
     }
-    
+
     public InstancesResult(String message, Instance[] instances) {
-    	this(Status.SUCCEEDED, message, instances);
+        this(Status.SUCCEEDED, message, instances);
     }
 
     public InstancesResult(Status status, String message,
                            Instance[] instanceExes) {
-    	super(status, message);
-    	this.instances = instanceExes;
+        super(status, message);
+        this.instances = instanceExes;
     }
 
-	public InstancesResult(Status status, String message) {
-	    super(status, message);
+    public InstancesResult(Status status, String message) {
+        super(status, message);
     }
 
 
@@ -55,24 +54,24 @@ public class InstancesResult extends APIResult {
         return instances;
     }
 
-	public void setInstances(Instance[] instances) {
-		this.instances = instances;
-	}
-	
-	@XmlRootElement(name = "instance")
-	public static class Instance {
-		@XmlElement
-		public String instance;
+    public void setInstances(Instance[] instances) {
+        this.instances = instances;
+    }
 
-		@XmlElement
-		public WorkflowStatus status;
+    @XmlRootElement(name = "instance")
+    public static class Instance {
+        @XmlElement
+        public String instance;
 
         @XmlElement
-		public String logFile;
+        public WorkflowStatus status;
+
+        @XmlElement
+        public String logFile;
 
         @XmlElement
         public String cluster;
-        
+
         @XmlElement
         public String sourceCluster;
 
@@ -81,108 +80,108 @@ public class InstancesResult extends APIResult {
 
         @XmlElement
         public Date endTime;
-        
+
         @XmlElement
         public String details;
 
-		@XmlElement
-		public InstanceAction[] actions;
+        @XmlElement
+        public InstanceAction[] actions;
 
-		public Instance() {
-		}
+        public Instance() {
+        }
 
-		public Instance(String cluster, String instance, WorkflowStatus status) {
-			this.cluster = cluster;
-			this.instance = instance;
-			this.status = status;
-		}
+        public Instance(String cluster, String instance, WorkflowStatus status) {
+            this.cluster = cluster;
+            this.instance = instance;
+            this.status = status;
+        }
 
         public String getInstance() {
             return instance;
         }
-        
+
         public WorkflowStatus getStatus() {
             return status;
         }
-        
-		public String getLogFile() {
-			return logFile;
-		}
-
-		public String getCluster() {
-			return cluster;
-		}
-
-		public String getSourceCluster() {
-			return sourceCluster;
-		}
-
-		public Date getStartTime() {
-			return startTime;
-		}
-
-		public Date getEndTime() {
-			return endTime;
-		}
-
-		public InstanceAction[] getActions() {
-			return actions;
-		}
-		
-		public String getDetails() {
-			return details;
-		}
-
-
-		@Override
-		public String toString() {
-			return "{instance:"
-					+ this.instance
-					+ ", status:"
-					+ this.status
-					+ (this.logFile == null ? "" : ", log:" + this.logFile)
-					+ (this.sourceCluster == null ? "" : ", source-cluster:"
-							+ this.sourceCluster)
-					+ (this.cluster == null ? "" : ", cluster:"
-							+ this.cluster) + "}";
-		}
+
+        public String getLogFile() {
+            return logFile;
+        }
+
+        public String getCluster() {
+            return cluster;
+        }
+
+        public String getSourceCluster() {
+            return sourceCluster;
+        }
+
+        public Date getStartTime() {
+            return startTime;
+        }
+
+        public Date getEndTime() {
+            return endTime;
+        }
+
+        public InstanceAction[] getActions() {
+            return actions;
+        }
+
+        public String getDetails() {
+            return details;
+        }
+
+
+        @Override
+        public String toString() {
+            return "{instance:"
+                    + this.instance
+                    + ", status:"
+                    + this.status
+                    + (this.logFile == null ? "" : ", log:" + this.logFile)
+                    + (this.sourceCluster == null ? "" : ", source-cluster:"
+                    + this.sourceCluster)
+                    + (this.cluster == null ? "" : ", cluster:"
+                    + this.cluster) + "}";
+        }
+    }
+
+    @XmlRootElement(name = "actions")
+    public static class InstanceAction {
+        @XmlElement
+        public String action;
+        @XmlElement
+        public String status;
+        @XmlElement
+        public String logFile;
+
+        public InstanceAction() {
+        }
+
+        public InstanceAction(String action, String status, String logFile) {
+            this.action = action;
+            this.status = status;
+            this.logFile = logFile;
+        }
+
+        public String getAction() {
+            return action;
+        }
+
+        public String getStatus() {
+            return status;
+        }
+
+        public String getLogFile() {
+            return logFile;
+        }
+
+        @Override
+        public String toString() {
+            return "{action:" + this.action + ", status:" + this.status
+                    + (this.logFile == null ? "" : ", log:" + this.logFile)
+                    + "}";
+        }
     }
-    
-	@XmlRootElement(name = "actions")
-	public static class InstanceAction {
-		@XmlElement
-		public String action;
-		@XmlElement
-		public String status;
-		@XmlElement
-		public String logFile;
-
-		public InstanceAction() {
-		}
-
-		public InstanceAction(String action, String status, String logFile) {
-			this.action = action;
-			this.status = status;
-			this.logFile = logFile;
-		}
-
-		public String getAction() {
-			return action;
-		}
-
-		public String getStatus() {
-			return status;
-		}
-
-		public String getLogFile() {
-			return logFile;
-		}
-
-		@Override
-		public String toString() {
-			return "{action:" + this.action + ", status:" + this.status
-					+ (this.logFile == null ? "" : ", log:" + this.logFile)
-					+ "}";
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/resources/cluster-0.1.xsd
----------------------------------------------------------------------
diff --git a/client/src/main/resources/cluster-0.1.xsd b/client/src/main/resources/cluster-0.1.xsd
index a7e6245..b8643a7 100644
--- a/client/src/main/resources/cluster-0.1.xsd
+++ b/client/src/main/resources/cluster-0.1.xsd
@@ -16,9 +16,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" attributeFormDefault="unqualified" elementFormDefault="qualified" 
-            targetNamespace="uri:falcon:cluster:0.1" xmlns="uri:falcon:cluster:0.1"
-            xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" attributeFormDefault="unqualified" elementFormDefault="qualified"
+           targetNamespace="uri:falcon:cluster:0.1" xmlns="uri:falcon:cluster:0.1"
+           xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
     <xs:annotation>
         <xs:documentation>
             Licensed to the Apache Software Foundation (ASF) under one or more
@@ -40,7 +40,7 @@
         </xs:documentation>
         <xs:appinfo>
             <jaxb:schemaBindings>
-                <jaxb:package name="org.apache.falcon.entity.v0.cluster" />
+                <jaxb:package name="org.apache.falcon.entity.v0.cluster"/>
             </jaxb:schemaBindings>
         </xs:appinfo>
     </xs:annotation>
@@ -57,13 +57,13 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="interfaces" name="interfaces" />
-            <xs:element type="locations" name="locations" />
-            <xs:element type="properties" name="properties" minOccurs="0" />
+            <xs:element type="interfaces" name="interfaces"/>
+            <xs:element type="locations" name="locations"/>
+            <xs:element type="properties" name="properties" minOccurs="0"/>
         </xs:sequence>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="xs:string" name="description" />
-        <xs:attribute type="xs:string" name="colo" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="description"/>
+        <xs:attribute type="xs:string" name="colo" use="required"/>
     </xs:complexType>
     <xs:complexType name="locations">
         <xs:annotation>
@@ -71,7 +71,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="location" name="location" maxOccurs="unbounded" minOccurs="1" />
+            <xs:element type="location" name="location" maxOccurs="unbounded" minOccurs="1"/>
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="property">
@@ -84,8 +84,8 @@
                 should be defined here.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="xs:string" name="name" use="required" />
-        <xs:attribute type="xs:string" name="value" use="required" />
+        <xs:attribute type="xs:string" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="value" use="required"/>
     </xs:complexType>
     <xs:complexType name="interface">
         <xs:annotation>
@@ -99,9 +99,9 @@
                 version: The current runtime version of each interface.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="interfacetype" name="type" use="required" />
-        <xs:attribute type="xs:string" name="endpoint" use="required" />
-        <xs:attribute type="xs:string" name="version" use="required" />
+        <xs:attribute type="interfacetype" name="type" use="required"/>
+        <xs:attribute type="xs:string" name="endpoint" use="required"/>
+        <xs:attribute type="xs:string" name="version" use="required"/>
     </xs:complexType>
     <xs:complexType name="properties">
         <xs:annotation>
@@ -110,7 +110,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0" />
+            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0"/>
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="location">
@@ -126,8 +126,8 @@
                 locations.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="xs:string" name="path" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="path" use="required"/>
     </xs:complexType>
     <xs:complexType name="interfaces">
         <xs:annotation>
@@ -136,7 +136,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="interface" name="interface" maxOccurs="unbounded" minOccurs="3" />
+            <xs:element type="interface" name="interface" maxOccurs="unbounded" minOccurs="3"/>
         </xs:sequence>
     </xs:complexType>
     <xs:simpleType name="interfacetype">
@@ -158,17 +158,17 @@
             </xs:documentation>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:enumeration value="readonly" />
-            <xs:enumeration value="write" />
-            <xs:enumeration value="execute" />
-            <xs:enumeration value="workflow" />
-            <xs:enumeration value="messaging" />
-            <xs:enumeration value="registry" />
+            <xs:enumeration value="readonly"/>
+            <xs:enumeration value="write"/>
+            <xs:enumeration value="execute"/>
+            <xs:enumeration value="workflow"/>
+            <xs:enumeration value="messaging"/>
+            <xs:enumeration value="registry"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="IDENTIFIER">
         <xs:restriction base="xs:string">
-            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})" />
+            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})"/>
         </xs:restriction>
     </xs:simpleType>
 </xs:schema>


[37/47] git commit: Removed ivory dir from project

Posted by sr...@apache.org.
Removed ivory dir from project


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

Branch: refs/heads/master
Commit: f739a7e82e47aefddbd82780305fb77acb67fcb3
Parents: 8dbbd5c
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:50:35 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:50:35 2013 +0530

----------------------------------------------------------------------
 .../org/apache/ivory/logging/java-test.jar         |  Bin 15728 -> 0 bytes
 1 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f739a7e8/webapp/src/test/resources/org/apache/ivory/logging/java-test.jar
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/org/apache/ivory/logging/java-test.jar b/webapp/src/test/resources/org/apache/ivory/logging/java-test.jar
deleted file mode 100644
index d743543..0000000
Binary files a/webapp/src/test/resources/org/apache/ivory/logging/java-test.jar and /dev/null differ


[19/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/resources/feed-0.1.xsd
----------------------------------------------------------------------
diff --git a/client/src/main/resources/feed-0.1.xsd b/client/src/main/resources/feed-0.1.xsd
index e0e2204..a2b73fe 100644
--- a/client/src/main/resources/feed-0.1.xsd
+++ b/client/src/main/resources/feed-0.1.xsd
@@ -16,8 +16,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" attributeFormDefault="unqualified" elementFormDefault="qualified" targetNamespace="uri:falcon:feed:0.1" xmlns="uri:falcon:feed:0.1"
-    xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" attributeFormDefault="unqualified" elementFormDefault="qualified"
+           targetNamespace="uri:falcon:feed:0.1" xmlns="uri:falcon:feed:0.1"
+           xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
 
     <xs:annotation>
         <xs:documentation>
@@ -44,7 +45,7 @@
         </xs:documentation>
         <xs:appinfo>
             <jaxb:schemaBindings>
-                <jaxb:package name="org.apache.falcon.entity.v0.feed" />
+                <jaxb:package name="org.apache.falcon.entity.v0.feed"/>
             </jaxb:schemaBindings>
         </xs:appinfo>
     </xs:annotation>
@@ -76,30 +77,30 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="partitions" name="partitions" minOccurs="0" />
-            <xs:element type="group-type" name="groups" minOccurs="0" />
-            <xs:element type="xs:string" name="availabilityFlag" minOccurs="0" />
-            <xs:element type="frequency-type" name="frequency" />
+            <xs:element type="partitions" name="partitions" minOccurs="0"/>
+            <xs:element type="group-type" name="groups" minOccurs="0"/>
+            <xs:element type="xs:string" name="availabilityFlag" minOccurs="0"/>
+            <xs:element type="frequency-type" name="frequency"/>
             <xs:element name="timezone" minOccurs="0" default="UTC">
                 <xs:simpleType>
                     <xs:annotation>
                         <xs:appinfo>
-                            <jaxb:javaType name="java.util.TimeZone" parseMethod="java.util.TimeZone.getTimeZone" 
-                                        printMethod="org.apache.falcon.entity.v0.SchemaHelper.getTimeZoneId" />
+                            <jaxb:javaType name="java.util.TimeZone" parseMethod="java.util.TimeZone.getTimeZone"
+                                           printMethod="org.apache.falcon.entity.v0.SchemaHelper.getTimeZoneId"/>
                         </xs:appinfo>
                     </xs:annotation>
-                    <xs:restriction base="xs:string" />
+                    <xs:restriction base="xs:string"/>
                 </xs:simpleType>
             </xs:element>
-            <xs:element type="late-arrival" name="late-arrival" minOccurs="0" />
-            <xs:element type="clusters" name="clusters" />
-            <xs:element type="locations" name="locations" />
-            <xs:element type="ACL" name="ACL" />
-            <xs:element type="schema" name="schema" />
-            <xs:element type="properties" name="properties" minOccurs="0" />
+            <xs:element type="late-arrival" name="late-arrival" minOccurs="0"/>
+            <xs:element type="clusters" name="clusters"/>
+            <xs:element type="locations" name="locations"/>
+            <xs:element type="ACL" name="ACL"/>
+            <xs:element type="schema" name="schema"/>
+            <xs:element type="properties" name="properties" minOccurs="0"/>
         </xs:sequence>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="xs:string" name="description" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="description"/>
     </xs:complexType>
 
     <xs:complexType name="cluster">
@@ -118,19 +119,20 @@
                 specified by expression frequency(times), ex: if
                 feed should be retained for at least 6 hours then retention's limit="hours(6)".
                 The field partitionExp contains
-                partition tags. Number of partition tags has to be equal to number of partitions specified in feed schema.
+                partition tags. Number of partition tags has to be equal to number of partitions specified in feed
+                schema.
                 A partition tag can be a wildcard(*), a static string or
                 an expression. Atleast one of the strings has to be an expression.
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="validity" name="validity" />
-            <xs:element type="retention" name="retention" />
+            <xs:element type="validity" name="validity"/>
+            <xs:element type="retention" name="retention"/>
             <xs:element type="locations" name="locations" minOccurs="0"/>
         </xs:sequence>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="cluster-type" name="type" use="optional" />
-        <xs:attribute type="xs:string" name="partition" use="optional" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="cluster-type" name="type" use="optional"/>
+        <xs:attribute type="xs:string" name="partition" use="optional"/>
     </xs:complexType>
     <xs:complexType name="partitions">
         <xs:annotation>
@@ -140,7 +142,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="partition" name="partition" maxOccurs="unbounded" minOccurs="0" />
+            <xs:element type="partition" name="partition" maxOccurs="unbounded" minOccurs="0"/>
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="schema">
@@ -149,8 +151,8 @@
                 for a feed and the provider of schema like protobuf, thrift etc.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="xs:string" name="location" use="required" />
-        <xs:attribute type="xs:string" name="provider" use="required" />
+        <xs:attribute type="xs:string" name="location" use="required"/>
+        <xs:attribute type="xs:string" name="provider" use="required"/>
     </xs:complexType>
     <xs:complexType name="properties">
         <xs:annotation>
@@ -159,7 +161,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0" />
+            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0"/>
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="validity">
@@ -174,8 +176,8 @@
                 validation.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="date-time-type" name="start" use="required" />
-        <xs:attribute type="date-time-type" name="end" use="required" />
+        <xs:attribute type="date-time-type" name="start" use="required"/>
+        <xs:attribute type="date-time-type" name="end" use="required"/>
     </xs:complexType>
     <xs:complexType name="locations">
         <xs:annotation>
@@ -184,7 +186,7 @@
             </xs:documentation>
         </xs:annotation>
         <xs:choice maxOccurs="unbounded" minOccurs="0">
-            <xs:element type="location" name="location" />
+            <xs:element type="location" name="location"/>
         </xs:choice>
     </xs:complexType>
     <xs:complexType name="late-arrival">
@@ -201,7 +203,7 @@
                 upto 8 hours then late-arrival's cut-off="hours(8)"
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="frequency-type" name="cut-off" use="required" />
+        <xs:attribute type="frequency-type" name="cut-off" use="required"/>
     </xs:complexType>
     <xs:complexType name="property">
         <xs:annotation>
@@ -210,8 +212,8 @@
                 workflow engine.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="xs:string" name="name" use="required" />
-        <xs:attribute type="xs:string" name="value" use="required" />
+        <xs:attribute type="xs:string" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="value" use="required"/>
     </xs:complexType>
     <xs:complexType name="clusters">
         <xs:annotation>
@@ -225,13 +227,13 @@
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="retention">
-        <xs:attribute type="retention-type" name="type" default="instance" />
-        <xs:attribute type="frequency-type" name="limit" use="required" />
-        <xs:attribute type="action-type" name="action" use="required" />
+        <xs:attribute type="retention-type" name="type" default="instance"/>
+        <xs:attribute type="frequency-type" name="limit" use="required"/>
+        <xs:attribute type="action-type" name="action" use="required"/>
     </xs:complexType>
     <xs:simpleType name="retention-type">
         <xs:restriction base="xs:string">
-            <xs:enumeration value="instance" />
+            <xs:enumeration value="instance"/>
             <!-- <xs:enumeration value="age" /> -->
         </xs:restriction>
     </xs:simpleType>
@@ -246,11 +248,11 @@
                 periodically. ex: type="data" path="/projects/TrafficHourly/${YEAR}-${MONTH}-${DAY}/traffic"
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="location-type" name="type" use="required" />
-        <xs:attribute type="xs:string" name="path" use="required" />
+        <xs:attribute type="location-type" name="type" use="required"/>
+        <xs:attribute type="xs:string" name="path" use="required"/>
     </xs:complexType>
     <xs:complexType name="partition">
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
     </xs:complexType>
     <xs:complexType name="ACL">
         <xs:annotation>
@@ -258,9 +260,9 @@
                 Access control list for this feed.
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="xs:string" name="owner" />
-        <xs:attribute type="xs:string" name="group" />
-        <xs:attribute type="xs:string" name="permission" />
+        <xs:attribute type="xs:string" name="owner"/>
+        <xs:attribute type="xs:string" name="group"/>
+        <xs:attribute type="xs:string" name="permission"/>
     </xs:complexType>
     <xs:simpleType name="action-type">
         <xs:restriction base="xs:string">
@@ -273,10 +275,10 @@
                     archive, delete, chown and chmod.
                 </xs:documentation>
             </xs:annotation>
-            <xs:enumeration value="archive" />
-            <xs:enumeration value="delete" />
-            <xs:enumeration value="chown" />
-            <xs:enumeration value="chmod" />
+            <xs:enumeration value="archive"/>
+            <xs:enumeration value="delete"/>
+            <xs:enumeration value="chown"/>
+            <xs:enumeration value="chmod"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="cluster-type">
@@ -291,47 +293,50 @@
             </xs:documentation>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:enumeration value="source" />
-            <xs:enumeration value="target" />
+            <xs:enumeration value="source"/>
+            <xs:enumeration value="target"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="location-type">
         <xs:restriction base="xs:string">
-            <xs:enumeration value="data" />
-            <xs:enumeration value="stats" />
-            <xs:enumeration value="meta" />
-            <xs:enumeration value="tmp" />
+            <xs:enumeration value="data"/>
+            <xs:enumeration value="stats"/>
+            <xs:enumeration value="meta"/>
+            <xs:enumeration value="tmp"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="IDENTIFIER">
         <xs:restriction base="xs:string">
-            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})" />
+            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="frequency-type">
         <xs:annotation>
             <xs:appinfo>
-                <jaxb:javaType name="org.apache.falcon.entity.v0.Frequency" parseMethod="org.apache.falcon.entity.v0.Frequency.fromString"
-                        printMethod="org.apache.falcon.entity.v0.Frequency.toString" />
+                <jaxb:javaType name="org.apache.falcon.entity.v0.Frequency"
+                               parseMethod="org.apache.falcon.entity.v0.Frequency.fromString"
+                               printMethod="org.apache.falcon.entity.v0.Frequency.toString"/>
             </xs:appinfo>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:pattern value="(minutes|hours|days|months)\([1-9]\d*\)" />
+            <xs:pattern value="(minutes|hours|days|months)\([1-9]\d*\)"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="date-time-type">
         <xs:annotation>
             <xs:appinfo>
-                <jaxb:javaType name="java.util.Date" parseMethod="org.apache.falcon.entity.v0.SchemaHelper.parseDateUTC" printMethod="org.apache.falcon.entity.v0.SchemaHelper.formatDateUTC" />
+                <jaxb:javaType name="java.util.Date" parseMethod="org.apache.falcon.entity.v0.SchemaHelper.parseDateUTC"
+                               printMethod="org.apache.falcon.entity.v0.SchemaHelper.formatDateUTC"/>
             </xs:appinfo>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:pattern value="((19|20)\d\d[- /.](0[1-9]|1[012])[- /.](0[1-9]|[12][0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9]))Z" />
+            <xs:pattern
+                    value="((19|20)\d\d[- /.](0[1-9]|1[012])[- /.](0[1-9]|[12][0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9]))Z"/>
         </xs:restriction>
     </xs:simpleType>
     <xs:simpleType name="group-type">
         <xs:restriction base="xs:string">
-            <xs:pattern value="(\w+(,\w+)*)" />
+            <xs:pattern value="(\w+(,\w+)*)"/>
         </xs:restriction>
     </xs:simpleType>
 </xs:schema>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/resources/jaxb-binding.xjb
----------------------------------------------------------------------
diff --git a/client/src/main/resources/jaxb-binding.xjb b/client/src/main/resources/jaxb-binding.xjb
index 7e4cb12..93e72af 100644
--- a/client/src/main/resources/jaxb-binding.xjb
+++ b/client/src/main/resources/jaxb-binding.xjb
@@ -17,19 +17,17 @@
   limitations under the License.
   -->
 <jaxb:bindings
-    jaxb:version="2.1"
-    jaxb:extensionBindingPrefixes="xjc jaxb xs inheritance annox"
-    xmlns:jaxb="http://java.sun.com/xml/ns/jaxb"
-    xmlns:xjc="http://java.sun.com/xml/ns/jaxb/xjc"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xmlns:inheritance="http://jaxb2-commons.dev.java.net/basic/inheritance"
-    xmlns:annox="http://annox.dev.java.net"
-    xmlns:xs="http://www.w3.org/2001/XMLSchema">
-    
+        jaxb:version="2.1"
+        jaxb:extensionBindingPrefixes="xjc jaxb xs inheritance annox"
+        xmlns:jaxb="http://java.sun.com/xml/ns/jaxb"
+        xmlns:xjc="http://java.sun.com/xml/ns/jaxb/xjc"
+        xmlns:inheritance="http://jaxb2-commons.dev.java.net/basic/inheritance"
+        xmlns:xs="http://www.w3.org/2001/XMLSchema">
+
     <jaxb:bindings schemaLocation="cluster-0.1.xsd" node="//xs:complexType[@name='cluster']">
         <inheritance:extends>org.apache.falcon.entity.v0.Entity</inheritance:extends>
     </jaxb:bindings>
-    
+
     <jaxb:bindings schemaLocation="feed-0.1.xsd" node="//xs:complexType[@name='feed']">
         <inheritance:extends>org.apache.falcon.entity.v0.Entity</inheritance:extends>
     </jaxb:bindings>
@@ -38,8 +36,8 @@
         <inheritance:extends>org.apache.falcon.entity.v0.Entity</inheritance:extends>
     </jaxb:bindings>
 
-      <jaxb:globalBindings>
+    <jaxb:globalBindings>
         <xjc:simple/>
-      </jaxb:globalBindings>
-      
-    </jaxb:bindings>
+    </jaxb:globalBindings>
+
+</jaxb:bindings>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/resources/process-0.1.xsd
----------------------------------------------------------------------
diff --git a/client/src/main/resources/process-0.1.xsd b/client/src/main/resources/process-0.1.xsd
index e322353..1368d85 100644
--- a/client/src/main/resources/process-0.1.xsd
+++ b/client/src/main/resources/process-0.1.xsd
@@ -16,8 +16,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<xs:schema attributeFormDefault="unqualified" elementFormDefault="qualified" xmlns:xs="http://www.w3.org/2001/XMLSchema" targetNamespace="uri:falcon:process:0.1" xmlns="uri:falcon:process:0.1"
-    xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
+<xs:schema attributeFormDefault="unqualified" elementFormDefault="qualified" xmlns:xs="http://www.w3.org/2001/XMLSchema"
+           targetNamespace="uri:falcon:process:0.1" xmlns="uri:falcon:process:0.1"
+           xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" jaxb:version="2.1">
     <xs:annotation>
         <xs:documentation>
             Licensed to the Apache Software Foundation (ASF) under one or more
@@ -43,7 +44,7 @@
         </xs:documentation>
         <xs:appinfo>
             <jaxb:schemaBindings>
-                <jaxb:package name="org.apache.falcon.entity.v0.process" />
+                <jaxb:package name="org.apache.falcon.entity.v0.process"/>
             </jaxb:schemaBindings>
         </xs:appinfo>
     </xs:annotation>
@@ -92,8 +93,8 @@
                 </xs:annotation>
                 <xs:simpleType>
                     <xs:restriction base="xs:unsignedShort">
-                        <xs:minInclusive value="1" />
-                        <xs:maxInclusive value="12" />
+                        <xs:minInclusive value="1"/>
+                        <xs:maxInclusive value="12"/>
                     </xs:restriction>
                 </xs:simpleType>
             </xs:element>
@@ -119,16 +120,17 @@
                 <xs:simpleType>
                     <xs:annotation>
                         <xs:appinfo>
-                            <jaxb:javaType name="java.util.TimeZone" parseMethod="java.util.TimeZone.getTimeZone" 
-                                        printMethod="org.apache.falcon.entity.v0.SchemaHelper.getTimeZoneId" />
+                            <jaxb:javaType name="java.util.TimeZone" parseMethod="java.util.TimeZone.getTimeZone"
+                                           printMethod="org.apache.falcon.entity.v0.SchemaHelper.getTimeZoneId"/>
                         </xs:appinfo>
                     </xs:annotation>
-                    <xs:restriction base="xs:string" />
+                    <xs:restriction base="xs:string"/>
                 </xs:simpleType>
             </xs:element>
             <xs:element type="inputs" name="inputs" minOccurs="0">
                 <xs:annotation>
-                    <xs:documentation>Defines inputs for the workflow. The workflow will run only when the scheduled time is up and all the inputs are available
+                    <xs:documentation>Defines inputs for the workflow. The workflow will run only when the scheduled
+                        time is up and all the inputs are available
                     </xs:documentation>
                 </xs:annotation>
             </xs:element>
@@ -142,15 +144,18 @@
             </xs:element>
             <xs:element type="workflow" name="workflow">
                 <xs:annotation>
-                    <xs:documentation>Defines the workflow that should run. The workflow should be defined with respect to the workflow specification of the workflow engine.
+                    <xs:documentation>Defines the workflow that should run. The workflow should be defined with respect
+                        to the workflow specification of the workflow engine.
                         Only
-                        oozie workflow engine is supported as of now. The workflow path is the path on hdfs which contains the workflow xml
+                        oozie workflow engine is supported as of now. The workflow path is the path on hdfs which
+                        contains the workflow xml
                     </xs:documentation>
                 </xs:annotation>
             </xs:element>
             <xs:element type="retry" name="retry" minOccurs="0">
                 <xs:annotation>
-                    <xs:documentation>Retry defines how to handle workflow failures. Policy type - backoff, exponention backoff along with the delay define how frequenctly
+                    <xs:documentation>Retry defines how to handle workflow failures. Policy type - backoff, exponention
+                        backoff along with the delay define how frequenctly
                         the
                         workflow should be re-tried. Number of attempts define how many times to re-try the failures.
                     </xs:documentation>
@@ -158,19 +163,21 @@
             </xs:element>
             <xs:element type="late-process" name="late-process" minOccurs="0">
                 <xs:annotation>
-                    <xs:documentation>Late process defines how the late data should be handled. Late policy - backoff, exponential backoff, final along with delay
+                    <xs:documentation>Late process defines how the late data should be handled. Late policy - backoff,
+                        exponential backoff, final along with delay
                         define how
-                        frequently Falcon should check for late data. The late data handling can be customized for each input separatly.
+                        frequently Falcon should check for late data. The late data handling can be customized for each
+                        input separatly.
                     </xs:documentation>
                 </xs:annotation>
             </xs:element>
         </xs:sequence>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
     </xs:complexType>
 
     <xs:simpleType name="IDENTIFIER">
         <xs:restriction base="xs:string">
-            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})" />
+            <xs:pattern value="(([a-zA-Z]([\-a-zA-Z0-9])*){1,39})"/>
         </xs:restriction>
     </xs:simpleType>
 
@@ -189,13 +196,14 @@
     <xs:complexType name="cluster">
         <xs:annotation>
             <xs:documentation>
-                Defines the cluster where the workflow should run. In addition, it also defines the validity of the workflow on this cluster
+                Defines the cluster where the workflow should run. In addition, it also defines the validity of the
+                workflow on this cluster
             </xs:documentation>
         </xs:annotation>
         <xs:sequence>
-            <xs:element type="validity" name="validity" />
+            <xs:element type="validity" name="validity"/>
         </xs:sequence>
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
     </xs:complexType>
 
     <xs:complexType name="validity">
@@ -204,37 +212,41 @@
                 Defines the vailidity of the workflow as start and end time
             </xs:documentation>
         </xs:annotation>
-        <xs:attribute type="date-time-type" name="start" use="required" />
-        <xs:attribute type="date-time-type" name="end" use="required" />
+        <xs:attribute type="date-time-type" name="start" use="required"/>
+        <xs:attribute type="date-time-type" name="end" use="required"/>
     </xs:complexType>
 
     <xs:simpleType name="date-time-type">
         <xs:annotation>
             <xs:appinfo>
-                <jaxb:javaType name="java.util.Date" parseMethod="org.apache.falcon.entity.v0.SchemaHelper.parseDateUTC" printMethod="org.apache.falcon.entity.v0.SchemaHelper.formatDateUTC" />
+                <jaxb:javaType name="java.util.Date" parseMethod="org.apache.falcon.entity.v0.SchemaHelper.parseDateUTC"
+                               printMethod="org.apache.falcon.entity.v0.SchemaHelper.formatDateUTC"/>
             </xs:appinfo>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:pattern value="((1|2)\d\d\d[- /.](0[1-9]|1[012])[- /.](0[1-9]|[12][0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9]))Z" />
+            <xs:pattern
+                    value="((1|2)\d\d\d[- /.](0[1-9]|1[012])[- /.](0[1-9]|[12][0-9]|3[01])T([0-1][0-9]|2[0-3]):([0-5][0-9]))Z"/>
         </xs:restriction>
     </xs:simpleType>
 
     <xs:simpleType name="execution-type">
         <xs:restriction base="xs:string">
-            <xs:enumeration value="FIFO" />
-            <xs:enumeration value="LIFO" />
-            <xs:enumeration value="ONLYLAST" />
+            <xs:enumeration value="FIFO"/>
+            <xs:enumeration value="LIFO"/>
+            <xs:enumeration value="ONLYLAST"/>
         </xs:restriction>
     </xs:simpleType>
 
     <xs:simpleType name="frequency-type">
         <xs:annotation>
             <xs:appinfo>
-                <jaxb:javaType name="org.apache.falcon.entity.v0.Frequency" parseMethod="org.apache.falcon.entity.v0.Frequency.fromString" printMethod="org.apache.falcon.entity.v0.Frequency.toString" />
+                <jaxb:javaType name="org.apache.falcon.entity.v0.Frequency"
+                               parseMethod="org.apache.falcon.entity.v0.Frequency.fromString"
+                               printMethod="org.apache.falcon.entity.v0.Frequency.toString"/>
             </xs:appinfo>
         </xs:annotation>
         <xs:restriction base="xs:string">
-            <xs:pattern value="(minutes|hours|days|months)\([1-9]\d*\)" />
+            <xs:pattern value="(minutes|hours|days|months)\([1-9]\d*\)"/>
         </xs:restriction>
     </xs:simpleType>
 
@@ -242,9 +254,11 @@
         <xs:sequence>
             <xs:element type="input" name="input" maxOccurs="unbounded" minOccurs="0">
                 <xs:annotation>
-                    <xs:documentation>Defines input for the workflow. Each input maps to a feed. Input path and frequency are picked from feed definition.
+                    <xs:documentation>Defines input for the workflow. Each input maps to a feed. Input path and
+                        frequency are picked from feed definition.
                         The input specifies the
-                        start and end instance for the workflow. Falcon creates a property with input name which contains paths of all input
+                        start and end instance for the workflow. Falcon creates a property with input name which
+                        contains paths of all input
                         instances between start and end. This
                         property will be available for the workflow to read inputs.
                         Input can also optionally specify the specific partition of feed that the workflow needs.
@@ -255,11 +269,11 @@
     </xs:complexType>
 
     <xs:complexType name="input">
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="IDENTIFIER" name="feed" use="required" />
-        <xs:attribute type="xs:string" name="start" use="required" />
-        <xs:attribute type="xs:string" name="end" use="required" />
-        <xs:attribute type="xs:string" name="partition" use="optional" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="IDENTIFIER" name="feed" use="required"/>
+        <xs:attribute type="xs:string" name="start" use="required"/>
+        <xs:attribute type="xs:string" name="end" use="required"/>
+        <xs:attribute type="xs:string" name="partition" use="optional"/>
         <xs:attribute type="xs:boolean" name="optional" use="optional" default="false"/>
     </xs:complexType>
 
@@ -267,7 +281,8 @@
         <xs:sequence>
             <xs:element type="output" name="output" maxOccurs="unbounded" minOccurs="0">
                 <xs:annotation>
-                    <xs:documentation> Each output maps to a feed. The Output path and frequency are picked from the corresponding feed definition.
+                    <xs:documentation>Each output maps to a feed. The Output path and frequency are picked from the
+                        corresponding feed definition.
                         The output also specifies the
                         instance that is created in terms of EL expression.
                         For each output, Falcon creates a property with the output name which can be used in workflows
@@ -278,40 +293,40 @@
     </xs:complexType>
 
     <xs:complexType name="output">
-        <xs:attribute type="IDENTIFIER" name="name" use="required" />
-        <xs:attribute type="IDENTIFIER" name="feed" use="required" />
-        <xs:attribute type="xs:string" name="instance" use="required" />
+        <xs:attribute type="IDENTIFIER" name="name" use="required"/>
+        <xs:attribute type="IDENTIFIER" name="feed" use="required"/>
+        <xs:attribute type="xs:string" name="instance" use="required"/>
     </xs:complexType>
 
     <xs:complexType name="workflow">
-        <xs:attribute type="engine-type" name="engine" use="optional" />
-        <xs:attribute type="xs:string" name="path" use="required" />
-        <xs:attribute type="xs:string" name="lib" use="optional" />
+        <xs:attribute type="engine-type" name="engine" use="optional"/>
+        <xs:attribute type="xs:string" name="path" use="required"/>
+        <xs:attribute type="xs:string" name="lib" use="optional"/>
     </xs:complexType>
 
     <xs:simpleType name="engine-type">
         <xs:restriction base="xs:string">
-            <xs:enumeration value="oozie" />
+            <xs:enumeration value="oozie"/>
         </xs:restriction>
     </xs:simpleType>
 
     <xs:complexType name="retry">
-        <xs:attribute type="policy-type" name="policy" use="required" />
-        <xs:attribute type="frequency-type" name="delay" use="required" />
+        <xs:attribute type="policy-type" name="policy" use="required"/>
+        <xs:attribute type="frequency-type" name="delay" use="required"/>
         <xs:attribute name="attempts" use="required">
             <xs:simpleType>
                 <xs:restriction base="xs:unsignedShort">
-                    <xs:minInclusive value="1" />
+                    <xs:minInclusive value="1"/>
                 </xs:restriction>
             </xs:simpleType>
         </xs:attribute>
     </xs:complexType>
-	
+
     <xs:simpleType name="policy-type">
         <xs:restriction base="xs:string">
-            <xs:enumeration value="periodic" />
-            <xs:enumeration value="exp-backoff" />
-            <xs:enumeration value="final" />
+            <xs:enumeration value="periodic"/>
+            <xs:enumeration value="exp-backoff"/>
+            <xs:enumeration value="final"/>
         </xs:restriction>
     </xs:simpleType>
 
@@ -325,22 +340,22 @@
                 </xs:annotation>
             </xs:element>
         </xs:sequence>
-        <xs:attribute type="policy-type" name="policy" use="required" />
-        <xs:attribute type="frequency-type" name="delay" use="required" />
+        <xs:attribute type="policy-type" name="policy" use="required"/>
+        <xs:attribute type="frequency-type" name="delay" use="required"/>
     </xs:complexType>
 
     <xs:complexType name="late-input">
-        <xs:attribute type="IDENTIFIER" name="input" use="required" />
-        <xs:attribute type="xs:string" name="workflow-path" use="required" />
+        <xs:attribute type="IDENTIFIER" name="input" use="required"/>
+        <xs:attribute type="xs:string" name="workflow-path" use="required"/>
     </xs:complexType>
 
     <xs:complexType name="properties">
         <xs:sequence>
-            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0" />
+            <xs:element type="property" name="property" maxOccurs="unbounded" minOccurs="0"/>
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="property">
-        <xs:attribute type="xs:string" name="name" use="required" />
-        <xs:attribute type="xs:string" name="value" use="required" />
+        <xs:attribute type="xs:string" name="name" use="required"/>
+        <xs:attribute type="xs:string" name="value" use="required"/>
     </xs:complexType>
 </xs:schema>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java b/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
index 943acba..a52da32 100644
--- a/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
+++ b/client/src/test/java/org/apache/falcon/cli/TestCLIParser.java
@@ -26,41 +26,41 @@ import org.testng.annotations.Test;
 
 public class TestCLIParser {
 
-	@Test
-	public void testEmptyParser() throws Exception {
-		try {
-			CLIParser parser = new CLIParser("falcon", new String[] {});
-			CLIParser.Command c = parser.parse(new String[] { "a" });
-			Assert.fail();
-		} catch (ParseException ex) {
-			// nop
-		}
-	}
+    @Test
+    public void testEmptyParser() throws Exception {
+        try {
+            CLIParser parser = new CLIParser("falcon", new String[]{});
+            CLIParser.Command c = parser.parse(new String[]{"a"});
+            Assert.fail();
+        } catch (ParseException ex) {
+            // nop
+        }
+    }
 
-	@Test
-	public void testCommandParser() throws Exception {
-		try {
-			CLIParser parser = new CLIParser("oozie", new String[] {});
-			parser.addCommand("a", "<A>", "AAAAA", new Options(), false);
-			CLIParser.Command c = parser.parse(new String[] { "a", "b" });
-			Assert.assertEquals("a", c.getName());
-			Assert.assertEquals("b", c.getCommandLine().getArgs()[0]);
-		} catch (ParseException ex) {
-			Assert.fail();
-		}
-	}
+    @Test
+    public void testCommandParser() throws Exception {
+        try {
+            CLIParser parser = new CLIParser("oozie", new String[]{});
+            parser.addCommand("a", "<A>", "AAAAA", new Options(), false);
+            CLIParser.Command c = parser.parse(new String[]{"a", "b"});
+            Assert.assertEquals("a", c.getName());
+            Assert.assertEquals("b", c.getCommandLine().getArgs()[0]);
+        } catch (ParseException ex) {
+            Assert.fail();
+        }
+    }
 
-	@Test
-	public void testCommandParserX() throws Exception {
-		Option opt = new Option("o", false, "O");
-		Options opts = new Options();
-		opts.addOption(opt);
-		CLIParser parser = new CLIParser("test", new String[] {});
-		parser.addCommand("c", "-X ",
-				"(everything after '-X' are pass-through parameters)", opts,
-				true);
-		CLIParser.Command c = parser.parse("c -o -X -o c".split(" "));
-		Assert.assertEquals("-X", c.getCommandLine().getArgList().get(0));
-		Assert.assertEquals(3, c.getCommandLine().getArgList().size());
-	}
+    @Test
+    public void testCommandParserX() throws Exception {
+        Option opt = new Option("o", false, "O");
+        Options opts = new Options();
+        opts.addOption(opt);
+        CLIParser parser = new CLIParser("test", new String[]{});
+        parser.addCommand("c", "-X ",
+                "(everything after '-X' are pass-through parameters)", opts,
+                true);
+        CLIParser.Command c = parser.parse("c -o -X -o c".split(" "));
+        Assert.assertEquals("-X", c.getCommandLine().getArgList().get(0));
+        Assert.assertEquals(3, c.getCommandLine().getArgList().size());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java b/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
index 4a34127..fc81896 100644
--- a/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
+++ b/client/src/test/java/org/apache/falcon/entity/v0/DateValidatorTest.java
@@ -17,66 +17,66 @@
  */
 
 package org.apache.falcon.entity.v0;
-import org.apache.falcon.entity.v0.DateValidator;
+
 import org.testng.Assert;
-import org.testng.annotations.*;
- 
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
 /**
  * Date format yyyy/mm/dd validator Testing
- *
  */
 public class DateValidatorTest {
- 
-	@DataProvider
-	public Object[][] ValidDateProvider() {
-		return new Object[][]{
-			new Object[] {"2011-11-01T00:00Z"}, new Object[] {"2020-01-01T00:00Z"}, 
-			new Object[] {"2010-01-31T00:59Z"}, new Object[] {"2020-01-31T00:00Z"}, 
-			new Object[] {"2008-02-29T01:00Z"}, new Object[] {"2008-02-29T00:00Z"},
-			new Object[] {"2009-02-28T01:01Z"}, new Object[] {"2009-02-28T00:00Z"},
-			new Object[] {"2010-03-31T23:00Z"}, new Object[] {"2010-03-31T00:00Z"},
-			new Object[] {"2010-04-30T23:59Z"}, new Object[] {"2010-04-30T00:00Z"},
-			new Object[] {"2010-05-31T23:23Z"}, new Object[] {"2010-05-31T00:00Z"},
-			new Object[] {"2010-06-30T00:00Z"}, new Object[] {"2010-06-30T00:00Z"},
-			new Object[] {"2010-07-31T00:00Z"}, new Object[] {"2010-07-31T00:00Z"},
-			new Object[] {"2010-08-31T00:00Z"}, new Object[] {"2010-08-31T00:00Z"},
-			new Object[] {"2010-09-30T00:00Z"}, new Object[] {"2010-09-30T00:00Z"},
-			new Object[] {"2010-10-31T00:00Z"}, new Object[] {"2010-10-31T00:00Z"},
-			new Object[] {"2010-11-30T00:00Z"}, new Object[] {"2010-11-30T00:00Z"},
-			new Object[] {"2010-12-31T00:00Z"}, new Object[] {"2010-12-31T00:00Z"},
-			new Object[] {"1999-01-30T01:00Z"}, new Object[] {"2999-12-31T00:00Z"}
-		};
-	}
- 
-	@DataProvider
-	public Object[][] InvalidDateProvider() {
-		return new Object[][]{
-			new Object[] {"2010-12-31T00:60Z"},new Object[] {"2010-12-31T24:00Z"},
-			new Object[] {"2010-01-32T00:00Z"}, new Object[] {"2020-01-32T00:00Z"},
-			new Object[] {"2010-13-1T00:00Z"}, new Object[] {"1820-01-01T00:00Z"},
-			new Object[] {"2007-2-29T00:00Z"}, new Object[] {"2007-02-29T00:00Z"},
-			new Object[] {"2008-2-30T00:00Z"}, new Object[] {"2008-02-31T00:00Z"},
-			new Object[] {"2008-a-29T00:00Z"}, new Object[] {"2008-02aT00:00Z"},
-			new Object[] {"2008-2-333T00:00Z"}, new Object[] {"200a-02-29T00:00Z"},
-			new Object[] {"2010-4-31T00:00Z"}, new Object[] {"2010-04-31T00:00Z"},
-			new Object[] {"2010-6-31T00:00Z"}, new Object[] {"2010-06-31T00:00Z"},
-			new Object[] {"2010-9-31T00:00Z"}, new Object[] {"2010-09-31T00:00Z"},
-			new Object[] {"2010-11-31T00:00Z"},new Object[] {"1999-04-31T01:00Z"}, 
-		};
-	}
- 
-	@Test(dataProvider = "ValidDateProvider")
-	public void ValidDateTest(String date) {
-		boolean valid = DateValidator.validate(date);
-		System.out.println("Date is valid : " + date + " , " + valid);
-		Assert.assertEquals(valid, true);
-	}
- 
-	@Test(dataProvider = "InvalidDateProvider", 
-                 dependsOnMethods="ValidDateTest")
-	public void InValidDateTest(String date) {
-		boolean valid = DateValidator.validate(date);
-		System.out.println("Date is valid : " + date + " , " + valid);
-		Assert.assertEquals(valid, false); 
-	}
+
+    @DataProvider
+    public Object[][] ValidDateProvider() {
+        return new Object[][]{
+                new Object[]{"2011-11-01T00:00Z"}, new Object[]{"2020-01-01T00:00Z"},
+                new Object[]{"2010-01-31T00:59Z"}, new Object[]{"2020-01-31T00:00Z"},
+                new Object[]{"2008-02-29T01:00Z"}, new Object[]{"2008-02-29T00:00Z"},
+                new Object[]{"2009-02-28T01:01Z"}, new Object[]{"2009-02-28T00:00Z"},
+                new Object[]{"2010-03-31T23:00Z"}, new Object[]{"2010-03-31T00:00Z"},
+                new Object[]{"2010-04-30T23:59Z"}, new Object[]{"2010-04-30T00:00Z"},
+                new Object[]{"2010-05-31T23:23Z"}, new Object[]{"2010-05-31T00:00Z"},
+                new Object[]{"2010-06-30T00:00Z"}, new Object[]{"2010-06-30T00:00Z"},
+                new Object[]{"2010-07-31T00:00Z"}, new Object[]{"2010-07-31T00:00Z"},
+                new Object[]{"2010-08-31T00:00Z"}, new Object[]{"2010-08-31T00:00Z"},
+                new Object[]{"2010-09-30T00:00Z"}, new Object[]{"2010-09-30T00:00Z"},
+                new Object[]{"2010-10-31T00:00Z"}, new Object[]{"2010-10-31T00:00Z"},
+                new Object[]{"2010-11-30T00:00Z"}, new Object[]{"2010-11-30T00:00Z"},
+                new Object[]{"2010-12-31T00:00Z"}, new Object[]{"2010-12-31T00:00Z"},
+                new Object[]{"1999-01-30T01:00Z"}, new Object[]{"2999-12-31T00:00Z"}
+        };
+    }
+
+    @DataProvider
+    public Object[][] InvalidDateProvider() {
+        return new Object[][]{
+                new Object[]{"2010-12-31T00:60Z"}, new Object[]{"2010-12-31T24:00Z"},
+                new Object[]{"2010-01-32T00:00Z"}, new Object[]{"2020-01-32T00:00Z"},
+                new Object[]{"2010-13-1T00:00Z"}, new Object[]{"1820-01-01T00:00Z"},
+                new Object[]{"2007-2-29T00:00Z"}, new Object[]{"2007-02-29T00:00Z"},
+                new Object[]{"2008-2-30T00:00Z"}, new Object[]{"2008-02-31T00:00Z"},
+                new Object[]{"2008-a-29T00:00Z"}, new Object[]{"2008-02aT00:00Z"},
+                new Object[]{"2008-2-333T00:00Z"}, new Object[]{"200a-02-29T00:00Z"},
+                new Object[]{"2010-4-31T00:00Z"}, new Object[]{"2010-04-31T00:00Z"},
+                new Object[]{"2010-6-31T00:00Z"}, new Object[]{"2010-06-31T00:00Z"},
+                new Object[]{"2010-9-31T00:00Z"}, new Object[]{"2010-09-31T00:00Z"},
+                new Object[]{"2010-11-31T00:00Z"}, new Object[]{"1999-04-31T01:00Z"},
+        };
+    }
+
+    @Test(dataProvider = "ValidDateProvider")
+    public void ValidDateTest(String date) {
+        boolean valid = DateValidator.validate(date);
+        System.out.println("Date is valid : " + date + " , " + valid);
+        Assert.assertEquals(valid, true);
+    }
+
+    @Test(dataProvider = "InvalidDateProvider",
+            dependsOnMethods = "ValidDateTest")
+    public void InValidDateTest(String date) {
+        boolean valid = DateValidator.validate(date);
+        System.out.println("Date is valid : " + date + " , " + valid);
+        Assert.assertEquals(valid, false);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/FalconException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/FalconException.java b/common/src/main/java/org/apache/falcon/FalconException.java
index 1e8b69b..391700e 100644
--- a/common/src/main/java/org/apache/falcon/FalconException.java
+++ b/common/src/main/java/org/apache/falcon/FalconException.java
@@ -19,34 +19,30 @@ package org.apache.falcon;
 
 /**
  * Common Exception thrown from Parsers. To the Falcon API class
- * 
  */
 public class FalconException extends Exception {
 
-	/**
-	 * 
-	 * @param e Exception
-	 */
-	public FalconException(Throwable e) {
-		super(e);
-	}
+    /**
+     * @param e Exception
+     */
+    public FalconException(Throwable e) {
+        super(e);
+    }
 
-	public FalconException(String message, Throwable e) {
-		super(message, e);
-	}
+    public FalconException(String message, Throwable e) {
+        super(message, e);
+    }
 
-	/**
-	 * 
-	 * @param message
-	 *            - custom exception message
-	 */
-	public FalconException(String message) {
-		super(message);
-	}
+    /**
+     * @param message - custom exception message
+     */
+    public FalconException(String message) {
+        super(message);
+    }
 
-	/**
-	 * 
-	 */
-	private static final long serialVersionUID = -1475818869309247014L;
+    /**
+     *
+     */
+    private static final long serialVersionUID = -1475818869309247014L;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/FalconRuntimException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/FalconRuntimException.java b/common/src/main/java/org/apache/falcon/FalconRuntimException.java
index d25043a..b603fbf 100644
--- a/common/src/main/java/org/apache/falcon/FalconRuntimException.java
+++ b/common/src/main/java/org/apache/falcon/FalconRuntimException.java
@@ -19,26 +19,25 @@
 package org.apache.falcon;
 
 /**
- *Runtime Execption class for Falcon application
- *
+ * Runtime Execption class for Falcon application
  */
-public class FalconRuntimException extends RuntimeException{
+public class FalconRuntimException extends RuntimeException {
+
+    /**
+     *
+     */
+    private static final long serialVersionUID = 1105135528999858955L;
 
-	/**
-	 * 
-	 */
-	private static final long serialVersionUID = 1105135528999858955L;
+    public FalconRuntimException(Exception e) {
+        super(e);
+    }
 
-	public FalconRuntimException(Exception e) {
-		super(e);
-	}
+    public FalconRuntimException(String message) {
+        super(message);
+    }
 
-	public FalconRuntimException(String message) {
-		super(message);
-	}
-	
-	public FalconRuntimException(String message, Throwable e) {
-		super(message);
-	}
+    public FalconRuntimException(String message, Throwable e) {
+        super(message);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
index 8fbe951..264ea28 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/AbstractCleanupHandler.java
@@ -17,16 +17,7 @@
  */
 package org.apache.falcon.cleanup;
 
-import java.io.IOException;
-
-import javax.servlet.jsp.el.ELException;
-import javax.servlet.jsp.el.ExpressionEvaluator;
-
 import org.apache.commons.el.ExpressionEvaluatorImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -37,110 +28,118 @@ import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.expression.ExpressionHelper;
 import org.apache.falcon.util.RuntimeProperties;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+import javax.servlet.jsp.el.ELException;
+import javax.servlet.jsp.el.ExpressionEvaluator;
+import java.io.IOException;
+
 public abstract class AbstractCleanupHandler {
 
-	protected static final Logger LOG = Logger
-			.getLogger(AbstractCleanupHandler.class);
-	protected final ConfigurationStore STORE = ConfigurationStore.get();
-	public static final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
-	public static final ExpressionHelper resolver = ExpressionHelper.get();
-
-	protected long getRetention(Entity entity, TimeUnit timeUnit)
-			throws FalconException {
-		String retention = getRetentionValue(timeUnit);
-		try {
-			return (Long) EVALUATOR.evaluate("${" + retention + "}",
-					Long.class, resolver, resolver);
-		} catch (ELException e) {
-			throw new FalconException("Unable to evalue retention limit: "
-					+ retention + " for entity: " + entity.getName());
-		}
-	}
-
-	private String getRetentionValue(Frequency.TimeUnit timeunit) {
-		return RuntimeProperties.get().getProperty(
-				"log.cleanup.frequency." + timeunit + ".retention", "days(1)");
-
-	}
-
-	protected FileStatus[] getAllLogs(
-			org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-			throws FalconException {
-		String stagingPath = ClusterHelper.getLocation(cluster, "staging");
-		Path logPath = getLogPath(entity, stagingPath);
-		FileSystem fs = getFileSystem(cluster);
-		FileStatus[] paths;
-		try {
-			paths = fs.globStatus(logPath);
-		} catch (IOException e) {
-			throw new FalconException(e);
-		}
-		return paths;
-	}
-
-	private FileSystem getFileSystem(
-			org.apache.falcon.entity.v0.cluster.Cluster cluster)
-			throws FalconException {
-
-		FileSystem fs;
-		try {
-			fs = new Path(ClusterHelper.getStorageUrl(cluster))
-					.getFileSystem(new Configuration());
-		} catch (IOException e) {
-			throw new FalconException(e);
-		}
-		return fs;
-	}
-
-	protected void delete(Cluster cluster, Entity entity, long retention)
-			throws FalconException {
-
-		FileStatus[] logs = getAllLogs(cluster, entity);
-		long now = System.currentTimeMillis();
-
-		for (FileStatus log : logs) {
-			if (now - log.getModificationTime() > retention) {
-				try {
-					boolean isDeleted = getFileSystem(cluster).delete(
-							log.getPath(), true);
-					if (isDeleted == false) {
-						LOG.error("Unable to delete path: " + log.getPath());
-					} else {
-						LOG.info("Deleted path: " + log.getPath());
-					}
-					deleteParentIfEmpty(getFileSystem(cluster),log.getPath().getParent());
-				} catch (IOException e) {
-					throw new FalconException(" Unable to delete log file : "
-							+ log.getPath() + " for entity " + entity.getName()
-							+ " for cluster: " + cluster.getName(), e);
-				}
-			} else {
-				LOG.info("Retention limit: " + retention
-						+ " is less than modification"
-						+ (now - log.getModificationTime()) + " for path: "
-						+ log.getPath());
-			}
-		}
-
-	}
-
-	private void deleteParentIfEmpty(FileSystem fs, Path parent) throws IOException {
-		 FileStatus[] files = fs.listStatus(parent);
-		if(files!=null && files.length==0){
-			LOG.info("Parent path: "+parent+ " is empty, deleting path");
-			fs.delete(parent, true);
-			deleteParentIfEmpty(fs,parent.getParent());
-		}
-		
-	}
-
-	public abstract void cleanup() throws FalconException;
-
-	protected abstract Path getLogPath(Entity entity, String stagingPath);
-	
-	protected String getCurrentColo(){
-		return StartupProperties.get().getProperty("current.colo", "default");
-	}
+    protected static final Logger LOG = Logger
+            .getLogger(AbstractCleanupHandler.class);
+    protected final ConfigurationStore STORE = ConfigurationStore.get();
+    public static final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
+    public static final ExpressionHelper resolver = ExpressionHelper.get();
+
+    protected long getRetention(Entity entity, TimeUnit timeUnit)
+            throws FalconException {
+        String retention = getRetentionValue(timeUnit);
+        try {
+            return (Long) EVALUATOR.evaluate("${" + retention + "}",
+                    Long.class, resolver, resolver);
+        } catch (ELException e) {
+            throw new FalconException("Unable to evalue retention limit: "
+                    + retention + " for entity: " + entity.getName());
+        }
+    }
+
+    private String getRetentionValue(Frequency.TimeUnit timeunit) {
+        return RuntimeProperties.get().getProperty(
+                "log.cleanup.frequency." + timeunit + ".retention", "days(1)");
+
+    }
+
+    protected FileStatus[] getAllLogs(
+            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+            throws FalconException {
+        String stagingPath = ClusterHelper.getLocation(cluster, "staging");
+        Path logPath = getLogPath(entity, stagingPath);
+        FileSystem fs = getFileSystem(cluster);
+        FileStatus[] paths;
+        try {
+            paths = fs.globStatus(logPath);
+        } catch (IOException e) {
+            throw new FalconException(e);
+        }
+        return paths;
+    }
+
+    private FileSystem getFileSystem(
+            org.apache.falcon.entity.v0.cluster.Cluster cluster)
+            throws FalconException {
+
+        FileSystem fs;
+        try {
+            fs = new Path(ClusterHelper.getStorageUrl(cluster))
+                    .getFileSystem(new Configuration());
+        } catch (IOException e) {
+            throw new FalconException(e);
+        }
+        return fs;
+    }
+
+    protected void delete(Cluster cluster, Entity entity, long retention)
+            throws FalconException {
+
+        FileStatus[] logs = getAllLogs(cluster, entity);
+        long now = System.currentTimeMillis();
+
+        for (FileStatus log : logs) {
+            if (now - log.getModificationTime() > retention) {
+                try {
+                    boolean isDeleted = getFileSystem(cluster).delete(
+                            log.getPath(), true);
+                    if (isDeleted == false) {
+                        LOG.error("Unable to delete path: " + log.getPath());
+                    } else {
+                        LOG.info("Deleted path: " + log.getPath());
+                    }
+                    deleteParentIfEmpty(getFileSystem(cluster), log.getPath().getParent());
+                } catch (IOException e) {
+                    throw new FalconException(" Unable to delete log file : "
+                            + log.getPath() + " for entity " + entity.getName()
+                            + " for cluster: " + cluster.getName(), e);
+                }
+            } else {
+                LOG.info("Retention limit: " + retention
+                        + " is less than modification"
+                        + (now - log.getModificationTime()) + " for path: "
+                        + log.getPath());
+            }
+        }
+
+    }
+
+    private void deleteParentIfEmpty(FileSystem fs, Path parent) throws IOException {
+        FileStatus[] files = fs.listStatus(parent);
+        if (files != null && files.length == 0) {
+            LOG.info("Parent path: " + parent + " is empty, deleting path");
+            fs.delete(parent, true);
+            deleteParentIfEmpty(fs, parent.getParent());
+        }
+
+    }
+
+    public abstract void cleanup() throws FalconException;
+
+    protected abstract Path getLogPath(Entity entity, String stagingPath);
+
+    protected String getCurrentColo() {
+        return StartupProperties.get().getProperty("current.colo", "default");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
index 6b7f34e..37fc653 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/FeedCleanupHandler.java
@@ -17,48 +17,47 @@
  */
 package org.apache.falcon.cleanup;
 
-import java.util.Collection;
-
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.util.DeploymentUtil;
+import org.apache.hadoop.fs.Path;
+
+import java.util.Collection;
 
 public class FeedCleanupHandler extends AbstractCleanupHandler {
 
-	@Override
-	public void cleanup() throws FalconException {
-		Collection<String> feeds = STORE.getEntities(EntityType.FEED);
-		for (String feedName : feeds) {
-			Feed feed;
-			feed = STORE.get(EntityType.FEED, feedName);
-			long retention = getRetention(feed, feed.getFrequency()
-					.getTimeUnit());
-			for (org.apache.falcon.entity.v0.feed.Cluster cluster : feed
-					.getClusters().getClusters()) {
-				Cluster currentCluster = STORE.get(EntityType.CLUSTER,
-						cluster.getName());
-				if(currentCluster.getColo().equals(getCurrentColo())){
-					LOG.info("Cleaning up logs for process:" + feedName
-							+ " in  cluster: " + cluster.getName() + " with retention: "+retention);
-					delete(currentCluster, feed, retention);
-				}else{
-					LOG.info("Ignoring cleanup for process:" + feedName
-							+ " in  cluster: " + cluster.getName()+ " as this does not belong to current colo" );
-				}
-			}
+    @Override
+    public void cleanup() throws FalconException {
+        Collection<String> feeds = STORE.getEntities(EntityType.FEED);
+        for (String feedName : feeds) {
+            Feed feed;
+            feed = STORE.get(EntityType.FEED, feedName);
+            long retention = getRetention(feed, feed.getFrequency()
+                    .getTimeUnit());
+            for (org.apache.falcon.entity.v0.feed.Cluster cluster : feed
+                    .getClusters().getClusters()) {
+                Cluster currentCluster = STORE.get(EntityType.CLUSTER,
+                        cluster.getName());
+                if (currentCluster.getColo().equals(getCurrentColo())) {
+                    LOG.info("Cleaning up logs for process:" + feedName
+                            + " in  cluster: " + cluster.getName() + " with retention: " + retention);
+                    delete(currentCluster, feed, retention);
+                } else {
+                    LOG.info("Ignoring cleanup for process:" + feedName
+                            + " in  cluster: " + cluster.getName() + " as this does not belong to current colo");
+                }
+            }
 
-		}
-	}
+        }
+    }
 
-	@Override
-	protected Path getLogPath(Entity entity, String stagingPath) {
-		Path logPath = new Path(stagingPath, "falcon/workflows/feed/"
-				+ entity.getName() + "/logs/job-*/*/*");
-		return logPath;
-	}
+    @Override
+    protected Path getLogPath(Entity entity, String stagingPath) {
+        Path logPath = new Path(stagingPath, "falcon/workflows/feed/"
+                + entity.getName() + "/logs/job-*/*/*");
+        return logPath;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java b/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
index a7f0852..9b96879 100644
--- a/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
+++ b/common/src/main/java/org/apache/falcon/cleanup/ProcessCleanupHandler.java
@@ -17,48 +17,47 @@
  */
 package org.apache.falcon.cleanup;
 
-import java.util.Collection;
-
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.process.Process;
-import org.apache.falcon.util.DeploymentUtil;
+import org.apache.hadoop.fs.Path;
+
+import java.util.Collection;
 
 public class ProcessCleanupHandler extends AbstractCleanupHandler {
 
-	@Override
-	public void cleanup() throws FalconException {
-		Collection<String> processes = STORE.getEntities(EntityType.PROCESS);
-		for (String processName : processes) {
-			Process process;
-			process = STORE.get(EntityType.PROCESS, processName);
-			long retention = getRetention(process, process.getFrequency()
-					.getTimeUnit());
-			for (org.apache.falcon.entity.v0.process.Cluster cluster : process
-					.getClusters().getClusters()) {
-				Cluster currentCluster = STORE.get(EntityType.CLUSTER,
-						cluster.getName());
-				if(currentCluster.getColo().equals(getCurrentColo())){
-					LOG.info("Cleaning up logs for process:" + processName
-							+ " in  cluster: " + cluster.getName() + " with retention: "+retention);
-					delete(currentCluster, process, retention);
-				}else{
-					LOG.info("Ignoring cleanup for process:" + processName
-							+ " in  cluster: " + cluster.getName()+ " as this does not belong to current colo" );
-				}
-			}
+    @Override
+    public void cleanup() throws FalconException {
+        Collection<String> processes = STORE.getEntities(EntityType.PROCESS);
+        for (String processName : processes) {
+            Process process;
+            process = STORE.get(EntityType.PROCESS, processName);
+            long retention = getRetention(process, process.getFrequency()
+                    .getTimeUnit());
+            for (org.apache.falcon.entity.v0.process.Cluster cluster : process
+                    .getClusters().getClusters()) {
+                Cluster currentCluster = STORE.get(EntityType.CLUSTER,
+                        cluster.getName());
+                if (currentCluster.getColo().equals(getCurrentColo())) {
+                    LOG.info("Cleaning up logs for process:" + processName
+                            + " in  cluster: " + cluster.getName() + " with retention: " + retention);
+                    delete(currentCluster, process, retention);
+                } else {
+                    LOG.info("Ignoring cleanup for process:" + processName
+                            + " in  cluster: " + cluster.getName() + " as this does not belong to current colo");
+                }
+            }
 
-		}
-	}
+        }
+    }
 
-	@Override
-	protected Path getLogPath(Entity entity, String stagingPath) {
-		Path logPath = new Path(stagingPath, "falcon/workflows/process/"
-				+ entity.getName() + "/logs/job-*/*");
-		return logPath;
-	}
+    @Override
+    protected Path getLogPath(Entity entity, String stagingPath) {
+        Path logPath = new Path(stagingPath, "falcon/workflows/process/"
+                + entity.getName() + "/logs/job-*/*");
+        return logPath;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
index b40d7b1..3d77887 100644
--- a/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/ClusterHelper.java
@@ -18,27 +18,25 @@
 
 package org.apache.falcon.entity;
 
+import org.apache.falcon.entity.v0.cluster.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.falcon.entity.v0.cluster.Cluster;
-import org.apache.falcon.entity.v0.cluster.Interface;
-import org.apache.falcon.entity.v0.cluster.Interfacetype;
-import org.apache.falcon.entity.v0.cluster.Location;
-import org.apache.falcon.entity.v0.cluster.Property;
 
 public final class ClusterHelper {
-	public static final String DEFAULT_BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    public static final String DEFAULT_BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
 
-    private ClusterHelper() {}
+    private ClusterHelper() {
+    }
 
     public static Configuration getConfiguration(Cluster cluster) {
         Configuration conf = new Configuration();
         conf.set("fs.default.name", getStorageUrl(cluster));
         conf.set("mapred.job.tracker", getMREndPoint(cluster));
-        if(cluster.getProperties() != null)
-            for(Property prop:cluster.getProperties().getProperties()) {
+        if (cluster.getProperties() != null) {
+            for (Property prop : cluster.getProperties().getProperties()) {
                 conf.set(prop.getName(), prop.getValue());
             }
+        }
         return conf;
     }
 
@@ -57,51 +55,58 @@ public final class ClusterHelper {
     public static String getMREndPoint(Cluster cluster) {
         return getInterface(cluster, Interfacetype.EXECUTE).getEndpoint();
     }
-    
+
     public static String getMessageBrokerUrl(Cluster cluster) {
         return getInterface(cluster, Interfacetype.MESSAGING).getEndpoint();
     }
-    
+
     public static String getMessageBrokerImplClass(Cluster cluster) {
-        if(cluster.getProperties() != null)
-            for(Property prop:cluster.getProperties().getProperties())
-                if(prop.getName().equals("brokerImplClass"))
+        if (cluster.getProperties() != null) {
+            for (Property prop : cluster.getProperties().getProperties()) {
+                if (prop.getName().equals("brokerImplClass")) {
                     return prop.getValue();
+                }
+            }
+        }
         return DEFAULT_BROKER_IMPL_CLASS;
     }
 
     public static Interface getInterface(Cluster cluster, Interfacetype type) {
-        for(Interface interf:cluster.getInterfaces().getInterfaces())
-            if(interf.getType() == type)
+        for (Interface interf : cluster.getInterfaces().getInterfaces()) {
+            if (interf.getType() == type) {
                 return interf;
+            }
+        }
         return null;
     }
 
     private static String getNormalizedUrl(Cluster cluster, Interfacetype type) {
-    	String normalizedUrl = getInterface(cluster, type).getEndpoint();
-    	String normalizedPath = new Path(normalizedUrl + "/").toString();
-    	return normalizedPath.substring(0, normalizedPath.length() - 1);
+        String normalizedUrl = getInterface(cluster, type).getEndpoint();
+        String normalizedPath = new Path(normalizedUrl + "/").toString();
+        return normalizedPath.substring(0, normalizedPath.length() - 1);
     }
 
     public static String getCompleteLocation(Cluster cluster, String locationKey) {
         return getStorageUrl(cluster) + "/" + getLocation(cluster, locationKey);
     }
-    
+
     public static String getLocation(Cluster cluster, String locationKey) {
-        for(Location loc:cluster.getLocations().getLocations()) {
-            if(loc.getName().equals(locationKey))
+        for (Location loc : cluster.getLocations().getLocations()) {
+            if (loc.getName().equals(locationKey)) {
                 return loc.getPath();
+            }
+        }
+        return null;
+    }
+
+    public static String getPropertyValue(Cluster cluster, String propName) {
+        if (cluster.getProperties() != null) {
+            for (Property prop : cluster.getProperties().getProperties()) {
+                if (prop.getName().equals(propName)) {
+                    return prop.getValue();
+                }
+            }
         }
         return null;
     }
-    
-	public static String getPropertyValue(Cluster cluster, String propName) {
-		if (cluster.getProperties() != null) {
-			for (Property prop : cluster.getProperties().getProperties()) {
-				if (prop.getName().equals(propName))
-					return prop.getValue();
-			}
-		}
-		return null;
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java b/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
index e620466..6c5d885 100644
--- a/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
+++ b/common/src/main/java/org/apache/falcon/entity/ColoClusterRelation.java
@@ -18,18 +18,19 @@
 
 package org.apache.falcon.entity;
 
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.service.ConfigurationChangeListener;
 
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 public class ColoClusterRelation implements ConfigurationChangeListener {
-    private static final ConcurrentHashMap<String, Set<String>> coloClusterMap = new ConcurrentHashMap<String, Set<String>>();
+    private static final ConcurrentHashMap<String, Set<String>> coloClusterMap
+            = new ConcurrentHashMap<String, Set<String>>();
     private static final ColoClusterRelation instance = new ColoClusterRelation();
 
     private ColoClusterRelation() {
@@ -40,15 +41,17 @@ public class ColoClusterRelation implements ConfigurationChangeListener {
     }
 
     public Set<String> getClusters(String colo) {
-        if (coloClusterMap.containsKey(colo))
+        if (coloClusterMap.containsKey(colo)) {
             return coloClusterMap.get(colo);
+        }
         return new HashSet<String>();
     }
 
     @Override
     public void onAdd(Entity entity) {
-        if (entity.getEntityType() != EntityType.CLUSTER)
+        if (entity.getEntityType() != EntityType.CLUSTER) {
             return;
+        }
 
         Cluster cluster = (Cluster) entity;
         coloClusterMap.putIfAbsent(cluster.getColo(), new HashSet<String>());
@@ -57,19 +60,22 @@ public class ColoClusterRelation implements ConfigurationChangeListener {
 
     @Override
     public void onRemove(Entity entity) {
-        if (entity.getEntityType() != EntityType.CLUSTER)
+        if (entity.getEntityType() != EntityType.CLUSTER) {
             return;
+        }
 
         Cluster cluster = (Cluster) entity;
         coloClusterMap.get(cluster.getColo()).remove(cluster.getName());
-        if (coloClusterMap.get(cluster.getColo()).isEmpty())
+        if (coloClusterMap.get(cluster.getColo()).isEmpty()) {
             coloClusterMap.remove(cluster.getColo());
+        }
     }
 
     @Override
     public void onChange(Entity oldEntity, Entity newEntity) throws FalconException {
-        if (oldEntity.getEntityType() != EntityType.CLUSTER)
+        if (oldEntity.getEntityType() != EntityType.CLUSTER) {
             return;
+        }
         throw new FalconException("change shouldn't be supported on cluster!");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java b/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
index 03b8d23..db922a4 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityNotRegisteredException.java
@@ -20,8 +20,8 @@ package org.apache.falcon.entity;
 
 import org.apache.falcon.FalconException;
 
-public class EntityNotRegisteredException extends FalconException{
-    
+public class EntityNotRegisteredException extends FalconException {
+
     public EntityNotRegisteredException(String message) {
         super(message);
     }


[44/47] git commit: checkstyle related fixes for rerun module.

Posted by sr...@apache.org.
checkstyle related fixes for rerun module.


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

Branch: refs/heads/master
Commit: 6a39baf35709c9757c36952c8f1d9c13fed8da45
Parents: dbd980e
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:27:20 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:27:20 2013 -0700

----------------------------------------------------------------------
 .../apache/falcon/latedata/LateDataHandler.java    |   14 +-
 .../apache/falcon/rerun/event/LaterunEvent.java    |    6 +-
 .../org/apache/falcon/rerun/event/RerunEvent.java  |   13 +-
 .../falcon/rerun/event/RerunEventFactory.java      |   20 +-
 .../org/apache/falcon/rerun/event/RetryEvent.java  |    5 +
 .../rerun/handler/AbstractRerunConsumer.java       |    6 +
 .../falcon/rerun/handler/AbstractRerunHandler.java |   14 +-
 .../falcon/rerun/handler/LateRerunConsumer.java    |    6 +-
 .../falcon/rerun/handler/LateRerunHandler.java     |   24 +-
 .../falcon/rerun/handler/RerunHandlerFactory.java  |   29 ++-
 .../apache/falcon/rerun/handler/RetryConsumer.java |    8 +-
 .../apache/falcon/rerun/handler/RetryHandler.java  |   14 +-
 .../falcon/rerun/policy/AbstractRerunPolicy.java   |   10 +-
 .../falcon/rerun/policy/ExpBackoffPolicy.java      |   18 +-
 .../apache/falcon/rerun/policy/FinalPolicy.java    |    3 +
 .../apache/falcon/rerun/policy/PeriodicPolicy.java |    4 +
 .../falcon/rerun/policy/RerunPolicyFactory.java    |   21 +-
 .../apache/falcon/rerun/queue/ActiveMQueue.java    |   17 +-
 .../apache/falcon/rerun/queue/DelayedQueue.java    |    5 +-
 .../apache/falcon/rerun/queue/InMemoryQueue.java   |   17 +-
 .../falcon/rerun/service/LateRunService.java       |   17 +-
 .../apache/falcon/rerun/service/RetryService.java  |   14 +-
 .../falcon/rerun/AbstractRerunPolicyTest.java      |   32 ++--
 .../apache/falcon/rerun/handler/TestLateData.java  |  184 +++++++-------
 .../apache/falcon/rerun/queue/ActiveMQTest.java    |   10 +-
 .../falcon/rerun/queue/InMemoryQueueTest.java      |    9 +-
 26 files changed, 297 insertions(+), 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
index 00a2d87..95a3511 100644
--- a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
@@ -32,11 +32,12 @@ import java.io.*;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
+/**
+ * A tool for late data handling.
+ */
 public class LateDataHandler extends Configured implements Tool {
 
-    private static Logger LOG = Logger.getLogger(LateDataHandler.class);
-
-    static PrintStream stream = System.out;
+    private static final Logger LOG = Logger.getLogger(LateDataHandler.class);
 
     public static void main(String[] args) throws Exception {
         Configuration conf = new Configuration();
@@ -110,9 +111,9 @@ public class LateDataHandler extends Configured implements Tool {
     }
 
     public String detectChanges(Path file, Map<String, Long> map, Configuration conf)
-            throws Exception {
+        throws Exception {
 
-        StringBuffer buffer = new StringBuffer();
+        StringBuilder buffer = new StringBuilder();
         BufferedReader in = new BufferedReader(new InputStreamReader(file
                 .getFileSystem(conf).open(file)));
         String line;
@@ -148,12 +149,11 @@ public class LateDataHandler extends Configured implements Tool {
         } finally {
             in.close();
         }
-
     }
 
     public long usage(Path inPath, Configuration conf) throws IOException {
         FileSystem fs = inPath.getFileSystem(conf);
-        FileStatus status[] = fs.globStatus(inPath);
+        FileStatus[] status = fs.globStatus(inPath);
         if (status == null || status.length == 0) {
             return 0;
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
index 7a22704..dcde876 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
@@ -17,14 +17,19 @@
  */
 package org.apache.falcon.rerun.event;
 
+/**
+ * Event representing a late rerun.
+ */
 public class LaterunEvent extends RerunEvent {
 
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     public LaterunEvent(String clusterName, String wfId, long msgInsertTime,
                         long delay, String entityType, String entityName, String instance,
                         int runId) {
         super(clusterName, wfId, msgInsertTime, delay, entityType, entityName,
                 instance, runId);
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     @Override
     public String toString() {
@@ -34,5 +39,4 @@ public class LaterunEvent extends RerunEvent {
                 + "entityName=" + entityName + SEP + "instance=" + instance
                 + SEP + "runId=" + runId;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
index 5a1e3e1..9ae6458 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
@@ -21,10 +21,16 @@ import java.util.Date;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * Event representing a rerun.
+ */
 public class RerunEvent implements Delayed {
 
     protected static final String SEP = "*";
 
+    /**
+     * Rerun Event type.
+     */
     public enum RerunType {
         RETRY, LATE
     }
@@ -38,9 +44,9 @@ public class RerunEvent implements Delayed {
     protected String instance;
     protected int runId;
 
-    public RerunEvent(String clusterName, String wfId,
-                      long msgInsertTime, long delay, String entityType, String entityName,
-                      String instance, int runId) {
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
+    public RerunEvent(String clusterName, String wfId, long msgInsertTime, long delay,
+                      String entityType, String entityName, String instance, int runId) {
         this.clusterName = clusterName;
         this.wfId = wfId;
         this.msgInsertTime = msgInsertTime;
@@ -50,6 +56,7 @@ public class RerunEvent implements Delayed {
         this.runId = runId;
         this.entityType = entityType;
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     public String getClusterName() {
         return clusterName;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
index c5e1e80..54bbecf 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
@@ -22,6 +22,11 @@ import org.apache.falcon.rerun.event.RerunEvent.RerunType;
 import java.util.HashMap;
 import java.util.Map;
 
+/**
+ * Factory implementation to dole out specific implementations of RerunEvent.
+ *
+ * @param <T>
+ */
 public class RerunEventFactory<T extends RerunEvent> {
 
     public T getRerunEvent(String type, String line) {
@@ -38,9 +43,8 @@ public class RerunEventFactory<T extends RerunEvent> {
     private T lateEventFromString(String line) {
         Map<String, String> map = getMap(line);
         return (T) new LaterunEvent(map.get("clusterName"), map.get("wfId"),
-                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
-                .get("delayInMilliSec")), map.get("entityType"),
-                map.get("entityName"), map.get("instance"),
+                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map.get("delayInMilliSec")),
+                map.get("entityType"), map.get("entityName"), map.get("instance"),
                 Integer.parseInt(map.get("runId")));
     }
 
@@ -48,12 +52,10 @@ public class RerunEventFactory<T extends RerunEvent> {
     public T retryEventFromString(String line) {
         Map<String, String> map = getMap(line);
         return (T) new RetryEvent(map.get("clusterName"), map.get("wfId"),
-                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
-                .get("delayInMilliSec")), map.get("entityType"),
-                map.get("entityName"), map.get("instance"),
-                Integer.parseInt(map.get("runId")), Integer.parseInt(map
-                .get("attempts")), Integer.parseInt(map
-                .get("failRetryCount")));
+                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map.get("delayInMilliSec")),
+                map.get("entityType"), map.get("entityName"), map.get("instance"),
+                Integer.parseInt(map.get("runId")), Integer.parseInt(map.get("attempts")),
+                Integer.parseInt(map.get("failRetryCount")));
 
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
index 33248b8..44bf96e 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
@@ -17,11 +17,15 @@
  */
 package org.apache.falcon.rerun.event;
 
+/**
+ * Event representing a retry.
+ */
 public class RetryEvent extends RerunEvent {
 
     private int attempts;
     private int failRetryCount;
 
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     public RetryEvent(String clusterName, String wfId, long msgInsertTime,
                       long delay, String entityType, String entityName, String instance,
                       int runId, int attempts, int failRetryCount) {
@@ -30,6 +34,7 @@ public class RetryEvent extends RerunEvent {
         this.attempts = attempts;
         this.failRetryCount = failRetryCount;
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     public int getAttempts() {
         return attempts;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
index fa1d9e3..b073117 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
@@ -26,6 +26,12 @@ import org.apache.falcon.rerun.policy.ExpBackoffPolicy;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 import org.apache.log4j.Logger;
 
+/**
+ * Base class for a rerun consumer.
+ *
+ * @param <T> a rerun event
+ * @param <M> a rerun handler
+ */
 public abstract class AbstractRerunConsumer<T extends RerunEvent, M extends AbstractRerunHandler<T, DelayedQueue<T>>>
         implements Runnable {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
index 4a90b9f..8a41bf8 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
@@ -27,6 +27,12 @@ import org.apache.falcon.workflow.WorkflowEngineFactory;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
 import org.apache.log4j.Logger;
 
+/**
+ * Base class for handling reruns.
+ *
+ * @param <T> a rerun event
+ * @param <M> queue
+ */
 public abstract class AbstractRerunHandler<T extends RerunEvent, M extends DelayedQueue<T>> {
 
     protected static final Logger LOG = Logger
@@ -34,9 +40,9 @@ public abstract class AbstractRerunHandler<T extends RerunEvent, M extends Delay
     protected M delayQueue;
     private AbstractWorkflowEngine wfEngine;
 
-    public void init(M delayQueue) throws FalconException {
+    public void init(M aDelayQueue) throws FalconException {
         this.wfEngine = WorkflowEngineFactory.getWorkflowEngine();
-        this.delayQueue = delayQueue;
+        this.delayQueue = aDelayQueue;
         this.delayQueue.init();
     }
 
@@ -60,13 +66,11 @@ public abstract class AbstractRerunHandler<T extends RerunEvent, M extends Delay
         delayQueue.reconnect();
     }
 
-    public Entity getEntity(String entityType, String entityName)
-            throws FalconException {
+    public Entity getEntity(String entityType, String entityName) throws FalconException {
         return EntityUtil.getEntity(entityType, entityName);
     }
 
     public Retry getRetry(Entity entity) throws FalconException {
         return EntityUtil.getRetry(entity);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
index 03561fc..4088a59 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
@@ -31,6 +31,11 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.*;
 
+/**
+ * A consumer of late reruns.
+ *
+ * @param <T>
+ */
 public class LateRerunConsumer<T extends LateRerunHandler<DelayedQueue<LaterunEvent>>>
         extends AbstractRerunConsumer<LaterunEvent, T> {
 
@@ -140,5 +145,4 @@ public class LateRerunConsumer<T extends LateRerunHandler<DelayedQueue<LaterunEv
 
         return late.detectChanges(lateLogPath, feedSizes, conf);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
index e24cc69..40c5d1c 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
@@ -40,6 +40,11 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.Date;
 
+/**
+ * An implementation of handler for late reruns.
+ *
+ * @param <M>
+ */
 public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
         AbstractRerunHandler<LaterunEvent, M> {
 
@@ -78,9 +83,8 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
                         wfId));
                 if (fs.exists(lateLogPath)) {
                     boolean deleted = fs.delete(lateLogPath, true);
-                    if (deleted == true) {
-                        LOG.info("Successfully deleted late file path:"
-                                + lateLogPath);
+                    if (deleted) {
+                        LOG.info("Successfully deleted late file path:" + lateLogPath);
                     }
                 }
                 return;
@@ -102,8 +106,7 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
         }
     }
 
-    private long getEventDelay(Entity entity, String nominalTime)
-            throws FalconException {
+    private long getEventDelay(Entity entity, String nominalTime) throws FalconException {
 
         Date instanceDate = EntityUtil.parseDateUTC(nominalTime);
         LateProcess lateProcess = EntityUtil.getLateProcess(entity);
@@ -135,8 +138,7 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
         return new Date(date.getTime() + milliSecondsToAdd);
     }
 
-    public static Date getCutOffTime(Entity entity, String nominalTime)
-            throws FalconException {
+    public static Date getCutOffTime(Entity entity, String nominalTime) throws FalconException {
 
         ConfigurationStore store = ConfigurationStore.get();
         ExpressionHelper evaluator = ExpressionHelper.get();
@@ -193,8 +195,8 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
     }
 
     @Override
-    public void init(M delayQueue) throws FalconException {
-        super.init(delayQueue);
+    public void init(M aDelayQueue) throws FalconException {
+        super.init(aDelayQueue);
         Thread daemon = new Thread(new LateRerunConsumer(this));
         daemon.setName("LaterunHandler");
         daemon.setDaemon(true);
@@ -211,8 +213,7 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
 
     }
 
-    public Configuration getConfiguration(String cluster, String wfId)
-            throws FalconException {
+    public Configuration getConfiguration(String cluster, String wfId) throws FalconException {
         Configuration conf = new Configuration();
         conf.set(
                 CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
@@ -220,5 +221,4 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
                         AbstractWorkflowEngine.NAME_NODE));
         return conf;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
index ce76842..3abfe5a 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
@@ -22,26 +22,29 @@ import org.apache.falcon.rerun.event.RerunEvent.RerunType;
 import org.apache.falcon.rerun.event.RetryEvent;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 
-public class RerunHandlerFactory {
+/**
+ * A factory implementation to dole out rerun handlers.
+ */
+public final class RerunHandlerFactory {
 
-    private static final RetryHandler<DelayedQueue<RetryEvent>> retryHandler
-            = new RetryHandler<DelayedQueue<RetryEvent>>();
-    private static final LateRerunHandler<DelayedQueue<LaterunEvent>> lateHandler
-            = new LateRerunHandler<DelayedQueue<LaterunEvent>>();
+    private static final RetryHandler<DelayedQueue<RetryEvent>> RETRY_HANDLER
+        = new RetryHandler<DelayedQueue<RetryEvent>>();
+    private static final LateRerunHandler<DelayedQueue<LaterunEvent>> LATE_HANDLER
+        = new LateRerunHandler<DelayedQueue<LaterunEvent>>();
 
     private RerunHandlerFactory() {
-
     }
 
     public static AbstractRerunHandler getRerunHandler(RerunType type) {
         switch (type) {
-            case RETRY:
-                return retryHandler;
-            case LATE:
-                return lateHandler;
-            default:
-                throw new RuntimeException("Invalid handler:" + type);
-        }
+        case RETRY:
+            return RETRY_HANDLER;
+
+        case LATE:
+            return LATE_HANDLER;
 
+        default:
+            throw new RuntimeException("Invalid handler:" + type);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
index c084233..63dade8 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
@@ -25,6 +25,11 @@ import org.apache.falcon.util.StartupProperties;
 
 import java.util.Date;
 
+/**
+ * A consumer of retry events which reruns the workflow in the workflow engine.
+ *
+ * @param <T>
+ */
 public class RetryConsumer<T extends RetryHandler<DelayedQueue<RetryEvent>>>
         extends AbstractRerunConsumer<RetryEvent, T> {
 
@@ -90,9 +95,6 @@ public class RetryConsumer<T extends RetryHandler<DelayedQueue<RetryEvent>>>
                         Integer.toString(message.getRunId()),
                         "Failure retry attempts exhausted");
             }
-
         }
-
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
index 188c397..5bd8fd8 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryHandler.java
@@ -28,6 +28,12 @@ import org.apache.falcon.rerun.policy.AbstractRerunPolicy;
 import org.apache.falcon.rerun.policy.RerunPolicyFactory;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 
+/**
+ * An implementation of retry handler that kicks off retries until the
+ * configured attempts are exhausted.
+ *
+ * @param <M>
+ */
 public class RetryHandler<M extends DelayedQueue<RetryEvent>> extends
         AbstractRerunHandler<RetryEvent, M> {
 
@@ -80,14 +86,12 @@ public class RetryHandler<M extends DelayedQueue<RetryEvent>> extends
     }
 
     @Override
-    public void init(M queue) throws FalconException {
-        super.init(queue);
+    public void init(M aDelayQueue) throws FalconException {
+        super.init(aDelayQueue);
         Thread daemon = new Thread(new RetryConsumer(this));
         daemon.setName("RetryHandler");
         daemon.setDaemon(true);
         daemon.start();
-        LOG.info("RetryHandler  thread started");
-
+        LOG.info("RetryHandler thread started.");
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
index 4bfbef2..6a30294 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/AbstractRerunPolicy.java
@@ -23,21 +23,21 @@ import org.apache.falcon.expression.ExpressionHelper;
 
 import java.util.Date;
 
+/**
+ * Base class for Rerun Policy.
+ */
 public abstract class AbstractRerunPolicy {
 
-    public long getDurationInMilliSec(Frequency frequency)
-            throws FalconException {
+    public long getDurationInMilliSec(Frequency frequency) throws FalconException {
         ExpressionHelper helper = ExpressionHelper.get();
         return helper.evaluate(frequency.toString(), Long.class);
-
     }
 
     public static Date addTime(Date date, int milliSecondsToAdd) {
         return new Date(date.getTime() + milliSecondsToAdd);
     }
 
-    public abstract long getDelay(Frequency delay, int eventNumber)
-            throws FalconException;
+    public abstract long getDelay(Frequency delay, int eventNumber) throws FalconException;
 
     public abstract long getDelay(Frequency delay, Date nominaltime,
                                   Date cutOffTime) throws FalconException;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
index b23e014..86832ef 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/ExpBackoffPolicy.java
@@ -23,23 +23,22 @@ import org.apache.falcon.expression.ExpressionHelper;
 
 import java.util.Date;
 
+/**
+ * An implementation of the Rerun policy that backs off exponentially.
+ */
 public class ExpBackoffPolicy extends AbstractRerunPolicy {
 
     @Override
-    public long getDelay(Frequency delay, int eventNumber)
-            throws FalconException {
-        return (long) (getDurationInMilliSec(delay) * Math.pow(getPower(),
-                eventNumber));
+    public long getDelay(Frequency delay, int eventNumber) throws FalconException {
+        return (long) (getDurationInMilliSec(delay) * Math.pow(getPower(), eventNumber));
     }
 
     @Override
-    public long getDelay(Frequency delay, Date nominalTime, Date cutOffTime)
-            throws FalconException {
+    public long getDelay(Frequency delay, Date nominalTime, Date cutOffTime) throws FalconException {
         ExpressionHelper evaluator = ExpressionHelper.get();
         Date now = new Date();
         Date lateTime = nominalTime;
-        long delayMilliSeconds = evaluator.evaluate(delay.toString(),
-                Long.class);
+        long delayMilliSeconds = evaluator.evaluate(delay.toString(), Long.class);
         int factor = 1;
         // TODO we can get rid of this using formula
         while (lateTime.compareTo(now) <= 0) {
@@ -49,12 +48,11 @@ public class ExpBackoffPolicy extends AbstractRerunPolicy {
         if (lateTime.after(cutOffTime)) {
             lateTime = cutOffTime;
         }
-        return (lateTime.getTime() - nominalTime.getTime());
 
+        return (lateTime.getTime() - nominalTime.getTime());
     }
 
     protected int getPower() {
         return 2;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
index 19fe8f7..8dd9c29 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/FinalPolicy.java
@@ -23,6 +23,9 @@ import org.apache.falcon.entity.v0.Frequency;
 
 import java.util.Date;
 
+/**
+ * An implementation of the Rerun policy that does rerun only once.
+ */
 public class FinalPolicy extends AbstractRerunPolicy {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
index 9bcca26..b8c69d2 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/PeriodicPolicy.java
@@ -17,7 +17,11 @@
  */
 package org.apache.falcon.rerun.policy;
 
+/**
+ * An implementation of the Rerun policy that is periodic.
+ */
 public class PeriodicPolicy extends ExpBackoffPolicy {
+
     @Override
     protected int getPower() {
         return 1;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
index 5ee902a..945b6c5 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/policy/RerunPolicyFactory.java
@@ -19,7 +19,10 @@ package org.apache.falcon.rerun.policy;
 
 import org.apache.falcon.entity.v0.process.PolicyType;
 
-public class RerunPolicyFactory {
+/**
+ * A factory implementation to dole out rerun/retry policy.
+ */
+public final class RerunPolicyFactory {
 
     private RerunPolicyFactory() {
         // factory
@@ -27,15 +30,17 @@ public class RerunPolicyFactory {
 
     public static AbstractRerunPolicy getRetryPolicy(PolicyType latePolicy) {
         switch (latePolicy) {
-            case PERIODIC:
-                return new PeriodicPolicy();
+        case PERIODIC:
+            return new PeriodicPolicy();
+
+        case EXP_BACKOFF:
+            return new ExpBackoffPolicy();
 
-            case EXP_BACKOFF:
-                return new ExpBackoffPolicy();
+        case FINAL:
+            return new FinalPolicy();
 
-            case FINAL:
-                return new FinalPolicy();
+        default:
+            throw new IllegalArgumentException("Unhandled Retry policy: " + latePolicy);
         }
-        throw new IllegalArgumentException("Unhandled Retry policy: " + latePolicy);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
index 82dfc22..3fa5282 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/ActiveMQueue.java
@@ -28,6 +28,11 @@ import javax.jms.*;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * An ActiveMQ implementation for DelayedQueue.
+ *
+ * @param <T>
+ */
 public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
 
     private ActiveMQConnection connection;
@@ -65,8 +70,8 @@ public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
         if (connection == null) {
             init();
         }
-        Session session = connection.createSession(false,
-                Session.AUTO_ACKNOWLEDGE);
+
+        Session session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE);
         return session;
     }
 
@@ -87,8 +92,6 @@ public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
 
     @Override
     public void populateQueue(List<T> events) {
-        // TODO Auto-generated method stub
-
     }
 
     @Override
@@ -127,21 +130,25 @@ public class ActiveMQueue<T extends RerunEvent> extends DelayedQueue<T> {
             producer.close();
             LOG.info("Producer closed successfully");
         } catch (Exception ignore) {
+            LOG.info("Producer could not be closed");
         }
+
         try {
             LOG.info("Attempting to close consumer");
             consumer.close();
             LOG.info("Consumer closed successfully");
         } catch (Exception ignore) {
+            LOG.info("Consumer could not be closed");
         }
+
         try {
             LOG.info("Attempting to close connection");
             connection.close();
             LOG.info("Connection closed successfully");
         } catch (Exception ignore) {
+            LOG.info("Connection could not be closed");
         }
 
         init();
-
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
index 7e55206..393a8e5 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/DelayedQueue.java
@@ -23,6 +23,10 @@ import org.apache.log4j.Logger;
 
 import java.util.List;
 
+/**
+ * Base class for a Queue implementation.
+ * @param <T>
+ */
 public abstract class DelayedQueue<T extends RerunEvent> {
     public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
 
@@ -35,5 +39,4 @@ public abstract class DelayedQueue<T extends RerunEvent> {
     public abstract void init();
 
     public abstract void reconnect() throws FalconException;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java b/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
index 7243f4d..06feb92 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/queue/InMemoryQueue.java
@@ -28,14 +28,19 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.DelayQueue;
 
+/**
+ * An in-memory implementation of a DelayedQueue.
+ * @param <T>
+ */
 public class InMemoryQueue<T extends RerunEvent> extends DelayedQueue<T> {
     public static final Logger LOG = Logger.getLogger(DelayedQueue.class);
-    protected DelayQueue<T> QUEUE = new DelayQueue<T>();
+
+    protected DelayQueue<T> delayQueue = new DelayQueue<T>();
     private File serializeFilePath;
 
     @Override
     public boolean offer(T event) {
-        boolean flag = QUEUE.offer(event);
+        boolean flag = delayQueue.offer(event);
         beforeRetry(event);
         LOG.debug("Enqueued Message:" + event.toString());
         return flag;
@@ -45,7 +50,7 @@ public class InMemoryQueue<T extends RerunEvent> extends DelayedQueue<T> {
     public T take() throws FalconException {
         T event;
         try {
-            event = QUEUE.take();
+            event = delayQueue.take();
             LOG.debug("Dequeued Message:" + event.toString());
             afterRetry(event);
         } catch (InterruptedException e) {
@@ -60,16 +65,14 @@ public class InMemoryQueue<T extends RerunEvent> extends DelayedQueue<T> {
 
     public void populateQueue(List<T> events) {
         for (T event : events) {
-            QUEUE.offer(event);
+            delayQueue.offer(event);
         }
-
     }
 
     @Override
     public void init() {
         List<T> events = bootstrap();
         populateQueue(events);
-
     }
 
     @Override
@@ -135,7 +138,7 @@ public class InMemoryQueue<T extends RerunEvent> extends DelayedQueue<T> {
                                 + rerunFile.getAbsolutePath(), e);
             }
         }
-        return rerunEvents;
 
+        return rerunEvents;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java b/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
index f8edfbc..cdbadaa 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/service/LateRunService.java
@@ -29,6 +29,9 @@ import org.apache.log4j.Logger;
 
 import java.io.File;
 
+/**
+ * A service implementation for Late Rerun initialized at startup.
+ */
 public class LateRunService implements FalconService {
 
     private static final Logger LOG = Logger.getLogger(LateRunService.class);
@@ -40,20 +43,18 @@ public class LateRunService implements FalconService {
 
     @Override
     public void init() throws FalconException {
-        AbstractRerunHandler<LaterunEvent, ActiveMQueue<LaterunEvent>> rerunHandler = RerunHandlerFactory
-                .getRerunHandler(RerunType.LATE);
+        AbstractRerunHandler<LaterunEvent, ActiveMQueue<LaterunEvent>> rerunHandler =
+            RerunHandlerFactory.getRerunHandler(RerunType.LATE);
         ActiveMQueue<LaterunEvent> queue = new ActiveMQueue<LaterunEvent>(
-                StartupProperties
-                        .get()
-                        .getProperty("broker.url",
-                                "failover:(tcp://localhost:61616)?initialReconnectDelay=5000"),
+                StartupProperties.get()
+                    .getProperty("broker.url", "failover:(tcp://localhost:61616)?initialReconnectDelay=5000"),
                 "falcon.late.queue");
         rerunHandler.init(queue);
     }
 
     @Override
     public void destroy() throws FalconException {
-        LOG.info("LateRun  thread destroyed");
+        LOG.info("LateRun thread destroyed");
     }
 
     private File getBasePath() {
@@ -64,7 +65,7 @@ public class LateRunService implements FalconService {
             throw new RuntimeException("Unable to initialize late recorder @"
                     + basePath);
         }
+
         return basePath;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java b/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
index 8a902be..b989acd 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/service/RetryService.java
@@ -30,6 +30,9 @@ import org.apache.log4j.Logger;
 
 import java.io.File;
 
+/**
+ * A service implementation for Retry initialized at startup.
+ */
 public class RetryService implements FalconService {
 
     private static final Logger LOG = Logger.getLogger(RetryService.class);
@@ -41,16 +44,15 @@ public class RetryService implements FalconService {
 
     @Override
     public void init() throws FalconException {
-        AbstractRerunHandler<RetryEvent, DelayedQueue<RetryEvent>> rerunHandler = RerunHandlerFactory
-                .getRerunHandler(RerunType.RETRY);
-        InMemoryQueue<RetryEvent> queue = new InMemoryQueue<RetryEvent>(
-                getBasePath());
+        AbstractRerunHandler<RetryEvent, DelayedQueue<RetryEvent>> rerunHandler =
+            RerunHandlerFactory.getRerunHandler(RerunType.RETRY);
+        InMemoryQueue<RetryEvent> queue = new InMemoryQueue<RetryEvent>(getBasePath());
         rerunHandler.init(queue);
     }
 
     @Override
     public void destroy() throws FalconException {
-        LOG.info("RetryHandler  thread destroyed");
+        LOG.info("RetryHandler thread destroyed");
     }
 
     private File getBasePath() {
@@ -61,7 +63,7 @@ public class RetryService implements FalconService {
             throw new RuntimeException("Unable to initialize retry recorder @"
                     + basePath);
         }
+
         return basePath;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java b/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
index be74032..630ef00 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/AbstractRerunPolicyTest.java
@@ -27,23 +27,23 @@ import org.testng.annotations.Test;
 
 import java.util.Date;
 
+/**
+ * Base test class for Rerun Policy.
+ */
 public class AbstractRerunPolicyTest {
 
     @Test
-    public void TestGetDurationInMillis() throws FalconException {
+    public void testGetDurationInMillis() throws FalconException {
         AbstractRerunPolicy policy = new AbstractRerunPolicy() {
 
             @Override
             public long getDelay(Frequency delay, Date nominaltime,
                                  Date cutOffTime) throws FalconException {
-                // TODO Auto-generated method stub
                 return 0;
             }
 
             @Override
-            public long getDelay(Frequency delay, int eventNumber)
-                    throws FalconException {
-                // TODO Auto-generated method stub
+            public long getDelay(Frequency delay, int eventNumber) throws FalconException {
                 return 0;
             }
         };
@@ -57,26 +57,26 @@ public class AbstractRerunPolicyTest {
     }
 
     @Test
-    public void TestExpBackoffPolicy() throws FalconException {
+    public void testExpBackoffPolicy() throws FalconException {
         AbstractRerunPolicy backoff = new ExpBackoffPolicy();
         long delay = backoff.getDelay(new Frequency("minutes(2)"), 2);
         Assert.assertEquals(delay, 480000);
 
         long currentTime = System.currentTimeMillis();
-        delay = backoff.getDelay(new Frequency("minutes(2)"), new Date(
-                currentTime - 1 * 4 * 60 * 1000), new Date(currentTime + 1 * 60
-                * 60 * 1000));
+        delay = backoff.getDelay(new Frequency("minutes(2)"),
+                new Date(currentTime - 1 * 4 * 60 * 1000),
+                new Date(currentTime + 1 * 60 * 60 * 1000));
         Assert.assertEquals(delay, 1 * 6 * 60 * 1000);
 
         currentTime = System.currentTimeMillis();
-        delay = backoff.getDelay(new Frequency("minutes(1)"), new Date(
-                currentTime - 1 * 9 * 60 * 1000), new Date(currentTime + 1 * 60
-                * 60 * 1000));
+        delay = backoff.getDelay(new Frequency("minutes(1)"),
+                new Date(currentTime - 1 * 9 * 60 * 1000),
+                new Date(currentTime + 1 * 60 * 60 * 1000));
         Assert.assertEquals(delay, 900000);
     }
 
     @Test
-    public void TestPeriodicPolicy() throws FalconException, InterruptedException {
+    public void testPeriodicPolicy() throws FalconException, InterruptedException {
         AbstractRerunPolicy periodic = new PeriodicPolicy();
         long delay = periodic.getDelay(new Frequency("minutes(2)"), 2);
         Assert.assertEquals(delay, 120000);
@@ -85,9 +85,9 @@ public class AbstractRerunPolicyTest {
 
         long currentTime = System.currentTimeMillis();
         //Thread.sleep(1000);
-        delay = periodic.getDelay(new Frequency("minutes(3)"), new Date(
-                currentTime), new Date(currentTime + 1 * 60
-                * 60 * 1000));
+        delay = periodic.getDelay(new Frequency("minutes(3)"),
+                new Date(currentTime),
+                new Date(currentTime + 1 * 60 * 60 * 1000));
         Assert.assertEquals(delay, 180000);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
index 6028e10..efd51b1 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/handler/TestLateData.java
@@ -18,7 +18,6 @@
 
 package org.apache.falcon.rerun.handler;
 
-import com.sun.jersey.api.client.WebResource;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -39,14 +38,16 @@ import javax.xml.bind.Unmarshaller;
 import java.io.StringWriter;
 import java.util.Collection;
 
+/**
+ * Test class for Late data handling.
+ */
 public class TestLateData {
 
     protected static final String FEED_XML = "/feed-template.xml";
-    protected static String CLUSTER_XML = "/cluster-template.xml";
+    protected static final String CLUSTER_XML = "/cluster-template.xml";
     protected static final String PROCESS_XML = "/process-template.xml";
     protected static final String PROCESS_XML2 = "/process-template2.xml";
 
-    protected WebResource service = null;
     protected Configuration conf = new Configuration();
 
     @BeforeClass
@@ -75,24 +76,27 @@ public class TestLateData {
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
         switch (type) {
-            case CLUSTER:
-                Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
-                cluster.setName(name);
-                ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
-                store.publish(type, cluster);
-                break;
-
-            case FEED:
-                Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
-                feed.setName(name);
-                store.publish(type, feed);
-                break;
-
-            case PROCESS:
-                Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
-                process.setName(name);
-                store.publish(type, process);
-                break;
+        case CLUSTER:
+            Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
+            cluster.setName(name);
+            ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
+            store.publish(type, cluster);
+            break;
+
+        case FEED:
+            Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
+            feed.setName(name);
+            store.publish(type, feed);
+            break;
+
+        case PROCESS:
+            Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
+            process.setName(name);
+            store.publish(type, process);
+            break;
+
+        default:
+            throw new IllegalArgumentException("Invalid entity type: " + type);
         }
     }
 
@@ -113,82 +117,78 @@ public class TestLateData {
         store.publish(EntityType.PROCESS, process);
     }
 
-    public String marshallEntity(final Entity entity) throws FalconException,
-                                                             JAXBException {
+    public String marshallEntity(final Entity entity) throws FalconException, JAXBException {
         Marshaller marshaller = entity.getEntityType().getMarshaller();
         StringWriter stringWriter = new StringWriter();
         marshaller.marshal(entity, stringWriter);
         return stringWriter.toString();
     }
 
-//	@Test
-//	private void TestLateWhenInstanceRunning() throws Exception
-//	{
-//		try{
-//        WorkflowEngine engine = Mockito.mock(WorkflowEngine.class);
-//        when(engine.instanceStatus("testCluster", "123")).thenReturn("RUNNING");
-//        
-//		ConfigurationStore store = ConfigurationStore.get();
-//		setup();
-//		String nominalTime = EntityUtil.formatDateUTC(new Date(System.currentTimeMillis() - 1800000));
-//        InMemoryQueue<LaterunEvent> queue = new InMemoryQueue<LaterunEvent>(new File("target/late"));
-//        latedataHandler.init(queue);
-//        
-//        AbstractRerunHandler handle = RerunHandlerFactory.getRerunHandler(RerunEvent.RerunType.LATE);
-//        handle.handleRerun("sample", nominalTime, "123", "123", engine, System.currentTimeMillis());
-//        
-//        File directory = new File("target/late");
-//        File[] files = directory.listFiles();
-//        int noFilesBefore = files.length;
-//        
-//        Thread.sleep(90000);
-//        
-//        files = directory.listFiles();
-//        int noFilesAfterRetry = files.length;        
-//        Assert.assertNotSame(noFilesBefore, noFilesAfterRetry);
-//		}
-//		catch (Exception e){
-//			Assert.fail("Not expecting any exception");
-//		}
-//        
-//	}
-//	
-//	
-//	@Test
-//	private void TestLateWhenDataPresent() throws Exception {
-//		WorkflowEngine engine = Mockito.mock(WorkflowEngine.class);
-//		when(engine.instanceStatus("testCluster", "123")).thenReturn(
-//				"SUCCEEDED");
-//
-//		LateRerunConsumer consumer = Mockito.mock(LateRerunConsumer.class);
-//		when(consumer.detectLate(Mockito.any(LaterunEvent.class))).thenReturn(
-//				"new data found");
-//
-//		String nominalTime = EntityUtil.formatDateUTC(new Date(System
-//				.currentTimeMillis() - 1800000));
-//		AbstractRerunHandler handle = RerunHandlerFactory
-//				.getRerunHandler(RerunEvent.RerunType.LATE);
-//
-//		ConfigurationStore store = ConfigurationStore.get();
-//		setup();
-//
-//		InMemoryQueue<LaterunEvent> queue = new InMemoryQueue<LaterunEvent>(
-//				new File("target/late"));
-//		latedataHandler.init(queue);
-//
-//		handle.handleRerun("sample", nominalTime, "123", "123", engine,
-//				System.currentTimeMillis());
-//
-//		File directory = new File("target/late");
-//		File[] files = directory.listFiles();
-//		int noFilesBefore = files.length;
-//
-//		Thread.sleep(90000);
-//
-//		files = directory.listFiles();
-//		int noFilesAfterRetry = files.length;
-//		Assert.assertNotSame(noFilesBefore, noFilesAfterRetry);
-//
-//	}
+/*
+    @Test
+    private void TestLateWhenInstanceRunning() throws Exception {
+        try {
+            WorkflowEngine engine = Mockito.mock(WorkflowEngine.class);
+            when(engine.instanceStatus("testCluster", "123")).thenReturn("RUNNING");
+
+            ConfigurationStore store = ConfigurationStore.get();
+            setup();
+            String nominalTime = EntityUtil.formatDateUTC(new Date(System.currentTimeMillis() - 1800000));
+            InMemoryQueue<LaterunEvent> queue = new InMemoryQueue<LaterunEvent>(new File("target/late"));
+            latedataHandler.init(queue);
+
+            AbstractRerunHandler handle = RerunHandlerFactory.getRerunHandler(RerunEvent.RerunType.LATE);
+            handle.handleRerun("sample", nominalTime, "123", "123", engine, System.currentTimeMillis());
+
+            File directory = new File("target/late");
+            File[] files = directory.listFiles();
+            int noFilesBefore = files.length;
+
+            Thread.sleep(90000);
+
+            files = directory.listFiles();
+            int noFilesAfterRetry = files.length;
+            Assert.assertNotSame(noFilesBefore, noFilesAfterRetry);
+        } catch (Exception e) {
+            Assert.fail("Not expecting any exception");
+        }
+    }
+
+    @Test
+    private void TestLateWhenDataPresent() throws Exception {
+        WorkflowEngine engine = Mockito.mock(WorkflowEngine.class);
+        when(engine.instanceStatus("testCluster", "123")).thenReturn(
+                "SUCCEEDED");
+
+        LateRerunConsumer consumer = Mockito.mock(LateRerunConsumer.class);
+        when(consumer.detectLate(Mockito.any(LaterunEvent.class))).thenReturn(
+                "new data found");
 
+        String nominalTime = EntityUtil.formatDateUTC(new Date(System
+                .currentTimeMillis() - 1800000));
+        AbstractRerunHandler handle = RerunHandlerFactory
+                .getRerunHandler(RerunEvent.RerunType.LATE);
+
+        ConfigurationStore store = ConfigurationStore.get();
+        setup();
+
+        InMemoryQueue<LaterunEvent> queue = new InMemoryQueue<LaterunEvent>(
+                new File("target/late"));
+        latedataHandler.init(queue);
+
+        handle.handleRerun("sample", nominalTime, "123", "123", engine,
+                System.currentTimeMillis());
+
+        File directory = new File("target/late");
+        File[] files = directory.listFiles();
+        int noFilesBefore = files.length;
+
+        Thread.sleep(90000);
+
+        files = directory.listFiles();
+        int noFilesAfterRetry = files.length;
+        Assert.assertNotSame(noFilesBefore, noFilesAfterRetry);
+
+    }
+*/
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
index a13aa02..a8f3885 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/ActiveMQTest.java
@@ -24,11 +24,14 @@ import org.apache.falcon.rerun.event.RerunEvent;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+/**
+ * Test class for ActiveMQ delayed queue implementation.
+ */
 public class ActiveMQTest {
 
     private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
     private BrokerService broker;
-    private static final String DESTI = "activemq.topic";
+    private static final String DESTINATION = "activemq.topic";
 
     @BeforeClass
     private void setup() throws Exception {
@@ -43,8 +46,9 @@ public class ActiveMQTest {
     @Test
     public void testBrokerStartAndEnqueue() {
         ActiveMQueue<RerunEvent> activeMQueue = new ActiveMQueue<RerunEvent>(
-                BROKER_URL, DESTI);
+                BROKER_URL, DESTINATION);
         activeMQueue.init();
+
         RerunEvent event = new LaterunEvent("clusterName", "wfId",
                 System.currentTimeMillis(), 60 * 1000, "entityType",
                 "entityName", "instance", 0);
@@ -58,7 +62,5 @@ public class ActiveMQTest {
         } catch (Exception e) {
             Assert.fail();
         }
-
-
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6a39baf3/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
----------------------------------------------------------------------
diff --git a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
index d41453b..6aafaa5 100644
--- a/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
+++ b/rerun/src/test/java/org/apache/falcon/rerun/queue/InMemoryQueueTest.java
@@ -25,6 +25,9 @@ import org.testng.annotations.Test;
 import java.io.File;
 import java.util.LinkedList;
 
+/**
+ * Test class for the InMemory Queue implementation of the DelayedQueue.
+ */
 public class InMemoryQueueTest {
 
     @Test(timeOut = 10000)
@@ -47,8 +50,8 @@ public class InMemoryQueueTest {
             boolean inserted = false;
             for (int posn = 0; posn < events.size(); posn++) {
                 MyEvent thisEvent = events.get(posn);
-                if (thisEvent.getDelayInMilliSec() + thisEvent.getMsgInsertTime() >
-                        event.getDelayInMilliSec() + event.getMsgInsertTime()) {
+                if (thisEvent.getDelayInMilliSec() + thisEvent.getMsgInsertTime()
+                        > event.getDelayInMilliSec() + event.getMsgInsertTime()) {
                     events.add(posn, event);
                     inserted = true;
                     break;
@@ -67,12 +70,14 @@ public class InMemoryQueueTest {
 
     private class MyEvent extends RerunEvent {
 
+        //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
         public MyEvent(String clusterName, String wfId,
                        long msgInsertTime, long delay, String entityType,
                        String entityName, String instance, int runId) {
             super(clusterName, wfId, msgInsertTime, delay,
                     entityType, entityName, instance, runId);
         }
+        //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
         @Override
         public RerunType getType() {


[40/47] git commit: checkstyle related changes - disables ALL CAPS for final local variables and enables failOnViolation for checkstyle errors.

Posted by sr...@apache.org.
checkstyle related changes - disables ALL CAPS for final local variables and enables failOnViolation for checkstyle errors.


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

Branch: refs/heads/master
Commit: 12889e2a8b0389d6f281e4eed4839caf10341099
Parents: ca7c16a
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:25:12 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:25:12 2013 -0700

----------------------------------------------------------------------
 .../src/main/resources/falcon/checkstyle.xml       |    2 +-
 pom.xml                                            |    2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/12889e2a/build-tools/src/main/resources/falcon/checkstyle.xml
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle.xml b/build-tools/src/main/resources/falcon/checkstyle.xml
index 3853c5c..0d60e27 100644
--- a/build-tools/src/main/resources/falcon/checkstyle.xml
+++ b/build-tools/src/main/resources/falcon/checkstyle.xml
@@ -87,7 +87,7 @@
             <property name="format" value="^[A-Z]+$"/>
         </module>
         <module name="LocalFinalVariableName">
-            <property name="format" value="^[A-Z][_A-Z0-9]*$"/>
+            <!--<property name="format" value="^[A-Z][_A-Z0-9]*$"/>-->
         </module>
         <module name="LocalVariableName"/>
         <module name="MemberName"/>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/12889e2a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ef53e37..67fd237 100644
--- a/pom.xml
+++ b/pom.xml
@@ -723,7 +723,7 @@
                             <consoleOutput>true</consoleOutput>
                             <includeTestSourceDirectory>true</includeTestSourceDirectory>
                             <configLocation>falcon/checkstyle.xml</configLocation>
-                            <failOnViolation>false</failOnViolation>
+                            <failOnViolation>true</failOnViolation>
                         </configuration>
                     </execution>
                 </executions>


[06/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
index 6ef4bdf..a59994a 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
@@ -18,557 +18,551 @@
 
 package org.apache.falcon.cli;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.util.Map;
-
 import junit.framework.Assert;
-
 import org.apache.falcon.resource.AbstractTestBase;
 import org.testng.annotations.Test;
 
+import java.io.*;
+import java.util.Map;
+
 //Refactor both the classes to move this methods to helper;
-public class FalconCLITest extends AbstractTestBase{
-
-	private InMemoryWriter stream = new InMemoryWriter(System.out);
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final boolean enableTest = true;
-
-	@Test(enabled = enableTest)
-	public void testSubmitEntityValidCommands() throws Exception {
-		
-		FalconCLI.OUT_STREAM = stream;
-
-		String filePath;
-		Map<String, String> overlay = getUniqueOverlay();
-
-		filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type cluster -file " + filePath));
-		Assert.assertEquals(stream.buffer.toString().trim(),
-				"default/Submit successful (cluster) " + clusterName);
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-		Assert.assertEquals(
-				stream.buffer.toString().trim(),
-				"default/Submit successful (feed) "
-						+ overlay.get("inputFeedName"));
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-		Assert.assertEquals(
-				stream.buffer.toString().trim(),
-				"default/Submit successful (feed) "
-						+ overlay.get("outputFeedName"));
-
-		filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type process -file " + filePath));
-		Assert.assertEquals(
-				stream.buffer.toString().trim(),
-				"default/Submit successful (process) "
-						+ overlay.get("processName"));
-	}
-	
-	@Test(enabled = enableTest)
-	public void testListWithEmptyConfigStore() throws Exception
-	{
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -list -type process "));
-	}
-	
-	@Test(enabled = enableTest)
-	public void testSubmitAndScheduleEntityValidCommands() throws Exception {
+public class FalconCLITest extends AbstractTestBase {
+
+    private InMemoryWriter stream = new InMemoryWriter(System.out);
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final boolean enableTest = true;
+
+    @Test(enabled = enableTest)
+    public void testSubmitEntityValidCommands() throws Exception {
+
+        FalconCLI.OUT.set(stream);
+
+        String filePath;
+        Map<String, String> overlay = getUniqueOverlay();
+
+        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type cluster -file " + filePath));
+        Assert.assertEquals(stream.buffer.toString().trim(),
+                "default/Submit successful (cluster) " + clusterName);
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+        Assert.assertEquals(
+                stream.buffer.toString().trim(),
+                "default/Submit successful (feed) "
+                        + overlay.get("inputFeedName"));
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+        Assert.assertEquals(
+                stream.buffer.toString().trim(),
+                "default/Submit successful (feed) "
+                        + overlay.get("outputFeedName"));
+
+        filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type process -file " + filePath));
+        Assert.assertEquals(
+                stream.buffer.toString().trim(),
+                "default/Submit successful (process) "
+                        + overlay.get("processName"));
+    }
+
+    @Test(enabled = enableTest)
+    public void testListWithEmptyConfigStore() throws Exception {
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -list -type process "));
+    }
+
+    @Test(enabled = enableTest)
+    public void testSubmitAndScheduleEntityValidCommands() throws Exception {
 
         Thread.sleep(5000);
-		String filePath;
-		Map<String, String> overlay = getUniqueOverlay();
-
-		filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
-		Assert.assertEquals(-1,
-				executeWithURL("entity -submitAndSchedule -type cluster -file "
-						+ filePath));
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submitAndSchedule -type feed -file "
-						+ filePath));
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submitAndSchedule -type feed -file "
-						+ filePath));
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-
-		filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submitAndSchedule -type process -file "
-						+ filePath));
+        String filePath;
+        Map<String, String> overlay = getUniqueOverlay();
+
+        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        Assert.assertEquals(-1,
+                executeWithURL("entity -submitAndSchedule -type cluster -file "
+                        + filePath));
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submitAndSchedule -type feed -file "
+                        + filePath));
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submitAndSchedule -type feed -file "
+                        + filePath));
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+
+        filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submitAndSchedule -type process -file "
+                        + filePath));
 
         Thread.sleep(5000);
-	}
+    }
 
-	@Test(enabled = enableTest)
-	public void testValidateValidCommands() throws Exception {
+    @Test(enabled = enableTest)
+    public void testValidateValidCommands() throws Exception {
 
-		String filePath;
-		Map<String, String> overlay = getUniqueOverlay();
+        String filePath;
+        Map<String, String> overlay = getUniqueOverlay();
 
-		filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -validate -type cluster -file "
-						+ filePath));
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type cluster -file " + filePath));
+        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -validate -type cluster -file "
+                        + filePath));
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type cluster -file " + filePath));
 
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -validate -type feed -file " + filePath));
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -validate -type feed -file " + filePath));
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
 
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -validate -type feed -file " + filePath));
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -validate -type feed -file " + filePath));
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
 
-		filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -validate -type process -file "
-						+ filePath));
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type process -file " + filePath));
+        filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -validate -type process -file "
+                        + filePath));
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type process -file " + filePath));
 
-	}
+    }
 
-	@Test(enabled = enableTest)
-	public void testDefinitionEntityValidCommands() throws Exception {
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
+    @Test(enabled = enableTest)
+    public void testDefinitionEntityValidCommands() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
 
-		Assert.assertEquals(0,
-				executeWithURL("entity -definition -type cluster -name "
-						+ overlay.get("cluster")));
+        Assert.assertEquals(0,
+                executeWithURL("entity -definition -type cluster -name "
+                        + overlay.get("cluster")));
 
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -definition -type feed -name "
-						+ overlay.get("inputFeedName")));
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -definition -type feed -name "
+                        + overlay.get("inputFeedName")));
 
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -definition -type feed -name "
-						+ overlay.get("outputFeedName")));
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -definition -type feed -name "
+                        + overlay.get("outputFeedName")));
 
-		Assert.assertEquals(0,
-				executeWithURL("entity -definition -type process -name "
-						+ overlay.get("processName")));
+        Assert.assertEquals(0,
+                executeWithURL("entity -definition -type process -name "
+                        + overlay.get("processName")));
 
-	}
+    }
 
-	@Test(enabled = enableTest)
-	public void testScheduleEntityValidCommands() throws Exception {
+    @Test(enabled = enableTest)
+    public void testScheduleEntityValidCommands() throws Exception {
 
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
 
-		Assert.assertEquals(-1,
-				executeWithURL("entity -schedule -type cluster -name "
-						+ overlay.get("cluster")));
+        Assert.assertEquals(-1,
+                executeWithURL("entity -schedule -type cluster -name "
+                        + overlay.get("cluster")));
 
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -schedule -type feed -name "
-						+ overlay.get("outputFeedName")));
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -schedule -type feed -name "
+                        + overlay.get("outputFeedName")));
 
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
 
-	}
+    }
 
-	@Test(enabled = enableTest)
-	public void testSuspendResumeStatusEntityValidCommands() throws Exception {
+    @Test(enabled = enableTest)
+    public void testSuspendResumeStatusEntityValidCommands() throws Exception {
 
         Thread.sleep(5000);
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -schedule -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-		
-		waitForProcessWFtoStart();
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -suspend -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -suspend -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -resume -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -resume -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -status -type process -name "
-						+ overlay.get("processName")));
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -schedule -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+        waitForProcessWFtoStart();
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -suspend -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -suspend -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -resume -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -resume -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -status -type process -name "
+                        + overlay.get("processName")));
 
         Thread.sleep(5000);
-	}
-
-	@Test(enabled = enableTest)
-	public void testSubCommandPresence() throws Exception {
-		Assert.assertEquals(-1, executeWithURL("entity -type cluster "));
-	}
-
-	@Test(enabled = enableTest)
-	public void testDeleteEntityValidCommands() throws Exception {
-
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(
-				-1,
-				executeWithURL("entity -delete -type cluster -name "
-						+ overlay.get("cluster")));
-
-		Assert.assertEquals(
-				-1,
-				executeWithURL("entity -delete -type feed -name "
-						+ overlay.get("inputFeedName")));
-
-		Assert.assertEquals(
-				-1,
-				executeWithURL("entity -delete -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -delete -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -delete -type feed -name "
-						+ overlay.get("inputFeedName")));
-
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -delete -type feed -name "
-						+ overlay.get("outputFeedName")));
-
-	}
-
-	@Test(enabled = enableTest)
-	public void testInvalidCLIEntitycommands() throws Exception {
-
-		Map<String, String> overlay = getUniqueOverlay();
-		overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(-1,
-				executeWithURL("entity -submit -type feed -name " + "name"));
-
-		Assert.assertEquals(-1,
-				executeWithURL("entity -schedule -type feed -file " + "name"));
-	}
-
-	@Test(enabled = enableTest)
-	public void testInstanceRunningAndStatusCommands() throws Exception {
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-		
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type feed -name "
-						+ overlay.get("outputFeedName")));
-		waitForProcessWFtoStart();
-		
-		Assert.assertEquals(0,
-				executeWithURL("instance -status -type feed -name "
-						+ overlay.get("outputFeedName")
-						+ " -start " + START_INSTANCE));
-
-		Assert.assertEquals(0,
-				executeWithURL("instance -running -type process -name "
-						+ overlay.get("processName")));
-
-		Assert.assertEquals(0,
-				executeWithURL("instance -status -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE));
-	}
-
-	@Test(enabled = enableTest)
-	public void testInstanceSuspendAndResume() throws Exception {
+    }
+
+    @Test(enabled = enableTest)
+    public void testSubCommandPresence() throws Exception {
+        Assert.assertEquals(-1, executeWithURL("entity -type cluster "));
+    }
+
+    @Test(enabled = enableTest)
+    public void testDeleteEntityValidCommands() throws Exception {
+
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(
+                -1,
+                executeWithURL("entity -delete -type cluster -name "
+                        + overlay.get("cluster")));
+
+        Assert.assertEquals(
+                -1,
+                executeWithURL("entity -delete -type feed -name "
+                        + overlay.get("inputFeedName")));
+
+        Assert.assertEquals(
+                -1,
+                executeWithURL("entity -delete -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -delete -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -delete -type feed -name "
+                        + overlay.get("inputFeedName")));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -delete -type feed -name "
+                        + overlay.get("outputFeedName")));
+
+    }
+
+    @Test(enabled = enableTest)
+    public void testInvalidCLIEntitycommands() throws Exception {
+
+        Map<String, String> overlay = getUniqueOverlay();
+        overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(-1,
+                executeWithURL("entity -submit -type feed -name " + "name"));
+
+        Assert.assertEquals(-1,
+                executeWithURL("entity -schedule -type feed -file " + "name"));
+    }
+
+    @Test(enabled = enableTest)
+    public void testInstanceRunningAndStatusCommands() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type feed -name "
+                        + overlay.get("outputFeedName")));
+        waitForProcessWFtoStart();
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -status -type feed -name "
+                        + overlay.get("outputFeedName")
+                        + " -start " + START_INSTANCE));
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -running -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -status -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE));
+    }
+
+    @Test(enabled = enableTest)
+    public void testInstanceSuspendAndResume() throws Exception {
         Thread.sleep(5000);
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-
-
-		Assert.assertEquals(0,
-				executeWithURL("instance -suspend -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -end " + START_INSTANCE));
-		
-		Assert.assertEquals(0,
-				executeWithURL("instance -resume -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -end " + START_INSTANCE));
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -suspend -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE));
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -resume -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE));
         Thread.sleep(5000);
-	}
+    }
 
     private static final String START_INSTANCE = "2012-04-20T00:00Z";
 
-	@Test(enabled = enableTest)
-	public void testInstanceKillAndRerun() throws Exception {
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-
-		waitForProcessWFtoStart();
-		Assert.assertEquals(
-				0,
-				executeWithURL("instance -kill -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -end " + START_INSTANCE));
-		
-		Assert.assertEquals(
-				0,
-				executeWithURL("instance -rerun -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -file "
-						+ createTempJobPropertiesFile()));
-	}
-	
-	@Test(enabled = enableTest)
-	public void testContinue() throws Exception{
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-
-		waitForProcessWFtoStart();
-		Assert.assertEquals(
-				0,
-				executeWithURL("instance -kill -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -end " + START_INSTANCE));
-		
-		Assert.assertEquals(
-				0,
-				executeWithURL("instance -continue -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE ));
-	}
-	
-	@Test(enabled = enableTest)
-	public void testInvalidCLIInstanceCommands() throws Exception {
-		// no command
-		Assert.assertEquals(-1, executeWithURL(" -kill -type process -name "
-				+ "name" + " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z"));
-
-		Assert.assertEquals(-1, executeWithURL("instance -kill  " + "name"
-				+ " -start 2010-01-01T01:00Z  -end 2010-01-01T01:00Z"));
-
-		Assert.assertEquals(-1,
-				executeWithURL("instance -kill -type process -name " + "name"
-						+ " -end 2010-01-01T03:00Z"));
-
-		Assert.assertEquals(-1,
-				executeWithURL("instance -kill -type process -name "
-						+ " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z"));
-
-	}
-
-	@Test(enabled = enableTest)
-	public void testFalconURL() throws Exception {
-		Assert.assertEquals(-1, new FalconCLI()
-				.run(("instance -status -type process -name " + "processName"
-						+ " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z")
-						.split("\\s")));
-
-		Assert.assertEquals(-1, new FalconCLI()
-				.run(("instance -status -type process -name "
-						+ "processName -url http://unknownhost:1234/"
-						+ " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z")
-						.split("\\s")));
-
-		
-	}
-	
-	@Test(enabled = enableTest)
-	public void testClientProperties() throws Exception {
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(
-				0,
-				new FalconCLI().run(("entity -schedule -type feed -name "
-						+ overlay.get("outputFeedName")).split("\\s")));
-
-		Assert.assertEquals(0,
-				new FalconCLI().run(("entity -schedule -type process -name "
-						+ overlay.get("processName")).split("\\s")));
-		
-	}
-	
-	@Test(enabled = enableTest)
-	public void testGetVersion() throws Exception {
-		Assert.assertEquals( 0,
-				new FalconCLI().run("admin -version".split("\\s")));
-		
-		Assert.assertEquals( 0,
-				new FalconCLI().run("admin -stack".split("\\s")));
-	}
-	
-	@Test(enabled = enableTest)
-	public void testInstanceGetLogs() throws Exception {
-		Map<String, String> overlay = getUniqueOverlay();
-		submitTestFiles(overlay);
-
-		Assert.assertEquals(0,
-				executeWithURL("entity -schedule -type process -name "
-						+ overlay.get("processName")));
-		
-		Assert.assertEquals(0,
-				executeWithURL("instance -logs -type process -name "
-						+ overlay.get("processName")
-						+ " -start " + START_INSTANCE + " -end " + START_INSTANCE));
-		
-	}
-
-	
-	private int executeWithURL(String command) throws Exception {
-		return new FalconCLI()
-				.run((command + " -url " + BASE_URL).split("\\s+"));
-	}
-
-	private String createTempJobPropertiesFile() throws IOException {
-		File target = new File("webapp/target");
-		if (!target.exists()) {
-			target = new File("target");
-		}
-		File tmpFile = File.createTempFile("job", ".properties", target);
-		OutputStream out = new FileOutputStream(tmpFile);
-		out.write("oozie.wf.rerun.failnodes=true\n".getBytes());
-		out.close();
-		return tmpFile.getAbsolutePath();
-	}
-
-	public void submitTestFiles(Map<String, String> overlay) throws Exception {
-
-		String filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE,
-				overlay);
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type cluster -file " + filePath));
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-
-		filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
-		Assert.assertEquals(0,
-				executeWithURL("entity -submit -type feed -file " + filePath));
-
-		filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
-		Assert.assertEquals(
-				0,
-				executeWithURL("entity -submit -type process -file " + filePath));
-	}
-
-	private static class InMemoryWriter extends PrintStream {
-
-		private StringBuffer buffer = new StringBuffer();
-
-		public InMemoryWriter(OutputStream out) {
-			super(out);
-		}
-
-		@Override
-		public void println(String x) {
-			clear();
-			buffer.append(x);
-			super.println(x);
-		}
-
-		public String getBuffer() {
-			return buffer.toString();
-		}
-
-		public void clear() {
-			buffer.delete(0, buffer.length());
-		}
-	}
+    @Test(enabled = enableTest)
+    public void testInstanceKillAndRerun() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+        waitForProcessWFtoStart();
+        Assert.assertEquals(
+                0,
+                executeWithURL("instance -kill -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("instance -rerun -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -file "
+                        + createTempJobPropertiesFile()));
+    }
+
+    @Test(enabled = enableTest)
+    public void testContinue() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+        waitForProcessWFtoStart();
+        Assert.assertEquals(
+                0,
+                executeWithURL("instance -kill -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE));
+
+        Assert.assertEquals(
+                0,
+                executeWithURL("instance -continue -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE));
+    }
+
+    @Test(enabled = enableTest)
+    public void testInvalidCLIInstanceCommands() throws Exception {
+        // no command
+        Assert.assertEquals(-1, executeWithURL(" -kill -type process -name "
+                + "name" + " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z"));
+
+        Assert.assertEquals(-1, executeWithURL("instance -kill  " + "name"
+                + " -start 2010-01-01T01:00Z  -end 2010-01-01T01:00Z"));
+
+        Assert.assertEquals(-1,
+                executeWithURL("instance -kill -type process -name " + "name"
+                        + " -end 2010-01-01T03:00Z"));
+
+        Assert.assertEquals(-1,
+                executeWithURL("instance -kill -type process -name "
+                        + " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z"));
+
+    }
+
+    @Test(enabled = enableTest)
+    public void testFalconURL() throws Exception {
+        Assert.assertEquals(-1, new FalconCLI()
+                .run(("instance -status -type process -name " + "processName"
+                        + " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z")
+                        .split("\\s")));
+
+        Assert.assertEquals(-1, new FalconCLI()
+                .run(("instance -status -type process -name "
+                        + "processName -url http://unknownhost:1234/"
+                        + " -start 2010-01-01T01:00Z  -end 2010-01-01T03:00Z")
+                        .split("\\s")));
+
+
+    }
+
+    @Test(enabled = enableTest)
+    public void testClientProperties() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(
+                0,
+                new FalconCLI().run(("entity -schedule -type feed -name "
+                        + overlay.get("outputFeedName")).split("\\s")));
+
+        Assert.assertEquals(0,
+                new FalconCLI().run(("entity -schedule -type process -name "
+                        + overlay.get("processName")).split("\\s")));
+
+    }
+
+    @Test(enabled = enableTest)
+    public void testGetVersion() throws Exception {
+        Assert.assertEquals(0,
+                new FalconCLI().run("admin -version".split("\\s")));
+
+        Assert.assertEquals(0,
+                new FalconCLI().run("admin -stack".split("\\s")));
+    }
+
+    @Test(enabled = enableTest)
+    public void testInstanceGetLogs() throws Exception {
+        Map<String, String> overlay = getUniqueOverlay();
+        submitTestFiles(overlay);
+
+        Assert.assertEquals(0,
+                executeWithURL("entity -schedule -type process -name "
+                        + overlay.get("processName")));
+
+        Assert.assertEquals(0,
+                executeWithURL("instance -logs -type process -name "
+                        + overlay.get("processName")
+                        + " -start " + START_INSTANCE + " -end " + START_INSTANCE));
+
+    }
+
+
+    private int executeWithURL(String command) throws Exception {
+        return new FalconCLI()
+                .run((command + " -url " + BASE_URL).split("\\s+"));
+    }
+
+    private String createTempJobPropertiesFile() throws IOException {
+        File target = new File("webapp/target");
+        if (!target.exists()) {
+            target = new File("target");
+        }
+        File tmpFile = File.createTempFile("job", ".properties", target);
+        OutputStream out = new FileOutputStream(tmpFile);
+        out.write("oozie.wf.rerun.failnodes=true\n".getBytes());
+        out.close();
+        return tmpFile.getAbsolutePath();
+    }
+
+    public void submitTestFiles(Map<String, String> overlay) throws Exception {
+
+        String filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE,
+                overlay);
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type cluster -file " + filePath));
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE1, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+
+        filePath = overlayParametersOverTemplate(FEED_TEMPLATE2, overlay);
+        Assert.assertEquals(0,
+                executeWithURL("entity -submit -type feed -file " + filePath));
+
+        filePath = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
+        Assert.assertEquals(
+                0,
+                executeWithURL("entity -submit -type process -file " + filePath));
+    }
+
+    private static class InMemoryWriter extends PrintStream {
+
+        private StringBuffer buffer = new StringBuffer();
+
+        public InMemoryWriter(OutputStream out) {
+            super(out);
+        }
+
+        @Override
+        public void println(String x) {
+            clear();
+            buffer.append(x);
+            super.println(x);
+        }
+
+        public String getBuffer() {
+            return buffer.toString();
+        }
+
+        public void clear() {
+            buffer.delete(0, buffer.length());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java b/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
index b8c4822..15b07fe 100644
--- a/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
+++ b/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
@@ -17,13 +17,7 @@
  */
 package org.apache.falcon.logging;
 
-import java.util.Collection;
-import java.util.List;
-
 import junit.framework.Assert;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -35,122 +29,126 @@ import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.security.CurrentUser;
 import org.apache.falcon.service.SharedLibraryHostingService;
 import org.apache.falcon.workflow.engine.OozieWorkflowEngine;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowJob;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import java.util.Collection;
+import java.util.List;
+
 /**
  * Requires Oozie to be running on localhost
- * 
  */
 public class LogMoverTest {
 
-	private static final ConfigurationStore store = ConfigurationStore.get();
-	private static EmbeddedCluster testCluster = null;
-	private static Process testProcess = null;
-	private static String processName = "testProcess"
-			+ System.currentTimeMillis();
-	FileSystem fs;
-
-	@BeforeClass
-	public void setup() throws Exception {
-		cleanupStore();
-		testCluster = EmbeddedCluster.newCluster("testCluster", true);
-		store.publish(EntityType.CLUSTER, testCluster.getCluster());
+    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static EmbeddedCluster testCluster = null;
+    private static Process testProcess = null;
+    private static String processName = "testProcess"
+            + System.currentTimeMillis();
+    FileSystem fs;
+
+    @BeforeClass
+    public void setup() throws Exception {
+        cleanupStore();
+        testCluster = EmbeddedCluster.newCluster("testCluster", true);
+        store.publish(EntityType.CLUSTER, testCluster.getCluster());
         SharedLibraryHostingService listener = new SharedLibraryHostingService();
         listener.onAdd(testCluster.getCluster());
-		fs = FileSystem.get(testCluster.getConf());
-		fs.mkdirs(new Path("/workflow/lib"));
-
-		fs.copyFromLocalFile(
-				new Path(LogMoverTest.class.getResource(
-						"/org/apache/falcon/logging/workflow.xml").toURI()),
-				new Path("/workflow"));
-		fs.copyFromLocalFile(
-				new Path(LogMoverTest.class.getResource(
-						"/org/apache/falcon/logging/java-test.jar").toURI()),
-				new Path("/workflow/lib"));
-
-		testProcess = new ProcessEntityParser().parse(LogMoverTest.class
-				.getResourceAsStream("/org/apache/falcon/logging/process.xml"));
-		testProcess.setName(processName);
-		store.publish(EntityType.PROCESS, testProcess);
-	}
-
-	@AfterClass
-	public void tearDown() {
-		testCluster.shutdown();
-	}
-
-	private void cleanupStore() throws FalconException {
-		for (EntityType type : EntityType.values()) {
-			Collection<String> entities = store.getEntities(type);
-			for (String entity : entities) {
-				store.remove(type, entity);
-			}
-		}
-	}
-
-	@Test
-	public void testLogMover() throws Exception {
-		CurrentUser.authenticate(System.getProperty("user.name"));
-		OozieWorkflowEngine engine = new OozieWorkflowEngine();
-		engine.schedule(testProcess);
-
-		OozieClient client = new OozieClient(
-				ClusterHelper.getOozieUrl(testCluster.getCluster()));
-		List<WorkflowJob> jobs;
-		while (true) {
-			jobs = client.getJobsInfo(OozieClient.FILTER_NAME + "="
-					+ "FALCON_PROCESS_DEFAULT_" + processName);
-			if (jobs.size() > 0) {
-				break;
-			} else {
-				Thread.sleep(100);
-			}
-		}
-
-		WorkflowJob job = jobs.get(0);
-		while (true) {
-			if (!(job.getStatus() == WorkflowJob.Status.RUNNING || job
-					.getStatus() == WorkflowJob.Status.PREP)) {
-				break;
-			} else {
-				Thread.sleep(100);
-				job = client.getJobInfo(job.getId());
-			}
-		}
-
-		Path oozieLogPath = new Path(getLogPath(),
-				"job-2010-01-01-01-00/000/oozie.log");
-		Assert.assertTrue(fs.exists(oozieLogPath));
-
-		testLogMoverWithNextRunId(job.getId());
-
-	}
-
-	private Path getLogPath() throws FalconException {
-		Path stagingPath = new Path(ClusterHelper.getLocation(
-				testCluster.getCluster(), "staging"),
-				EntityUtil.getStagingPath(testProcess) + "/../logs");
-		Path logPath = new Path(ClusterHelper.getStorageUrl(testCluster
-				.getCluster()), stagingPath);
-		return logPath;
-	}
-
-	private void testLogMoverWithNextRunId(String jobId) throws Exception {
-		LogMover.main(new String[] { "-workflowEngineUrl",
-				ClusterHelper.getOozieUrl(testCluster.getCluster()),
-				"-subflowId", jobId + "@user-workflow", "-runId", "1",
-				"-logDir", getLogPath().toString() + "/job-2010-01-01-01-00",
-				"-status", "SUCCEEDED", "-entityType", "process" });
-
-		Path oozieLogPath = new Path(getLogPath(),
-				"job-2010-01-01-01-00/001/oozie.log");
-		Assert.assertTrue(fs.exists(oozieLogPath));
-
-	}
+        fs = FileSystem.get(testCluster.getConf());
+        fs.mkdirs(new Path("/workflow/lib"));
+
+        fs.copyFromLocalFile(
+                new Path(LogMoverTest.class.getResource(
+                        "/org/apache/falcon/logging/workflow.xml").toURI()),
+                new Path("/workflow"));
+        fs.copyFromLocalFile(
+                new Path(LogMoverTest.class.getResource(
+                        "/org/apache/falcon/logging/java-test.jar").toURI()),
+                new Path("/workflow/lib"));
+
+        testProcess = new ProcessEntityParser().parse(LogMoverTest.class
+                .getResourceAsStream("/org/apache/falcon/logging/process.xml"));
+        testProcess.setName(processName);
+        store.publish(EntityType.PROCESS, testProcess);
+    }
+
+    @AfterClass
+    public void tearDown() {
+        testCluster.shutdown();
+    }
+
+    private void cleanupStore() throws FalconException {
+        for (EntityType type : EntityType.values()) {
+            Collection<String> entities = store.getEntities(type);
+            for (String entity : entities) {
+                store.remove(type, entity);
+            }
+        }
+    }
+
+    @Test
+    public void testLogMover() throws Exception {
+        CurrentUser.authenticate(System.getProperty("user.name"));
+        OozieWorkflowEngine engine = new OozieWorkflowEngine();
+        engine.schedule(testProcess);
+
+        OozieClient client = new OozieClient(
+                ClusterHelper.getOozieUrl(testCluster.getCluster()));
+        List<WorkflowJob> jobs;
+        while (true) {
+            jobs = client.getJobsInfo(OozieClient.FILTER_NAME + "="
+                    + "FALCON_PROCESS_DEFAULT_" + processName);
+            if (jobs.size() > 0) {
+                break;
+            } else {
+                Thread.sleep(100);
+            }
+        }
+
+        WorkflowJob job = jobs.get(0);
+        while (true) {
+            if (!(job.getStatus() == WorkflowJob.Status.RUNNING || job
+                    .getStatus() == WorkflowJob.Status.PREP)) {
+                break;
+            } else {
+                Thread.sleep(100);
+                job = client.getJobInfo(job.getId());
+            }
+        }
+
+        Path oozieLogPath = new Path(getLogPath(),
+                "job-2010-01-01-01-00/000/oozie.log");
+        Assert.assertTrue(fs.exists(oozieLogPath));
+
+        testLogMoverWithNextRunId(job.getId());
+
+    }
+
+    private Path getLogPath() throws FalconException {
+        Path stagingPath = new Path(ClusterHelper.getLocation(
+                testCluster.getCluster(), "staging"),
+                EntityUtil.getStagingPath(testProcess) + "/../logs");
+        Path logPath = new Path(ClusterHelper.getStorageUrl(testCluster
+                .getCluster()), stagingPath);
+        return logPath;
+    }
+
+    private void testLogMoverWithNextRunId(String jobId) throws Exception {
+        LogMover.main(new String[]{"-workflowEngineUrl",
+                                   ClusterHelper.getOozieUrl(testCluster.getCluster()),
+                                   "-subflowId", jobId + "@user-workflow", "-runId", "1",
+                                   "-logDir", getLogPath().toString() + "/job-2010-01-01-01-00",
+                                   "-status", "SUCCEEDED", "-entityType", "process"});
+
+        Path oozieLogPath = new Path(getLogPath(),
+                "job-2010-01-01-01-00/001/oozie.log");
+        Assert.assertTrue(fs.exists(oozieLogPath));
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java b/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
index 5938701..ecb14ed 100644
--- a/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
+++ b/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
@@ -17,10 +17,6 @@
  */
 package org.apache.falcon.logging;
 
-import java.util.Collection;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.parser.ProcessEntityParser;
@@ -30,116 +26,123 @@ import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.resource.InstancesResult.Instance;
 import org.apache.falcon.resource.InstancesResult.InstanceAction;
 import org.apache.falcon.resource.InstancesResult.WorkflowStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import java.util.Collection;
+
 public class LogProviderTest {
 
-	private static final ConfigurationStore store = ConfigurationStore.get();
-	private static EmbeddedCluster testCluster = null;
-	private static Process testProcess = null;
-	private static String processName = "testProcess";
-	private static FileSystem fs;
-	private Instance instance;
-
-	@BeforeClass
-	public void setup() throws Exception {
-		testCluster = EmbeddedCluster.newCluster("testCluster", false);
-		cleanupStore();
-		store.publish(EntityType.CLUSTER, testCluster.getCluster());
-		fs = FileSystem.get(testCluster.getConf());
-		Path instanceLogPath = new Path(
-				"/workflow/staging/falcon/workflows/process/" + processName
-						+ "/logs/job-2010-01-01-01-00/000");
-		fs.mkdirs(instanceLogPath);
-		fs.createNewFile(new Path(instanceLogPath, "oozie.log"));
-		fs.createNewFile(new Path(instanceLogPath, "pigAction_SUCCEEDED.log"));
-		fs.createNewFile(new Path(instanceLogPath, "mr_Action_FAILED.log"));
-		fs.createNewFile(new Path(instanceLogPath, "mr_Action2_SUCCEEDED.log"));
-
-		fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
-				+ processName + "/logs/job-2010-01-01-01-00/001"));
-		fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
-				+ processName + "/logs/job-2010-01-01-01-00/002"));
-		Path run3 = new Path("/workflow/staging/falcon/workflows/process/"
-				+ processName + "/logs/job-2010-01-01-01-00/003");
-		fs.mkdirs(run3);
-		fs.createNewFile(new Path(run3, "oozie.log"));
-
-		testProcess = new ProcessEntityParser().parse(LogMoverTest.class
-				.getResourceAsStream("/org/apache/falcon/logging/process.xml"));
-		testProcess.setName(processName);
-		store.publish(EntityType.PROCESS, testProcess);
-	}
-
-	@BeforeMethod
-	public void setInstance() {
-		instance = new Instance();
-		instance.status = WorkflowStatus.SUCCEEDED;
-		instance.instance = "2010-01-01T01:00Z";
-		instance.cluster = "testCluster";
-		instance.logFile = "http://localhost:15000/oozie/wflog";
-	}
-	
-	private void cleanupStore() throws FalconException {
-		for (EntityType type : EntityType.values()) {
-			Collection<String> entities = store.getEntities(type);
-			for (String entity : entities) {
-				store.remove(type, entity);
-			}
-		}
-	}
-
-	@Test
-	public void testLogProviderWithValidRunId() throws FalconException {
-		LogProvider provider = new LogProvider();
-		Instance instanceWithLog = provider.populateLogUrls(testProcess,
-				instance, "0");
-		Assert.assertEquals(
-				instance.logFile,
-				"http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/000/oozie.log");
-
-		InstanceAction action = instanceWithLog.actions[0];
-		Assert.assertEquals(action.action, "mr_Action2");
-		Assert.assertEquals(action.status, "SUCCEEDED");
-		Assert.assertEquals(
-				action.logFile,
-				"http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/000/mr_Action2_SUCCEEDED.log");
-
-		action = instanceWithLog.actions[1];
-		Assert.assertEquals(action.action, "mr_Action");
-		Assert.assertEquals(action.status, "FAILED");
-		Assert.assertEquals(
-				action.logFile,
-				"http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/000/mr_Action_FAILED.log");
-
-	}
-
-	@Test
-	public void testLogProviderWithInvalidRunId() throws FalconException {
-		LogProvider provider = new LogProvider();
-		provider.populateLogUrls(testProcess, instance, "x");
-		Assert.assertEquals(instance.logFile,
-				"http://localhost:15000/oozie/wflog");
-	}
-
-	@Test
-	public void testLogProviderWithUnavailableRunId() throws FalconException {
-		LogProvider provider = new LogProvider();
-		instance.logFile = null;
-		provider.populateLogUrls(testProcess, instance, "7");
-		Assert.assertEquals(instance.logFile, "-");
-	}
-
-	@Test
-	public void testLogProviderWithEmptyRunId() throws FalconException {
-		LogProvider provider = new LogProvider();
-		instance.logFile = null;
-		provider.populateLogUrls(testProcess, instance, null);
-		Assert.assertEquals(
-				instance.logFile,
-				"http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/003/oozie.log");
-	}
+    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static EmbeddedCluster testCluster = null;
+    private static Process testProcess = null;
+    private static String processName = "testProcess";
+    private static FileSystem fs;
+    private Instance instance;
+
+    @BeforeClass
+    public void setup() throws Exception {
+        testCluster = EmbeddedCluster.newCluster("testCluster", false);
+        cleanupStore();
+        store.publish(EntityType.CLUSTER, testCluster.getCluster());
+        fs = FileSystem.get(testCluster.getConf());
+        Path instanceLogPath = new Path(
+                "/workflow/staging/falcon/workflows/process/" + processName
+                        + "/logs/job-2010-01-01-01-00/000");
+        fs.mkdirs(instanceLogPath);
+        fs.createNewFile(new Path(instanceLogPath, "oozie.log"));
+        fs.createNewFile(new Path(instanceLogPath, "pigAction_SUCCEEDED.log"));
+        fs.createNewFile(new Path(instanceLogPath, "mr_Action_FAILED.log"));
+        fs.createNewFile(new Path(instanceLogPath, "mr_Action2_SUCCEEDED.log"));
+
+        fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
+                + processName + "/logs/job-2010-01-01-01-00/001"));
+        fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
+                + processName + "/logs/job-2010-01-01-01-00/002"));
+        Path run3 = new Path("/workflow/staging/falcon/workflows/process/"
+                + processName + "/logs/job-2010-01-01-01-00/003");
+        fs.mkdirs(run3);
+        fs.createNewFile(new Path(run3, "oozie.log"));
+
+        testProcess = new ProcessEntityParser().parse(LogMoverTest.class
+                .getResourceAsStream("/org/apache/falcon/logging/process.xml"));
+        testProcess.setName(processName);
+        store.publish(EntityType.PROCESS, testProcess);
+    }
+
+    @BeforeMethod
+    public void setInstance() {
+        instance = new Instance();
+        instance.status = WorkflowStatus.SUCCEEDED;
+        instance.instance = "2010-01-01T01:00Z";
+        instance.cluster = "testCluster";
+        instance.logFile = "http://localhost:15000/oozie/wflog";
+    }
+
+    private void cleanupStore() throws FalconException {
+        for (EntityType type : EntityType.values()) {
+            Collection<String> entities = store.getEntities(type);
+            for (String entity : entities) {
+                store.remove(type, entity);
+            }
+        }
+    }
+
+    @Test
+    public void testLogProviderWithValidRunId() throws FalconException {
+        LogProvider provider = new LogProvider();
+        Instance instanceWithLog = provider.populateLogUrls(testProcess,
+                instance, "0");
+        Assert.assertEquals(
+                instance.logFile,
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
+                        "-01-01-00/000/oozie.log");
+
+        InstanceAction action = instanceWithLog.actions[0];
+        Assert.assertEquals(action.action, "mr_Action2");
+        Assert.assertEquals(action.status, "SUCCEEDED");
+        Assert.assertEquals(
+                action.logFile,
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
+                        "-01-01-00/000/mr_Action2_SUCCEEDED.log");
+
+        action = instanceWithLog.actions[1];
+        Assert.assertEquals(action.action, "mr_Action");
+        Assert.assertEquals(action.status, "FAILED");
+        Assert.assertEquals(
+                action.logFile,
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
+                        "-01-01-00/000/mr_Action_FAILED.log");
+
+    }
+
+    @Test
+    public void testLogProviderWithInvalidRunId() throws FalconException {
+        LogProvider provider = new LogProvider();
+        provider.populateLogUrls(testProcess, instance, "x");
+        Assert.assertEquals(instance.logFile,
+                "http://localhost:15000/oozie/wflog");
+    }
+
+    @Test
+    public void testLogProviderWithUnavailableRunId() throws FalconException {
+        LogProvider provider = new LogProvider();
+        instance.logFile = null;
+        provider.populateLogUrls(testProcess, instance, "7");
+        Assert.assertEquals(instance.logFile, "-");
+    }
+
+    @Test
+    public void testLogProviderWithEmptyRunId() throws FalconException {
+        LogProvider provider = new LogProvider();
+        instance.logFile = null;
+        provider.populateLogUrls(testProcess, instance, null);
+        Assert.assertEquals(
+                instance.logFile,
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/003/oozie.log");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java b/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
index f80a522..de9958d 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
@@ -18,37 +18,12 @@
 
 package org.apache.falcon.resource;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.servlet.ServletInputStream;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriBuilder;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.cluster.util.StandAloneCluster;
@@ -60,22 +35,28 @@ import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.util.EmbeddedServer;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.falcon.workflow.engine.OozieClientFactory;
-import org.apache.oozie.client.BundleJob;
-import org.apache.oozie.client.CoordinatorJob;
-import org.apache.oozie.client.Job;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.oozie.client.*;
 import org.apache.oozie.client.Job.Status;
-import org.apache.oozie.client.OozieClient;
-import org.apache.oozie.client.WorkflowJob;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeClass;
 
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
+import javax.servlet.ServletInputStream;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.UriBuilder;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import java.io.*;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 public class AbstractTestBase {
     protected static final String FEED_TEMPLATE1 = "/feed-template1.xml";
@@ -86,7 +67,7 @@ public class AbstractTestBase {
     protected static final String PROCESS_TEMPLATE = "/process-template.xml";
 
     protected static final String BASE_URL = "http://localhost:15000/";
-    protected static final String REMOTE_USER=System.getProperty("user.name");
+    protected static final String REMOTE_USER = System.getProperty("user.name");
 
     protected EmbeddedServer server;
 
@@ -114,16 +95,18 @@ public class AbstractTestBase {
         response = submitToFalcon(processTemplate, overlay, EntityType.PROCESS);
         assertSuccessful(response);
         ClientResponse clientRepsonse = this.service.path("api/entities/schedule/process/" + processName)
-                .header("Remote-User", REMOTE_USER).accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML).post(ClientResponse.class);
+                .header("Remote-User", REMOTE_USER).accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML).post(
+                        ClientResponse.class);
         assertSuccessful(clientRepsonse);
     }
 
     protected void scheduleProcess() throws Exception {
-    	scheduleProcess(PROCESS_TEMPLATE, getUniqueOverlay());
+        scheduleProcess(PROCESS_TEMPLATE, getUniqueOverlay());
     }
-    
+
     private List<WorkflowJob> getRunningJobs(String entityName) throws Exception {
-        OozieClient ozClient = OozieClientFactory.get((Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, clusterName));
+        OozieClient ozClient = OozieClientFactory.get(
+                (Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, clusterName));
         StringBuilder builder = new StringBuilder();
         builder.append(OozieClient.FILTER_STATUS).append('=').append(Job.Status.RUNNING).append(';');
         builder.append(OozieClient.FILTER_NAME).append('=').append("FALCON_PROCESS_DEFAULT_").append(entityName);
@@ -133,45 +116,51 @@ public class AbstractTestBase {
     protected void waitForWorkflowStart(String entityName) throws Exception {
         for (int i = 0; i < 10; i++) {
             List<WorkflowJob> jobs = getRunningJobs(entityName);
-            if (jobs != null && !jobs.isEmpty())
+            if (jobs != null && !jobs.isEmpty()) {
                 return;
+            }
 
             System.out.println("Waiting for workflow to start");
             Thread.sleep(i * 1000);
         }
         throw new Exception("Workflow for " + entityName + " hasn't started in oozie");
     }
-    
-    protected void waitForProcessWFtoStart() throws Exception{
-    	waitForWorkflowStart(processName);
+
+    protected void waitForProcessWFtoStart() throws Exception {
+        waitForWorkflowStart(processName);
     }
-    
-    protected void waitForOutputFeedWFtoStart() throws Exception{
-    	waitForWorkflowStart(outputFeedName);
+
+    protected void waitForOutputFeedWFtoStart() throws Exception {
+        waitForWorkflowStart(outputFeedName);
     }
 
     protected void waitForBundleStart(Status status) throws Exception {
         OozieClient ozClient = OozieClientFactory.get(clusterName);
         List<BundleJob> bundles = getBundles();
-        if(bundles.isEmpty())
+        if (bundles.isEmpty()) {
             return;
-        
+        }
+
         String bundleId = bundles.get(0).getId();
         for (int i = 0; i < 15; i++) {
             Thread.sleep(i * 1000);
             BundleJob bundle = ozClient.getBundleJobInfo(bundleId);
             if (bundle.getStatus() == status) {
-            	if(status == Status.FAILED)
-            		return;
-            	
+                if (status == Status.FAILED) {
+                    return;
+                }
+
                 boolean done = false;
-                for (CoordinatorJob coord : bundle.getCoordinators())
-                    if (coord.getStatus() == status)
+                for (CoordinatorJob coord : bundle.getCoordinators()) {
+                    if (coord.getStatus() == status) {
                         done = true;
-                if (done == true)
+                    }
+                }
+                if (done == true) {
                     return;
+                }
             }
-            System.out.println("Waiting for bundle " + bundleId  + " in " + status + " state");
+            System.out.println("Waiting for bundle " + bundleId + " in " + status + " state");
         }
         throw new Exception("Bundle " + bundleId + " is not " + status + " in oozie");
     }
@@ -233,8 +222,9 @@ public class AbstractTestBase {
         fs.delete(wfParent, true);
         Path wfPath = new Path(wfParent, "workflow");
         fs.mkdirs(wfPath);
-        fs.copyFromLocalFile(false, true, new Path(this.getClass().getResource("/fs-workflow.xml").getPath()), new Path(wfPath,
-                "workflow.xml"));
+        fs.copyFromLocalFile(false, true, new Path(this.getClass().getResource("/fs-workflow.xml").getPath()),
+                new Path(wfPath,
+                        "workflow.xml"));
         fs.mkdirs(new Path(wfParent, "input/2012/04/20/00"));
         Path outPath = new Path(wfParent, "output");
         fs.mkdirs(outPath);
@@ -243,7 +233,7 @@ public class AbstractTestBase {
 
     /**
      * Converts a InputStream into ServletInputStream
-     * 
+     *
      * @param fileName
      * @return ServletInputStream
      * @throws java.io.IOException
@@ -275,7 +265,8 @@ public class AbstractTestBase {
         }
     }
 
-    protected ClientResponse submitAndSchedule(String template, Map<String, String> overlay, EntityType entityType) throws Exception {
+    protected ClientResponse submitAndSchedule(String template, Map<String, String> overlay, EntityType entityType)
+            throws Exception {
         String tmpFile = overlayParametersOverTemplate(template, overlay);
         ServletInputStream rawlogStream = getServletInputStream(tmpFile);
 
@@ -284,7 +275,8 @@ public class AbstractTestBase {
                 .post(ClientResponse.class, rawlogStream);
     }
 
-    protected ClientResponse submitToFalcon(String template, Map<String, String> overlay, EntityType entityType) throws IOException {
+    protected ClientResponse submitToFalcon(String template, Map<String, String> overlay, EntityType entityType)
+            throws IOException {
         String tmpFile = overlayParametersOverTemplate(template, overlay);
         return submitFileToFalcon(entityType, tmpFile);
     }
@@ -293,7 +285,8 @@ public class AbstractTestBase {
 
         ServletInputStream rawlogStream = getServletInputStream(tmpFile);
 
-        return this.service.path("api/entities/submit/" + entityType.name().toLowerCase()).header("Remote-User", "testuser")
+        return this.service.path("api/entities/submit/" + entityType.name().toLowerCase()).header("Remote-User",
+                "testuser")
                 .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML).post(ClientResponse.class, rawlogStream);
     }
 
@@ -365,13 +358,14 @@ public class AbstractTestBase {
 
     protected List<BundleJob> getBundles() throws Exception {
         List<BundleJob> bundles = new ArrayList<BundleJob>();
-        if (clusterName == null)
+        if (clusterName == null) {
             return bundles;
-        
+        }
+
         OozieClient ozClient = OozieClientFactory.get(cluster.getCluster());
         return ozClient.getBundleJobsInfo("name=FALCON_PROCESS_" + processName, 0, 10);
     }
-    
+
     @AfterClass
     public void cleanup() throws Exception {
         tearDown();
@@ -380,31 +374,33 @@ public class AbstractTestBase {
 
     @AfterMethod
     public boolean killOozieJobs() throws Exception {
-        if (clusterName == null)
+        if (clusterName == null) {
             return true;
+        }
 
         OozieClient ozClient = OozieClientFactory.get(cluster.getCluster());
         List<BundleJob> bundles = getBundles();
         if (bundles != null) {
-            for (BundleJob bundle : bundles)
+            for (BundleJob bundle : bundles) {
                 ozClient.kill(bundle.getId());
+            }
         }
         return false;
     }
-    
-	protected Map<String, String> getUniqueOverlay() throws FalconException {
-		Map<String, String> overlay = new HashMap<String, String>();
-		long time = System.currentTimeMillis();
-		clusterName = "cluster" + time;
-		overlay.put("cluster", clusterName);
-		overlay.put("inputFeedName", "in" + time);
-		//only feeds with future dates can be scheduled
-		Date endDate = new Date(System.currentTimeMillis() + 15 * 60 * 1000);
-		overlay.put("feedEndDate", SchemaHelper.formatDateUTC(endDate));
-		overlay.put("outputFeedName", "out" + time);
-		processName = "p" + time;
-		overlay.put("processName", processName);
-		outputFeedName = "out"+time;
-		return overlay;
-	}
+
+    protected Map<String, String> getUniqueOverlay() throws FalconException {
+        Map<String, String> overlay = new HashMap<String, String>();
+        long time = System.currentTimeMillis();
+        clusterName = "cluster" + time;
+        overlay.put("cluster", clusterName);
+        overlay.put("inputFeedName", "in" + time);
+        //only feeds with future dates can be scheduled
+        Date endDate = new Date(System.currentTimeMillis() + 15 * 60 * 1000);
+        overlay.put("feedEndDate", SchemaHelper.formatDateUTC(endDate));
+        overlay.put("outputFeedName", "out" + time);
+        processName = "p" + time;
+        overlay.put("processName", processName);
+        outputFeedName = "out" + time;
+        return overlay;
+    }
 }


[21/47] git commit: Fixes for Checkstyle

Posted by sr...@apache.org.
Fixes for Checkstyle


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

Branch: refs/heads/master
Commit: a4d79f0c516580d92935fcded7b5d25fb89168b3
Parents: b047ce2
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Tue Apr 16 10:07:25 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Tue Apr 16 10:07:27 2013 +0530

----------------------------------------------------------------------
 client/src/main/java/org/apache/falcon/Pair.java   |   18 +-
 .../main/java/org/apache/falcon/cli/CLIParser.java |   27 +-
 .../main/java/org/apache/falcon/cli/FalconCLI.java | 1095 ++++++------
 .../apache/falcon/client/FalconCLIException.java   |   12 +-
 .../org/apache/falcon/client/FalconClient.java     | 1287 +++++++--------
 .../org/apache/falcon/entity/v0/DateValidator.java |  102 +-
 .../java/org/apache/falcon/entity/v0/Entity.java   |   16 +-
 .../org/apache/falcon/entity/v0/EntityType.java    |   29 +-
 .../org/apache/falcon/entity/v0/Frequency.java     |   38 +-
 .../org/apache/falcon/entity/v0/SchemaHelper.java  |    3 +-
 .../java/org/apache/falcon/resource/APIResult.java |    8 +-
 .../org/apache/falcon/resource/EntityList.java     |    5 +-
 .../apache/falcon/resource/InstancesResult.java    |  229 ++--
 client/src/main/resources/cluster-0.1.xsd          |   54 +-
 client/src/main/resources/feed-0.1.xsd             |  127 +-
 client/src/main/resources/jaxb-binding.xjb         |   26 +-
 client/src/main/resources/process-0.1.xsd          |  129 +-
 .../java/org/apache/falcon/cli/TestCLIParser.java  |   70 +-
 .../apache/falcon/entity/v0/DateValidatorTest.java |  114 +-
 .../java/org/apache/falcon/FalconException.java    |   42 +-
 .../org/apache/falcon/FalconRuntimException.java   |   33 +-
 .../falcon/cleanup/AbstractCleanupHandler.java     |  221 ++--
 .../apache/falcon/cleanup/FeedCleanupHandler.java  |   65 +-
 .../falcon/cleanup/ProcessCleanupHandler.java      |   65 +-
 .../org/apache/falcon/entity/ClusterHelper.java    |   69 +-
 .../apache/falcon/entity/ColoClusterRelation.java  |   26 +-
 .../entity/EntityNotRegisteredException.java       |    4 +-
 .../java/org/apache/falcon/entity/EntityUtil.java  |  867 +++++-----
 .../java/org/apache/falcon/entity/ExternalId.java  |   35 +-
 .../java/org/apache/falcon/entity/FeedHelper.java  |   82 +-
 .../org/apache/falcon/entity/ProcessHelper.java    |    6 +-
 .../apache/falcon/entity/WorkflowNameBuilder.java  |  194 ++--
 .../apache/falcon/entity/common/Configuration.java |   66 +-
 .../apache/falcon/entity/common/FeedDataPath.java  |   62 +-
 .../falcon/entity/parser/ClusterEntityParser.java  |   40 +-
 .../entity/parser/CrossEntityValidations.java      |   48 +-
 .../apache/falcon/entity/parser/EntityParser.java  |   45 +-
 .../falcon/entity/parser/EntityParserFactory.java  |   46 +-
 .../falcon/entity/parser/FeedEntityParser.java     |  165 ++-
 .../falcon/entity/parser/ProcessEntityParser.java  |   99 +-
 .../falcon/entity/parser/ValidationException.java  |   16 +-
 .../falcon/entity/store/ConfigurationStore.java    |  144 +-
 .../falcon/entity/store/StoreAccessException.java  |   18 +-
 .../org/apache/falcon/entity/v0/EntityGraph.java   |   32 +-
 .../falcon/entity/v0/EntityIntegrityChecker.java   |   21 +-
 .../entity/v0/UnschedulableEntityException.java    |    1 -
 .../apache/falcon/expression/ExpressionHelper.java |   44 +-
 .../java/org/apache/falcon/group/FeedGroup.java    |  122 +-
 .../java/org/apache/falcon/group/FeedGroupMap.java |  170 +-
 .../org/apache/falcon/security/CurrentUser.java    |    6 +-
 .../apache/falcon/security/FalconLoginModule.java  |    2 +-
 .../security/FalconSecurityConfiguration.java      |    2 +-
 .../apache/falcon/service/LogCleanupService.java   |   99 +-
 .../apache/falcon/service/ServiceInitializer.java  |    8 +-
 .../java/org/apache/falcon/service/Services.java   |   17 +-
 .../org/apache/falcon/update/UpdateHelper.java     |  129 +-
 .../apache/falcon/util/ApplicationProperties.java  |   24 +-
 .../org/apache/falcon/util/BuildProperties.java    |   22 +-
 .../apache/falcon/util/DeploymentProperties.java   |   22 +-
 .../org/apache/falcon/util/DeploymentUtil.java     |   26 +-
 .../org/apache/falcon/util/ReflectionUtils.java    |    4 +-
 .../org/apache/falcon/util/RuntimeProperties.java  |  102 +-
 .../org/apache/falcon/util/StartupProperties.java  |   52 +-
 .../apache/falcon/workflow/WorkflowBuilder.java    |   13 +-
 .../falcon/workflow/WorkflowEngineFactory.java     |    7 +-
 .../workflow/engine/AbstractWorkflowEngine.java    |   41 +-
 common/src/main/resources/log4j.xml                |  120 +-
 .../falcon/cleanup/LogCleanupServiceTest.java      |  218 ++--
 .../org/apache/falcon/entity/AbstractTestBase.java |   71 +-
 .../falcon/entity/ColoClusterRelationTest.java     |   18 +-
 .../org/apache/falcon/entity/EntityTypeTest.java   |   22 +-
 .../org/apache/falcon/entity/EntityUtilTest.java   |   25 +-
 .../org/apache/falcon/entity/FeedHelperTest.java   |    7 +-
 .../falcon/entity/TestWorkflowNameBuilder.java     |   94 +-
 .../entity/parser/ClusterEntityParserTest.java     |   12 +-
 .../falcon/entity/parser/FeedEntityParserTest.java |  702 ++++----
 .../falcon/entity/parser/FeedUpdateTest.java       |    8 +-
 .../entity/parser/ProcessEntityParserTest.java     |  235 ++--
 .../entity/store/ConfigurationStoreTest.java       |   84 +-
 .../apache/falcon/entity/v0/EntityGraphTest.java   |   25 +-
 .../org/apache/falcon/group/FeedGroupMapTest.java  |  282 ++--
 .../apache/falcon/security/CurrentUserTest.java    |    2 +-
 .../org/apache/falcon/update/UpdateHelperTest.java |   48 +-
 .../apache/falcon/util/StartupPropertiesTest.java  |    5 +-
 .../test/resources/config/cluster/cluster-0.1.xml  |   45 +-
 common/src/test/resources/config/feed/feed-0.1.xml |   41 +-
 common/src/test/resources/config/feed/feed-0.2.xml |   27 +-
 .../test/resources/config/process/process-0.1.xml  |   24 +-
 .../test/resources/config/process/process-0.2.xml  |   66 +-
 .../resources/config/process/process-invalid.xml   |   38 +-
 docs/pom.xml                                       |    4 +-
 docs/src/site/site.xml                             |   10 +-
 .../apache/falcon/converter/OozieFeedMapper.java   |  147 +-
 .../falcon/workflow/OozieFeedWorkflowBuilder.java  |   41 +-
 .../config/coordinator/replication-coordinator.xml |   64 +-
 .../config/workflow/replication-workflow.xml       |  292 ++--
 .../config/workflow/retention-workflow.xml         |  138 +-
 .../falcon/converter/OozieFeedMapperTest.java      |  167 +-
 feed/src/test/resources/feed.xml                   |   66 +-
 feed/src/test/resources/src-cluster.xml            |   42 +-
 feed/src/test/resources/trg-cluster.xml            |   42 +-
 .../falcon/messaging/EntityInstanceMessage.java    |  375 +++---
 .../messaging/EntityInstanceMessageCreator.java    |   41 +-
 .../apache/falcon/messaging/MessageProducer.java   |  303 ++--
 messaging/src/main/resources/log4j.xml             |   78 +-
 .../falcon/messaging/FalconTopicProducerTest.java  |  273 ++--
 .../apache/falcon/messaging/FeedProducerTest.java  |  312 ++--
 .../falcon/messaging/ProcessProducerTest.java      |  234 ++--
 .../apache/falcon/aspect/AbstractFalconAspect.java |  114 +-
 .../org/apache/falcon/aspect/GenericAlert.java     |  132 +-
 .../org/apache/falcon/aspect/ResourceMessage.java  |   65 +-
 .../falcon/util/ResourcesReflectionUtil.java       |  167 +-
 .../converter/AbstractOozieEntityMapper.java       |  125 +-
 .../java/org/apache/falcon/logging/LogMover.java   |  334 ++--
 .../org/apache/falcon/logging/LogProvider.java     |  248 ++--
 .../apache/falcon/service/FalconPathFilter.java    |    4 +-
 .../service/SharedLibraryHostingService.java       |   72 +-
 .../java/org/apache/falcon/util/OozieUtils.java    |    4 +-
 .../falcon/workflow/FalconPostProcessing.java      |  359 ++--
 .../falcon/workflow/OozieWorkflowBuilder.java      |   40 +-
 .../falcon/workflow/engine/NullBundleJob.java      |    8 +-
 .../falcon/workflow/engine/OozieClientFactory.java |    6 +-
 .../workflow/engine/OozieHouseKeepingService.java  |   14 +-
 .../workflow/engine/OozieWorkflowEngine.java       |  135 +-
 .../org/apache/oozie/client/CustomOozieClient.java |    9 +-
 oozie/src/main/resources/oozie-bundle-0.1.xsd      |    2 +-
 oozie/src/main/resources/oozie-workflow-0.3.xsd    |    2 +-
 .../oozie/bundle/BundleUnmarshallingTest.java      |   12 +-
 .../coordinator/CoordinatorUnmarshallingTest.java  |   10 +-
 .../oozie/workflow/FalconPostProcessingTest.java   |  257 ++--
 .../oozie/workflow/WorkflowUnmarshallingTest.java  |   16 +-
 .../apache/oozie/client/CustomOozieClientTest.java |    2 +-
 oozie/src/test/resources/oozie/xmls/bundle.xml     |   28 +-
 .../src/test/resources/oozie/xmls/coordinator.xml  |   74 +-
 oozie/src/test/resources/oozie/xmls/workflow.xml   |    2 +-
 .../java/org/apache/falcon/FalconWebException.java |    9 +-
 .../falcon/listener/ContextStartupListener.java    |   13 +-
 .../falcon/plugin/ChainableMonitoringPlugin.java   |    2 +-
 .../falcon/resource/AbstractEntityManager.java     |  120 +-
 .../falcon/resource/AbstractInstanceManager.java   |  202 ++--
 .../resource/AbstractSchedulableEntityManager.java |   55 +-
 .../falcon/resource/channel/AbstractChannel.java   |    2 +-
 .../falcon/resource/channel/HTTPChannel.java       |   49 +-
 .../apache/falcon/resource/channel/IPCChannel.java |   18 +-
 .../apache/falcon/resource/channel/MethodKey.java  |   14 +-
 .../resource/provider/JAXBContextResolver.java     |   22 +-
 .../falcon/resource/proxy/BufferedRequest.java     |    2 +-
 .../resource/proxy/InstanceManagerProxy.java       |  109 +-
 .../proxy/SchedulableEntityManagerProxy.java       |  127 +-
 .../apache/falcon/security/BasicAuthFilter.java    |   21 +-
 .../falcon/service/FalconTopicSubscriber.java      |  240 ++--
 .../falcon/service/ProcessSubscriberService.java   |   56 +-
 .../falcon/service/SLAMonitoringService.java       |   18 +-
 .../org/apache/falcon/util/EmbeddedServer.java     |   32 +-
 prism/src/main/resources/log4j.xml                 |  120 +-
 prism/src/main/webapp/WEB-INF/web.xml              |   66 +-
 .../org/apache/falcon/aspect/GenericAlertTest.java |   10 +-
 .../apache/falcon/aspect/LoggingAspectTest.java    |   69 +-
 .../apache/falcon/resource/EntityManagerTest.java  |    5 +-
 .../falcon/service/FalconTopicSubscriberTest.java  |  151 +-
 process/pom.xml                                    |   88 +-
 .../falcon/converter/OozieProcessMapper.java       |  148 +-
 .../workflow/OozieProcessWorkflowBuilder.java      |   77 +-
 .../config/workflow/process-parent-workflow.xml    |  281 ++--
 .../apache/falcon/converter/AbstractTestBase.java  |   20 +-
 .../OozieProcessMapperLateProcessTest.java         |  121 +-
 .../falcon/converter/OozieProcessMapperTest.java   |  153 +-
 .../test/resources/config/cluster/cluster-0.1.xml  |   45 +-
 .../src/test/resources/config/feed/feed-0.1.xml    |   43 +-
 .../test/resources/config/late/late-cluster.xml    |   45 +-
 .../src/test/resources/config/late/late-feed1.xml  |   28 +-
 .../src/test/resources/config/late/late-feed2.xml  |   28 +-
 .../src/test/resources/config/late/late-feed3.xml  |   30 +-
 .../test/resources/config/late/late-process1.xml   |   17 +-
 .../test/resources/config/late/late-process2.xml   |   23 +-
 .../test/resources/config/process/process-0.1.xml  |   22 +-
 .../falcon/replication/CustomReplicator.java       |    1 +
 .../apache/falcon/replication/FeedReplicator.java  |  144 +-
 .../falcon/replication/FilteredCopyListing.java    |   33 +-
 .../falcon/repliation/FeedReplicatorTest.java      |    8 +-
 .../replication/FilteredCopyListingTest.java       |   15 +-
 .../apache/falcon/latedata/LateDataHandler.java    |  272 ++--
 .../apache/falcon/rerun/event/LaterunEvent.java    |   28 +-
 .../org/apache/falcon/rerun/event/RerunEvent.java  |  166 +-
 .../falcon/rerun/event/RerunEventFactory.java      |   79 +-
 .../org/apache/falcon/rerun/event/RetryEvent.java  |   68 +-
 .../rerun/handler/AbstractRerunConsumer.java       |   72 +-
 .../falcon/rerun/handler/AbstractRerunHandler.java |   64 +-
 .../falcon/rerun/handler/LateRerunConsumer.java    |  209 ++--
 .../falcon/rerun/handler/LateRerunHandler.java     |  374 +++---
 .../falcon/rerun/handler/RerunHandlerFactory.java  |   32 +-
 .../apache/falcon/rerun/handler/RetryConsumer.java |  132 +-
 .../apache/falcon/rerun/handler/RetryHandler.java  |  105 +-
 .../falcon/rerun/policy/AbstractRerunPolicy.java   |   28 +-
 .../falcon/rerun/policy/ExpBackoffPolicy.java      |   65 +-
 .../apache/falcon/rerun/policy/FinalPolicy.java    |    4 +-
 .../apache/falcon/rerun/policy/PeriodicPolicy.java |    8 +-
 .../falcon/rerun/policy/RerunPolicyFactory.java    |    2 +-
 .../apache/falcon/rerun/queue/ActiveMQueue.java    |  243 ++--
 .../apache/falcon/rerun/queue/DelayedQueue.java    |   27 +-
 .../apache/falcon/rerun/queue/InMemoryQueue.java   |  233 ++--
 .../falcon/rerun/service/LateRunService.java       |   66 +-
 .../apache/falcon/rerun/service/RetryService.java  |   58 +-
 .../falcon/rerun/AbstractRerunPolicyTest.java      |  112 +-
 .../apache/falcon/rerun/handler/TestLateData.java  |   59 +-
 .../apache/falcon/rerun/queue/ActiveMQTest.java    |   65 +-
 .../falcon/rerun/queue/InMemoryQueueTest.java      |    2 +-
 rerun/src/test/resources/cluster-template.xml      |   37 +-
 rerun/src/test/resources/feed-template.xml         |   41 +-
 rerun/src/test/resources/process-template.xml      |   22 +-
 rerun/src/test/resources/process-template2.xml     |   46 +-
 .../org/apache/falcon/retention/FeedEvictor.java   |  220 ++--
 .../apache/falcon/retention/FeedEvictorTest.java   |  764 +++++-----
 .../falcon/cluster/util/EmbeddedCluster.java       |   37 +-
 .../falcon/cluster/util/StandAloneCluster.java     |   24 +-
 webapp/src/main/java/org/apache/falcon/Debug.java  |   11 +-
 webapp/src/main/java/org/apache/falcon/Main.java   |    6 +-
 .../apache/falcon/resource/ConfigSyncService.java  |    8 +-
 .../apache/falcon/resource/InstanceManager.java    |   64 +-
 .../falcon/resource/SchedulableEntityManager.java  |   14 +-
 webapp/src/main/resources/log4j.xml                |  120 +-
 webapp/src/main/webapp/WEB-INF/distributed/web.xml |    8 +-
 webapp/src/main/webapp/WEB-INF/embedded/web.xml    |   66 +-
 webapp/src/main/webapp/WEB-INF/web.xml             |   66 +-
 .../java/org/apache/falcon/cli/FalconCLITest.java  | 1024 ++++++------
 .../org/apache/falcon/logging/LogMoverTest.java    |  218 ++--
 .../org/apache/falcon/logging/LogProviderTest.java |  221 ++--
 .../apache/falcon/resource/AbstractTestBase.java   |  172 +-
 .../falcon/resource/EntityManagerJerseyTest.java   |  244 ++--
 .../resource/ProcessInstanceManagerTest.java       |   17 +-
 .../falcon/security/BasicAuthFilterTest.java       |    4 +-
 .../falcon/util/ResourcesReflectionUtilTest.java   |   42 +-
 webapp/src/test/resources/cluster-template.xml     |   41 +-
 webapp/src/test/resources/feed-template1.xml       |   20 +-
 webapp/src/test/resources/feed-template2.xml       |   20 +-
 .../org/apache/falcon/logging/process.xml          |   12 +-
 .../org/apache/falcon/logging/workflow.xml         |   48 +-
 webapp/src/test/resources/process-template.xml     |   14 +-
 webapp/src/test/resources/process-version-0.xml    |   30 +-
 239 files changed, 11669 insertions(+), 11389 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/Pair.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/Pair.java b/client/src/main/java/org/apache/falcon/Pair.java
index 5b25419..300be27 100644
--- a/client/src/main/java/org/apache/falcon/Pair.java
+++ b/client/src/main/java/org/apache/falcon/Pair.java
@@ -31,7 +31,7 @@ public class Pair<A, B> {
     public static <A, B> Pair<A, B> of(A a, B b) {
         return new Pair<A, B>(a, b);
     }
-    
+
     @Override
     public String toString() {
         return "(" + first + "," + second + ")";
@@ -39,13 +39,21 @@ public class Pair<A, B> {
 
     @Override
     public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
 
         Pair pair = (Pair) o;
 
-        if (first != null ? !first.equals(pair.first) : pair.first != null) return false;
-        if (second != null ? !second.equals(pair.second) : pair.second != null) return false;
+        if (first != null ? !first.equals(pair.first) : pair.first != null) {
+            return false;
+        }
+        if (second != null ? !second.equals(pair.second) : pair.second != null) {
+            return false;
+        }
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/cli/CLIParser.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/CLIParser.java b/client/src/main/java/org/apache/falcon/cli/CLIParser.java
index 733cfb8..18fc487 100644
--- a/client/src/main/java/org/apache/falcon/cli/CLIParser.java
+++ b/client/src/main/java/org/apache/falcon/cli/CLIParser.java
@@ -18,17 +18,13 @@
 
 package org.apache.falcon.cli;
 
+import org.apache.commons.cli.*;
+
 import java.io.PrintWriter;
 import java.text.MessageFormat;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-
 
 /**
  * Cannibalized from Oozie CLIParser into Falcon
@@ -57,9 +53,9 @@ public class CLIParser {
     /**
      * Add a command to the parser.
      *
-     * @param command comand name.
-     * @param argsHelp command arguments help.
-     * @param commandHelp command description.
+     * @param command        comand name.
+     * @param argsHelp       command arguments help.
+     * @param commandHelp    command description.
      * @param commandOptions command options.
      * @param hasArguments
      */
@@ -74,7 +70,7 @@ public class CLIParser {
     /**
      * Bean that represents a parsed command.
      */
-    public class Command {
+    public final class Command {
         private String name;
         private CommandLine commandLine;
 
@@ -112,16 +108,14 @@ public class CLIParser {
     public Command parse(String[] args) throws ParseException {
         if (args.length == 0) {
             throw new ParseException("missing sub-command");
-        }
-        else {
+        } else {
             if (commands.containsKey(args[0])) {
                 GnuParser parser = new GnuParser();
                 String[] minusCommand = new String[args.length - 1];
                 System.arraycopy(args, 1, minusCommand, 0, minusCommand.length);
                 return new Command(args[0], parser.parse(commands.get(args[0]), minusCommand,
-                                                         commandWithArgs.get(args[0])));
-            }
-            else {
+                        commandWithArgs.get(args[0])));
+            } else {
                 throw new ParseException(MessageFormat.format("invalid sub-command [{0}]", args[0]));
             }
         }
@@ -147,8 +141,7 @@ public class CLIParser {
             if (entry.getValue().getOptions().size() > 0) {
                 pw.println(s + "<OPTIONS> " + commandsHelp.get(entry.getKey()));
                 formatter.printOptions(pw, 100, entry.getValue(), s.length(), 3);
-            }
-            else {
+            } else {
                 pw.println(s + commandsHelp.get(entry.getKey()));
             }
             pw.println();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
index 92243ea..575d7a5 100644
--- a/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
+++ b/client/src/main/java/org/apache/falcon/cli/FalconCLI.java
@@ -18,13 +18,7 @@
 
 package org.apache.falcon.cli;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PrintStream;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.Set;
-
+import com.sun.jersey.api.client.ClientHandlerException;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
@@ -33,560 +27,563 @@ import org.apache.commons.cli.ParseException;
 import org.apache.falcon.client.FalconCLIException;
 import org.apache.falcon.client.FalconClient;
 
-import com.sun.jersey.api.client.ClientHandlerException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
 
+/**
+ * Falcon Command Line Interface - wraps the RESTful API.
+ */
 public class FalconCLI {
 
-	public static PrintStream ERR_STREAM = System.err;
-	public static PrintStream OUT_STREAM = System.out;
-	
-	public static final String FALCON_URL = "FALCON_URL";
-	public static final String URL_OPTION = "url";
-	public static final String VERSION_OPTION = "version";
-	public static final String ADMIN_CMD = "admin";
-	public static final String HELP_CMD = "help";
-	private static final String VERSION_CMD = "version";
-	private static final String STACK_OPTION = "stack";
-	
-	public static final String ENTITY_CMD = "entity";
-	public static final String ENTITY_TYPE_OPT = "type";
-	public static final String COLO_OPT = "colo";
-	public static final String ENTITY_NAME_OPT = "name";
-	public static final String FILE_PATH_OPT = "file";
-	public static final String SUBMIT_OPT = "submit";
-	public static final String UPDATE_OPT = "update";
-	public static final String SCHEDULE_OPT = "schedule";
-	public static final String SUSPEND_OPT = "suspend";
-	public static final String RESUME_OPT = "resume";
-	public static final String DELETE_OPT = "delete";
-	public static final String SUBMIT_AND_SCHEDULE_OPT = "submitAndSchedule";
-	public static final String VALIDATE_OPT = "validate";
-	public static final String STATUS_OPT = "status";
-	public static final String DEFINITION_OPT = "definition";
-	public static final String DEPENDENCY_OPT = "dependency";
-	public static final String LIST_OPT = "list";
-
-	public static final String INSTANCE_CMD = "instance";
-	public static final String START_OPT = "start";
-	public static final String END_OPT = "end";
-	public static final String RUNNING_OPT = "running";
-	public static final String KILL_OPT = "kill";
-	public static final String RERUN_OPT = "rerun";
-	public static final String CONTINUE_OPT = "continue";
-	public static final String LOG_OPT = "logs";
-	public static final String RUNID_OPT = "runid";
-	public static final String CLUSTERS_OPT = "clusters";
-	public static final String SOURCECLUSTER_OPT = "sourceClusters";
-	public static final String CURRENT_COLO = "current.colo";
-	public static final String CLIENT_PROPERTIES = "/client.properties";
-
-	/**
-	 * Entry point for the Falcon CLI when invoked from the command line. Upon
-	 * completion this method exits the JVM with '0' (success) or '-1'
-	 * (failure).
-	 * 
-	 * @param args
-	 *            options and arguments for the Falcon CLI.
-	 */
-	public static void main(String[] args) {
-		System.exit(new FalconCLI().run(args));
-	}
-
-	// TODO help and headers
-	private static final String[] FALCON_HELP = {
-			"the env variable '" + FALCON_URL
-					+ "' is used as default value for the '-" + URL_OPTION
-					+ "' option",
-			"custom headers for Falcon web services can be specified using '-D"
-					+ FalconClient.WS_HEADER_PREFIX + "NAME=VALUE'" };
-
-	/**
-	 * Run a CLI programmatically.
-	 * <p/>
-	 * It does not exit the JVM.
-	 * <p/>
-	 * A CLI instance can be used only once.
-	 * 
-	 * @param args
-	 *            options and arguments for the Oozie CLI.
-	 * @return '0' (success), '-1' (failure).
-	 */
-	public synchronized int run(String[] args) {
-
-		CLIParser parser = new CLIParser("falcon", FALCON_HELP);
-
-		parser.addCommand(ADMIN_CMD, "", "admin operations",
-				createAdminOptions(), true);
-		parser.addCommand(HELP_CMD, "", "display usage", new Options(), false);
-		parser.addCommand(VERSION_CMD, "", "show client version",
-				new Options(), false);
-		parser.addCommand(
-				ENTITY_CMD,
-				"",
-				"Entity opertions like submit, suspend, resume, delete, status, defintion, submitAndSchedule",
-				entityOptions(), false);
-		parser.addCommand(
-				INSTANCE_CMD,
-				"",
-				"Process instances operations like running, status, kill, suspend, resume, rerun, logs",
-				instanceOptions(), false);
-
-		try {
-			CLIParser.Command command = parser.parse(args);
-			if (command.getName().equals(HELP_CMD)) {
-				parser.showHelp();
-			} else if (command.getName().equals(ADMIN_CMD)) {
-				adminCommand(command.getCommandLine());
-			} else if (command.getName().equals(ENTITY_CMD)) {
-				entityCommand(command.getCommandLine());
-			} else if (command.getName().equals(INSTANCE_CMD)) {
-				instanceCommand(command.getCommandLine());
-			}
-
-			return 0;
-		} catch (FalconCLIException ex) {
-			ERR_STREAM.println("Error: " + ex.getMessage());
-			return -1;
-		} catch (ParseException ex) {
-			ERR_STREAM.println("Invalid sub-command: " + ex.getMessage());
-			ERR_STREAM.println();
-			ERR_STREAM.println(parser.shortHelp());
-			return -1;
-		} catch (ClientHandlerException ex) {
-			ERR_STREAM
-					.print("Unable to connect to Falcon server, please check if the URL is correct and Falcon server is up and running\n");
-			ERR_STREAM.println(ex.getMessage());
-			return -1;
-		} catch (Exception ex) {
-			ex.printStackTrace();
-			ERR_STREAM.println(ex.getMessage());
-			return -1;
-		}
-
-	}
-
-	private void instanceCommand(CommandLine commandLine)
-			throws FalconCLIException, IOException {
-		String falconUrl = getFalconEndpoint(commandLine);
-		FalconClient client = new FalconClient(falconUrl);
-
-		Set<String> optionsList = new HashSet<String>();
-		for (Option option : commandLine.getOptions()) {
-			optionsList.add(option.getOpt());
-		}
-
-		String result = null;
-		String type = commandLine.getOptionValue(ENTITY_TYPE_OPT);
-		String entity = commandLine.getOptionValue(ENTITY_NAME_OPT);
-		String start = commandLine.getOptionValue(START_OPT);
-		String end = commandLine.getOptionValue(END_OPT);
-		String filePath = commandLine.getOptionValue(FILE_PATH_OPT);
-		String runid = commandLine.getOptionValue(RUNID_OPT);
-		String colo = commandLine.getOptionValue(COLO_OPT);
-		String clusters = commandLine.getOptionValue(CLUSTERS_OPT);
-		String sourceClusters = commandLine.getOptionValue(SOURCECLUSTER_OPT);
-		
-		colo = getColo(colo);
-		
-		validateInstanceCommands(optionsList, entity, type, start, end,
-				filePath, colo, clusters, sourceClusters);
-		
-		if (optionsList.contains(RUNNING_OPT)) {
-			result = client.getRunningInstances(type, entity, colo);
-		} else if (optionsList.contains(STATUS_OPT)) {
-			result = client.getStatusOfInstances(type, entity, start, end,
-					runid, colo);
-		} else if (optionsList.contains(KILL_OPT)) {
-			result = client.killInstances(type, entity, start, end, colo, clusters, sourceClusters);
-		} else if (optionsList.contains(SUSPEND_OPT)) {
-			result = client.suspendInstances(type, entity, start, end, colo, clusters, sourceClusters);
-		} else if (optionsList.contains(RESUME_OPT)) {
-			result = client.resumeInstances(type, entity, start, end, colo, clusters, sourceClusters);
-		} else if (optionsList.contains(RERUN_OPT)) {
-			result = client.rerunInstances(type, entity, start, end, filePath, colo, clusters, sourceClusters);
-		}  else if (optionsList.contains(CONTINUE_OPT)) {
-			result = client.rerunInstances(type, entity, start, end, colo, clusters, sourceClusters);
-		} else if(optionsList.contains(LOG_OPT)){
-			result = client.getLogsOfInstances(type, entity, start, end, colo, runid);
-		} else {
-			throw new FalconCLIException("Invalid command");
-		}
-		OUT_STREAM.println(result);
-
-	}
-
-	private void validateInstanceCommands(Set<String> optionsList,
-			String entity, String type, String start, String end,
-			String filePath, String colo, String clusters, String sourceClusters) 
-					throws FalconCLIException {
-
-		if (entity == null || entity.equals("")) {
-			throw new FalconCLIException("Missing argument: name");
-		}
-
-		if (type == null || type.equals("")) {
-			throw new FalconCLIException("Missing argument: type");
-		}
-		
-		if (colo == null || colo.equals("")) {
-			throw new FalconCLIException("Missing argument: colo");
-		}
-		
-		if (!optionsList.contains(RUNNING_OPT)) {
-			if (start == null || start.equals("")) {
-				throw new FalconCLIException("Missing argument: start");
-			}
-		}
-		
-		if (optionsList.contains(CLUSTERS_OPT)) {
-			if (optionsList.contains(RUNNING_OPT)
-					|| optionsList.contains(LOG_OPT)
-					|| optionsList.contains(STATUS_OPT)) {
-				throw new FalconCLIException("Invalid argument: clusters");
-			}
-		}
-		
-		if (optionsList.contains(SOURCECLUSTER_OPT)) {
-			if ( optionsList.contains(RUNNING_OPT)
-					|| optionsList.contains(LOG_OPT)
-					|| optionsList.contains(STATUS_OPT) || !type.equals("feed") ) {
-				throw new FalconCLIException("Invalid argument: sourceClusters");
-			}
-		}
-
-	}
-
-	private void entityCommand(CommandLine commandLine)
-			throws FalconCLIException, IOException {
-		String falconUrl = getFalconEndpoint(commandLine);
-		FalconClient client = new FalconClient(falconUrl);
-
-		Set<String> optionsList = new HashSet<String>();
-		for (Option option : commandLine.getOptions()) {
-			optionsList.add(option.getOpt());
-		}
-
-		String result = null;
-		String entityType = commandLine.getOptionValue(ENTITY_TYPE_OPT);
-		String entityName = commandLine.getOptionValue(ENTITY_NAME_OPT);
-		String filePath = commandLine.getOptionValue(FILE_PATH_OPT);
-		String colo = commandLine.getOptionValue(COLO_OPT);
-		
-		
-		validateEntityType(optionsList, entityType);
-
-		if (optionsList.contains(SUBMIT_OPT)) {
-			validateFilePath(optionsList, filePath);
-			validateColo(optionsList);
-			result = client.submit(entityType, filePath);
-		} else if (optionsList.contains(UPDATE_OPT)) {
-			validateFilePath(optionsList, filePath);
-			validateColo(optionsList);
-			validateEntityName(optionsList, entityName);
-			result = client.update(entityType, entityName, filePath);
-		} else if (optionsList.contains(SUBMIT_AND_SCHEDULE_OPT)) {
-			validateFilePath(optionsList, filePath);
-			validateColo(optionsList);
-			result = client.submitAndSchedule(entityType, filePath);
-		} else if (optionsList.contains(VALIDATE_OPT)) {
-			validateFilePath(optionsList, filePath);
-			validateColo(optionsList);
-			result = client.validate(entityType, filePath);
-		} else if (optionsList.contains(SCHEDULE_OPT)) {
-			validateEntityName(optionsList, entityName);
-			colo = getColo(colo);
-			result = client.schedule(entityType, entityName, colo);
-		} else if (optionsList.contains(SUSPEND_OPT)) {
-			validateEntityName(optionsList, entityName);
-			colo = getColo(colo);
-			result = client.suspend(entityType, entityName, colo);
-		} else if (optionsList.contains(RESUME_OPT)) {
-			validateEntityName(optionsList, entityName);
-			colo = getColo(colo);
-			result = client.resume(entityType, entityName, colo);
-		} else if (optionsList.contains(DELETE_OPT)) {
-			validateColo(optionsList);
-			validateEntityName(optionsList, entityName);
-			result = client.delete(entityType, entityName);
-		} else if (optionsList.contains(STATUS_OPT)) {
-			validateEntityName(optionsList, entityName);
-			colo = getColo(colo);
-			result = client.getStatus(entityType, entityName, colo);
-		} else if (optionsList.contains(DEFINITION_OPT)) {
-			validateColo(optionsList);
-			validateEntityName(optionsList, entityName);
-			result = client.getDefinition(entityType, entityName);
-		} else if (optionsList.contains(DEPENDENCY_OPT)) {
-			validateColo(optionsList);
-			validateEntityName(optionsList, entityName);
-			result = client.getDependency(entityType, entityName);
-		} else if (optionsList.contains(LIST_OPT)) {
-			validateColo(optionsList);
-			result = client.getEntityList(entityType);
-		} else if (optionsList.contains(HELP_CMD)) {
-			OUT_STREAM.println("Falcon Help");
-		} else {
-			throw new FalconCLIException("Invalid command");
-		}
-		OUT_STREAM.println(result);
-	}
-	
-	private String getColo(String colo) throws FalconCLIException, IOException
-	{
-		if (colo == null) {
-			Properties prop = getClientProperties();
-			colo = prop.getProperty(CURRENT_COLO, "*");
-		}
-		return colo;
-	}
-	private void validateFilePath(Set<String> optionsList, String filePath)
-			throws FalconCLIException {
-		if (filePath == null || filePath.equals("")) {
-			throw new FalconCLIException("Missing argument: file");
-		}
-	}
-	
-	private void validateColo(Set<String> optionsList)
-			throws FalconCLIException {
-		if (optionsList.contains(COLO_OPT)) {
-			throw new FalconCLIException("Invalid argument : " + COLO_OPT);
-		}
-	}
-
-	private void validateEntityName(Set<String> optionsList, String entityName)
-			throws FalconCLIException {
-		if (entityName == null || entityName.equals("")) {
-			throw new FalconCLIException("Missing argument: name");
-		}
-	}
-
-	private void validateEntityType(Set<String> optionsList, String entityType)
-			throws FalconCLIException {
-		if (entityType == null || entityType.equals("")) {
-			throw new FalconCLIException("Missing argument: type");
-		}
-	}
-	
-
-	private Options createAdminOptions() {
-		Options adminOptions = new Options();
-		Option url = new Option(URL_OPTION, true, "Falcon URL");
-		adminOptions.addOption(url);
-
-		OptionGroup group = new OptionGroup();
-		// Option status = new Option(STATUS_OPTION, false,
-		// "show the current system status");
-		Option version = new Option(VERSION_OPTION, false,
-				"show Falcon server build version");
-		Option help = new Option("help", false, "show Falcon help");
-		group.addOption(version);
-		group.addOption(help);
-
-		adminOptions.addOptionGroup(group);
-		return adminOptions;
-	}
-
-	private Options entityOptions() {
-
-		Options entityOptions = new Options();
-
-		Option submit = new Option(SUBMIT_OPT, false,
-				"Submits an entity xml to Falcon");
-		Option update = new Option(UPDATE_OPT, false,
-				"Updates an existing entity xml");
-		Option schedule = new Option(SCHEDULE_OPT, false,
-				"Schedules a submited entity in Falcon");
-		Option suspend = new Option(SUSPEND_OPT, false,
-				"Suspends a running entity in Falcon");
-		Option resume = new Option(RESUME_OPT, false,
-				"Resumes a suspended entity in Falcon");
-		Option delete = new Option(DELETE_OPT, false,
-				"Deletes an entity in Falcon, and kills its instance from workflow engine");
-		Option submitAndSchedule = new Option(SUBMIT_AND_SCHEDULE_OPT, false,
-				"Submits and entity to Falcon and schedules it immediately");
-		Option validate = new Option(VALIDATE_OPT, false,
-				"Validates an entity based on the entity type");
-		Option status = new Option(STATUS_OPT, false,
-				"Gets the status of entity");
-		Option definition = new Option(DEFINITION_OPT, false,
-				"Gets the Definition of entity");
-		Option dependency = new Option(DEPENDENCY_OPT, false,
-				"Gets the dependencies of entity");
-		Option list = new Option(LIST_OPT, false,
-				"List entities registerd for a type");
-
-		OptionGroup group = new OptionGroup();
-		group.addOption(submit);
-		group.addOption(update);
-		group.addOption(schedule);
-		group.addOption(suspend);
-		group.addOption(resume);
-		group.addOption(delete);
-		group.addOption(submitAndSchedule);
-		group.addOption(validate);
-		group.addOption(status);
-		group.addOption(definition);
-		group.addOption(dependency);
-		group.addOption(list);
-
-		Option url = new Option(URL_OPTION, true, "Falcon URL");
-		Option entityType = new Option(ENTITY_TYPE_OPT, true,
-				"Entity type, can be cluster, feed or process xml");
-		entityType.setRequired(true);
-		Option filePath = new Option(FILE_PATH_OPT, true,
-				"Path to entity xml file");
-		Option entityName = new Option(ENTITY_NAME_OPT, true,
-				"Entity type, can be cluster, feed or process xml");
+    public static final AtomicReference<PrintStream> ERR = new AtomicReference<PrintStream>(System.err);
+    public static final AtomicReference<PrintStream> OUT = new AtomicReference<PrintStream>(System.out);
+
+    public static final String FALCON_URL = "FALCON_URL";
+    public static final String URL_OPTION = "url";
+    public static final String VERSION_OPTION = "version";
+    public static final String ADMIN_CMD = "admin";
+    public static final String HELP_CMD = "help";
+    private static final String VERSION_CMD = "version";
+    private static final String STACK_OPTION = "stack";
+
+    public static final String ENTITY_CMD = "entity";
+    public static final String ENTITY_TYPE_OPT = "type";
+    public static final String COLO_OPT = "colo";
+    public static final String ENTITY_NAME_OPT = "name";
+    public static final String FILE_PATH_OPT = "file";
+    public static final String SUBMIT_OPT = "submit";
+    public static final String UPDATE_OPT = "update";
+    public static final String SCHEDULE_OPT = "schedule";
+    public static final String SUSPEND_OPT = "suspend";
+    public static final String RESUME_OPT = "resume";
+    public static final String DELETE_OPT = "delete";
+    public static final String SUBMIT_AND_SCHEDULE_OPT = "submitAndSchedule";
+    public static final String VALIDATE_OPT = "validate";
+    public static final String STATUS_OPT = "status";
+    public static final String DEFINITION_OPT = "definition";
+    public static final String DEPENDENCY_OPT = "dependency";
+    public static final String LIST_OPT = "list";
+
+    public static final String INSTANCE_CMD = "instance";
+    public static final String START_OPT = "start";
+    public static final String END_OPT = "end";
+    public static final String RUNNING_OPT = "running";
+    public static final String KILL_OPT = "kill";
+    public static final String RERUN_OPT = "rerun";
+    public static final String CONTINUE_OPT = "continue";
+    public static final String LOG_OPT = "logs";
+    public static final String RUNID_OPT = "runid";
+    public static final String CLUSTERS_OPT = "clusters";
+    public static final String SOURCECLUSTER_OPT = "sourceClusters";
+    public static final String CURRENT_COLO = "current.colo";
+    public static final String CLIENT_PROPERTIES = "/client.properties";
+
+    /**
+     * Entry point for the Falcon CLI when invoked from the command line. Upon
+     * completion this method exits the JVM with '0' (success) or '-1'
+     * (failure).
+     *
+     * @param args options and arguments for the Falcon CLI.
+     */
+    public static void main(final String[] args) {
+        System.exit(new FalconCLI().run(args));
+    }
+
+    // TODO help and headers
+    private static final String[] FALCON_HELP = {
+            "the env variable '" + FALCON_URL
+                    + "' is used as default value for the '-" + URL_OPTION + "' option",
+            "custom headers for Falcon web services can be specified using '-D"
+                    + FalconClient.WS_HEADER_PREFIX + "NAME=VALUE'"};
+
+    /**
+     * Run a CLI programmatically.
+     * <p/>
+     * It does not exit the JVM.
+     * <p/>
+     * A CLI instance can be used only once.
+     *
+     * @param args options and arguments for the Oozie CLI.
+     * @return '0' (success), '-1' (failure).
+     */
+    public synchronized int run(final String[] args) {
+
+        CLIParser parser = new CLIParser("falcon", FALCON_HELP);
+
+        parser.addCommand(ADMIN_CMD, "", "admin operations", createAdminOptions(), true);
+        parser.addCommand(HELP_CMD, "", "display usage", new Options(), false);
+        parser.addCommand(VERSION_CMD, "", "show client version", new Options(), false);
+        parser.addCommand(ENTITY_CMD,
+                "",
+                "Entity opertions like submit, suspend, resume, delete, status, defintion, submitAndSchedule",
+                entityOptions(), false);
+        parser.addCommand(INSTANCE_CMD,
+                "",
+                "Process instances operations like running, status, kill, suspend, resume, rerun, logs",
+                instanceOptions(), false);
+
+        try {
+            CLIParser.Command command = parser.parse(args);
+            if (command.getName().equals(HELP_CMD)) {
+                parser.showHelp();
+            } else if (command.getName().equals(ADMIN_CMD)) {
+                adminCommand(command.getCommandLine());
+            } else if (command.getName().equals(ENTITY_CMD)) {
+                entityCommand(command.getCommandLine());
+            } else if (command.getName().equals(INSTANCE_CMD)) {
+                instanceCommand(command.getCommandLine());
+            }
+
+            return 0;
+        } catch (FalconCLIException ex) {
+            ERR.get().println("Error: " + ex.getMessage());
+            return -1;
+        } catch (ParseException ex) {
+            ERR.get().println("Invalid sub-command: " + ex.getMessage());
+            ERR.get().println();
+            ERR.get().println(parser.shortHelp());
+            return -1;
+        } catch (ClientHandlerException ex) {
+            ERR.get().print("Unable to connect to Falcon server, "
+                    + "please check if the URL is correct and Falcon server is up and running\n");
+            ERR.get().println(ex.getMessage());
+            return -1;
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            ERR.get().println(ex.getMessage());
+            return -1;
+        }
+    }
+
+    private void instanceCommand(CommandLine commandLine) throws FalconCLIException, IOException {
+        String falconUrl = getFalconEndpoint(commandLine);
+        FalconClient client = new FalconClient(falconUrl);
+
+        Set<String> optionsList = new HashSet<String>();
+        for (Option option : commandLine.getOptions()) {
+            optionsList.add(option.getOpt());
+        }
+
+        String result;
+        String type = commandLine.getOptionValue(ENTITY_TYPE_OPT);
+        String entity = commandLine.getOptionValue(ENTITY_NAME_OPT);
+        String start = commandLine.getOptionValue(START_OPT);
+        String end = commandLine.getOptionValue(END_OPT);
+        String filePath = commandLine.getOptionValue(FILE_PATH_OPT);
+        String runid = commandLine.getOptionValue(RUNID_OPT);
+        String colo = commandLine.getOptionValue(COLO_OPT);
+        String clusters = commandLine.getOptionValue(CLUSTERS_OPT);
+        String sourceClusters = commandLine.getOptionValue(SOURCECLUSTER_OPT);
+
+        colo = getColo(colo);
+
+        validateInstanceCommands(optionsList, entity, type, start, end,
+                filePath, colo, clusters, sourceClusters);
+
+        if (optionsList.contains(RUNNING_OPT)) {
+            result = client.getRunningInstances(type, entity, colo);
+        } else if (optionsList.contains(STATUS_OPT)) {
+            result = client.getStatusOfInstances(type, entity, start, end,
+                    runid, colo);
+        } else if (optionsList.contains(KILL_OPT)) {
+            result = client.killInstances(type, entity, start, end, colo, clusters, sourceClusters);
+        } else if (optionsList.contains(SUSPEND_OPT)) {
+            result = client.suspendInstances(type, entity, start, end, colo, clusters, sourceClusters);
+        } else if (optionsList.contains(RESUME_OPT)) {
+            result = client.resumeInstances(type, entity, start, end, colo, clusters, sourceClusters);
+        } else if (optionsList.contains(RERUN_OPT)) {
+            result = client.rerunInstances(type, entity, start, end, filePath, colo, clusters, sourceClusters);
+        } else if (optionsList.contains(CONTINUE_OPT)) {
+            result = client.rerunInstances(type, entity, start, end, colo, clusters, sourceClusters);
+        } else if (optionsList.contains(LOG_OPT)) {
+            result = client.getLogsOfInstances(type, entity, start, end, colo, runid);
+        } else {
+            throw new FalconCLIException("Invalid command");
+        }
+
+        OUT.get().println(result);
+    }
+
+    private void validateInstanceCommands(Set<String> optionsList, String entity, String type,
+                                          String start, String end, String filePath, String colo,
+                                          String clusters, String sourceClusters) throws FalconCLIException {
+
+        if (entity == null || entity.equals("")) {
+            throw new FalconCLIException("Missing argument: name");
+        }
+
+        if (type == null || type.equals("")) {
+            throw new FalconCLIException("Missing argument: type");
+        }
+
+        if (colo == null || colo.equals("")) {
+            throw new FalconCLIException("Missing argument: colo");
+        }
+
+        if (!optionsList.contains(RUNNING_OPT)) {
+            if (start == null || start.equals("")) {
+                throw new FalconCLIException("Missing argument: start");
+            }
+        }
+
+        if (optionsList.contains(CLUSTERS_OPT)) {
+            if (optionsList.contains(RUNNING_OPT)
+                    || optionsList.contains(LOG_OPT)
+                    || optionsList.contains(STATUS_OPT)) {
+                throw new FalconCLIException("Invalid argument: clusters");
+            }
+        }
+
+        if (optionsList.contains(SOURCECLUSTER_OPT)) {
+            if (optionsList.contains(RUNNING_OPT)
+                    || optionsList.contains(LOG_OPT)
+                    || optionsList.contains(STATUS_OPT) || !type.equals("feed")) {
+                throw new FalconCLIException("Invalid argument: sourceClusters");
+            }
+        }
+    }
+
+    private void entityCommand(CommandLine commandLine)
+            throws FalconCLIException, IOException {
+        String falconUrl = getFalconEndpoint(commandLine);
+        FalconClient client = new FalconClient(falconUrl);
+
+        Set<String> optionsList = new HashSet<String>();
+        for (Option option : commandLine.getOptions()) {
+            optionsList.add(option.getOpt());
+        }
+
+        String result = null;
+        String entityType = commandLine.getOptionValue(ENTITY_TYPE_OPT);
+        String entityName = commandLine.getOptionValue(ENTITY_NAME_OPT);
+        String filePath = commandLine.getOptionValue(FILE_PATH_OPT);
+        String colo = commandLine.getOptionValue(COLO_OPT);
+
+
+        validateEntityType(optionsList, entityType);
+
+        if (optionsList.contains(SUBMIT_OPT)) {
+            validateFilePath(optionsList, filePath);
+            validateColo(optionsList);
+            result = client.submit(entityType, filePath);
+        } else if (optionsList.contains(UPDATE_OPT)) {
+            validateFilePath(optionsList, filePath);
+            validateColo(optionsList);
+            validateEntityName(optionsList, entityName);
+            result = client.update(entityType, entityName, filePath);
+        } else if (optionsList.contains(SUBMIT_AND_SCHEDULE_OPT)) {
+            validateFilePath(optionsList, filePath);
+            validateColo(optionsList);
+            result = client.submitAndSchedule(entityType, filePath);
+        } else if (optionsList.contains(VALIDATE_OPT)) {
+            validateFilePath(optionsList, filePath);
+            validateColo(optionsList);
+            result = client.validate(entityType, filePath);
+        } else if (optionsList.contains(SCHEDULE_OPT)) {
+            validateEntityName(optionsList, entityName);
+            colo = getColo(colo);
+            result = client.schedule(entityType, entityName, colo);
+        } else if (optionsList.contains(SUSPEND_OPT)) {
+            validateEntityName(optionsList, entityName);
+            colo = getColo(colo);
+            result = client.suspend(entityType, entityName, colo);
+        } else if (optionsList.contains(RESUME_OPT)) {
+            validateEntityName(optionsList, entityName);
+            colo = getColo(colo);
+            result = client.resume(entityType, entityName, colo);
+        } else if (optionsList.contains(DELETE_OPT)) {
+            validateColo(optionsList);
+            validateEntityName(optionsList, entityName);
+            result = client.delete(entityType, entityName);
+        } else if (optionsList.contains(STATUS_OPT)) {
+            validateEntityName(optionsList, entityName);
+            colo = getColo(colo);
+            result = client.getStatus(entityType, entityName, colo);
+        } else if (optionsList.contains(DEFINITION_OPT)) {
+            validateColo(optionsList);
+            validateEntityName(optionsList, entityName);
+            result = client.getDefinition(entityType, entityName);
+        } else if (optionsList.contains(DEPENDENCY_OPT)) {
+            validateColo(optionsList);
+            validateEntityName(optionsList, entityName);
+            result = client.getDependency(entityType, entityName);
+        } else if (optionsList.contains(LIST_OPT)) {
+            validateColo(optionsList);
+            result = client.getEntityList(entityType);
+        } else if (optionsList.contains(HELP_CMD)) {
+            OUT.get().println("Falcon Help");
+        } else {
+            throw new FalconCLIException("Invalid command");
+        }
+        OUT.get().println(result);
+    }
+
+    private String getColo(String colo) throws FalconCLIException, IOException {
+        if (colo == null) {
+            Properties prop = getClientProperties();
+            colo = prop.getProperty(CURRENT_COLO, "*");
+        }
+        return colo;
+    }
+
+    private void validateFilePath(Set<String> optionsList, String filePath)
+            throws FalconCLIException {
+        if (filePath == null || filePath.equals("")) {
+            throw new FalconCLIException("Missing argument: file");
+        }
+    }
+
+    private void validateColo(Set<String> optionsList)
+            throws FalconCLIException {
+        if (optionsList.contains(COLO_OPT)) {
+            throw new FalconCLIException("Invalid argument : " + COLO_OPT);
+        }
+    }
+
+    private void validateEntityName(Set<String> optionsList, String entityName)
+            throws FalconCLIException {
+        if (entityName == null || entityName.equals("")) {
+            throw new FalconCLIException("Missing argument: name");
+        }
+    }
+
+    private void validateEntityType(Set<String> optionsList, String entityType)
+            throws FalconCLIException {
+        if (entityType == null || entityType.equals("")) {
+            throw new FalconCLIException("Missing argument: type");
+        }
+    }
+
+
+    private Options createAdminOptions() {
+        Options adminOptions = new Options();
+        Option url = new Option(URL_OPTION, true, "Falcon URL");
+        adminOptions.addOption(url);
+
+        OptionGroup group = new OptionGroup();
+        // Option status = new Option(STATUS_OPTION, false,
+        // "show the current system status");
+        Option version = new Option(VERSION_OPTION, false,
+                "show Falcon server build version");
+        Option help = new Option("help", false, "show Falcon help");
+        group.addOption(version);
+        group.addOption(help);
+
+        adminOptions.addOptionGroup(group);
+        return adminOptions;
+    }
+
+    private Options entityOptions() {
+
+        Options entityOptions = new Options();
+
+        Option submit = new Option(SUBMIT_OPT, false,
+                "Submits an entity xml to Falcon");
+        Option update = new Option(UPDATE_OPT, false,
+                "Updates an existing entity xml");
+        Option schedule = new Option(SCHEDULE_OPT, false,
+                "Schedules a submited entity in Falcon");
+        Option suspend = new Option(SUSPEND_OPT, false,
+                "Suspends a running entity in Falcon");
+        Option resume = new Option(RESUME_OPT, false,
+                "Resumes a suspended entity in Falcon");
+        Option delete = new Option(DELETE_OPT, false,
+                "Deletes an entity in Falcon, and kills its instance from workflow engine");
+        Option submitAndSchedule = new Option(SUBMIT_AND_SCHEDULE_OPT, false,
+                "Submits and entity to Falcon and schedules it immediately");
+        Option validate = new Option(VALIDATE_OPT, false,
+                "Validates an entity based on the entity type");
+        Option status = new Option(STATUS_OPT, false,
+                "Gets the status of entity");
+        Option definition = new Option(DEFINITION_OPT, false,
+                "Gets the Definition of entity");
+        Option dependency = new Option(DEPENDENCY_OPT, false,
+                "Gets the dependencies of entity");
+        Option list = new Option(LIST_OPT, false,
+                "List entities registerd for a type");
+
+        OptionGroup group = new OptionGroup();
+        group.addOption(submit);
+        group.addOption(update);
+        group.addOption(schedule);
+        group.addOption(suspend);
+        group.addOption(resume);
+        group.addOption(delete);
+        group.addOption(submitAndSchedule);
+        group.addOption(validate);
+        group.addOption(status);
+        group.addOption(definition);
+        group.addOption(dependency);
+        group.addOption(list);
+
+        Option url = new Option(URL_OPTION, true, "Falcon URL");
+        Option entityType = new Option(ENTITY_TYPE_OPT, true,
+                "Entity type, can be cluster, feed or process xml");
+        entityType.setRequired(true);
+        Option filePath = new Option(FILE_PATH_OPT, true,
+                "Path to entity xml file");
+        Option entityName = new Option(ENTITY_NAME_OPT, true,
+                "Entity type, can be cluster, feed or process xml");
         Option colo = new Option(COLO_OPT, true,
                 "Colo name");
         colo.setRequired(false);
 
-		entityOptions.addOption(url);
-		entityOptions.addOptionGroup(group);
-		entityOptions.addOption(entityType);
-		entityOptions.addOption(entityName);
-		entityOptions.addOption(filePath);
-		entityOptions.addOption(colo);
-
-		return entityOptions;
-
-	}
-
-	private Options instanceOptions() {
-
-		Options instanceOptions = new Options();
-
-		Option running = new Option(RUNNING_OPT, false,
-				"Gets running process instances for a given process");
-		Option status = new Option(
-				STATUS_OPT,
-				false,
-				"Gets status of process instances for a given process in the range start time and optional end time");
-		Option kill = new Option(
-				KILL_OPT,
-				false,
-				"Kills active process instances for a given process in the range start time and optional end time");
-		Option suspend = new Option(
-				SUSPEND_OPT,
-				false,
-				"Suspends active process instances for a given process in the range start time and optional end time");
-		Option resume = new Option(
-				RESUME_OPT,
-				false,
-				"Resumes suspended process instances for a given process in the range start time and optional end time");
-		Option rerun = new Option(
-				RERUN_OPT,
-				false,
-				"Reruns process instances for a given process in the range start time and optional end time and overrides properties present in job.properties file");
-		
-		Option continues = new Option(
-				CONTINUE_OPT,
-				false,
-				"resume process instance execution for a given process in the range start time and optional end time and overrides properties present in job.properties file");
-		
-		Option logs = new Option(
-				LOG_OPT,
-				false,
-				"Logs print the logs for process instances for a given process in the range start time and optional end time");
-		OptionGroup group = new OptionGroup();
-		group.addOption(running);
-		group.addOption(status);
-		group.addOption(kill);
-		group.addOption(resume);
-		group.addOption(suspend);
-		group.addOption(resume);
-		group.addOption(rerun);
-		group.addOption(logs);
-		group.addOption(continues);
-
-		Option url = new Option(URL_OPTION, true, "Falcon URL");
-		Option start = new Option(START_OPT, true,
-				"Start time is required for commands, status, kill, suspend, resume and re-run");
-		Option end = new Option(
-				END_OPT,
-				true,
-				"End time is optional for commands, status, kill, suspend, resume and re-run; if not specified then current time is considered as end time");
-		Option runid = new Option(RUNID_OPT, true,
-				"Instance runid  is optional and user can specify the runid, defaults to 0");
-		Option clusters = new Option(CLUSTERS_OPT, true,
-				"clusters is optional for commands kill, suspend and resume, should not be specified for other commands");
-		Option sourceClusters = new Option(SOURCECLUSTER_OPT, true,
-				" source cluster is optional for commands kill, suspend and resume, should not be specified for other commands (required for only feed)");
-		Option filePath = new Option(
-				FILE_PATH_OPT,
-				true,
-				"Path to job.properties file is required for rerun command, it should contain name=value pair for properties to override for rerun");
-		Option entityType = new Option(ENTITY_TYPE_OPT, true,
-				"Entity type, can be feed or process xml");
-		Option entityName = new Option(ENTITY_NAME_OPT, true,
-				"Entity type, can be feed or process xml");
+        entityOptions.addOption(url);
+        entityOptions.addOptionGroup(group);
+        entityOptions.addOption(entityType);
+        entityOptions.addOption(entityName);
+        entityOptions.addOption(filePath);
+        entityOptions.addOption(colo);
+
+        return entityOptions;
+    }
+
+    private Options instanceOptions() {
+
+        Options instanceOptions = new Options();
+
+        Option running = new Option(RUNNING_OPT, false,
+                "Gets running process instances for a given process");
+        Option status = new Option(
+                STATUS_OPT,
+                false,
+                "Gets status of process instances for a given process in the range start time and optional end time");
+        Option kill = new Option(
+                KILL_OPT,
+                false,
+                "Kills active process instances for a given process in the range start time and optional end time");
+        Option suspend = new Option(
+                SUSPEND_OPT,
+                false,
+                "Suspends active process instances for a given process in the range start time and optional end time");
+        Option resume = new Option(
+                RESUME_OPT,
+                false,
+                "Resumes suspended process instances for a given process "
+                        + "in the range start time and optional end time");
+        Option rerun = new Option(
+                RERUN_OPT,
+                false,
+                "Reruns process instances for a given process in the range start time and "
+                        + "optional end time and overrides properties present in job.properties file");
+
+        Option continues = new Option(
+                CONTINUE_OPT,
+                false,
+                "resume process instance execution for a given process in the range start time and "
+                        + "optional end time and overrides properties present in job.properties file");
+
+        Option logs = new Option(
+                LOG_OPT,
+                false,
+                "Logs print the logs for process instances for a given process in "
+                        + "the range start time and optional end time");
+
+        OptionGroup group = new OptionGroup();
+        group.addOption(running);
+        group.addOption(status);
+        group.addOption(kill);
+        group.addOption(resume);
+        group.addOption(suspend);
+        group.addOption(resume);
+        group.addOption(rerun);
+        group.addOption(logs);
+        group.addOption(continues);
+
+        Option url = new Option(URL_OPTION, true, "Falcon URL");
+        Option start = new Option(START_OPT, true,
+                "Start time is required for commands, status, kill, suspend, resume and re-run");
+        Option end = new Option(
+                END_OPT,
+                true,
+                "End time is optional for commands, status, kill, suspend, resume and re-run; "
+                        + "if not specified then current time is considered as end time");
+        Option runid = new Option(RUNID_OPT, true,
+                "Instance runid  is optional and user can specify the runid, defaults to 0");
+        Option clusters = new Option(CLUSTERS_OPT, true,
+                "clusters is optional for commands kill, suspend and resume, "
+                        + "should not be specified for other commands");
+        Option sourceClusters = new Option(SOURCECLUSTER_OPT, true,
+                " source cluster is optional for commands kill, suspend and resume, "
+                        + "should not be specified for other commands (required for only feed)");
+        Option filePath = new Option(
+                FILE_PATH_OPT,
+                true,
+                "Path to job.properties file is required for rerun command, "
+                        + "it should contain name=value pair for properties to override for rerun");
+        Option entityType = new Option(ENTITY_TYPE_OPT, true,
+                "Entity type, can be feed or process xml");
+        Option entityName = new Option(ENTITY_NAME_OPT, true,
+                "Entity type, can be feed or process xml");
         Option colo = new Option(COLO_OPT, true,
                 "Colo on which the cmd has to be executed");
 
-		instanceOptions.addOption(url);
-		instanceOptions.addOptionGroup(group);
-		instanceOptions.addOption(start);
-		instanceOptions.addOption(end);
-		instanceOptions.addOption(filePath);
-		instanceOptions.addOption(entityType);
-		instanceOptions.addOption(entityName);
-		instanceOptions.addOption(runid);
-		instanceOptions.addOption(clusters);
-		instanceOptions.addOption(sourceClusters);
-		instanceOptions.addOption(colo);
-		
-		return instanceOptions;
-
-	}
-
-	protected String getFalconEndpoint(CommandLine commandLine)
-			throws FalconCLIException, IOException {
-		String url = commandLine.getOptionValue(URL_OPTION);
-		if (url == null) {
-		    url = System.getenv(FALCON_URL);
-		}
-		if(url == null) {
+        instanceOptions.addOption(url);
+        instanceOptions.addOptionGroup(group);
+        instanceOptions.addOption(start);
+        instanceOptions.addOption(end);
+        instanceOptions.addOption(filePath);
+        instanceOptions.addOption(entityType);
+        instanceOptions.addOption(entityName);
+        instanceOptions.addOption(runid);
+        instanceOptions.addOption(clusters);
+        instanceOptions.addOption(sourceClusters);
+        instanceOptions.addOption(colo);
+
+        return instanceOptions;
+    }
+
+    protected String getFalconEndpoint(CommandLine commandLine) throws FalconCLIException, IOException {
+        String url = commandLine.getOptionValue(URL_OPTION);
+        if (url == null) {
+            url = System.getenv(FALCON_URL);
+        }
+        if (url == null) {
             Properties prop = getClientProperties();
-            if (prop.containsKey("falcon.url"))
+            if (prop.containsKey("falcon.url")) {
                 url = prop.getProperty("falcon.url");
-		}
-		if(url == null)
-		    throw new FalconCLIException("Failed to get falcon url from cmdline, or environment or client properties");
-		return url;
-	}
-
-	private void adminCommand(CommandLine commandLine) throws FalconCLIException, IOException {
-		String result;
-		String falconUrl = getFalconEndpoint(commandLine);
-		FalconClient client = new FalconClient(falconUrl);
-		
-		Set<String> optionsList = new HashSet<String>();
-		for (Option option : commandLine.getOptions()) {
-			optionsList.add(option.getOpt());
-		}
-		
-		if(optionsList.contains(STACK_OPTION)){
-			result = client.getThreadDump();
-			OUT_STREAM.println(result);
-		}
-		if (optionsList.contains(VERSION_OPTION)) {
-			result = client.getVersion();
-			OUT_STREAM.println("Falcon server build version: " + result);
-		}
-
-		else if (optionsList.contains(HELP_CMD)) {
-			OUT_STREAM.println("Falcon Help");
-		}
-	}
-	
-	private Properties getClientProperties() throws IOException
-	{
+            }
+        }
+        if (url == null) {
+            throw new FalconCLIException("Failed to get falcon url from cmdline, or environment or client properties");
+        }
+
+        return url;
+    }
+
+    private void adminCommand(CommandLine commandLine) throws FalconCLIException, IOException {
+        String result;
+        String falconUrl = getFalconEndpoint(commandLine);
+        FalconClient client = new FalconClient(falconUrl);
+
+        Set<String> optionsList = new HashSet<String>();
+        for (Option option : commandLine.getOptions()) {
+            optionsList.add(option.getOpt());
+        }
+
+        if (optionsList.contains(STACK_OPTION)) {
+            result = client.getThreadDump();
+            OUT.get().println(result);
+        }
+        if (optionsList.contains(VERSION_OPTION)) {
+            result = client.getVersion();
+            OUT.get().println("Falcon server build version: " + result);
+        } else if (optionsList.contains(HELP_CMD)) {
+            OUT.get().println("Falcon Help");
+        }
+    }
+
+    private Properties getClientProperties() throws IOException {
         Properties prop = new Properties();
         InputStream input = FalconCLI.class.getResourceAsStream(CLIENT_PROPERTIES);
-		if (input != null) {
-	        prop.load(input);
-		}
-		return prop;
-	}
-
+        if (input != null) {
+            prop.load(input);
+        }
+        return prop;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/client/src/main/java/org/apache/falcon/client/FalconCLIException.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconCLIException.java b/client/src/main/java/org/apache/falcon/client/FalconCLIException.java
index 367f5f5..ad13b4b 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconCLIException.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconCLIException.java
@@ -18,15 +18,15 @@
 
 package org.apache.falcon.client;
 
-import java.io.IOException;
-import java.io.InputStream;
-
 import com.sun.jersey.api.client.ClientResponse;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.InstancesResult;
 
+import java.io.IOException;
+import java.io.InputStream;
+
 /**
- * Exception thrown by FalconClient
+ * Exception thrown by FalconClient.
  */
 public class FalconCLIException extends Exception {
 
@@ -40,9 +40,9 @@ public class FalconCLIException extends Exception {
         super(msg, throwable);
     }
 
-	public static FalconCLIException fromReponse(ClientResponse clientResponse) {
+    public static FalconCLIException fromReponse(ClientResponse clientResponse) {
         return new FalconCLIException(getMessage(clientResponse));
-	}
+    }
 
     private static String getMessage(ClientResponse clientResponse) {
         String message;


[45/47] git commit: checkstyle related fixes for retention module.

Posted by sr...@apache.org.
checkstyle related fixes for retention module.


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

Branch: refs/heads/master
Commit: 4e39c7a960a7ed0dd2d4a934e462ca92be074dd7
Parents: 6a39baf
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:27:39 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:27:39 2013 -0700

----------------------------------------------------------------------
 .../org/apache/falcon/retention/FeedEvictor.java   |   51 ++++----
 .../apache/falcon/retention/FeedEvictorTest.java   |  101 ++++++---------
 2 files changed, 65 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/4e39c7a9/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 14c2c1c..155c847 100644
--- a/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
+++ b/retention/src/main/java/org/apache/falcon/retention/FeedEvictor.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -44,6 +44,7 @@ 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;
 
@@ -53,27 +54,25 @@ import java.util.regex.Pattern;
  */
 public class FeedEvictor extends Configured implements Tool {
 
-    private static Logger LOG = Logger.getLogger(FeedEvictor.class);
+    private static final Logger LOG = Logger.getLogger(FeedEvictor.class);
 
-    private static final ExpressionEvaluator EVALUATOR = new
-            ExpressionEvaluatorImpl();
-    private static final ExpressionHelper resolver = ExpressionHelper.get();
+    private static final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
+    private static final ExpressionHelper RESOLVER = ExpressionHelper.get();
 
-    static PrintStream stream = System.out;
+    public static final AtomicReference<PrintStream> OUT = new AtomicReference<PrintStream>(System.out);
+//    static PrintStream stream = System.out;
 
-    private static final String format = "yyyyMMddHHmm";
+    private static final String FORMAT = "yyyyMMddHHmm";
 
     public static void main(String[] args) throws Exception {
         Configuration conf = new Configuration();
         Path confPath = new Path("file:///" + System.getProperty("oozie.action.conf.xml"));
 
-        LOG.info(confPath + " found ? " +
-                confPath.getFileSystem(conf).exists(confPath));
+        LOG.info(confPath + " found ? " + confPath.getFileSystem(conf).exists(confPath));
         conf.addResource(confPath);
         int ret = ToolRunner.run(conf, new FeedEvictor(), args);
         if (ret != 0) {
-            throw new Exception("Unable to perform eviction action args: " +
-                    Arrays.toString(args));
+            throw new Exception("Unable to perform eviction action args: " + Arrays.toString(args));
         }
     }
 
@@ -98,12 +97,12 @@ public class FeedEvictor extends Configured implements Tool {
             evictor(path, retentionType, retentionLimit, timeZone, frequency, logFile);
         }
 
-        logInstancePaths(new Path(logFile), instancePaths.toString());
+        logInstancePaths(new Path(logFile));
         int len = buffer.length();
         if (len > 0) {
-            stream.println("instances=" + buffer.substring(0, len - 1));
+            OUT.get().println("instances=" + buffer.substring(0, len - 1));
         } else {
-            stream.println("instances=NULL");
+            OUT.get().println("instances=NULL");
         }
         return 0;
     }
@@ -124,7 +123,7 @@ public class FeedEvictor extends Configured implements Tool {
                 + retentionType + ", Limit: " + retentionLimit + ", timezone: "
                 + timeZone + ", frequency: " + frequency);
 
-        DateFormat dateFormat = new SimpleDateFormat(format);
+        DateFormat dateFormat = new SimpleDateFormat(FORMAT);
         dateFormat.setTimeZone(TimeZone.getTimeZone(timeZone));
         for (Path path : toBeDeleted) {
             if (deleteInstance(path)) {
@@ -137,27 +136,26 @@ public class FeedEvictor extends Configured implements Tool {
 
     }
 
-    private void logInstancePaths(Path path, String instancePaths) throws IOException {
+    private void logInstancePaths(Path path) throws IOException {
         LOG.info("Writing deleted instances to path " + path);
         OutputStream out = fs.create(path);
-        out.write(instancePaths.getBytes());
+        out.write(instancePaths.toString().getBytes());
         out.close();
         if (LOG.isDebugEnabled()) {
-            debug(path, fs);
+            debug(path);
         }
     }
 
     private Pair<Date, Date> getDateRange(String period) throws ELException {
         Long duration = (Long) EVALUATOR.evaluate("${" + period + "}",
-                Long.class, resolver, resolver);
+                Long.class, RESOLVER, RESOLVER);
         Date end = new Date();
         Date start = new Date(end.getTime() - duration);
         return Pair.of(start, end);
     }
 
     private List<Path> discoverInstanceToDelete(String inPath, String timeZone,
-                                                String dateMask, Date start)
-            throws IOException {
+                                                String dateMask, Date start) throws IOException {
 
         FileStatus[] files = findFilesForFeed(inPath);
         if (files == null || files.length == 0) {
@@ -187,8 +185,7 @@ public class FeedEvictor extends Configured implements Tool {
                 .replaceAll(VARS.MINUTE.regex(), "mm");
     }
 
-    private FileStatus[] findFilesForFeed(String feedBasePath)
-            throws IOException {
+    private FileStatus[] findFilesForFeed(String feedBasePath) throws IOException {
 
         Matcher matcher = FeedDataPath.PATTERN.matcher(feedBasePath);
         while (matcher.find()) {
@@ -234,7 +231,7 @@ public class FeedEvictor extends Configured implements Tool {
         }
 
         try {
-            DateFormat dateFormat = new SimpleDateFormat(format.
+            DateFormat dateFormat = new SimpleDateFormat(FORMAT.
                     substring(0, date.length()));
             dateFormat.setTimeZone(TimeZone.getTimeZone(timeZone));
             return dateFormat.parse(date);
@@ -268,7 +265,7 @@ public class FeedEvictor extends Configured implements Tool {
         return fs.delete(path, true);
     }
 
-    private void debug(Path outPath, FileSystem fs) throws IOException {
+    private void debug(Path outPath) throws IOException {
         ByteArrayOutputStream writer = new ByteArrayOutputStream();
         InputStream instance = fs.open(outPath);
         IOUtils.copyBytes(instance, writer, 4096, true);
@@ -276,8 +273,7 @@ public class FeedEvictor extends Configured implements Tool {
         LOG.debug("Written " + writer);
     }
 
-    private CommandLine getCommand(String[] args)
-            throws org.apache.commons.cli.ParseException {
+    private CommandLine getCommand(String[] args) throws org.apache.commons.cli.ParseException {
         Options options = new Options();
         Option opt;
         opt = new Option("feedBasePath", true,
@@ -304,5 +300,4 @@ public class FeedEvictor extends Configured implements Tool {
         options.addOption(opt);
         return new GnuParser().parse(options, args);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/4e39c7a9/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 66e99a6..c5a2013 100644
--- a/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
+++ b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -35,6 +35,9 @@ import java.text.SimpleDateFormat;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+/**
+ * Test for FeedEvictor.
+ */
 public class FeedEvictorTest {
 
     private EmbeddedCluster cluster;
@@ -44,7 +47,7 @@ public class FeedEvictorTest {
     @BeforeClass
     public void start() throws Exception {
         cluster = EmbeddedCluster.newCluster("test", false);
-        FeedEvictor.stream = stream;
+        FeedEvictor.OUT.set(stream);
     }
 
     @AfterClass
@@ -58,12 +61,14 @@ public class FeedEvictorTest {
             FeedEvictor.main(null);
             Assert.fail("Expected an exception to be thrown");
         } catch (Exception ignore) {
+            // ignore
         }
 
         try {
             FeedEvictor.main(new String[]{"1", "2"});
             Assert.fail("Expected an exception to be thrown");
         } catch (Exception ignore) {
+            // ignore
         }
     }
 
@@ -71,7 +76,8 @@ public class FeedEvictorTest {
     public void testEviction1() throws Exception {
         try {
             FeedEvictor.main(new String[]{"1", "2", "3", "4", "5", "6", "7"});
-        } catch (Exception e) {
+        } catch (Exception ignore) {
+            // ignore
         }
     }
 
@@ -89,18 +95,15 @@ public class FeedEvictorTest {
             String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
 
             FeedEvictor.main(new String[]{
-                    "-feedBasePath", cluster.getConf().get("fs.default.name")
-                    + dataPath,
-                    "-retentionType", "instance", "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency",
-                    "daily",
-                    "-logFile", logFile});
+                "-feedBasePath", cluster.getConf().get("fs.default.name") + dataPath,
+                "-retentionType", "instance", "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency",
+                "daily", "-logFile", logFile, });
 
             assertFailures(fs, pair);
             compare(map.get("feed1"), stream.getBuffer());
 
             Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
 
-
         } catch (Exception e) {
             Assert.fail("Unknown exception", e);
         }
@@ -170,15 +173,12 @@ public class FeedEvictorTest {
 
             Pair<List<String>, List<String>> pair;
             pair = createTestData("feed2", "yyyyMMddHH/'more'/yyyy", 5, TimeUnit.HOURS, "/data");
-            String dataPath = "/data/YYYY/feed2/mmHH/dd/MM/" +
-                    "?{YEAR}?{MONTH}?{DAY}?{HOUR}/more/?{YEAR}";
+            String dataPath = "/data/YYYY/feed2/mmHH/dd/MM/?{YEAR}?{MONTH}?{DAY}?{HOUR}/more/?{YEAR}";
             String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
             FeedEvictor.main(new String[]{
-                    "-feedBasePath", cluster.getConf().get("fs.default.name")
-                    + dataPath,
-                    "-retentionType", "instance", "-retentionLimit", "hours(5)", "-timeZone", "UTC", "-frequency",
-                    "hourly",
-                    "-logFile", logFile});
+                "-feedBasePath", cluster.getConf().get("fs.default.name") + dataPath,
+                "-retentionType", "instance", "-retentionLimit", "hours(5)", "-timeZone", "UTC", "-frequency",
+                "hourly", "-logFile", logFile, });
             assertFailures(fs, pair);
 
             compare(map.get("feed2"), stream.getBuffer());
@@ -201,25 +201,19 @@ public class FeedEvictorTest {
 
             Pair<List<String>, List<String>> pair;
             pair = createTestData("/data");
-            FeedEvictor.main(new String[]{
-                    "-feedBasePath",
-                    cluster.getConf().get("fs.default.name")
-                            + "/data/YYYY/feed3/dd/MM/"
-                            + "?{MONTH}/more/?{HOUR}", "-retentionType",
-                    "instance", "-retentionLimit", "months(5)", "-timeZone",
-                    "UTC", "-frequency", "hourly", "-logFile",
-                    "/falcon/staging/feed/2012-01-01-04-00"});
+            FeedEvictor.main(new String[] {
+                "-feedBasePath",
+                cluster.getConf().get("fs.default.name") + "/data/YYYY/feed3/dd/MM/?{MONTH}/more/?{HOUR}",
+                "-retentionType", "instance", "-retentionLimit", "months(5)", "-timeZone",
+                "UTC", "-frequency", "hourly", "-logFile", "/falcon/staging/feed/2012-01-01-04-00", });
             Assert.assertEquals("instances=NULL", stream.getBuffer());
 
             stream.clear();
-            String dataPath = "/data/YYYY/feed4/dd/MM/" +
-                    "02/more/hello";
+            String dataPath = "/data/YYYY/feed4/dd/MM/02/more/hello";
             String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
-            FeedEvictor.main(new String[]{"-feedBasePath",
-                                          cluster.getConf().get("fs.default.name") + dataPath,
-                                          "-retentionType", "instance", "-retentionLimit",
-                                          "hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
-                                          "-logFile", logFile});
+            FeedEvictor.main(new String[] {"-feedBasePath",
+                cluster.getConf().get("fs.default.name") + dataPath, "-retentionType", "instance",
+                "-retentionLimit", "hours(5)", "-timeZone", "UTC", "-frequency", "hourly", "-logFile", logFile, });
             Assert.assertEquals("instances=NULL", stream.getBuffer());
 
             Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
@@ -243,26 +237,23 @@ public class FeedEvictorTest {
             statsPair = createTestData("/stats");
             metaPair = createTestData("/meta");
             tmpPair = createTestData("/tmp");
-            FeedEvictor.main(new String[]{
-                    "-feedBasePath",
-                    getFeedBasePath(cluster, "/data") + "#"
-                            + getFeedBasePath(cluster, "/stats") + "#"
-                            + getFeedBasePath(cluster, "/meta") + "#"
-                            + getFeedBasePath(cluster, "/tmp"),
-                    "-retentionType", "instance", "-retentionLimit",
-                    "months(5)", "-timeZone", "UTC", "-frequency", "hourly",
-                    "-logFile", "/falcon/staging/feed/2012-01-01-04-00"});
+            FeedEvictor.main(new String[] {"-feedBasePath",
+                getFeedBasePath("/data") + "#"
+                    + getFeedBasePath("/stats") + "#"
+                    + getFeedBasePath("/meta") + "#"
+                    + getFeedBasePath("/tmp"),
+                "-retentionType", "instance", "-retentionLimit", "months(5)", "-timeZone",
+                "UTC", "-frequency", "hourly", "-logFile", "/falcon/staging/feed/2012-01-01-04-00", });
             Assert.assertEquals("instances=NULL", stream.getBuffer());
 
             stream.clear();
-            String dataPath = "/data/YYYY/feed4/dd/MM/" +
-                    "02/more/hello";
+            String dataPath = "/data/YYYY/feed4/dd/MM/02/more/hello";
             String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
             FeedEvictor.main(new String[]{"-feedBasePath",
                                           cluster.getConf().get("fs.default.name") + dataPath,
                                           "-retentionType", "instance", "-retentionLimit",
                                           "hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
-                                          "-logFile", logFile});
+                                          "-logFile", logFile, });
             Assert.assertEquals("instances=NULL", stream.getBuffer());
 
             Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
@@ -295,12 +286,8 @@ public class FeedEvictorTest {
                     + "/meta/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}";
             String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
 
-            FeedEvictor.main(new String[]{
-                    "-feedBasePath",
-                    dataPath,
-                    "-retentionType", "instance", "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency",
-                    "daily",
-                    "-logFile", logFile});
+            FeedEvictor.main(new String[] {"-feedBasePath", dataPath, "-retentionType", "instance",
+                "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency", "daily", "-logFile", logFile, });
 
             assertFailures(fs, pair);
 
@@ -333,12 +320,9 @@ public class FeedEvictorTest {
         return Pair.of(inRange, outOfRange);
     }
 
-    private Pair<List<String>, List<String>> createTestData(String feed,
-                                                            String mask,
-                                                            int period,
-                                                            TimeUnit timeUnit, String locationType)
-            throws Exception {
-
+    private Pair<List<String>, List<String>> createTestData(String feed, String mask,
+                                                            int period, TimeUnit timeUnit,
+                                                            String locationType) throws Exception {
         Configuration conf = cluster.getConf();
         FileSystem fs = FileSystem.get(conf);
 
@@ -370,12 +354,11 @@ public class FeedEvictorTest {
                 SimpleDateFormat(timeUnit == TimeUnit.HOURS ? "yyyyMMddHH" : "yyyyMMdd");
         displayFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
 
-        StringBuffer buffer = new StringBuffer();
+        StringBuilder buffer = new StringBuilder();
         for (long date = now;
              date > now - timeUnit.toMillis(range + 6);
              date -= timeUnit.toMillis(1)) {
-            String path = locationType + "/YYYY/" + feed + "/mmHH/dd/MM/" +
-                    format.format(date);
+            String path = locationType + "/YYYY/" + feed + "/mmHH/dd/MM/" + format.format(date);
             touch(fs, path);
             if (date <= now && date > now - timeUnit.toMillis(range)) {
                 outOfRange.add(path);
@@ -393,7 +376,7 @@ public class FeedEvictorTest {
         fs.create(new Path(path)).close();
     }
 
-    private String getFeedBasePath(EmbeddedCluster cluster, String locationType) {
+    private String getFeedBasePath(String locationType) {
         return cluster.getConf().get("fs.default.name")
                 + "/data/YYYY/feed3/dd/MM/"
                 + "?{MONTH}/more/?{HOUR}";


[33/47] git commit: More check style fixes relating to feed module

Posted by sr...@apache.org.
More check style fixes relating to feed module


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

Branch: refs/heads/master
Commit: 2a2fa50a3a1ec5138d1a6dd9b10342373432596d
Parents: 9bfdf3b
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:46:36 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:46:36 2013 +0530

----------------------------------------------------------------------
 .../falcon/workflow/OozieFeedWorkflowBuilder.java  |    2 +-
 1 files changed, 1 insertions(+), 1 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a2fa50a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
index 0cbdf77..7b9095f 100644
--- a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
+++ b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
@@ -63,7 +63,7 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
             return null;
         }
 
-        Cluster cluster = configStore.get(EntityType.CLUSTER, feedCluster.getName());
+        Cluster cluster = CONFIG_STORE.get(EntityType.CLUSTER, feedCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(feed));
         Feed feedClone = (Feed) feed.copy();
         EntityUtil.setStartDate(feedClone, clusterName, startDate);


[04/47] git commit: Check style plugin changes

Posted by sr...@apache.org.
Check style plugin changes


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

Branch: refs/heads/master
Commit: b047ce2b4af86e823bca01be0db2f9bd0c619be6
Parents: 2e147a6
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Tue Apr 16 10:04:21 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Tue Apr 16 10:04:21 2013 +0530

----------------------------------------------------------------------
 build-tools/pom.xml                                |   28 ++
 .../resources/falcon/checkstyle-java-header.txt    |   17 +
 .../main/resources/falcon/checkstyle-noframes.xsl  |  218 ++++++++++++++
 .../src/main/resources/falcon/checkstyle.xml       |  226 +++++++++++++++
 pom.xml                                            |   68 ++++-
 5 files changed, 548 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b047ce2b/build-tools/pom.xml
----------------------------------------------------------------------
diff --git a/build-tools/pom.xml b/build-tools/pom.xml
new file mode 100644
index 0000000..e8001b3
--- /dev/null
+++ b/build-tools/pom.xml
@@ -0,0 +1,28 @@
+<?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.
+  -->
+
+<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/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <groupId>org.apache.falcon</groupId>
+    <artifactId>build-tools</artifactId>
+    <version>0.2-SNAPSHOT</version>
+    <name>Build Tools</name>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b047ce2b/build-tools/src/main/resources/falcon/checkstyle-java-header.txt
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle-java-header.txt b/build-tools/src/main/resources/falcon/checkstyle-java-header.txt
new file mode 100644
index 0000000..5d5f1e3
--- /dev/null
+++ b/build-tools/src/main/resources/falcon/checkstyle-java-header.txt
@@ -0,0 +1,17 @@
+/**
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b047ce2b/build-tools/src/main/resources/falcon/checkstyle-noframes.xsl
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle-noframes.xsl b/build-tools/src/main/resources/falcon/checkstyle-noframes.xsl
new file mode 100644
index 0000000..6308ef8
--- /dev/null
+++ b/build-tools/src/main/resources/falcon/checkstyle-noframes.xsl
@@ -0,0 +1,218 @@
+<!--
+  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.
+  -->
+
+<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
+
+    <xsl:output method="html" indent="yes"/>
+    <xsl:decimal-format decimal-separator="." grouping-separator=","/>
+
+    <xsl:key name="files" match="file" use="@name"/>
+
+    <!-- Checkstyle XML Style Sheet by Stephane Bailliez <sb...@apache.org>         -->
+    <!-- Part of the Checkstyle distribution found at http://checkstyle.sourceforge.net -->
+    <!-- Usage (generates checkstyle_report.html):                                      -->
+    <!--    <checkstyle failonviolation="false" config="${check.config}">               -->
+    <!--      <fileset dir="${src.dir}" includes="**/*.java"/>                          -->
+    <!--      <formatter type="xml" toFile="${doc.dir}/checkstyle_report.xml"/>         -->
+    <!--    </checkstyle>                                                               -->
+    <!--    <style basedir="${doc.dir}" destdir="${doc.dir}"                            -->
+    <!--            includes="checkstyle_report.xml"                                    -->
+    <!--            style="${doc.dir}/checkstyle-noframes-sorted.xsl"/>                 -->
+
+    <xsl:template match="checkstyle">
+        <html>
+            <head>
+                <style type="text/css">
+                    .bannercell {
+                    border: 0px;
+                    padding: 0px;
+                    }
+                    body {
+                    margin-left: 10;
+                    margin-right: 10;
+                    font:normal 80% arial,helvetica,sanserif;
+                    background-color:#FFFFFF;
+                    color:#000000;
+                    }
+                    .a td {
+                    background: #efefef;
+                    }
+                    .b td {
+                    background: #fff;
+                    }
+                    th, td {
+                    text-align: left;
+                    vertical-align: top;
+                    }
+                    th {
+                    font-weight:bold;
+                    background: #ccc;
+                    color: black;
+                    }
+                    table, th, td {
+                    font-size:100%;
+                    border: none
+                    }
+                    table.log tr td, tr th {
+
+                    }
+                    h2 {
+                    font-weight:bold;
+                    font-size:140%;
+                    margin-bottom: 5;
+                    }
+                    h3 {
+                    font-size:100%;
+                    font-weight:bold;
+                    background: #525D76;
+                    color: white;
+                    text-decoration: none;
+                    padding: 5px;
+                    margin-right: 2px;
+                    margin-left: 2px;
+                    margin-bottom: 0;
+                    }
+                </style>
+            </head>
+            <body>
+                <a name="top"></a>
+                <!-- jakarta logo -->
+                <table border="0" cellpadding="0" cellspacing="0" width="100%">
+                    <tr>
+                        <td class="bannercell" rowspan="2">
+                            <!--a href="http://jakarta.apache.org/">
+                            <img src="http://jakarta.apache.org/images/jakarta-logo.gif" alt="http://jakarta.apache.org" align="left" border="0"/>
+                            </a-->
+                        </td>
+                        <td class="text-align:right">
+                            <h2>CheckStyle Audit</h2>
+                        </td>
+                    </tr>
+                    <tr>
+                        <td class="text-align:right">Designed for use with
+                            <a href='http://checkstyle.sourceforge.net/'>CheckStyle</a>
+                            and<a href='http://jakarta.apache.org'>Ant</a>.
+                        </td>
+                    </tr>
+                </table>
+                <hr size="1"/>
+
+                <!-- Summary part -->
+                <xsl:apply-templates select="." mode="summary"/>
+                <hr size="1" width="100%" align="left"/>
+
+                <!-- Package List part -->
+                <xsl:apply-templates select="." mode="filelist"/>
+                <hr size="1" width="100%" align="left"/>
+
+                <!-- For each package create its part -->
+                <xsl:apply-templates select="file[@name and generate-id(.) = generate-id(key('files', @name))]"/>
+
+                <hr size="1" width="100%" align="left"/>
+
+
+            </body>
+        </html>
+    </xsl:template>
+
+
+    <xsl:template match="checkstyle" mode="filelist">
+        <h3>Files</h3>
+        <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+            <tr>
+                <th>Name</th>
+                <th>Errors</th>
+            </tr>
+            <xsl:for-each select="file[@name and generate-id(.) = generate-id(key('files', @name))]">
+                <xsl:sort data-type="number" order="descending" select="count(key('files', @name)/error)"/>
+                <xsl:variable name="errorCount" select="count(error)"/>
+                <tr>
+                    <xsl:call-template name="alternated-row"/>
+                    <td>
+                        <a href="#f-{@name}">
+                            <xsl:value-of select="@name"/>
+                        </a>
+                    </td>
+                    <td>
+                        <xsl:value-of select="$errorCount"/>
+                    </td>
+                </tr>
+            </xsl:for-each>
+        </table>
+    </xsl:template>
+
+
+    <xsl:template match="file">
+        <a name="f-{@name}"></a>
+        <h3>File
+            <xsl:value-of select="@name"/>
+        </h3>
+
+        <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+            <tr>
+                <th>Error Description</th>
+                <th>Line</th>
+            </tr>
+            <xsl:for-each select="key('files', @name)/error">
+                <xsl:sort data-type="number" order="ascending" select="@line"/>
+                <tr>
+                    <xsl:call-template name="alternated-row"/>
+                    <td>
+                        <xsl:value-of select="@message"/>
+                    </td>
+                    <td>
+                        <xsl:value-of select="@line"/>
+                    </td>
+                </tr>
+            </xsl:for-each>
+        </table>
+        <a href="#top">Back to top</a>
+    </xsl:template>
+
+
+    <xsl:template match="checkstyle" mode="summary">
+        <h3>Summary</h3>
+        <xsl:variable name="fileCount"
+                      select="count(file[@name and generate-id(.) = generate-id(key('files', @name))])"/>
+        <xsl:variable name="errorCount" select="count(file/error)"/>
+        <table class="log" border="0" cellpadding="5" cellspacing="2" width="100%">
+            <tr>
+                <th>Files</th>
+                <th>Errors</th>
+            </tr>
+            <tr>
+                <xsl:call-template name="alternated-row"/>
+                <td>
+                    <xsl:value-of select="$fileCount"/>
+                </td>
+                <td>
+                    <xsl:value-of select="$errorCount"/>
+                </td>
+            </tr>
+        </table>
+    </xsl:template>
+
+    <xsl:template name="alternated-row">
+        <xsl:attribute name="class">
+            <xsl:if test="position() mod 2 = 1">a</xsl:if>
+            <xsl:if test="position() mod 2 = 0">b</xsl:if>
+        </xsl:attribute>
+    </xsl:template>
+</xsl:stylesheet>
+
+

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b047ce2b/build-tools/src/main/resources/falcon/checkstyle.xml
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle.xml b/build-tools/src/main/resources/falcon/checkstyle.xml
new file mode 100644
index 0000000..9854d6c
--- /dev/null
+++ b/build-tools/src/main/resources/falcon/checkstyle.xml
@@ -0,0 +1,226 @@
+<?xml version="1.0"?>
+<!--
+  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.
+  -->
+
+<!DOCTYPE module PUBLIC
+        "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
+        "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+
+<!--
+
+  Checkstyle configuration for Falcon that is based on the sun_checks.xml file
+  that is bundled with Checkstyle and includes checks for:
+
+    - the Java Language Specification at
+      http://java.sun.com/docs/books/jls/second_edition/html/index.html
+
+    - the Sun Code Conventions at http://java.sun.com/docs/codeconv/
+
+    - the Javadoc guidelines at
+      http://java.sun.com/j2se/javadoc/writingdoccomments/index.html
+
+    - the JDK Api documentation http://java.sun.com/j2se/docs/api/index.html
+
+    - some best practices
+
+  Checkstyle is very configurable. Be sure to read the documentation at
+  http://checkstyle.sf.net (or in your downloaded distribution).
+
+  Most Checks are configurable, be sure to consult the documentation.
+
+  To completely disable a check, just comment it out or delete it from the file.
+
+  Finally, it is worth reading the documentation.
+
+-->
+
+<module name="Checker">
+
+    <!-- Checks that a package.html file exists for each package.     -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
+    <!-- module name="PackageHtml"/ -->
+
+    <!-- Checks whether files end with a new line.                        -->
+    <!-- See http://checkstyle.sf.net/config_misc.html#NewlineAtEndOfFile -->
+    <module name="NewlineAtEndOfFile"/>
+
+    <!-- Checks for Headers                                -->
+    <!-- See http://checkstyle.sf.net/config_header.html   -->
+    <module name="Header">
+        <property name="headerFile" value="build-tools/src/main/resources/falcon/checkstyle-java-header.txt"/>
+    </module>
+
+    <module name="FileLength"/>
+    <module name="FileTabCharacter"/>
+
+    <module name="TreeWalker">
+        <!-- Checks for Javadoc comments.                     -->
+        <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+        <module name="JavadocType">
+            <property name="scope" value="public"/>
+            <property name="allowMissingParamTags" value="true"/>
+        </module>
+        <module name="JavadocStyle"/>
+
+        <module name="SuperClone"/>
+        <module name="SuperFinalize"/>
+
+        <!-- Checks for Naming Conventions.                  -->
+        <!-- See http://checkstyle.sf.net/config_naming.html -->
+        <module name="ConstantName"/>
+        <module name="ClassTypeParameterName">
+            <property name="format" value="^[A-Z]+$"/>
+        </module>
+        <module name="LocalFinalVariableName">
+            <property name="format" value="^[A-Z][_A-Z0-9]*$"/>
+        </module>
+        <module name="LocalVariableName"/>
+        <module name="MemberName"/>
+        <module name="MethodName"/>
+        <module name="MethodTypeParameterName">
+            <property name="format" value="^[A-Z]+$"/>
+        </module>
+        <module name="PackageName"/>
+        <module name="ParameterName"/>
+        <module name="StaticVariableName"/>
+        <module name="TypeName"/>
+
+        <!-- Checks for imports                              -->
+        <!-- See http://checkstyle.sf.net/config_import.html -->
+        <module name="IllegalImport"/>
+        <!-- defaults to sun.* packages -->
+        <module name="RedundantImport"/>
+        <module name="UnusedImports"/>
+
+
+        <!-- Checks for Size Violations.                    -->
+        <!-- See http://checkstyle.sf.net/config_sizes.html -->
+        <module name="LineLength">
+            <property name="max" value="120"/>
+        </module>
+        <module name="MethodLength"/>
+        <module name="ParameterNumber"/>
+        <module name="OuterTypeNumber"/>
+
+        <!-- Checks for whitespace                               -->
+        <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+        <module name="GenericWhitespace"/>
+        <module name="EmptyForIteratorPad"/>
+        <module name="MethodParamPad"/>
+        <module name="WhitespaceAround">
+            <property name="tokens" value="LITERAL_IF"/>
+        </module>
+        <module name="NoWhitespaceAfter">
+            <property name="tokens"
+                      value="BNOT, DEC, DOT, INC, LNOT, UNARY_MINUS, UNARY_PLUS"/>
+        </module>
+        <module name="NoWhitespaceBefore"/>
+        <module name="OperatorWrap"/>
+        <module name="ParenPad"/>
+        <module name="TypecastParenPad"/>
+        <module name="WhitespaceAfter">
+            <property name="tokens" value="COMMA, SEMI"/>
+        </module>
+
+        <module name="Regexp">
+            <property name="format" value="[ \t]+$"/>
+            <property name="illegalPattern" value="true"/>
+            <property name="message" value="Trailing whitespace"/>
+        </module>
+
+        <!-- Modifier Checks                                    -->
+        <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+        <module name="ModifierOrder"/>
+        <module name="RedundantModifier"/>
+
+
+        <!-- Checks for blocks. You know, those {}'s         -->
+        <!-- See http://checkstyle.sf.net/config_blocks.html -->
+        <module name="AvoidNestedBlocks"/>
+        <module name="EmptyBlock">
+            <!-- catch blocks need a statement or a comment. -->
+            <property name="option" value="text"/>
+            <property name="tokens" value="LITERAL_CATCH"/>
+        </module>
+        <module name="EmptyBlock">
+            <!-- all other blocks need a real statement. -->
+            <property name="option" value="stmt"/>
+            <property name="tokens" value="LITERAL_DO, LITERAL_ELSE, LITERAL_FINALLY,
+          LITERAL_IF, LITERAL_FOR, LITERAL_TRY, LITERAL_WHILE, INSTANCE_INIT,
+          STATIC_INIT"/>
+        </module>
+        <module name="LeftCurly"/>
+        <module name="NeedBraces"/>
+        <module name="RightCurly"/>
+
+
+        <!-- Checks for common coding problems               -->
+        <!-- See http://checkstyle.sf.net/config_coding.html -->
+        <!-- module name="AvoidInlineConditionals"/-->
+        <module name="DoubleCheckedLocking"/>
+        <module name="EmptyStatement"/>
+        <module name="EqualsHashCode"/>
+        <module name="StringLiteralEquality"/>
+        <module name="HiddenField">
+            <property name="ignoreConstructorParameter" value="true"/>
+        </module>
+        <module name="IllegalInstantiation"/>
+        <module name="InnerAssignment"/>
+        <module name="MissingSwitchDefault"/>
+        <module name="RedundantThrows"/>
+        <module name="SimplifyBooleanExpression"/>
+        <module name="SimplifyBooleanReturn"/>
+        <module name="DefaultComesLast"/>
+
+        <!-- Checks for class design                         -->
+        <!-- See http://checkstyle.sf.net/config_design.html -->
+        <module name="FinalClass"/>
+        <module name="HideUtilityClassConstructor"/>
+        <module name="InterfaceIsType"/>
+        <module name="VisibilityModifier">
+            <property name="protectedAllowed" value="true"/>
+        </module>
+        <module name="MissingOverride"/>
+
+
+        <!-- Miscellaneous other checks.                   -->
+        <!-- See http://checkstyle.sf.net/config_misc.html -->
+        <module name="ArrayTypeStyle"/>
+        <module name="ArrayTrailingComma"/>
+        <!--
+          This generates too many false-positives on wrapped 'throws' clauses
+          to be really useful. Disabled for now.
+
+          Falcon style is:
+          * Spaces, not tabs.
+          * Indent by four spaces.
+          * Indent by four spaces when wrapping a line.
+        -->
+        <module name="Indentation">
+            <property name="basicOffset" value="4"/>
+            <property name="caseIndent" value="0"/>
+        </module>
+        <module name="TodoComment"/>
+        <module name="UpperEll"/>
+
+        <module name="FileContentsHolder"/>
+    </module>
+
+    <!-- allow warnings to be suppressed -->
+    <module name="SuppressionCommentFilter"/>
+</module>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b047ce2b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4f67662..52d6ffb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -56,6 +56,7 @@
                 <module>prism</module>
                 <module>webapp</module>
                 <module>docs</module>
+                <module>build-tools</module>
             </modules>
         </profile>
     </profiles>
@@ -76,6 +77,7 @@
         <module>rerun</module>
         <module>prism</module>
         <module>webapp</module>
+        <module>build-tools</module>
     </modules>
 
     <scm>
@@ -84,11 +86,11 @@
     </scm>
 
     <distributionManagement>
-      <repository>
-        <id>internal.repo</id>
-        <name>Internal Repository</name>
-        <url>${internal.maven.repo}</url>
-      </repository>
+        <repository>
+            <id>internal.repo</id>
+            <name>Internal Repository</name>
+            <url>${internal.maven.repo}</url>
+        </repository>
     </distributionManagement>
 
     <licenses>
@@ -585,6 +587,16 @@
                     <artifactId>apache-rat-plugin</artifactId>
                     <version>0.7</version>
                 </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-checkstyle-plugin</artifactId>
+                    <version>2.9.1</version>
+                </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-site-plugin</artifactId>
+                    <version>3.2</version>
+                </plugin>
             </plugins>
         </pluginManagement>
 
@@ -624,7 +636,9 @@
                     <execution>
                         <id>attach-sources</id>
                         <phase>deploy</phase>
-                        <goals><goal>jar-no-fork</goal></goals>
+                        <goals>
+                            <goal>jar-no-fork</goal>
+                        </goals>
                     </execution>
                 </executions>
             </plugin>
@@ -635,7 +649,9 @@
                     <execution>
                         <id>attach-javadocs</id>
                         <phase>deploy</phase>
-                        <goals><goal>jar</goal></goals>
+                        <goals>
+                            <goal>jar</goal>
+                        </goals>
                     </execution>
                 </executions>
             </plugin>
@@ -653,7 +669,9 @@
                     <execution>
                         <id>deploy</id>
                         <phase>deploy</phase>
-                        <goals><goal>deploy</goal></goals>
+                        <goals>
+                            <goal>deploy</goal>
+                        </goals>
                     </execution>
                 </executions>
             </plugin>
@@ -661,10 +679,18 @@
                 <groupId>org.apache.rat</groupId>
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
+                    <useDefaultExcludes>true</useDefaultExcludes>
+                    <useMavenDefaultExcludes>true</useMavenDefaultExcludes>
+                    <useIdeaDefaultExcludes>true</useIdeaDefaultExcludes>
+                    <useEclipseDefaultExcludes>true</useEclipseDefaultExcludes>
+                    <excludeSubProjects>true</excludeSubProjects>
                     <excludes>
                         <exclude>.git/**</exclude>
                         <exclude>.idea/**</exclude>
-                        <exclude>**/*.twiki</exclude>
+                        <exclude>docs/src/site/twiki/*.twiki</exclude>
+                        <exclude>**/*.iml</exclude>
+                        <exclude>**/target/**</exclude>
+                        <exclude>**/activemq-data/**</exclude>
                     </excludes>
                 </configuration>
                 <executions>
@@ -677,6 +703,30 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.falcon</groupId>
+                        <artifactId>build-tools</artifactId>
+                        <version>0.2-SNAPSHOT</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                        <configuration>
+                            <consoleOutput>true</consoleOutput>
+                            <includeTestSourceDirectory>true</includeTestSourceDirectory>
+                            <configLocation>falcon/checkstyle.xml</configLocation>
+                            <failOnViolation>false</failOnViolation>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>


[09/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-process2.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-process2.xml b/process/src/test/resources/config/late/late-process2.xml
index a9d3576..bc507ad 100644
--- a/process/src/test/resources/config/late/late-process2.xml
+++ b/process/src/test/resources/config/late/late-process2.xml
@@ -16,8 +16,8 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<process name="late-process2"  xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<!-- where -->
+<process name="late-process2" xmlns="uri:falcon:process:0.1">
+    <!-- where -->
     <clusters>
         <cluster name="late-cluster">
             <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z"/>
@@ -32,25 +32,26 @@
 
     <!-- what -->
     <inputs>
-        <input name="impression" feed="late-feed1" start-instance="today(0,0)" end-instance="today(0,2)" /> 
-        <input name="clicks" feed="late-feed2" start-instance="yesterday(0,0)" end-instance="today(0,0)" partition="*/US"/>
+        <input name="impression" feed="late-feed1" start-instance="today(0,0)" end-instance="today(0,2)"/>
+        <input name="clicks" feed="late-feed2" start-instance="yesterday(0,0)" end-instance="today(0,0)"
+               partition="*/US"/>
     </inputs>
 
     <outputs>
-        <output name="clicksummary" feed="late-feed3" instance="today(0,0)" />
+        <output name="clicksummary" feed="late-feed3" instance="today(0,0)"/>
     </outputs>
 
     <!-- how -->
     <properties>
-    	<property name="procprop" value="procprop"/>
+        <property name="procprop" value="procprop"/>
     </properties>
-    
-    <workflow engine="oozie" path="/user/guest/workflow" />
 
-    <retry policy="periodic" delay="hours(10)" attempts="3" />
+    <workflow engine="oozie" path="/user/guest/workflow"/>
+
+    <retry policy="periodic" delay="hours(10)" attempts="3"/>
 
     <late-process policy="exp-backoff" delay="hours(1)">
-        <late-input feed="impression" workflow-path="hdfs://impression/late/workflow" />
-        <late-input feed="clicks" workflow-path="hdfs://clicks/late/workflow" />
+        <late-input feed="impression" workflow-path="hdfs://impression/late/workflow"/>
+        <late-input feed="clicks" workflow-path="hdfs://clicks/late/workflow"/>
     </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/process/process-0.1.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/process/process-0.1.xml b/process/src/test/resources/config/process/process-0.1.xml
index b5a7f50..91d5e0f 100644
--- a/process/src/test/resources/config/process/process-0.1.xml
+++ b/process/src/test/resources/config/process/process-0.1.xml
@@ -4,11 +4,11 @@
     License. You may obtain a copy of the License at ~ ~ http://www.apache.org/licenses/LICENSE-2.0 ~ ~ Unless required by applicable law or agreed to in writing, software ~ distributed 
     under the License is distributed on an "AS IS" BASIS, ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ~ See the License for the specific language governing 
     permissions and ~ limitations under the License. -->
-<process name="sample" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample" xmlns="uri:falcon:process:0.1">
     <!-- where -->
     <clusters>
         <cluster name="corp">
-            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z" />
+            <validity start="2011-11-02T00:00Z" end="2011-12-30T00:00Z"/>
         </cluster>
     </clusters>
 
@@ -20,26 +20,26 @@
 
     <!-- what -->
     <inputs>
-        <input name="impression" feed="impressions" start="today(0,0)" end="today(0,2)" />
-        <input name="click" feed="clicks" start="yesterday(0,0)" end="latest(0)" partition="*/US" />
+        <input name="impression" feed="impressions" start="today(0,0)" end="today(0,2)"/>
+        <input name="click" feed="clicks" start="yesterday(0,0)" end="latest(0)" partition="*/US"/>
     </inputs>
 
     <outputs>
-        <output name="clicksummary" feed="impressions" instance="today(0,0)" />
+        <output name="clicksummary" feed="impressions" instance="today(0,0)"/>
     </outputs>
 
     <!-- how -->
     <properties>
-        <property name="procprop" value="procprop" />
-        <property name="mapred.job.priority" value="LOW" />
+        <property name="procprop" value="procprop"/>
+        <property name="mapred.job.priority" value="LOW"/>
     </properties>
 
-    <workflow engine="oozie" path="/user/guest/workflow" />
+    <workflow engine="oozie" path="/user/guest/workflow"/>
 
-    <retry policy="periodic" delay="hours(10)" attempts="3" />
+    <retry policy="periodic" delay="hours(10)" attempts="3"/>
 
     <late-process policy="exp-backoff" delay="hours(1)">
-        <late-input input="impression" workflow-path="hdfs://impression/late/workflow" />
-        <late-input input="clicks" workflow-path="hdfs://clicks/late/workflow" />
+        <late-input input="impression" workflow-path="hdfs://impression/late/workflow"/>
+        <late-input input="clicks" workflow-path="hdfs://clicks/late/workflow"/>
     </late-process>
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java b/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
index 969a59c..7dfd406 100644
--- a/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
+++ b/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
@@ -30,6 +30,7 @@ import java.io.IOException;
 public class CustomReplicator extends DistCp {
 
     private static Logger LOG = Logger.getLogger(CustomReplicator.class);
+
     /**
      * Public Constructor. Creates DistCp object with specified input-parameters.
      * (E.g. source-paths, target-location, etc.)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
index a693d75..fc0b5ac 100644
--- a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
+++ b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
@@ -17,16 +17,7 @@
  */
 package org.apache.falcon.replication;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Pattern;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -39,31 +30,36 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Logger;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
 public class FeedReplicator extends Configured implements Tool {
 
-	private static Logger LOG = Logger.getLogger(FeedReplicator.class);
+    private static Logger LOG = Logger.getLogger(FeedReplicator.class);
 
     public static void main(String[] args) throws Exception {
-		ToolRunner.run(new Configuration(), new FeedReplicator(), args);
-	}
+        ToolRunner.run(new Configuration(), new FeedReplicator(), args);
+    }
 
-	@Override
-	public int run(String[] args) throws Exception {
+    @Override
+    public int run(String[] args) throws Exception {
 
         DistCpOptions options = getDistCpOptions(args);
-        
+
         Configuration conf = this.getConf();
-		// inject wf configs
-		Path confPath = new Path("file:///"
-				+ System.getProperty("oozie.action.conf.xml"));
-
-		LOG.info(confPath + " found conf ? "
-				+ confPath.getFileSystem(conf).exists(confPath));
-		conf.addResource(confPath);
-        
-		DistCp distCp = new CustomReplicator(conf, options);
-		LOG.info("Started DistCp");
-		distCp.execute();
+        // inject wf configs
+        Path confPath = new Path("file:///"
+                + System.getProperty("oozie.action.conf.xml"));
+
+        LOG.info(confPath + " found conf ? "
+                + confPath.getFileSystem(conf).exists(confPath));
+        conf.addResource(confPath);
+
+        DistCp distCp = new CustomReplicator(conf, options);
+        LOG.info("Started DistCp");
+        distCp.execute();
 
         Path targetPath = options.getTargetPath();
         FileSystem fs = targetPath.getFileSystem(getConf());
@@ -79,28 +75,30 @@ public class FeedReplicator extends Configured implements Tool {
         String fixedPath = getFixedPath(relativePath);
 
         FileStatus[] files = fs.globStatus(new Path(targetPath.toString() + "/" + fixedPath));
-		if (files != null) {
-			for (FileStatus file : files) {
-            fs.create(new Path(file.getPath(), FileOutputCommitter.SUCCEEDED_FILE_NAME)).close();
-            LOG.info("Created " + new Path(file.getPath(), FileOutputCommitter.SUCCEEDED_FILE_NAME));
-			}
-		} else {
-			LOG.info("No files present in path: "
-					+ new Path(targetPath.toString() + "/" + fixedPath)
-							.toString());
-		}
-		LOG.info("Completed DistCp");
-		return 0;
-	}
+        if (files != null) {
+            for (FileStatus file : files) {
+                fs.create(new Path(file.getPath(), FileOutputCommitter.SUCCEEDED_FILE_NAME)).close();
+                LOG.info("Created " + new Path(file.getPath(), FileOutputCommitter.SUCCEEDED_FILE_NAME));
+            }
+        } else {
+            LOG.info("No files present in path: "
+                    + new Path(targetPath.toString() + "/" + fixedPath)
+                    .toString());
+        }
+        LOG.info("Completed DistCp");
+        return 0;
+    }
 
     private String getFixedPath(String relativePath) throws IOException {
         String[] patterns = relativePath.split("/");
         int part = patterns.length - 1;
         for (int index = patterns.length - 1; index >= 0; index--) {
             String pattern = patterns[index];
-            if (pattern.isEmpty()) continue;
+            if (pattern.isEmpty()) {
+                continue;
+            }
             Pattern r = FilteredCopyListing.getRegEx(pattern);
-            if (!r.toString().equals("(" + pattern + "/)|(" + pattern + "$)"))  {
+            if (!r.toString().equals("(" + pattern + "/)|(" + pattern + "$)")) {
                 continue;
             }
             part = index;
@@ -114,42 +112,42 @@ public class FeedReplicator extends Configured implements Tool {
     }
 
     public DistCpOptions getDistCpOptions(String[] args) throws ParseException {
-		Options options = new Options();
-		Option opt;
-		opt = new Option("maxMaps", true,
-				"max number of maps to use for this copy");
-		opt.setRequired(true);
-		options.addOption(opt);
+        Options options = new Options();
+        Option opt;
+        opt = new Option("maxMaps", true,
+                "max number of maps to use for this copy");
+        opt.setRequired(true);
+        options.addOption(opt);
 
         opt = new Option("sourcePaths", true,
-				"comma separtated list of source paths to be copied");
-		opt.setRequired(true);
-		options.addOption(opt);
+                "comma separtated list of source paths to be copied");
+        opt.setRequired(true);
+        options.addOption(opt);
 
         opt = new Option("targetPath", true, "target path");
-		opt.setRequired(true);
-		options.addOption(opt);
+        opt.setRequired(true);
+        options.addOption(opt);
 
-		CommandLine cmd = new GnuParser().parse(options, args);
-		String[] paths = cmd.getOptionValue("sourcePaths").trim().split(",");
-		List<Path> srcPaths = getPaths(paths);
-		String trgPath = cmd.getOptionValue("targetPath").trim();
+        CommandLine cmd = new GnuParser().parse(options, args);
+        String[] paths = cmd.getOptionValue("sourcePaths").trim().split(",");
+        List<Path> srcPaths = getPaths(paths);
+        String trgPath = cmd.getOptionValue("targetPath").trim();
 
-		DistCpOptions distcpOptions = new DistCpOptions(srcPaths, new Path(
-				trgPath));
+        DistCpOptions distcpOptions = new DistCpOptions(srcPaths, new Path(
+                trgPath));
         distcpOptions.setSyncFolder(true);
-		distcpOptions.setBlocking(true);
-		distcpOptions
-				.setMaxMaps(Integer.valueOf(cmd.getOptionValue("maxMaps")));
-
-		return distcpOptions;
-	}
-
-	private List<Path> getPaths(String[] paths) {
-		List<Path> listPaths = new ArrayList<Path>();
-		for (String path : paths) {
-			listPaths.add(new Path(path));
-		}
-		return listPaths;
-	}
+        distcpOptions.setBlocking(true);
+        distcpOptions
+                .setMaxMaps(Integer.valueOf(cmd.getOptionValue("maxMaps")));
+
+        return distcpOptions;
+    }
+
+    private List<Path> getPaths(String[] paths) {
+        List<Path> listPaths = new ArrayList<Path>();
+        for (String path : paths) {
+            listPaths.add(new Path(path));
+        }
+        return listPaths;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java b/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
index e1b6276..c1698e3 100644
--- a/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
+++ b/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
@@ -32,12 +32,18 @@ import java.util.regex.Pattern;
 public class FilteredCopyListing extends SimpleCopyListing {
     private static final Logger LOG = Logger.getLogger(FilteredCopyListing.class);
 
-    /** Default pattern character: Escape any special meaning. */
-    private static final char  PAT_ESCAPE = '\\';
-    /** Default pattern character: Any single character. */
-    private static final char  PAT_ANY = '.';
-    /** Default pattern character: Character set close. */
-    private static final char  PAT_SET_CLOSE = ']';
+    /**
+     * Default pattern character: Escape any special meaning.
+     */
+    private static final char PAT_ESCAPE = '\\';
+    /**
+     * Default pattern character: Any single character.
+     */
+    private static final char PAT_ANY = '.';
+    /**
+     * Default pattern character: Character set close.
+     */
+    private static final char PAT_SET_CLOSE = ']';
 
     private Pattern regex;
 
@@ -55,7 +61,9 @@ public class FilteredCopyListing extends SimpleCopyListing {
 
     @Override
     protected boolean shouldCopy(Path path, DistCpOptions options) {
-        if (path.getName().equals(FileOutputCommitter.SUCCEEDED_FILE_NAME)) return false;
+        if (path.getName().equals(FileOutputCommitter.SUCCEEDED_FILE_NAME)) {
+            return false;
+        }
         return regex == null || regex.matcher(path.toString()).find();
     }
 
@@ -74,8 +82,9 @@ public class FilteredCopyListing extends SimpleCopyListing {
 
         // Validate the pattern
         len = filePattern.length();
-        if (len == 0)
+        if (len == 0) {
             return null;
+        }
 
         setOpen = 0;
         setRange = false;
@@ -89,8 +98,9 @@ public class FilteredCopyListing extends SimpleCopyListing {
             if (pCh == PAT_ESCAPE) {
                 fileRegex.append(pCh);
                 i++;
-                if (i >= len)
+                if (i >= len) {
                     error("An escaped character does not present", filePattern, i);
+                }
                 pCh = filePattern.charAt(i);
             } else if (isJavaRegexSpecialChar(pCh)) {
                 fileRegex.append(PAT_ESCAPE);
@@ -121,8 +131,9 @@ public class FilteredCopyListing extends SimpleCopyListing {
                 error("Incomplete character set range", filePattern, i);
             } else if (pCh == PAT_SET_CLOSE && setOpen > 0) {
                 // End of a character set
-                if (setOpen < 2)
+                if (setOpen < 2) {
                     error("Unexpected end of set", filePattern, i);
+                }
                 setOpen = 0;
             } else if (setOpen > 0) {
                 // Normal character, or the end of a character set range
@@ -143,7 +154,7 @@ public class FilteredCopyListing extends SimpleCopyListing {
 
     private static void error(String s, String pattern, int pos) throws IOException {
         throw new IOException("Illegal file pattern: "
-                +s+ " for glob "+ pattern + " at " + pos);
+                + s + " for glob " + pattern + " at " + pos);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
----------------------------------------------------------------------
diff --git a/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java b/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
index f2ccfd8..1935e51 100644
--- a/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
+++ b/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
@@ -17,9 +17,9 @@
  */
 package org.apache.falcon.repliation;
 
+import org.apache.falcon.replication.FeedReplicator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.DistCpOptions;
-import org.apache.falcon.replication.FeedReplicator;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -37,9 +37,9 @@ public class FeedReplicatorTest {
          * <arg>-targetPath</arg><arg>${distcpTargetPaths}</arg>
          */
         FeedReplicator replicator = new FeedReplicator();
-        DistCpOptions options = replicator.getDistCpOptions(new String[] { "true", "-maxMaps", "5", "-sourcePaths",
-                "hdfs://localhost:8020/tmp/", "-targetPath",
-                "hdfs://localhost1:8020/tmp/" });
+        DistCpOptions options = replicator.getDistCpOptions(new String[]{"true", "-maxMaps", "5", "-sourcePaths",
+                                                                         "hdfs://localhost:8020/tmp/", "-targetPath",
+                                                                         "hdfs://localhost1:8020/tmp/"});
 
         List<Path> srcPaths = new ArrayList<Path>();
         srcPaths.add(new Path("hdfs://localhost:8020/tmp/"));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
----------------------------------------------------------------------
diff --git a/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java b/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
index dc17cc2..5054bf8 100644
--- a/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
+++ b/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
@@ -34,7 +34,9 @@ import org.testng.annotations.Test;
 
 import java.io.DataOutputStream;
 import java.net.URI;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 
 public class FilteredCopyListingTest {
 
@@ -69,8 +71,7 @@ public class FilteredCopyListingTest {
             fileSystem = FileSystem.getLocal(new Configuration());
             fileSystem.mkdirs(new Path(path));
             recordInExpectedValues(path);
-        }
-        finally {
+        } finally {
             IOUtils.cleanup(null, fileSystem);
         }
     }
@@ -80,8 +81,7 @@ public class FilteredCopyListingTest {
         try {
             fileSystem = FileSystem.getLocal(new Configuration());
             fileSystem.delete(new Path(path), true);
-        }
-        finally {
+        } finally {
             IOUtils.cleanup(null, fileSystem);
         }
     }
@@ -93,8 +93,7 @@ public class FilteredCopyListingTest {
             fileSystem = FileSystem.getLocal(new Configuration());
             outputStream = fileSystem.create(new Path(path), true, 10);
             recordInExpectedValues(path);
-        }
-        finally {
+        } finally {
             IOUtils.cleanup(null, fileSystem, outputStream);
         }
     }
@@ -209,7 +208,7 @@ public class FilteredCopyListingTest {
     private void verifyContents(Path listingPath, int expected) throws Exception {
         SequenceFile.Reader reader = new SequenceFile.Reader(FileSystem.getLocal(new Configuration()),
                 listingPath, new Configuration());
-        Text key   = new Text();
+        Text key = new Text();
         FileStatus value = new FileStatus();
         Map<String, String> actualValues = new HashMap<String, String>();
         while (reader.next(key, value)) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
index 73dc882..00a2d87 100644
--- a/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/latedata/LateDataHandler.java
@@ -18,19 +18,7 @@
 
 package org.apache.falcon.latedata;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -40,131 +28,139 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Logger;
 
+import java.io.*;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
 public class LateDataHandler extends Configured implements Tool {
 
-	private static Logger LOG = Logger.getLogger(LateDataHandler.class);
-
-	static PrintStream stream = System.out;
-
-	public static void main(String[] args) throws Exception {
-		Configuration conf = new Configuration();
-		Path confPath = new Path("file:///"
-				+ System.getProperty("oozie.action.conf.xml"));
-
-		LOG.info(confPath + " found ? "
-				+ confPath.getFileSystem(conf).exists(confPath));
-		conf.addResource(confPath);
-		ToolRunner.run(conf, new LateDataHandler(), args);
-	}
-
-	private static CommandLine getCommand(String[] args) throws ParseException {
-		Options options = new Options();
-
-		Option opt = new Option("out", true, "Out file name");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("paths", true,
-				"Comma separated path list, further separated by #");
-		opt.setRequired(true);
-		options.addOption(opt);
-		opt = new Option("falconInputFeeds", true,
-				"Input feed names, further separated by #");
-		opt.setRequired(true);
-		options.addOption(opt);
-
-		return new GnuParser().parse(options, args);
-	}
-
-	@Override
-	public int run(String[] args) throws Exception {
-
-		CommandLine command = getCommand(args);
-
-		Path file = new Path(command.getOptionValue("out"));
-		Map<String, Long> map = new LinkedHashMap<String, Long>();
-		String pathStr = getOptionValue(command, "paths");
-		if(pathStr == null)
-		    return 0;
-		
-		String[] pathGroups = pathStr.split("#");
-		String[] inputFeeds = getOptionValue(command, "falconInputFeeds").split(
-				"#");
-		for (int index = 0; index < pathGroups.length; index++) {
-			long usage = 0;
-			for (String pathElement : pathGroups[index].split(",")) {
-				Path inPath = new Path(pathElement);
-				usage += usage(inPath, getConf());
-			}
-			map.put(inputFeeds[index], usage);
-		}
-		LOG.info("MAP data: " + map);
-
-		OutputStream out = file.getFileSystem(getConf()).create(file);
-		for (Map.Entry<String, Long> entry : map.entrySet()) {
-			out.write((entry.getKey() + "=" + entry.getValue() + "\n")
-					.getBytes());
-		}
-		out.close();
-		return 0;
-	}
-
-	private String getOptionValue(CommandLine command, String option) {
-	    String value = command.getOptionValue(option);
-	    if(value.equals("null"))
-	        return null;
-	    return value;
-	}
-	
-	public String detectChanges(Path file, Map<String, Long> map, Configuration conf)
-			throws Exception {
-
-		StringBuffer buffer = new StringBuffer();
-		BufferedReader in = new BufferedReader(new InputStreamReader(file
-				.getFileSystem(conf).open(file)));
-		String line;
-		try {
-			Map<String, Long> recorded = new LinkedHashMap<String, Long>();
-			while ((line = in.readLine()) != null) {
-				if (line.isEmpty())
-					continue;
-				int index = line.indexOf('=');
-				String key = line.substring(0, index);
-				long size = Long.parseLong(line.substring(index + 1));
-				recorded.put(key, size);
-			}
-
-			for (Map.Entry<String, Long> entry : map.entrySet()) {
-				if (recorded.get(entry.getKey()) == null) {
-					LOG.info("No matching key " + entry.getKey());
-					continue;
-				}
-				if (!recorded.get(entry.getKey()).equals(entry.getValue())) {
-					LOG.info("Recorded size:"+recorded.get(entry.getKey())+"  is different from new size" + entry.getValue());
-					buffer.append(entry.getKey()).append(',');
-				}
-			}
-			if (buffer.length() == 0) {
-				return "";
-			} else {
-				return buffer.substring(0, buffer.length() - 1);
-			}
-
-		} finally {
-			in.close();
-		}
-
-	}
-
-	public long usage(Path inPath, Configuration conf) throws IOException {
-		FileSystem fs = inPath.getFileSystem(conf);
-		FileStatus status[] = fs.globStatus(inPath);
-		if (status == null || status.length == 0) {
-			return 0;
-		}
-		long totalSize = 0;
-		for (FileStatus statu : status) {
-			totalSize += fs.getContentSummary(statu.getPath()).getLength();
-		}
-		return totalSize;
-	}
+    private static Logger LOG = Logger.getLogger(LateDataHandler.class);
+
+    static PrintStream stream = System.out;
+
+    public static void main(String[] args) throws Exception {
+        Configuration conf = new Configuration();
+        Path confPath = new Path("file:///"
+                + System.getProperty("oozie.action.conf.xml"));
+
+        LOG.info(confPath + " found ? "
+                + confPath.getFileSystem(conf).exists(confPath));
+        conf.addResource(confPath);
+        ToolRunner.run(conf, new LateDataHandler(), args);
+    }
+
+    private static CommandLine getCommand(String[] args) throws ParseException {
+        Options options = new Options();
+
+        Option opt = new Option("out", true, "Out file name");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("paths", true,
+                "Comma separated path list, further separated by #");
+        opt.setRequired(true);
+        options.addOption(opt);
+        opt = new Option("falconInputFeeds", true,
+                "Input feed names, further separated by #");
+        opt.setRequired(true);
+        options.addOption(opt);
+
+        return new GnuParser().parse(options, args);
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+        CommandLine command = getCommand(args);
+
+        Path file = new Path(command.getOptionValue("out"));
+        Map<String, Long> map = new LinkedHashMap<String, Long>();
+        String pathStr = getOptionValue(command, "paths");
+        if (pathStr == null) {
+            return 0;
+        }
+
+        String[] pathGroups = pathStr.split("#");
+        String[] inputFeeds = getOptionValue(command, "falconInputFeeds").split(
+                "#");
+        for (int index = 0; index < pathGroups.length; index++) {
+            long usage = 0;
+            for (String pathElement : pathGroups[index].split(",")) {
+                Path inPath = new Path(pathElement);
+                usage += usage(inPath, getConf());
+            }
+            map.put(inputFeeds[index], usage);
+        }
+        LOG.info("MAP data: " + map);
+
+        OutputStream out = file.getFileSystem(getConf()).create(file);
+        for (Map.Entry<String, Long> entry : map.entrySet()) {
+            out.write((entry.getKey() + "=" + entry.getValue() + "\n")
+                    .getBytes());
+        }
+        out.close();
+        return 0;
+    }
+
+    private String getOptionValue(CommandLine command, String option) {
+        String value = command.getOptionValue(option);
+        if (value.equals("null")) {
+            return null;
+        }
+        return value;
+    }
+
+    public String detectChanges(Path file, Map<String, Long> map, Configuration conf)
+            throws Exception {
+
+        StringBuffer buffer = new StringBuffer();
+        BufferedReader in = new BufferedReader(new InputStreamReader(file
+                .getFileSystem(conf).open(file)));
+        String line;
+        try {
+            Map<String, Long> recorded = new LinkedHashMap<String, Long>();
+            while ((line = in.readLine()) != null) {
+                if (line.isEmpty()) {
+                    continue;
+                }
+                int index = line.indexOf('=');
+                String key = line.substring(0, index);
+                long size = Long.parseLong(line.substring(index + 1));
+                recorded.put(key, size);
+            }
+
+            for (Map.Entry<String, Long> entry : map.entrySet()) {
+                if (recorded.get(entry.getKey()) == null) {
+                    LOG.info("No matching key " + entry.getKey());
+                    continue;
+                }
+                if (!recorded.get(entry.getKey()).equals(entry.getValue())) {
+                    LOG.info("Recorded size:" + recorded.get(entry.getKey()) + "  is different from new size"
+                            + entry.getValue());
+                    buffer.append(entry.getKey()).append(',');
+                }
+            }
+            if (buffer.length() == 0) {
+                return "";
+            } else {
+                return buffer.substring(0, buffer.length() - 1);
+            }
+
+        } finally {
+            in.close();
+        }
+
+    }
+
+    public long usage(Path inPath, Configuration conf) throws IOException {
+        FileSystem fs = inPath.getFileSystem(conf);
+        FileStatus status[] = fs.globStatus(inPath);
+        if (status == null || status.length == 0) {
+            return 0;
+        }
+        long totalSize = 0;
+        for (FileStatus statu : status) {
+            totalSize += fs.getContentSummary(statu.getPath()).getLength();
+        }
+        return totalSize;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
index 5f050ca..7a22704 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/LaterunEvent.java
@@ -19,20 +19,20 @@ package org.apache.falcon.rerun.event;
 
 public class LaterunEvent extends RerunEvent {
 
-	public LaterunEvent(String clusterName, String wfId, long msgInsertTime,
-			long delay, String entityType, String entityName, String instance,
-			int runId) {
-		super(clusterName, wfId, msgInsertTime, delay, entityType, entityName,
-				instance, runId);
-	}
+    public LaterunEvent(String clusterName, String wfId, long msgInsertTime,
+                        long delay, String entityType, String entityName, String instance,
+                        int runId) {
+        super(clusterName, wfId, msgInsertTime, delay, entityType, entityName,
+                instance, runId);
+    }
 
-	@Override
-	public String toString() {
-		return "clusterName=" + clusterName + SEP + "wfId=" + wfId + SEP
-				+ "msgInsertTime=" + msgInsertTime + SEP + "delayInMilliSec="
-				+ delayInMilliSec + SEP + "entityType=" + entityType + SEP
-				+ "entityName=" + entityName + SEP + "instance=" + instance
-				+ SEP + "runId=" + runId;
-	}
+    @Override
+    public String toString() {
+        return "clusterName=" + clusterName + SEP + "wfId=" + wfId + SEP
+                + "msgInsertTime=" + msgInsertTime + SEP + "delayInMilliSec="
+                + delayInMilliSec + SEP + "entityType=" + entityType + SEP
+                + "entityName=" + entityName + SEP + "instance=" + instance
+                + SEP + "runId=" + runId;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
index 9526e0a..5a1e3e1 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEvent.java
@@ -23,91 +23,91 @@ import java.util.concurrent.TimeUnit;
 
 public class RerunEvent implements Delayed {
 
-	protected static final String SEP = "*";
-	
-	public  enum RerunType{
-		RETRY, LATE
-	}
-
-	protected String clusterName;
-	protected String wfId;
-	protected long msgInsertTime;
-	protected long delayInMilliSec;
-	protected String entityType;
-	protected String entityName;
-	protected String instance;
-	protected int runId;
-
-	public RerunEvent(String clusterName, String wfId,
-			long msgInsertTime, long delay, String entityType, String entityName,
-			String instance, int runId) {
-		this.clusterName = clusterName;
-		this.wfId = wfId;
-		this.msgInsertTime = msgInsertTime;
-		this.delayInMilliSec = delay;
-		this.entityName = entityName;
-		this.instance = instance;
-		this.runId = runId;
-		this.entityType=entityType;
-	}
-
-	public String getClusterName() {
-		return clusterName;
-	}
-
-	public String getWfId() {
-		return wfId;
-	}
-
-	public long getDelayInMilliSec() {
-		return delayInMilliSec;
-	}
-
-	public String getEntityName() {
-		return entityName;
-	}
-
-	public String getInstance() {
-		return instance;
-	}
-
-	public int getRunId() {
-		return runId;
-	}
-	
-	public String getEntityType(){
-		return entityType;
-	}
-
-	@Override
-	public int compareTo(Delayed o) {
+    protected static final String SEP = "*";
+
+    public enum RerunType {
+        RETRY, LATE
+    }
+
+    protected String clusterName;
+    protected String wfId;
+    protected long msgInsertTime;
+    protected long delayInMilliSec;
+    protected String entityType;
+    protected String entityName;
+    protected String instance;
+    protected int runId;
+
+    public RerunEvent(String clusterName, String wfId,
+                      long msgInsertTime, long delay, String entityType, String entityName,
+                      String instance, int runId) {
+        this.clusterName = clusterName;
+        this.wfId = wfId;
+        this.msgInsertTime = msgInsertTime;
+        this.delayInMilliSec = delay;
+        this.entityName = entityName;
+        this.instance = instance;
+        this.runId = runId;
+        this.entityType = entityType;
+    }
+
+    public String getClusterName() {
+        return clusterName;
+    }
+
+    public String getWfId() {
+        return wfId;
+    }
+
+    public long getDelayInMilliSec() {
+        return delayInMilliSec;
+    }
+
+    public String getEntityName() {
+        return entityName;
+    }
+
+    public String getInstance() {
+        return instance;
+    }
+
+    public int getRunId() {
+        return runId;
+    }
+
+    public String getEntityType() {
+        return entityType;
+    }
+
+    @Override
+    public int compareTo(Delayed o) {
         RerunEvent event = (RerunEvent) o;
         return new Date(msgInsertTime + delayInMilliSec).
                 compareTo(new Date(event.msgInsertTime + event.delayInMilliSec));
-	}
-
-	@Override
-	public long getDelay(TimeUnit unit) {
-		return unit.convert((msgInsertTime - System.currentTimeMillis())
-				+ delayInMilliSec, TimeUnit.MILLISECONDS);
-	}
-
-	public long getMsgInsertTime() {
-		return msgInsertTime;
-	}
-
-	public void setMsgInsertTime(long msgInsertTime) {
-		this.msgInsertTime = msgInsertTime;
-	}
-
-	public RerunType getType() {
-		if (this instanceof RetryEvent) {
-			return RerunType.RETRY;
-		} else if (this instanceof LaterunEvent) {
-			return RerunType.LATE;
-		} else {
-			return null;
-		}
-	}
+    }
+
+    @Override
+    public long getDelay(TimeUnit unit) {
+        return unit.convert((msgInsertTime - System.currentTimeMillis())
+                + delayInMilliSec, TimeUnit.MILLISECONDS);
+    }
+
+    public long getMsgInsertTime() {
+        return msgInsertTime;
+    }
+
+    public void setMsgInsertTime(long msgInsertTime) {
+        this.msgInsertTime = msgInsertTime;
+    }
+
+    public RerunType getType() {
+        if (this instanceof RetryEvent) {
+            return RerunType.RETRY;
+        } else if (this instanceof LaterunEvent) {
+            return RerunType.LATE;
+        } else {
+            return null;
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
index fcdb836..c5e1e80 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RerunEventFactory.java
@@ -17,53 +17,54 @@
  */
 package org.apache.falcon.rerun.event;
 
+import org.apache.falcon.rerun.event.RerunEvent.RerunType;
+
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.falcon.rerun.event.RerunEvent.RerunType;
-
 public class RerunEventFactory<T extends RerunEvent> {
 
-	public T getRerunEvent(String type, String line) {
-		if (type.startsWith(RerunType.RETRY.name())) {
-			return retryEventFromString(line);
-		} else if (type.startsWith(RerunType.LATE.name())) {
-			return lateEventFromString(line);
-		} else
-			return null;
-	}
+    public T getRerunEvent(String type, String line) {
+        if (type.startsWith(RerunType.RETRY.name())) {
+            return retryEventFromString(line);
+        } else if (type.startsWith(RerunType.LATE.name())) {
+            return lateEventFromString(line);
+        } else {
+            return null;
+        }
+    }
 
-	@SuppressWarnings("unchecked")
-	private T lateEventFromString(String line) {
-		Map<String, String> map = getMap(line);
-		return (T) new LaterunEvent(map.get("clusterName"), map.get("wfId"),
-				Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
-						.get("delayInMilliSec")), map.get("entityType"),
-				map.get("entityName"), map.get("instance"),
-				Integer.parseInt(map.get("runId")));
-	}
+    @SuppressWarnings("unchecked")
+    private T lateEventFromString(String line) {
+        Map<String, String> map = getMap(line);
+        return (T) new LaterunEvent(map.get("clusterName"), map.get("wfId"),
+                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
+                .get("delayInMilliSec")), map.get("entityType"),
+                map.get("entityName"), map.get("instance"),
+                Integer.parseInt(map.get("runId")));
+    }
 
-	@SuppressWarnings("unchecked")
-	public T retryEventFromString(String line) {
-		Map<String, String> map = getMap(line);
-		return (T) new RetryEvent(map.get("clusterName"), map.get("wfId"),
-				Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
-						.get("delayInMilliSec")), map.get("entityType"),
-				map.get("entityName"), map.get("instance"),
-				Integer.parseInt(map.get("runId")), Integer.parseInt(map
-						.get("attempts")), Integer.parseInt(map
-						.get("failRetryCount")));
+    @SuppressWarnings("unchecked")
+    public T retryEventFromString(String line) {
+        Map<String, String> map = getMap(line);
+        return (T) new RetryEvent(map.get("clusterName"), map.get("wfId"),
+                Long.parseLong(map.get("msgInsertTime")), Long.parseLong(map
+                .get("delayInMilliSec")), map.get("entityType"),
+                map.get("entityName"), map.get("instance"),
+                Integer.parseInt(map.get("runId")), Integer.parseInt(map
+                .get("attempts")), Integer.parseInt(map
+                .get("failRetryCount")));
 
-	}
+    }
 
-	private Map<String, String> getMap(String message) {
-		String[] items = message.split("\\" + RerunEvent.SEP);
-		Map<String, String> map = new HashMap<String, String>();
-		for (String item : items) {
-			String[] pair = item.split("=");
-			map.put(pair[0], pair[1]);
-		}
-		return map;
-	}
+    private Map<String, String> getMap(String message) {
+        String[] items = message.split("\\" + RerunEvent.SEP);
+        Map<String, String> map = new HashMap<String, String>();
+        for (String item : items) {
+            String[] pair = item.split("=");
+            map.put(pair[0], pair[1]);
+        }
+        return map;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java b/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
index 7ff4361..33248b8 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/event/RetryEvent.java
@@ -19,39 +19,39 @@ package org.apache.falcon.rerun.event;
 
 public class RetryEvent extends RerunEvent {
 
-	private int attempts;
-	private int failRetryCount;
-
-	public RetryEvent(String clusterName, String wfId, long msgInsertTime,
-			long delay, String entityType, String entityName, String instance,
-			int runId, int attempts, int failRetryCount) {
-		super(clusterName, wfId, msgInsertTime, delay, entityType, entityName,
-				instance, runId);
-		this.attempts = attempts;
-		this.failRetryCount = failRetryCount;
-	}
-
-	public int getAttempts() {
-		return attempts;
-	}
-
-	public int getFailRetryCount() {
-		return failRetryCount;
-	}
-
-	public void setFailRetryCount(int failRetryCount) {
-		this.failRetryCount = failRetryCount;
-	}
-
-	@Override
-	public String toString() {
-
-		return "clusterName=" + clusterName + SEP + "wfId=" + wfId + SEP
-				+ "msgInsertTime=" + msgInsertTime + SEP + "delayInMilliSec="
-				+ delayInMilliSec + SEP + "entityType=" + entityType + SEP
-				+ "entityName=" + entityName + SEP + "instance=" + instance
-				+ SEP + "runId=" + runId + SEP + "attempts=" + attempts + SEP
-				+ "failRetryCount=" + failRetryCount;
-	}
+    private int attempts;
+    private int failRetryCount;
+
+    public RetryEvent(String clusterName, String wfId, long msgInsertTime,
+                      long delay, String entityType, String entityName, String instance,
+                      int runId, int attempts, int failRetryCount) {
+        super(clusterName, wfId, msgInsertTime, delay, entityType, entityName,
+                instance, runId);
+        this.attempts = attempts;
+        this.failRetryCount = failRetryCount;
+    }
+
+    public int getAttempts() {
+        return attempts;
+    }
+
+    public int getFailRetryCount() {
+        return failRetryCount;
+    }
+
+    public void setFailRetryCount(int failRetryCount) {
+        this.failRetryCount = failRetryCount;
+    }
+
+    @Override
+    public String toString() {
+
+        return "clusterName=" + clusterName + SEP + "wfId=" + wfId + SEP
+                + "msgInsertTime=" + msgInsertTime + SEP + "delayInMilliSec="
+                + delayInMilliSec + SEP + "entityType=" + entityType + SEP
+                + "entityName=" + entityName + SEP + "instance=" + instance
+                + SEP + "runId=" + runId + SEP + "attempts=" + attempts + SEP
+                + "failRetryCount=" + failRetryCount;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
index 5e2fa50..fa1d9e3 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunConsumer.java
@@ -27,47 +27,47 @@ import org.apache.falcon.rerun.queue.DelayedQueue;
 import org.apache.log4j.Logger;
 
 public abstract class AbstractRerunConsumer<T extends RerunEvent, M extends AbstractRerunHandler<T, DelayedQueue<T>>>
-		implements Runnable {
+        implements Runnable {
 
-	protected static final Logger LOG = Logger
-			.getLogger(AbstractRerunConsumer.class);
+    protected static final Logger LOG = Logger
+            .getLogger(AbstractRerunConsumer.class);
 
-	protected M handler;
+    protected M handler;
 
-	public AbstractRerunConsumer(M handler) {
-		this.handler = handler;
-	}
+    public AbstractRerunConsumer(M handler) {
+        this.handler = handler;
+    }
 
-	@Override
-	public void run() {
-		int attempt = 1;
-		AbstractRerunPolicy policy = new ExpBackoffPolicy();
-		Frequency frequency = new Frequency("minutes(1)");
-		while (true) {
-			try {
-				T message = null;
-				try {
-					message = handler.takeFromQueue();
-					attempt = 1;
-				} catch (FalconException e) {
-					LOG.error("Error while reading message from the queue: ", e);
-					GenericAlert.alertRerunConsumerFailed(
-							"Error while reading message from the queue: ", e);
-					Thread.sleep(policy.getDelay(frequency, attempt));
-					handler.reconnect();
-					attempt++;
-					continue;
-				}
-				String jobStatus = handler.getWfEngine().getWorkflowStatus(
-						message.getClusterName(), message.getWfId());
-				handleRerun(message.getClusterName(), jobStatus, message);
+    @Override
+    public void run() {
+        int attempt = 1;
+        AbstractRerunPolicy policy = new ExpBackoffPolicy();
+        Frequency frequency = new Frequency("minutes(1)");
+        while (true) {
+            try {
+                T message = null;
+                try {
+                    message = handler.takeFromQueue();
+                    attempt = 1;
+                } catch (FalconException e) {
+                    LOG.error("Error while reading message from the queue: ", e);
+                    GenericAlert.alertRerunConsumerFailed(
+                            "Error while reading message from the queue: ", e);
+                    Thread.sleep(policy.getDelay(frequency, attempt));
+                    handler.reconnect();
+                    attempt++;
+                    continue;
+                }
+                String jobStatus = handler.getWfEngine().getWorkflowStatus(
+                        message.getClusterName(), message.getWfId());
+                handleRerun(message.getClusterName(), jobStatus, message);
 
-			} catch (Throwable e) {
-				LOG.error("Error in rerun consumer:", e);
-			}
-		}
+            } catch (Throwable e) {
+                LOG.error("Error in rerun consumer:", e);
+            }
+        }
 
-	}
+    }
 
-	protected abstract void handleRerun(String cluster, String jobStatus, T message);
+    protected abstract void handleRerun(String cluster, String jobStatus, T message);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
index 66f9c2a..4a90b9f 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/AbstractRerunHandler.java
@@ -29,44 +29,44 @@ import org.apache.log4j.Logger;
 
 public abstract class AbstractRerunHandler<T extends RerunEvent, M extends DelayedQueue<T>> {
 
-	protected static final Logger LOG = Logger
-			.getLogger(LateRerunHandler.class);
-	protected M delayQueue;
-	private AbstractWorkflowEngine wfEngine;
+    protected static final Logger LOG = Logger
+            .getLogger(LateRerunHandler.class);
+    protected M delayQueue;
+    private AbstractWorkflowEngine wfEngine;
 
-	public void init(M delayQueue) throws FalconException {
-		this.wfEngine = WorkflowEngineFactory.getWorkflowEngine();
-		this.delayQueue = delayQueue;
-		this.delayQueue.init();
-	}
+    public void init(M delayQueue) throws FalconException {
+        this.wfEngine = WorkflowEngineFactory.getWorkflowEngine();
+        this.delayQueue = delayQueue;
+        this.delayQueue.init();
+    }
 
-	public abstract void handleRerun(String cluster, String entityType,
-			String entityName, String nominalTime, String runId, String wfId,
-			long msgReceivedTime);
+    public abstract void handleRerun(String cluster, String entityType,
+                                     String entityName, String nominalTime, String runId, String wfId,
+                                     long msgReceivedTime);
 
-	public AbstractWorkflowEngine getWfEngine() {
-		return wfEngine;
-	}
+    public AbstractWorkflowEngine getWfEngine() {
+        return wfEngine;
+    }
 
-	public boolean offerToQueue(T event) throws FalconException {
-		return delayQueue.offer(event);
-	}
+    public boolean offerToQueue(T event) throws FalconException {
+        return delayQueue.offer(event);
+    }
 
-	public T takeFromQueue() throws FalconException {
-		return delayQueue.take();
-	}
-	
-	public void reconnect() throws FalconException {
-		delayQueue.reconnect();
-	}
+    public T takeFromQueue() throws FalconException {
+        return delayQueue.take();
+    }
 
-	public Entity getEntity(String entityType, String entityName)
-			throws FalconException {
-		return EntityUtil.getEntity(entityType, entityName);
-	}
+    public void reconnect() throws FalconException {
+        delayQueue.reconnect();
+    }
 
-	public Retry getRetry(Entity entity) throws FalconException {
-		return EntityUtil.getRetry(entity);
-	}
+    public Entity getEntity(String entityType, String entityName)
+            throws FalconException {
+        return EntityUtil.getEntity(entityType, entityName);
+    }
+
+    public Retry getRetry(Entity entity) throws FalconException {
+        return EntityUtil.getRetry(entity);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
index fc88f0e..03561fc 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunConsumer.java
@@ -17,16 +17,6 @@
  */
 package org.apache.falcon.rerun.handler;
 
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.v0.Entity;
@@ -35,115 +25,120 @@ import org.apache.falcon.entity.v0.process.LateInput;
 import org.apache.falcon.latedata.LateDataHandler;
 import org.apache.falcon.rerun.event.LaterunEvent;
 import org.apache.falcon.rerun.queue.DelayedQueue;
-import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.*;
 
 public class LateRerunConsumer<T extends LateRerunHandler<DelayedQueue<LaterunEvent>>>
-		extends AbstractRerunConsumer<LaterunEvent, T> {
+        extends AbstractRerunConsumer<LaterunEvent, T> {
 
-	public LateRerunConsumer(T handler) {
-		super(handler);
-	}
+    public LateRerunConsumer(T handler) {
+        super(handler);
+    }
 
-	@Override
-	protected void handleRerun(String cluster, String jobStatus,
-			LaterunEvent message) {
-		try {
-			if (jobStatus.equals("RUNNING") || jobStatus.equals("PREP")
-					|| jobStatus.equals("SUSPENDED")) {
-				LOG.debug("Re-enqueing message in LateRerunHandler for workflow with same delay as job status is running:"
-						+ message.getWfId());
-				message.setMsgInsertTime(System.currentTimeMillis());
-				handler.offerToQueue(message);
-				return;
-			}
+    @Override
+    protected void handleRerun(String cluster, String jobStatus,
+                               LaterunEvent message) {
+        try {
+            if (jobStatus.equals("RUNNING") || jobStatus.equals("PREP")
+                    || jobStatus.equals("SUSPENDED")) {
+                LOG.debug(
+                        "Re-enqueing message in LateRerunHandler for workflow with same delay as job status is running:"
+                                + message.getWfId());
+                message.setMsgInsertTime(System.currentTimeMillis());
+                handler.offerToQueue(message);
+                return;
+            }
 
-			String detectLate = detectLate(message);
+            String detectLate = detectLate(message);
 
-			if (detectLate.equals("")) {
-				LOG.debug("No Late Data Detected, scheduling next late rerun for wf-id: "
-						+ message.getWfId()
-						+ " at "
-						+ SchemaHelper.formatDateUTC(new Date()));
-				handler.handleRerun(cluster, message.getEntityType(),
-						message.getEntityName(), message.getInstance(),
-						Integer.toString(message.getRunId()),
-						message.getWfId(), System.currentTimeMillis());
-				return;
-			}
+            if (detectLate.equals("")) {
+                LOG.debug("No Late Data Detected, scheduling next late rerun for wf-id: "
+                        + message.getWfId()
+                        + " at "
+                        + SchemaHelper.formatDateUTC(new Date()));
+                handler.handleRerun(cluster, message.getEntityType(),
+                        message.getEntityName(), message.getInstance(),
+                        Integer.toString(message.getRunId()),
+                        message.getWfId(), System.currentTimeMillis());
+                return;
+            }
 
-			LOG.info("Late changes detected in the following feeds: "
-					+ detectLate);
+            LOG.info("Late changes detected in the following feeds: "
+                    + detectLate);
 
-			handler.getWfEngine().reRun(message.getClusterName(),
-					message.getWfId(), null);
-			LOG.info("Scheduled late rerun for wf-id: " + message.getWfId()
-					+ " on cluster: " + message.getClusterName());
-		} catch (Exception e) {
-			LOG.warn(
-					"Late Re-run failed for instance "
-							+ message.getEntityName() + ":"
-							+ message.getInstance() + " after "
-							+ message.getDelayInMilliSec() + " with message:",
-					e);
-			GenericAlert.alertLateRerunFailed(message.getEntityType(),
-					message.getEntityName(), message.getInstance(),
-					message.getWfId(), Integer.toString(message.getRunId()),
-					e.getMessage());
-		}
+            handler.getWfEngine().reRun(message.getClusterName(),
+                    message.getWfId(), null);
+            LOG.info("Scheduled late rerun for wf-id: " + message.getWfId()
+                    + " on cluster: " + message.getClusterName());
+        } catch (Exception e) {
+            LOG.warn(
+                    "Late Re-run failed for instance "
+                            + message.getEntityName() + ":"
+                            + message.getInstance() + " after "
+                            + message.getDelayInMilliSec() + " with message:",
+                    e);
+            GenericAlert.alertLateRerunFailed(message.getEntityType(),
+                    message.getEntityName(), message.getInstance(),
+                    message.getWfId(), Integer.toString(message.getRunId()),
+                    e.getMessage());
+        }
 
-	}
+    }
 
-	public String detectLate(LaterunEvent message) throws Exception {
-		LateDataHandler late = new LateDataHandler();
-		String falconInputFeeds = handler.getWfEngine().getWorkflowProperty(
-				message.getClusterName(), message.getWfId(), "falconInputFeeds");
-		String logDir = handler.getWfEngine().getWorkflowProperty(
-				message.getClusterName(), message.getWfId(), "logDir");
-		String falconInPaths = handler.getWfEngine().getWorkflowProperty(
-				message.getClusterName(), message.getWfId(), "falconInPaths");
-		String nominalTime = handler.getWfEngine().getWorkflowProperty(
-				message.getClusterName(), message.getWfId(), "nominalTime");
-		String srcClusterName = handler.getWfEngine().getWorkflowProperty(
-				message.getClusterName(), message.getWfId(), "srcClusterName");
+    public String detectLate(LaterunEvent message) throws Exception {
+        LateDataHandler late = new LateDataHandler();
+        String falconInputFeeds = handler.getWfEngine().getWorkflowProperty(
+                message.getClusterName(), message.getWfId(), "falconInputFeeds");
+        String logDir = handler.getWfEngine().getWorkflowProperty(
+                message.getClusterName(), message.getWfId(), "logDir");
+        String falconInPaths = handler.getWfEngine().getWorkflowProperty(
+                message.getClusterName(), message.getWfId(), "falconInPaths");
+        String nominalTime = handler.getWfEngine().getWorkflowProperty(
+                message.getClusterName(), message.getWfId(), "nominalTime");
+        String srcClusterName = handler.getWfEngine().getWorkflowProperty(
+                message.getClusterName(), message.getWfId(), "srcClusterName");
 
-		Configuration conf = handler.getConfiguration(message.getClusterName(),
-				message.getWfId());
-		Path lateLogPath = handler.getLateLogPath(logDir, nominalTime,
-				srcClusterName);
-		FileSystem fs = FileSystem.get(conf);
-		if (!fs.exists(lateLogPath)) {
-			LOG.warn("Late log file:" + lateLogPath + " not found:");
-			return "";
-		}
-		Map<String, Long> feedSizes = new LinkedHashMap<String, Long>();
-		String[] pathGroups = falconInPaths.split("#");
-		String[] inputFeeds = falconInputFeeds.split("#");
-		Entity entity = EntityUtil.getEntity(message.getEntityType(),
-				message.getEntityName());
+        Configuration conf = handler.getConfiguration(message.getClusterName(),
+                message.getWfId());
+        Path lateLogPath = handler.getLateLogPath(logDir, nominalTime,
+                srcClusterName);
+        FileSystem fs = FileSystem.get(conf);
+        if (!fs.exists(lateLogPath)) {
+            LOG.warn("Late log file:" + lateLogPath + " not found:");
+            return "";
+        }
+        Map<String, Long> feedSizes = new LinkedHashMap<String, Long>();
+        String[] pathGroups = falconInPaths.split("#");
+        String[] inputFeeds = falconInputFeeds.split("#");
+        Entity entity = EntityUtil.getEntity(message.getEntityType(),
+                message.getEntityName());
 
-		List<String> lateFeed = new ArrayList<String>();
-		if (EntityUtil.getLateProcess(entity) != null) {
-			for (LateInput li : EntityUtil.getLateProcess(entity)
-					.getLateInputs()) {
-				lateFeed.add(li.getInput());
-			}
-			for (int index = 0; index < pathGroups.length; index++) {
-				if (lateFeed.contains(inputFeeds[index])) {
-					long usage = 0;
-					for (String pathElement : pathGroups[index].split(",")) {
-						Path inPath = new Path(pathElement);
-						usage += late.usage(inPath, conf);
-					}
-					feedSizes.put(inputFeeds[index], usage);
-				}
-			}
-		} else {
-			LOG.warn("Late process is not configured for entity: "
-					+ message.getEntityType() + "(" + message.getEntityName()
-					+ ")");
-		}
+        List<String> lateFeed = new ArrayList<String>();
+        if (EntityUtil.getLateProcess(entity) != null) {
+            for (LateInput li : EntityUtil.getLateProcess(entity)
+                    .getLateInputs()) {
+                lateFeed.add(li.getInput());
+            }
+            for (int index = 0; index < pathGroups.length; index++) {
+                if (lateFeed.contains(inputFeeds[index])) {
+                    long usage = 0;
+                    for (String pathElement : pathGroups[index].split(",")) {
+                        Path inPath = new Path(pathElement);
+                        usage += late.usage(inPath, conf);
+                    }
+                    feedSizes.put(inputFeeds[index], usage);
+                }
+            }
+        } else {
+            LOG.warn("Late process is not configured for entity: "
+                    + message.getEntityType() + "(" + message.getEntityName()
+                    + ")");
+        }
 
-		return late.detectChanges(lateLogPath, feedSizes, conf);
-	}
+        return late.detectChanges(lateLogPath, feedSizes, conf);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
index e2145cb..ad19157 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
@@ -17,12 +17,6 @@
  */
 package org.apache.falcon.rerun.handler;
 
-import java.util.Date;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.entity.EntityUtil;
@@ -31,10 +25,7 @@ import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.Input;
-import org.apache.falcon.entity.v0.process.LateInput;
-import org.apache.falcon.entity.v0.process.LateProcess;
-import org.apache.falcon.entity.v0.process.PolicyType;
+import org.apache.falcon.entity.v0.process.*;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.expression.ExpressionHelper;
 import org.apache.falcon.rerun.event.LaterunEvent;
@@ -42,185 +33,192 @@ import org.apache.falcon.rerun.policy.AbstractRerunPolicy;
 import org.apache.falcon.rerun.policy.RerunPolicyFactory;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.util.Date;
 
 public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
-		AbstractRerunHandler<LaterunEvent, M> {
-
-	@Override
-	public void handleRerun(String cluster, String entityType,
-			String entityName, String nominalTime, String runId, String wfId,
-			long msgReceivedTime) {
-
-		try {
-			Entity entity = EntityUtil.getEntity(entityType, entityName);
-			try {
-				if (EntityUtil.getLateProcess(entity) == null
-						|| EntityUtil.getLateProcess(entity).getLateInputs() == null
-						|| EntityUtil.getLateProcess(entity).getLateInputs()
-								.size() == 0) {
-					LOG.info("Late rerun not configured for entity: " + entityName);
-					return;
-				}
-			} catch (FalconException e) {
-				LOG.error("Unable to get Late Process for entity:" + entityName);
-				return;
-			}
-			int intRunId = Integer.parseInt(runId);
-			Date msgInsertTime = EntityUtil.parseDateUTC(nominalTime);
-			Long wait = getEventDelay(entity, nominalTime);
-			if (wait == -1) {
-				LOG.info("Late rerun expired for entity: "+entityType+"("+entityName+")");
-				String logDir = this.getWfEngine().getWorkflowProperty(cluster,
-						wfId, "logDir");
-				String srcClusterName = this.getWfEngine().getWorkflowProperty(
-						cluster, wfId, "srcClusterName");
-				Path lateLogPath = this.getLateLogPath(logDir,
-						EntityUtil.UTCtoURIDate(nominalTime), srcClusterName);
-				LOG.info("Going to delete path:" +lateLogPath);
-				FileSystem fs = FileSystem.get(getConfiguration(cluster,
-						wfId));
-				if (fs.exists(lateLogPath)) {
-					boolean deleted = fs.delete(lateLogPath, true);
-					if (deleted == true) {
-						LOG.info("Successfully deleted late file path:"
-								+ lateLogPath);
-					}
-				}
-				return;
-			}
-
-			LOG.debug("Scheduling the late rerun for entity instance : "
-					+ entityType + "(" + entityName + ")" + ":" + nominalTime
-					+ " And WorkflowId: " + wfId);
-			LaterunEvent event = new LaterunEvent(cluster, wfId,
-					msgInsertTime.getTime(), wait, entityType, entityName,
-					nominalTime, intRunId);
-			offerToQueue(event);
-		} catch (Exception e) {
-			LOG.error("Unable to schedule late rerun for entity instance : "
-					+ entityType + "(" + entityName + ")" + ":" + nominalTime
-					+ " And WorkflowId: " + wfId, e);
-			GenericAlert.alertLateRerunFailed(entityType, entityName,
-					nominalTime, wfId, runId, e.getMessage());
-		}
-	}
-
-	private long getEventDelay(Entity entity, String nominalTime)
-			throws FalconException {
-
-		Date instanceDate = EntityUtil.parseDateUTC(nominalTime);
-		LateProcess lateProcess = EntityUtil.getLateProcess(entity);
-		if (lateProcess == null) {
-			LOG.warn("Late run not applicable for entity:"
-					+ entity.getEntityType() + "(" + entity.getName() + ")");
-			return -1;
-		}
-		PolicyType latePolicy = lateProcess.getPolicy();
-		Date cutOffTime = getCutOffTime(entity, nominalTime);
-		Date now = new Date();
-		Long wait = null;
-
-		if (now.after(cutOffTime)) {
-			LOG.warn("Feed Cut Off time: "
-					+ SchemaHelper.formatDateUTC(cutOffTime)
-					+ " has expired, Late Rerun can not be scheduled");
-			return -1;
-		} else {
-			AbstractRerunPolicy rerunPolicy = RerunPolicyFactory
-					.getRetryPolicy(latePolicy);
-			wait = rerunPolicy.getDelay(lateProcess.getDelay(), instanceDate,
-					cutOffTime);
-		}
-		return wait;
-	}
-
-	public static Date addTime(Date date, int milliSecondsToAdd) {
-		return new Date(date.getTime() + milliSecondsToAdd);
-	}
-
-	public static Date getCutOffTime(Entity entity, String nominalTime)
-			throws FalconException {
-
-		ConfigurationStore store = ConfigurationStore.get();
-		ExpressionHelper evaluator = ExpressionHelper.get();
-		Date instanceStart = EntityUtil.parseDateUTC(nominalTime);
-		ExpressionHelper.setReferenceDate(instanceStart);
-		Date endTime = new Date();
-		Date feedCutOff = new Date(0);
-		if (entity.getEntityType() == EntityType.FEED) {
-			if (((Feed) entity).getLateArrival() == null) {
-				LOG.debug("Feed's " + entity.getName()
-						+ " late arrival cut-off is not configured, returning");
-				return feedCutOff;
-			}
-			String lateCutOff = ((Feed) entity).getLateArrival().getCutOff()
-					.toString();
-			endTime = EntityUtil.parseDateUTC(nominalTime);
-			long feedCutOffPeriod = evaluator.evaluate(lateCutOff, Long.class);
-			endTime = addTime(endTime, (int) feedCutOffPeriod);
-			return endTime;
-		} else if (entity.getEntityType() == EntityType.PROCESS) {
-			Process process = (Process) entity;
-			for (LateInput lp : process.getLateProcess().getLateInputs()) {
-				Feed feed = null;
-				String endInstanceTime = "";
-				for (Input input : process.getInputs().getInputs()) {
-					if (input.getName().equals(lp.getInput())) {
-						endInstanceTime = input.getEnd();
-						feed = store.get(EntityType.FEED, input.getFeed());
-						break;
-					}
-				}
-				if (feed.getLateArrival() == null) {
-					LOG.debug("Feed's " + feed.getName()
-							+ " late arrival cut-off is not configured, ignoring this feed");
-					continue;
-				}
-				String lateCutOff = feed.getLateArrival().getCutOff()
-						.toString();
-				endTime = evaluator.evaluate(endInstanceTime, Date.class);
-				long feedCutOffPeriod = evaluator.evaluate(lateCutOff,
-						Long.class);
-				endTime = addTime(endTime, (int) feedCutOffPeriod);
-
-				if (endTime.after(feedCutOff))
-					feedCutOff = endTime;
-			}
-			return feedCutOff;
-		} else {
-			throw new FalconException(
-					"Invalid entity while getting cut-off time:"
-							+ entity.getName());
-		}
-	}
-
-	@Override
-	public void init(M delayQueue) throws FalconException {
-		super.init(delayQueue);
-		Thread daemon = new Thread(new LateRerunConsumer(this));
-		daemon.setName("LaterunHandler");
-		daemon.setDaemon(true);
-		daemon.start();
-		LOG.info("Laterun Handler  thread started");
-	}
-	
-	public Path getLateLogPath(String logDir, String nominalTime,
-			String srcClusterName) {
-		//SrcClusterName valid only in case of feed
-		return new Path(logDir + "/latedata/" + nominalTime + "/"
-				+ (srcClusterName == null
-				? "" : srcClusterName));
-
-	}
-	
-	public Configuration getConfiguration(String cluster, String wfId)
-			throws FalconException {
-		Configuration conf = new Configuration();
-		conf.set(
-				CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
-				this.getWfEngine().getWorkflowProperty(cluster, wfId,
-						AbstractWorkflowEngine.NAME_NODE));
-		return conf;
-	}
+        AbstractRerunHandler<LaterunEvent, M> {
+
+    @Override
+    public void handleRerun(String cluster, String entityType,
+                            String entityName, String nominalTime, String runId, String wfId,
+                            long msgReceivedTime) {
+
+        try {
+            Entity entity = EntityUtil.getEntity(entityType, entityName);
+            try {
+                if (EntityUtil.getLateProcess(entity) == null
+                        || EntityUtil.getLateProcess(entity).getLateInputs() == null
+                        || EntityUtil.getLateProcess(entity).getLateInputs()
+                        .size() == 0) {
+                    LOG.info("Late rerun not configured for entity: " + entityName);
+                    return;
+                }
+            } catch (FalconException e) {
+                LOG.error("Unable to get Late Process for entity:" + entityName);
+                return;
+            }
+            int intRunId = Integer.parseInt(runId);
+            Date msgInsertTime = EntityUtil.parseDateUTC(nominalTime);
+            Long wait = getEventDelay(entity, nominalTime);
+            if (wait == -1) {
+                LOG.info("Late rerun expired for entity: " + entityType + "(" + entityName + ")");
+                String logDir = this.getWfEngine().getWorkflowProperty(cluster,
+                        wfId, "logDir");
+                String srcClusterName = this.getWfEngine().getWorkflowProperty(
+                        cluster, wfId, "srcClusterName");
+                Path lateLogPath = this.getLateLogPath(logDir,
+                        EntityUtil.UTCtoURIDate(nominalTime), srcClusterName);
+                LOG.info("Going to delete path:" + lateLogPath);
+                FileSystem fs = FileSystem.get(getConfiguration(cluster,
+                        wfId));
+                if (fs.exists(lateLogPath)) {
+                    boolean deleted = fs.delete(lateLogPath, true);
+                    if (deleted == true) {
+                        LOG.info("Successfully deleted late file path:"
+                                + lateLogPath);
+                    }
+                }
+                return;
+            }
+
+            LOG.debug("Scheduling the late rerun for entity instance : "
+                    + entityType + "(" + entityName + ")" + ":" + nominalTime
+                    + " And WorkflowId: " + wfId);
+            LaterunEvent event = new LaterunEvent(cluster, wfId,
+                    msgInsertTime.getTime(), wait, entityType, entityName,
+                    nominalTime, intRunId);
+            offerToQueue(event);
+        } catch (Exception e) {
+            LOG.error("Unable to schedule late rerun for entity instance : "
+                    + entityType + "(" + entityName + ")" + ":" + nominalTime
+                    + " And WorkflowId: " + wfId, e);
+            GenericAlert.alertLateRerunFailed(entityType, entityName,
+                    nominalTime, wfId, runId, e.getMessage());
+        }
+    }
+
+    private long getEventDelay(Entity entity, String nominalTime)
+            throws FalconException {
+
+        Date instanceDate = EntityUtil.parseDateUTC(nominalTime);
+        LateProcess lateProcess = EntityUtil.getLateProcess(entity);
+        if (lateProcess == null) {
+            LOG.warn("Late run not applicable for entity:"
+                    + entity.getEntityType() + "(" + entity.getName() + ")");
+            return -1;
+        }
+        PolicyType latePolicy = lateProcess.getPolicy();
+        Date cutOffTime = getCutOffTime(entity, nominalTime);
+        Date now = new Date();
+        Long wait = null;
+
+        if (now.after(cutOffTime)) {
+            LOG.warn("Feed Cut Off time: "
+                    + SchemaHelper.formatDateUTC(cutOffTime)
+                    + " has expired, Late Rerun can not be scheduled");
+            return -1;
+        } else {
+            AbstractRerunPolicy rerunPolicy = RerunPolicyFactory
+                    .getRetryPolicy(latePolicy);
+            wait = rerunPolicy.getDelay(lateProcess.getDelay(), instanceDate,
+                    cutOffTime);
+        }
+        return wait;
+    }
+
+    public static Date addTime(Date date, int milliSecondsToAdd) {
+        return new Date(date.getTime() + milliSecondsToAdd);
+    }
+
+    public static Date getCutOffTime(Entity entity, String nominalTime)
+            throws FalconException {
+
+        ConfigurationStore store = ConfigurationStore.get();
+        ExpressionHelper evaluator = ExpressionHelper.get();
+        Date instanceStart = EntityUtil.parseDateUTC(nominalTime);
+        ExpressionHelper.setReferenceDate(instanceStart);
+        Date endTime = new Date();
+        Date feedCutOff = new Date(0);
+        if (entity.getEntityType() == EntityType.FEED) {
+            if (((Feed) entity).getLateArrival() == null) {
+                LOG.debug("Feed's " + entity.getName()
+                        + " late arrival cut-off is not configured, returning");
+                return feedCutOff;
+            }
+            String lateCutOff = ((Feed) entity).getLateArrival().getCutOff()
+                    .toString();
+            endTime = EntityUtil.parseDateUTC(nominalTime);
+            long feedCutOffPeriod = evaluator.evaluate(lateCutOff, Long.class);
+            endTime = addTime(endTime, (int) feedCutOffPeriod);
+            return endTime;
+        } else if (entity.getEntityType() == EntityType.PROCESS) {
+            Process process = (Process) entity;
+            for (LateInput lp : process.getLateProcess().getLateInputs()) {
+                Feed feed = null;
+                String endInstanceTime = "";
+                for (Input input : process.getInputs().getInputs()) {
+                    if (input.getName().equals(lp.getInput())) {
+                        endInstanceTime = input.getEnd();
+                        feed = store.get(EntityType.FEED, input.getFeed());
+                        break;
+                    }
+                }
+                if (feed.getLateArrival() == null) {
+                    LOG.debug("Feed's " + feed.getName()
+                            + " late arrival cut-off is not configured, ignoring this feed");
+                    continue;
+                }
+                String lateCutOff = feed.getLateArrival().getCutOff()
+                        .toString();
+                endTime = evaluator.evaluate(endInstanceTime, Date.class);
+                long feedCutOffPeriod = evaluator.evaluate(lateCutOff,
+                        Long.class);
+                endTime = addTime(endTime, (int) feedCutOffPeriod);
+
+                if (endTime.after(feedCutOff)) {
+                    feedCutOff = endTime;
+                }
+            }
+            return feedCutOff;
+        } else {
+            throw new FalconException(
+                    "Invalid entity while getting cut-off time:"
+                            + entity.getName());
+        }
+    }
+
+    @Override
+    public void init(M delayQueue) throws FalconException {
+        super.init(delayQueue);
+        Thread daemon = new Thread(new LateRerunConsumer(this));
+        daemon.setName("LaterunHandler");
+        daemon.setDaemon(true);
+        daemon.start();
+        LOG.info("Laterun Handler  thread started");
+    }
+
+    public Path getLateLogPath(String logDir, String nominalTime,
+                               String srcClusterName) {
+        //SrcClusterName valid only in case of feed
+        return new Path(logDir + "/latedata/" + nominalTime + "/"
+                + (srcClusterName == null
+                ? "" : srcClusterName));
+
+    }
+
+    public Configuration getConfiguration(String cluster, String wfId)
+            throws FalconException {
+        Configuration conf = new Configuration();
+        conf.set(
+                CommonConfigurationKeys.FS_DEFAULT_NAME_KEY,
+                this.getWfEngine().getWorkflowProperty(cluster, wfId,
+                        AbstractWorkflowEngine.NAME_NODE));
+        return conf;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
index 1901890..ce76842 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RerunHandlerFactory.java
@@ -17,29 +17,31 @@
  */
 package org.apache.falcon.rerun.handler;
 
-import org.apache.falcon.rerun.event.RerunEvent.RerunType;
 import org.apache.falcon.rerun.event.LaterunEvent;
+import org.apache.falcon.rerun.event.RerunEvent.RerunType;
 import org.apache.falcon.rerun.event.RetryEvent;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 
 public class RerunHandlerFactory {
 
-	private static final RetryHandler<DelayedQueue<RetryEvent>> retryHandler = new RetryHandler<DelayedQueue<RetryEvent>>();
-	private static final LateRerunHandler<DelayedQueue<LaterunEvent>> lateHandler = new LateRerunHandler<DelayedQueue<LaterunEvent>>();
+    private static final RetryHandler<DelayedQueue<RetryEvent>> retryHandler
+            = new RetryHandler<DelayedQueue<RetryEvent>>();
+    private static final LateRerunHandler<DelayedQueue<LaterunEvent>> lateHandler
+            = new LateRerunHandler<DelayedQueue<LaterunEvent>>();
 
-	private RerunHandlerFactory() {
+    private RerunHandlerFactory() {
 
-	}
+    }
 
-	public static AbstractRerunHandler getRerunHandler(RerunType type) {
-		switch (type) {
-		case RETRY:
-			return retryHandler;
-		case LATE:
-			return lateHandler;
-		default:
-			throw new RuntimeException("Invalid handler:" + type);
-		}
+    public static AbstractRerunHandler getRerunHandler(RerunType type) {
+        switch (type) {
+            case RETRY:
+                return retryHandler;
+            case LATE:
+                return lateHandler;
+            default:
+                throw new RuntimeException("Invalid handler:" + type);
+        }
 
-	}
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
index a30d2da..c084233 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/RetryConsumer.java
@@ -17,82 +17,82 @@
  */
 package org.apache.falcon.rerun.handler;
 
-import java.util.Date;
-
 import org.apache.falcon.aspect.GenericAlert;
 import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.rerun.event.RetryEvent;
 import org.apache.falcon.rerun.queue.DelayedQueue;
 import org.apache.falcon.util.StartupProperties;
 
+import java.util.Date;
+
 public class RetryConsumer<T extends RetryHandler<DelayedQueue<RetryEvent>>>
-		extends AbstractRerunConsumer<RetryEvent, T> {
+        extends AbstractRerunConsumer<RetryEvent, T> {
 
-	public RetryConsumer(T handler) {
-		super(handler);
-	}
+    public RetryConsumer(T handler) {
+        super(handler);
+    }
 
-	@Override
-	protected void handleRerun(String cluster, String jobStatus,
-			RetryEvent message) {
-		try {
-			if (!jobStatus.equals("KILLED")) {
-				LOG.debug("Re-enqueing message in RetryHandler for workflow with same delay as job status is running:"
-						+ message.getWfId());
-				message.setMsgInsertTime(System.currentTimeMillis());
-				handler.offerToQueue(message);
-				return;
-			}
-			LOG.info("Retrying attempt:"
-					+ (message.getRunId() + 1)
-					+ " out of configured: "
-					+ message.getAttempts()
-					+ " attempt for instance::"
-					+ message.getEntityName()
-					+ ":"
-					+ message.getInstance()
-					+ " And WorkflowId: "
-					+ message.getWfId()
-					+ " At time: "
-					+ SchemaHelper.formatDateUTC(new Date(System
-							.currentTimeMillis())));
-			handler.getWfEngine().reRun(message.getClusterName(),
-					message.getWfId(), null);
-		} catch (Exception e) {
-			int maxFailRetryCount = Integer.parseInt(StartupProperties.get()
-					.getProperty("max.retry.failure.count", "1"));
-			if (message.getFailRetryCount() < maxFailRetryCount) {
-				LOG.warn(
-						"Retrying again for process instance "
-								+ message.getEntityName() + ":"
-								+ message.getInstance() + " after "
-								+ message.getDelayInMilliSec()
-								+ " seconds as Retry failed with message:", e);
-				message.setFailRetryCount(message.getFailRetryCount() + 1);
-				try {
-					handler.offerToQueue(message);
-				} catch (Exception ex) {
-					LOG.error("Unable to re-offer to queue:", ex);
-					GenericAlert.alertRetryFailed(message.getEntityType(),
-							message.getEntityName(), message.getInstance(),
-							message.getWfId(),
-							Integer.toString(message.getRunId()),
-							ex.getMessage());
-				}
-			} else {
-				LOG.warn(
-						"Failure retry attempts exhausted for instance: "
-								+ message.getEntityName() + ":"
-								+ message.getInstance(), e);
-				GenericAlert.alertRetryFailed(message.getEntityType(),
-						message.getEntityName(), message.getInstance(),
-						message.getWfId(),
-						Integer.toString(message.getRunId()),
-						"Failure retry attempts exhausted");
-			}
+    @Override
+    protected void handleRerun(String cluster, String jobStatus,
+                               RetryEvent message) {
+        try {
+            if (!jobStatus.equals("KILLED")) {
+                LOG.debug("Re-enqueing message in RetryHandler for workflow with same delay as job status is running:"
+                        + message.getWfId());
+                message.setMsgInsertTime(System.currentTimeMillis());
+                handler.offerToQueue(message);
+                return;
+            }
+            LOG.info("Retrying attempt:"
+                    + (message.getRunId() + 1)
+                    + " out of configured: "
+                    + message.getAttempts()
+                    + " attempt for instance::"
+                    + message.getEntityName()
+                    + ":"
+                    + message.getInstance()
+                    + " And WorkflowId: "
+                    + message.getWfId()
+                    + " At time: "
+                    + SchemaHelper.formatDateUTC(new Date(System
+                    .currentTimeMillis())));
+            handler.getWfEngine().reRun(message.getClusterName(),
+                    message.getWfId(), null);
+        } catch (Exception e) {
+            int maxFailRetryCount = Integer.parseInt(StartupProperties.get()
+                    .getProperty("max.retry.failure.count", "1"));
+            if (message.getFailRetryCount() < maxFailRetryCount) {
+                LOG.warn(
+                        "Retrying again for process instance "
+                                + message.getEntityName() + ":"
+                                + message.getInstance() + " after "
+                                + message.getDelayInMilliSec()
+                                + " seconds as Retry failed with message:", e);
+                message.setFailRetryCount(message.getFailRetryCount() + 1);
+                try {
+                    handler.offerToQueue(message);
+                } catch (Exception ex) {
+                    LOG.error("Unable to re-offer to queue:", ex);
+                    GenericAlert.alertRetryFailed(message.getEntityType(),
+                            message.getEntityName(), message.getInstance(),
+                            message.getWfId(),
+                            Integer.toString(message.getRunId()),
+                            ex.getMessage());
+                }
+            } else {
+                LOG.warn(
+                        "Failure retry attempts exhausted for instance: "
+                                + message.getEntityName() + ":"
+                                + message.getInstance(), e);
+                GenericAlert.alertRetryFailed(message.getEntityType(),
+                        message.getEntityName(), message.getInstance(),
+                        message.getWfId(),
+                        Integer.toString(message.getRunId()),
+                        "Failure retry attempts exhausted");
+            }
 
-		}
+        }
 
-	}
+    }
 
 }


[14/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 1b4a93d..3a7f4ae 100644
--- a/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
+++ b/feed/src/test/java/org/apache/falcon/converter/OozieFeedMapperTest.java
@@ -18,7 +18,6 @@
 package org.apache.falcon.converter;
 
 import junit.framework.Assert;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -31,6 +30,7 @@ import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.oozie.coordinator.CONFIGURATION.Property;
 import org.apache.falcon.oozie.coordinator.COORDINATORAPP;
 import org.apache.falcon.oozie.coordinator.SYNCDATASET;
+import org.apache.hadoop.fs.Path;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -42,96 +42,97 @@ import java.util.List;
 import static org.testng.Assert.assertEquals;
 
 public class OozieFeedMapperTest {
-	private EmbeddedCluster srcMiniDFS;
-	private EmbeddedCluster trgMiniDFS;
-	ConfigurationStore store = ConfigurationStore.get();
-	Cluster srcCluster;
-	Cluster trgCluster;
-	Feed feed;
+    private EmbeddedCluster srcMiniDFS;
+    private EmbeddedCluster trgMiniDFS;
+    ConfigurationStore store = ConfigurationStore.get();
+    Cluster srcCluster;
+    Cluster trgCluster;
+    Feed feed;
 
-	private static final String SRC_CLUSTER_PATH = "/src-cluster.xml";
-	private static final String TRG_CLUSTER_PATH = "/trg-cluster.xml";
-	private static final String FEED = "/feed.xml";
+    private static final String SRC_CLUSTER_PATH = "/src-cluster.xml";
+    private static final String TRG_CLUSTER_PATH = "/trg-cluster.xml";
+    private static final String FEED = "/feed.xml";
 
-	@BeforeClass
-	public void setUpDFS() throws Exception {
-		srcMiniDFS = EmbeddedCluster.newCluster("cluster1", false);
-		String srcHdfsUrl = srcMiniDFS.getConf().get("fs.default.name");
+    @BeforeClass
+    public void setUpDFS() throws Exception {
+        srcMiniDFS = EmbeddedCluster.newCluster("cluster1", false);
+        String srcHdfsUrl = srcMiniDFS.getConf().get("fs.default.name");
 
-		trgMiniDFS = EmbeddedCluster.newCluster("cluster2", false);
-		String trgHdfsUrl = trgMiniDFS.getConf().get("fs.default.name");
+        trgMiniDFS = EmbeddedCluster.newCluster("cluster2", false);
+        String trgHdfsUrl = trgMiniDFS.getConf().get("fs.default.name");
 
-		cleanupStore();
+        cleanupStore();
 
-		srcCluster = (Cluster) storeEntity(EntityType.CLUSTER, SRC_CLUSTER_PATH);
-		ClusterHelper.getInterface(srcCluster, Interfacetype.WRITE).setEndpoint(srcHdfsUrl);
+        srcCluster = (Cluster) storeEntity(EntityType.CLUSTER, SRC_CLUSTER_PATH);
+        ClusterHelper.getInterface(srcCluster, Interfacetype.WRITE).setEndpoint(srcHdfsUrl);
 
-		trgCluster = (Cluster) storeEntity(EntityType.CLUSTER, TRG_CLUSTER_PATH);
+        trgCluster = (Cluster) storeEntity(EntityType.CLUSTER, TRG_CLUSTER_PATH);
         ClusterHelper.getInterface(trgCluster, Interfacetype.WRITE).setEndpoint(trgHdfsUrl);
 
-		feed = (Feed) storeEntity(EntityType.FEED, FEED);
-
-	}
-
-	protected Entity storeEntity(EntityType type, String path) throws Exception {
-		Unmarshaller unmarshaller = type.getUnmarshaller();
-		Entity entity = (Entity) unmarshaller
-				.unmarshal(OozieFeedMapperTest.class.getResource(path));
-		store.publish(type, entity);
-		return entity;
-	}
-
-	protected void cleanupStore() throws FalconException {
-		for (EntityType type : EntityType.values()) {
-			Collection<String> entities = store.getEntities(type);
-			for (String entity : entities)
-				store.remove(type, entity);
-		}
-	}
-
-	@AfterClass
-	public void stopDFS() {
-		srcMiniDFS.shutdown();
-		trgMiniDFS.shutdown();
-	}
-
-	@Test
-	public void testReplicationCoords() throws FalconException {
-		OozieFeedMapper feedMapper = new OozieFeedMapper(feed);
-		List<COORDINATORAPP> coords = feedMapper.getCoordinators(trgCluster,
-				new Path("/projects/falcon/"));
-		COORDINATORAPP coord = coords.get(0);
-
-		Assert.assertEquals("${nameNode}/projects/falcon/REPLICATION", coord
-				.getAction().getWorkflow().getAppPath());
-		Assert.assertEquals("FALCON_FEED_REPLICATION_" + feed.getName() + "_"
-				+ srcCluster.getName(), coord.getName());
-		Assert.assertEquals("${coord:minutes(20)}", coord.getFrequency());
-		SYNCDATASET inputDataset = (SYNCDATASET) coord.getDatasets()
-				.getDatasetOrAsyncDataset().get(0);
-		SYNCDATASET outputDataset = (SYNCDATASET) coord.getDatasets()
-				.getDatasetOrAsyncDataset().get(1);
-
-		Assert.assertEquals("${coord:minutes(20)}", inputDataset.getFrequency());
-		Assert.assertEquals("input-dataset", inputDataset.getName());
-		Assert.assertEquals(
-				ClusterHelper.getStorageUrl(srcCluster)
-						+ "/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}",
-				inputDataset.getUriTemplate());
-
-		Assert.assertEquals("${coord:minutes(20)}",
-				outputDataset.getFrequency());
-		Assert.assertEquals("output-dataset", outputDataset.getName());
-		Assert.assertEquals(
-				"${nameNode}"
-						+ "/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}",
-				outputDataset.getUriTemplate());
-        for(Property prop:coord.getAction().getWorkflow().getConfiguration().getProperty()){
-        	if(prop.getName().equals("mapred.job.priority")){
-        		assertEquals(prop.getValue(), "NORMAL");
-        		break;
-        	}
+        feed = (Feed) storeEntity(EntityType.FEED, FEED);
+
+    }
+
+    protected Entity storeEntity(EntityType type, String path) throws Exception {
+        Unmarshaller unmarshaller = type.getUnmarshaller();
+        Entity entity = (Entity) unmarshaller
+                .unmarshal(OozieFeedMapperTest.class.getResource(path));
+        store.publish(type, entity);
+        return entity;
+    }
+
+    protected void cleanupStore() throws FalconException {
+        for (EntityType type : EntityType.values()) {
+            Collection<String> entities = store.getEntities(type);
+            for (String entity : entities) {
+                store.remove(type, entity);
+            }
+        }
+    }
+
+    @AfterClass
+    public void stopDFS() {
+        srcMiniDFS.shutdown();
+        trgMiniDFS.shutdown();
+    }
+
+    @Test
+    public void testReplicationCoords() throws FalconException {
+        OozieFeedMapper feedMapper = new OozieFeedMapper(feed);
+        List<COORDINATORAPP> coords = feedMapper.getCoordinators(trgCluster,
+                new Path("/projects/falcon/"));
+        COORDINATORAPP coord = coords.get(0);
+
+        Assert.assertEquals("${nameNode}/projects/falcon/REPLICATION", coord
+                .getAction().getWorkflow().getAppPath());
+        Assert.assertEquals("FALCON_FEED_REPLICATION_" + feed.getName() + "_"
+                + srcCluster.getName(), coord.getName());
+        Assert.assertEquals("${coord:minutes(20)}", coord.getFrequency());
+        SYNCDATASET inputDataset = (SYNCDATASET) coord.getDatasets()
+                .getDatasetOrAsyncDataset().get(0);
+        SYNCDATASET outputDataset = (SYNCDATASET) coord.getDatasets()
+                .getDatasetOrAsyncDataset().get(1);
+
+        Assert.assertEquals("${coord:minutes(20)}", inputDataset.getFrequency());
+        Assert.assertEquals("input-dataset", inputDataset.getName());
+        Assert.assertEquals(
+                ClusterHelper.getStorageUrl(srcCluster)
+                        + "/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}",
+                inputDataset.getUriTemplate());
+
+        Assert.assertEquals("${coord:minutes(20)}",
+                outputDataset.getFrequency());
+        Assert.assertEquals("output-dataset", outputDataset.getName());
+        Assert.assertEquals(
+                "${nameNode}"
+                        + "/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}",
+                outputDataset.getUriTemplate());
+        for (Property prop : coord.getAction().getWorkflow().getConfiguration().getProperty()) {
+            if (prop.getName().equals("mapred.job.priority")) {
+                assertEquals(prop.getValue(), "NORMAL");
+                break;
+            }
         }
 
-	}
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/test/resources/feed.xml
----------------------------------------------------------------------
diff --git a/feed/src/test/resources/feed.xml b/feed/src/test/resources/feed.xml
index aac717c..d5948b0 100644
--- a/feed/src/test/resources/feed.xml
+++ b/feed/src/test/resources/feed.xml
@@ -16,39 +16,41 @@
   limitations under the License.
   -->
 <feed description="clicks log" name="raw-logs" xmlns="uri:falcon:feed:0.1"
-	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+        >
 
-	<groups>online,bi</groups>
+    <groups>online,bi</groups>
 
-	<frequency>minutes(20)</frequency>
+    <frequency>minutes(20)</frequency>
     <timezone>UTC</timezone>
-    
-	<late-arrival cut-off="minutes(3)" />
-	<clusters>
-		<cluster name="corp1" type="source">
-			<validity start="2010-01-01T00:00Z" end="2010-01-01T02:00Z"/>
-			<retention limit="minutes(5)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
-		</cluster>
-		<cluster name="corp2" type="target">
-			<validity start="2010-01-01T00:00Z" end="2010-01-01T02:00Z"/>
-			<retention limit="minutes(7)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
-		</cluster>
-	</clusters>
-
-	<locations>
-		<location type="data"
-			path="/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}" />
-		<location type="stats" path="/projects/falcon/clicksStats" />
-		<location type="meta" path="/projects/falcon/clicksMetaData" />
-	</locations>
-
-	<ACL owner="testuser" group="group" permission="0x755" />
-	<schema location="/schema/clicks" provider="protobuf" />
-
-	<properties>
-		<property name="field3" value="value3" />
-		<property name="field2" value="value2" />
-
-		<property name="field4" value="value2" />
-	</properties>
+
+    <late-arrival cut-off="minutes(3)"/>
+    <clusters>
+        <cluster name="corp1" type="source">
+            <validity start="2010-01-01T00:00Z" end="2010-01-01T02:00Z"/>
+            <retention limit="minutes(5)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+        </cluster>
+        <cluster name="corp2" type="target">
+            <validity start="2010-01-01T00:00Z" end="2010-01-01T02:00Z"/>
+            <retention limit="minutes(7)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+        </cluster>
+    </clusters>
+
+    <locations>
+        <location type="data"
+                  path="/examples/input-data/rawLogs/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
+    </locations>
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
+
+    <properties>
+        <property name="field3" value="value3"/>
+        <property name="field2" value="value2"/>
+
+        <property name="field4" value="value2"/>
+    </properties>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/test/resources/src-cluster.xml
----------------------------------------------------------------------
diff --git a/feed/src/test/resources/src-cluster.xml b/feed/src/test/resources/src-cluster.xml
index 0567c34..75d8ed0 100644
--- a/feed/src/test/resources/src-cluster.xml
+++ b/feed/src/test/resources/src-cluster.xml
@@ -16,25 +16,25 @@
   limitations under the License.
   -->
 <cluster colo="gs1" description="" name="corp1" xmlns="uri:falcon:cluster:0.1"
-	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="http://localhost:50070"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/falcon/working" />
-		<location name="staging" path="/projects/falcon/staging" />
-	</locations>
-	<properties>
-		<property name="separator" value="-" />
-	</properties>
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="http://localhost:50070"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+        <location name="staging" path="/projects/falcon/staging"/>
+    </locations>
+    <properties>
+        <property name="separator" value="-"/>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/feed/src/test/resources/trg-cluster.xml
----------------------------------------------------------------------
diff --git a/feed/src/test/resources/trg-cluster.xml b/feed/src/test/resources/trg-cluster.xml
index d05c3f2..9a99b62 100644
--- a/feed/src/test/resources/trg-cluster.xml
+++ b/feed/src/test/resources/trg-cluster.xml
@@ -16,25 +16,25 @@
   limitations under the License.
   -->
 <cluster colo="gs2" description="" name="corp2" xmlns="uri:falcon:cluster:0.1"
-	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="http://localhost:50070"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/falcon/working" />
-		<location name="staging" path="/projects/falcon/staging2" />
-	</locations>
-	<properties>
-		<property name="separator" value="-" />
-	</properties>
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="http://localhost:50070"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+        <location name="staging" path="/projects/falcon/staging2"/>
+    </locations>
+    <properties>
+        <property name="separator" value="-"/>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
index 6112e98..f6b90df 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
@@ -18,6 +18,13 @@
 
 package org.apache.falcon.messaging;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.log4j.Logger;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -28,200 +35,190 @@ import java.util.Date;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.log4j.Logger;
-
 /**
  * Value Object which is stored in JMS Topic as MapMessage
- * 
  */
 public class EntityInstanceMessage {
 
-	private final Map<ARG, String> keyValueMap = new LinkedHashMap<ARG, String>();
-	private static final Logger LOG = Logger
-			.getLogger(EntityInstanceMessage.class);
-	private static final String FALCON_ENTITY_TOPIC_NAME = "FALCON.ENTITY.TOPIC";
-
-	public enum EntityOps {
-		GENERATE, DELETE, ARCHIVE, REPLICATE, CHMOD
-	}
-
-	public enum ARG {
-		entityName("entityName"), feedNames("feedNames"), feedInstancePaths(
-				"feedInstancePaths"), workflowId("workflowId"), runId("runId"), nominalTime(
-				"nominalTime"), timeStamp("timeStamp"), brokerUrl("broker.url"), brokerImplClass(
-				"broker.impl.class"), entityType("entityType"), operation(
-				"operation"), logFile("logFile"), topicName("topicName"), status(
-				"status"), brokerTTL("broker.ttlInMins"),cluster("cluster");
-
-		private String propName;
-
-		private ARG(String propName) {
-			this.propName = propName;
-		}
-
-		/**
-		 * 
-		 * @return Name of the Argument used in the parent workflow to pass
-		 *         arguments to MessageProducer Main class.
-		 */
-		public String getArgName() {
-			return this.name();
-		}
-
-		/**
-		 * 
-		 * @return Name of the property used in the startup.properties,
-		 *         coordinator and parent workflow.
-		 */
-		public String getPropName() {
-			return this.propName;
-		}
-	}
-
-	public Map<ARG, String> getKeyValueMap() {
-		return this.keyValueMap;
-	}
-
-	public String getTopicName() {
-		return this.keyValueMap.get(ARG.topicName);
-	}
-
-	public String getFeedName() {
-		return this.keyValueMap.get(ARG.feedNames);
-	}
-
-	public void setFeedName(String feedName) {
-		this.keyValueMap.remove(ARG.feedNames);
-		this.keyValueMap.put(ARG.feedNames, feedName);
-	}
-
-	public String getFeedInstancePath() {
-		return this.keyValueMap.get(ARG.feedInstancePaths);
-	}
-
-	public void setFeedInstancePath(String feedInstancePath) {
-		this.keyValueMap.remove(ARG.feedInstancePaths);
-		this.keyValueMap.put(ARG.feedInstancePaths, feedInstancePath);
-	}
-
-	public String getEntityType() {
-		return this.keyValueMap.get(ARG.entityType);
-	}
-
-	public String getBrokerTTL() {
-		return this.keyValueMap.get(ARG.brokerTTL);
-	}
-
-	public void convertDateFormat() throws ParseException {
-		String date = this.keyValueMap.remove(ARG.nominalTime);
-		this.keyValueMap.put(ARG.nominalTime, getFalconDate(date));
-		date = this.keyValueMap.remove(ARG.timeStamp);
-		this.keyValueMap.put(ARG.timeStamp, getFalconDate(date));
-	}
-
-	public static EntityInstanceMessage[] getMessages(CommandLine cmd)
-			throws ParseException {
-		String[] feedNames = getFeedNames(cmd);
-		if(feedNames == null){
-		    return null;
-		}
-		
-		String[] feedPaths;
-		try {
-			feedPaths = getFeedPaths(cmd);
-		} catch (IOException e) {
-			LOG.error("Error getting instance paths: ", e);
-			throw new RuntimeException(e);
-		}
-
-		EntityInstanceMessage[] messages = new EntityInstanceMessage[feedPaths.length];
-		for (int i = 0; i < feedPaths.length; i++) {
-			EntityInstanceMessage message = new EntityInstanceMessage();
-			setDefaultValues(cmd, message);
-			// override default values
-			if (message.getEntityType().equalsIgnoreCase("PROCESS")) {
-				message.setFeedName(feedNames[i]);
-			} else {
-				message.setFeedName(message.getFeedName());
-			}
-			message.setFeedInstancePath(feedPaths[i]);
-			message.convertDateFormat();
-			messages[i] = message;
-		}
-
-		return messages;
-	}
-
-	private static void setDefaultValues(CommandLine cmd,
-			EntityInstanceMessage message) {
-		for (ARG arg : ARG.values()) {
-			message.keyValueMap.put(arg, cmd.getOptionValue(arg.name()));
-		}
-	}
-
-	private static String[] getFeedNames(CommandLine cmd) {
-		String feedNameStr = cmd.getOptionValue(ARG.feedNames.getArgName());
+    private final Map<ARG, String> keyValueMap = new LinkedHashMap<ARG, String>();
+    private static final Logger LOG = Logger
+            .getLogger(EntityInstanceMessage.class);
+    private static final String FALCON_ENTITY_TOPIC_NAME = "FALCON.ENTITY.TOPIC";
+
+    public enum EntityOps {
+        GENERATE, DELETE, ARCHIVE, REPLICATE, CHMOD
+    }
+
+    public enum ARG {
+        entityName("entityName"), feedNames("feedNames"), feedInstancePaths(
+                "feedInstancePaths"), workflowId("workflowId"), runId("runId"), nominalTime(
+                "nominalTime"), timeStamp("timeStamp"), brokerUrl("broker.url"), brokerImplClass(
+                "broker.impl.class"), entityType("entityType"), operation(
+                "operation"), logFile("logFile"), topicName("topicName"), status(
+                "status"), brokerTTL("broker.ttlInMins"), cluster("cluster");
+
+        private String propName;
+
+        private ARG(String propName) {
+            this.propName = propName;
+        }
+
+        /**
+         * @return Name of the Argument used in the parent workflow to pass
+         *         arguments to MessageProducer Main class.
+         */
+        public String getArgName() {
+            return this.name();
+        }
+
+        /**
+         * @return Name of the property used in the startup.properties,
+         *         coordinator and parent workflow.
+         */
+        public String getPropName() {
+            return this.propName;
+        }
+    }
+
+    public Map<ARG, String> getKeyValueMap() {
+        return this.keyValueMap;
+    }
+
+    public String getTopicName() {
+        return this.keyValueMap.get(ARG.topicName);
+    }
+
+    public String getFeedName() {
+        return this.keyValueMap.get(ARG.feedNames);
+    }
+
+    public void setFeedName(String feedName) {
+        this.keyValueMap.remove(ARG.feedNames);
+        this.keyValueMap.put(ARG.feedNames, feedName);
+    }
+
+    public String getFeedInstancePath() {
+        return this.keyValueMap.get(ARG.feedInstancePaths);
+    }
+
+    public void setFeedInstancePath(String feedInstancePath) {
+        this.keyValueMap.remove(ARG.feedInstancePaths);
+        this.keyValueMap.put(ARG.feedInstancePaths, feedInstancePath);
+    }
+
+    public String getEntityType() {
+        return this.keyValueMap.get(ARG.entityType);
+    }
+
+    public String getBrokerTTL() {
+        return this.keyValueMap.get(ARG.brokerTTL);
+    }
+
+    public void convertDateFormat() throws ParseException {
+        String date = this.keyValueMap.remove(ARG.nominalTime);
+        this.keyValueMap.put(ARG.nominalTime, getFalconDate(date));
+        date = this.keyValueMap.remove(ARG.timeStamp);
+        this.keyValueMap.put(ARG.timeStamp, getFalconDate(date));
+    }
+
+    public static EntityInstanceMessage[] getMessages(CommandLine cmd)
+            throws ParseException {
+        String[] feedNames = getFeedNames(cmd);
+        if (feedNames == null) {
+            return null;
+        }
+
+        String[] feedPaths;
+        try {
+            feedPaths = getFeedPaths(cmd);
+        } catch (IOException e) {
+            LOG.error("Error getting instance paths: ", e);
+            throw new RuntimeException(e);
+        }
+
+        EntityInstanceMessage[] messages = new EntityInstanceMessage[feedPaths.length];
+        for (int i = 0; i < feedPaths.length; i++) {
+            EntityInstanceMessage message = new EntityInstanceMessage();
+            setDefaultValues(cmd, message);
+            // override default values
+            if (message.getEntityType().equalsIgnoreCase("PROCESS")) {
+                message.setFeedName(feedNames[i]);
+            } else {
+                message.setFeedName(message.getFeedName());
+            }
+            message.setFeedInstancePath(feedPaths[i]);
+            message.convertDateFormat();
+            messages[i] = message;
+        }
+
+        return messages;
+    }
+
+    private static void setDefaultValues(CommandLine cmd,
+                                         EntityInstanceMessage message) {
+        for (ARG arg : ARG.values()) {
+            message.keyValueMap.put(arg, cmd.getOptionValue(arg.name()));
+        }
+    }
+
+    private static String[] getFeedNames(CommandLine cmd) {
+        String feedNameStr = cmd.getOptionValue(ARG.feedNames.getArgName());
+        String topicName = cmd.getOptionValue(ARG.topicName.getArgName());
+        if (topicName.equals(FALCON_ENTITY_TOPIC_NAME)) {
+            return new String[]{feedNameStr};
+        }
+        if (feedNameStr.equals("null")) {
+            return null;
+        }
+
+        return feedNameStr.split(",");
+    }
+
+    private static String[] getFeedPaths(CommandLine cmd) throws IOException {
         String topicName = cmd.getOptionValue(ARG.topicName.getArgName());
-		if (topicName.equals(FALCON_ENTITY_TOPIC_NAME)) {
-			return new String[] { feedNameStr };
-		}
-		if (feedNameStr.equals("null")) {
-			return null;
-		}
-		
-		return feedNameStr.split(",");
-	}
-
-	private static String[] getFeedPaths(CommandLine cmd) throws IOException {
-		String topicName = cmd.getOptionValue(ARG.topicName.getArgName());
-		String operation = cmd.getOptionValue(ARG.operation.getArgName());
-
-		if (topicName.equals(FALCON_ENTITY_TOPIC_NAME)) {
-			LOG.debug("Returning instance paths for Falcon Topic: "
-					+ cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
-			return new String[] { cmd.getOptionValue(ARG.feedInstancePaths
-					.getArgName()) };
-		}
-
-		if (operation.equals(EntityOps.GENERATE.name())
-				|| operation.equals(EntityOps.REPLICATE.name())) {
-			LOG.debug("Returning instance paths: "
-					+ cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
-			return cmd.getOptionValue(ARG.feedInstancePaths.getArgName())
-					.split(",");
-		}
-		//else case of feed retention
-		Path logFile = new Path(cmd.getOptionValue(ARG.logFile.getArgName()));
-		FileSystem fs = FileSystem.get(logFile.toUri(), new Configuration());
-		ByteArrayOutputStream writer = new ByteArrayOutputStream();
-		InputStream instance = fs.open(logFile);
-		IOUtils.copyBytes(instance, writer, 4096, true);
-		String[] instancePaths = writer.toString().split("=");
-		fs.delete(logFile, true);
-		LOG.info("Deleted feed instance paths file:"+logFile);
-		if (instancePaths.length == 1) {
-			LOG.debug("Returning 0 instance paths for feed ");
-			return new String[0];
-		} else {
-			LOG.debug("Returning instance paths for feed " + instancePaths[1]);
-			return instancePaths[1].split(",");
-		}
-
-	}
-
-	public String getFalconDate(String nominalTime) throws ParseException {
-		DateFormat nominalFormat = new SimpleDateFormat(
-				"yyyy'-'MM'-'dd'-'HH'-'mm");
-		Date nominalDate = nominalFormat.parse(nominalTime);
-		DateFormat falconFormat = new SimpleDateFormat(
-				"yyyy'-'MM'-'dd'T'HH':'mm'Z'");
-		return falconFormat.format(nominalDate);
-
-	}
+        String operation = cmd.getOptionValue(ARG.operation.getArgName());
+
+        if (topicName.equals(FALCON_ENTITY_TOPIC_NAME)) {
+            LOG.debug("Returning instance paths for Falcon Topic: "
+                    + cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
+            return new String[]{cmd.getOptionValue(ARG.feedInstancePaths
+                    .getArgName())};
+        }
+
+        if (operation.equals(EntityOps.GENERATE.name())
+                || operation.equals(EntityOps.REPLICATE.name())) {
+            LOG.debug("Returning instance paths: "
+                    + cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
+            return cmd.getOptionValue(ARG.feedInstancePaths.getArgName())
+                    .split(",");
+        }
+        //else case of feed retention
+        Path logFile = new Path(cmd.getOptionValue(ARG.logFile.getArgName()));
+        FileSystem fs = FileSystem.get(logFile.toUri(), new Configuration());
+        ByteArrayOutputStream writer = new ByteArrayOutputStream();
+        InputStream instance = fs.open(logFile);
+        IOUtils.copyBytes(instance, writer, 4096, true);
+        String[] instancePaths = writer.toString().split("=");
+        fs.delete(logFile, true);
+        LOG.info("Deleted feed instance paths file:" + logFile);
+        if (instancePaths.length == 1) {
+            LOG.debug("Returning 0 instance paths for feed ");
+            return new String[0];
+        } else {
+            LOG.debug("Returning instance paths for feed " + instancePaths[1]);
+            return instancePaths[1].split(",");
+        }
+
+    }
+
+    public String getFalconDate(String nominalTime) throws ParseException {
+        DateFormat nominalFormat = new SimpleDateFormat(
+                "yyyy'-'MM'-'dd'-'HH'-'mm");
+        Date nominalDate = nominalFormat.parse(nominalTime);
+        DateFormat falconFormat = new SimpleDateFormat(
+                "yyyy'-'MM'-'dd'T'HH':'mm'Z'");
+        return falconFormat.format(nominalDate);
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
index 867eb44..0a8e7df 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
@@ -18,42 +18,41 @@
 
 package org.apache.falcon.messaging;
 
-import java.util.Map.Entry;
+import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
 
 import javax.jms.JMSException;
 import javax.jms.MapMessage;
 import javax.jms.Message;
 import javax.jms.Session;
-
-import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
+import java.util.Map.Entry;
 
 /**
  * Falcon JMS message creator- creates JMS TextMessage
  */
 public class EntityInstanceMessageCreator {
 
-	private MapMessage mapMessage;
+    private MapMessage mapMessage;
 
-	private final EntityInstanceMessage instanceMessage;
+    private final EntityInstanceMessage instanceMessage;
 
-	public EntityInstanceMessageCreator(EntityInstanceMessage instanceMessage) {
-		this.instanceMessage = instanceMessage;
-	}
+    public EntityInstanceMessageCreator(EntityInstanceMessage instanceMessage) {
+        this.instanceMessage = instanceMessage;
+    }
 
-	public Message createMessage(Session session) throws JMSException {
-		mapMessage = session.createMapMessage();
-		for (Entry<ARG, String> entry : instanceMessage.getKeyValueMap()
-				.entrySet()) {
-			mapMessage.setString(entry.getKey().getArgName(), instanceMessage
-					.getKeyValueMap().get(entry.getKey()));
-		}
-		return mapMessage;
+    public Message createMessage(Session session) throws JMSException {
+        mapMessage = session.createMapMessage();
+        for (Entry<ARG, String> entry : instanceMessage.getKeyValueMap()
+                .entrySet()) {
+            mapMessage.setString(entry.getKey().getArgName(), instanceMessage
+                    .getKeyValueMap().get(entry.getKey()));
+        }
+        return mapMessage;
 
-	}
+    }
 
-	@Override
-	public String toString() {
-		return this.mapMessage.toString();
-	}
+    @Override
+    public String toString() {
+        return this.mapMessage.toString();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java b/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
index dbbf70d..cb0ad8a 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
@@ -18,167 +18,160 @@
 
 package org.apache.falcon.messaging;
 
-import java.lang.reflect.InvocationTargetException;
-
-import javax.jms.Connection;
-import javax.jms.ConnectionFactory;
-import javax.jms.DeliveryMode;
-import javax.jms.JMSException;
-import javax.jms.Session;
-import javax.jms.Topic;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.*;
+import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
 import org.apache.log4j.Logger;
 
+import javax.jms.*;
+import java.lang.reflect.InvocationTargetException;
+
 public class MessageProducer extends Configured implements Tool {
 
-	private Connection connection;
-	private static final Logger LOG = Logger.getLogger(MessageProducer.class);
-	private static final long DEFAULT_TTL = 3 * 24 * 60 * 60 * 1000;
-
-	/**
-	 * 
-	 * @param arguments
-	 *            - Accepts a Message to be send to JMS topic, creates a new
-	 *            Topic based on topic name if it does not exist or else
-	 *            existing topic with the same name is used to send the message.
-	 * @throws JMSException
-	 */
-	protected void sendMessage(EntityInstanceMessage entityInstanceMessage)
-			throws JMSException {
-
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Topic entityTopic = session.createTopic(entityInstanceMessage
-				.getTopicName());
-		javax.jms.MessageProducer producer = session
-				.createProducer(entityTopic);
-		producer.setDeliveryMode(DeliveryMode.PERSISTENT);
-		long messageTTL = DEFAULT_TTL;
-		try {
-			long messageTTLinMins = Long.parseLong(entityInstanceMessage
-					.getBrokerTTL());
-			messageTTL = messageTTLinMins * 60 * 1000;
-		} catch (NumberFormatException e) {
-			LOG.error("Error in parsing broker.ttl, setting TTL to:"
-					+ DEFAULT_TTL + " milli-seconds");
-		}
-		producer.setTimeToLive(messageTTL);
-		producer.send(new EntityInstanceMessageCreator(entityInstanceMessage)
-				.createMessage(session));
-
-	}
-
-	public static void main(String[] args) throws Exception {
-		ToolRunner.run(new MessageProducer(), args);
-	}
-
-	private void createAndStartConnection(String implementation,
-			String userName, String password, String url) throws JMSException,
-			ClassNotFoundException, IllegalArgumentException,
-			SecurityException, InstantiationException, IllegalAccessException,
-			InvocationTargetException, NoSuchMethodException {
-
-		Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) MessageProducer.class
-				.getClassLoader().loadClass(implementation);
-
-		ConnectionFactory connectionFactory = clazz.getConstructor(
-				String.class, String.class, String.class).newInstance(userName,
-				password, url);
-
-		connection = connectionFactory.createConnection();
-		connection.start();
-	}
-
-	private static CommandLine getCommand(String[] arguments)
-			throws ParseException {
-		Options options = new Options();
-		addOption(options, new Option(ARG.brokerImplClass.getArgName(), true,
-				"message broker Implementation class"));
-		addOption(options, new Option(ARG.brokerTTL.getArgName(), true,
-				"message time-to-live"));
-		addOption(options, new Option(ARG.brokerUrl.getArgName(), true,
-				"message broker url"));
-		addOption(options, new Option(ARG.entityName.getArgName(), true,
-				"name of the entity"));
-		addOption(options, new Option(ARG.entityType.getArgName(), true,
-				"type of the entity"));
-		addOption(options, new Option(ARG.feedInstancePaths.getArgName(),
-				true, "feed instance paths"));
-		addOption(options, new Option(ARG.feedNames.getArgName(), true,
-				"feed names"));
-		addOption(options, new Option(ARG.logFile.getArgName(), true,
-				"log file path"));
-		addOption(options, new Option(ARG.nominalTime.getArgName(), true,
-				"instance time"));
-		addOption(options, new Option(ARG.operation.getArgName(), true,
-				"operation like generate, delete, archive"));
-		addOption(options, new Option(ARG.runId.getArgName(), true,
-				"current run-id of the instance"));
-		addOption(options, new Option(ARG.status.getArgName(), true,
-				"status of workflow instance"));
-		addOption(options, new Option(ARG.timeStamp.getArgName(), true,
-				"current timestamp"));
-		addOption(options, new Option(ARG.topicName.getArgName(), true,
-				"name of the topic to be used to send message"));
-		addOption(options, new Option(ARG.workflowId.getArgName(), true,
-				"workflow id"));
-		addOption(options, new Option(ARG.cluster.getArgName(), true,
-				"cluster name"));
-
-		return new GnuParser().parse(options, arguments);
-	}
-
-	private static void addOption(Options options, Option opt) {
-		opt.setRequired(true);
-		options.addOption(opt);
-	}
-
-	@Override
-	public int run(String[] args) throws Exception {
-		CommandLine cmd;
-		try {
-			cmd = getCommand(args);
-		} catch (ParseException e) {
-			throw new Exception("Unable to parse arguments: ", e);
-		}
-		EntityInstanceMessage[] entityInstanceMessage = EntityInstanceMessage
-				.getMessages(cmd);
-		if (entityInstanceMessage == null || entityInstanceMessage.length == 0) {
-			LOG.warn("No operation on output feed");
-			return 0;
-		}
-
-		MessageProducer falconMessageProducer = new MessageProducer();
-		try {
-			falconMessageProducer.createAndStartConnection(
-					cmd.getOptionValue(ARG.brokerImplClass.name()), "",
-					"", cmd.getOptionValue(ARG.brokerUrl.name()));
-			for (EntityInstanceMessage message : entityInstanceMessage) {
-				LOG.info("Sending message:" + message.getKeyValueMap());
-				falconMessageProducer.sendMessage(message);
-			}
-		} catch (JMSException e) {
-			LOG.error("Error in getConnection:", e);
-		} catch (Exception e) {
-			LOG.error("Error in getConnection:", e);
-		} finally {
-			try {
-			    if(falconMessageProducer.connection != null)
-			        falconMessageProducer.connection.close();
-			} catch (JMSException e) {
-				LOG.error("Error in closing connection:", e);
-			}
-		}
-		return 0;
-	}
+    private Connection connection;
+    private static final Logger LOG = Logger.getLogger(MessageProducer.class);
+    private static final long DEFAULT_TTL = 3 * 24 * 60 * 60 * 1000;
+
+    /**
+     * @param arguments - Accepts a Message to be send to JMS topic, creates a new
+     *                  Topic based on topic name if it does not exist or else
+     *                  existing topic with the same name is used to send the message.
+     * @throws JMSException
+     */
+    protected void sendMessage(EntityInstanceMessage entityInstanceMessage)
+            throws JMSException {
+
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Topic entityTopic = session.createTopic(entityInstanceMessage
+                .getTopicName());
+        javax.jms.MessageProducer producer = session
+                .createProducer(entityTopic);
+        producer.setDeliveryMode(DeliveryMode.PERSISTENT);
+        long messageTTL = DEFAULT_TTL;
+        try {
+            long messageTTLinMins = Long.parseLong(entityInstanceMessage
+                    .getBrokerTTL());
+            messageTTL = messageTTLinMins * 60 * 1000;
+        } catch (NumberFormatException e) {
+            LOG.error("Error in parsing broker.ttl, setting TTL to:"
+                    + DEFAULT_TTL + " milli-seconds");
+        }
+        producer.setTimeToLive(messageTTL);
+        producer.send(new EntityInstanceMessageCreator(entityInstanceMessage)
+                .createMessage(session));
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        ToolRunner.run(new MessageProducer(), args);
+    }
+
+    private void createAndStartConnection(String implementation,
+                                          String userName, String password, String url) throws JMSException,
+                                                                                               ClassNotFoundException,
+                                                                                               IllegalArgumentException,
+                                                                                               SecurityException,
+                                                                                               InstantiationException,
+                                                                                               IllegalAccessException,
+                                                                                               InvocationTargetException,
+                                                                                               NoSuchMethodException {
+
+        Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) MessageProducer.class
+                .getClassLoader().loadClass(implementation);
+
+        ConnectionFactory connectionFactory = clazz.getConstructor(
+                String.class, String.class, String.class).newInstance(userName,
+                password, url);
+
+        connection = connectionFactory.createConnection();
+        connection.start();
+    }
+
+    private static CommandLine getCommand(String[] arguments)
+            throws ParseException {
+        Options options = new Options();
+        addOption(options, new Option(ARG.brokerImplClass.getArgName(), true,
+                "message broker Implementation class"));
+        addOption(options, new Option(ARG.brokerTTL.getArgName(), true,
+                "message time-to-live"));
+        addOption(options, new Option(ARG.brokerUrl.getArgName(), true,
+                "message broker url"));
+        addOption(options, new Option(ARG.entityName.getArgName(), true,
+                "name of the entity"));
+        addOption(options, new Option(ARG.entityType.getArgName(), true,
+                "type of the entity"));
+        addOption(options, new Option(ARG.feedInstancePaths.getArgName(),
+                true, "feed instance paths"));
+        addOption(options, new Option(ARG.feedNames.getArgName(), true,
+                "feed names"));
+        addOption(options, new Option(ARG.logFile.getArgName(), true,
+                "log file path"));
+        addOption(options, new Option(ARG.nominalTime.getArgName(), true,
+                "instance time"));
+        addOption(options, new Option(ARG.operation.getArgName(), true,
+                "operation like generate, delete, archive"));
+        addOption(options, new Option(ARG.runId.getArgName(), true,
+                "current run-id of the instance"));
+        addOption(options, new Option(ARG.status.getArgName(), true,
+                "status of workflow instance"));
+        addOption(options, new Option(ARG.timeStamp.getArgName(), true,
+                "current timestamp"));
+        addOption(options, new Option(ARG.topicName.getArgName(), true,
+                "name of the topic to be used to send message"));
+        addOption(options, new Option(ARG.workflowId.getArgName(), true,
+                "workflow id"));
+        addOption(options, new Option(ARG.cluster.getArgName(), true,
+                "cluster name"));
+
+        return new GnuParser().parse(options, arguments);
+    }
+
+    private static void addOption(Options options, Option opt) {
+        opt.setRequired(true);
+        options.addOption(opt);
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        CommandLine cmd;
+        try {
+            cmd = getCommand(args);
+        } catch (ParseException e) {
+            throw new Exception("Unable to parse arguments: ", e);
+        }
+        EntityInstanceMessage[] entityInstanceMessage = EntityInstanceMessage
+                .getMessages(cmd);
+        if (entityInstanceMessage == null || entityInstanceMessage.length == 0) {
+            LOG.warn("No operation on output feed");
+            return 0;
+        }
+
+        MessageProducer falconMessageProducer = new MessageProducer();
+        try {
+            falconMessageProducer.createAndStartConnection(
+                    cmd.getOptionValue(ARG.brokerImplClass.name()), "",
+                    "", cmd.getOptionValue(ARG.brokerUrl.name()));
+            for (EntityInstanceMessage message : entityInstanceMessage) {
+                LOG.info("Sending message:" + message.getKeyValueMap());
+                falconMessageProducer.sendMessage(message);
+            }
+        } catch (JMSException e) {
+            LOG.error("Error in getConnection:", e);
+        } catch (Exception e) {
+            LOG.error("Error in getConnection:", e);
+        } finally {
+            try {
+                if (falconMessageProducer.connection != null) {
+                    falconMessageProducer.connection.close();
+                }
+            } catch (JMSException e) {
+                LOG.error("Error in closing connection:", e);
+            }
+        }
+        return 0;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/main/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/messaging/src/main/resources/log4j.xml b/messaging/src/main/resources/log4j.xml
index a69599b..2788c63 100644
--- a/messaging/src/main/resources/log4j.xml
+++ b/messaging/src/main/resources/log4j.xml
@@ -20,44 +20,44 @@
 <!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
 
 <log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-  <appender name="console" class="org.apache.log4j.ConsoleAppender">
-    <param name="Target" value="System.out"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
-
-  <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/application.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
-
-  <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/audit.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - %m%n"/>
-    </layout>
-  </appender>
-
-  <logger name="org.apache.falcon" additivity="false">
-    <level value="debug"/>
-    <appender-ref ref="console" />
-  </logger>
-
-  <logger name="AUDIT">
-    <level value="info"/>
-    <appender-ref ref="AUDIT" />
-  </logger>
-
-  <root>
-    <priority value ="info" />
-    <appender-ref ref="console" />
-  </root>
+    <appender name="console" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/application.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/audit.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - %m%n"/>
+        </layout>
+    </appender>
+
+    <logger name="org.apache.falcon" additivity="false">
+        <level value="debug"/>
+        <appender-ref ref="console"/>
+    </logger>
+
+    <logger name="AUDIT">
+        <level value="info"/>
+        <appender-ref ref="AUDIT"/>
+    </logger>
+
+    <root>
+        <priority value="info"/>
+        <appender-ref ref="console"/>
+    </root>
 
 </log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
index 90c4689..9a13738 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
@@ -17,14 +17,6 @@
  */
 package org.apache.falcon.messaging;
 
-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.falcon.messaging.EntityInstanceMessage.ARG;
@@ -33,137 +25,140 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import javax.jms.*;
+
 public class FalconTopicProducerTest {
 
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
-	private static final String TOPIC_NAME = "FALCON.ENTITY.TOPIC";
-	private BrokerService broker;
-
-	private volatile AssertionError error;
-
-	@BeforeClass
-	public void setup() throws Exception {
-		broker = new BrokerService();
-		broker.addConnector(BROKER_URL);
-		broker.setDataDirectory("target/activemq");
-		broker.setBrokerName("localhost");
-		broker.start();
-	}
-
-	@AfterClass
-	public void tearDown() throws Exception {
-		broker.deleteAllMessages();
-		broker.stop();
-	}
-	
-	@Test
-	public void testWithFeedOutputPaths() throws Exception{
-		String [] args = new String[] { "-" + ARG.entityName.getArgName(), "agg-coord",
-				"-" + ARG.feedNames.getArgName(), "click-logs,raw-logs",
-				"-" + ARG.feedInstancePaths.getArgName(),
-				"/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
-				"-" + ARG.workflowId.getArgName(), "workflow-01-00",
-				"-" + ARG.runId.getArgName(), "1",
-				"-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
-				"-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
-				"-" + ARG.brokerUrl.getArgName(), BROKER_URL,
-				"-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
-				"-" + ARG.entityType.getArgName(), ("process"),
-				"-" + ARG.operation.getArgName(), ("GENERATE"),
-				"-" + ARG.logFile.getArgName(), ("/logFile"),
-				"-" + ARG.topicName.getArgName(), (TOPIC_NAME),
-				"-" + ARG.status.getArgName(), ("SUCCEEDED"),
-				"-" + ARG.brokerTTL.getArgName(), "10",
-				"-" + ARG.cluster.getArgName(), "corp" };
-		testProcessMessageCreator(args);
-	}
-	
-	@Test
-	public void testWithEmptyFeedOutputPaths() throws Exception{
-		String [] args = new String[] { "-" + ARG.entityName.getArgName(), "agg-coord",
-				"-" + ARG.feedNames.getArgName(), "null",
-				"-" + ARG.feedInstancePaths.getArgName(),
-				"null",
-				"-" + ARG.workflowId.getArgName(), "workflow-01-00",
-				"-" + ARG.runId.getArgName(), "1",
-				"-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
-				"-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
-				"-" + ARG.brokerUrl.getArgName(), BROKER_URL,
-				"-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
-				"-" + ARG.entityType.getArgName(), ("process"),
-				"-" + ARG.operation.getArgName(), ("GENERATE"),
-				"-" + ARG.logFile.getArgName(), ("/logFile"),
-				"-" + ARG.topicName.getArgName(), (TOPIC_NAME),
-				"-" + ARG.status.getArgName(), ("SUCCEEDED"),
-				"-" + ARG.brokerTTL.getArgName(), "10",
-				"-" + ARG.cluster.getArgName(), "corp" };
-		testProcessMessageCreator(args);
-	}
-
-	private void testProcessMessageCreator(String[] args) throws Exception {
-
-		Thread t = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumer();
-				} catch (AssertionError e) {
-					error = e;
-				} catch (JMSException ignore) {
-
-				}
-			}
-		};
-		t.start();
-		Thread.sleep(1500);
-		new MessageProducer().run(args);
-		t.join();
-		if (error != null) {
-			throw error;
-		}
-	}
-
-	private void consumer() throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				BROKER_URL);
-		Connection connection = connectionFactory.createConnection();
-		connection.start();
-
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Destination destination = session.createTopic(TOPIC_NAME);
-		MessageConsumer consumer = session.createConsumer(destination);
-
-		// wait till you get atleast one message
-		MapMessage m;
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-
-		assertMessage(m);
-		Assert.assertTrue((m.getString(ARG.feedNames.getArgName())
-				.equals("click-logs,raw-logs"))
-				|| (m.getString(ARG.feedNames.getArgName()).equals("null")));
-		Assert.assertTrue(m.getString(ARG.feedInstancePaths.getArgName())
-				.equals("/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20")
-				|| (m.getString(ARG.feedInstancePaths.getArgName()).equals("null")));
-
-		connection.close();
-	}
-
-	private void assertMessage(MapMessage m) throws JMSException {
-		Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
-				"agg-coord");
-		Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
-				"workflow-01-00");
-		Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
-		Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
-				"2011-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
-				"2012-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
-	}
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String TOPIC_NAME = "FALCON.ENTITY.TOPIC";
+    private BrokerService broker;
+
+    private volatile AssertionError error;
+
+    @BeforeClass
+    public void setup() throws Exception {
+        broker = new BrokerService();
+        broker.addConnector(BROKER_URL);
+        broker.setDataDirectory("target/activemq");
+        broker.setBrokerName("localhost");
+        broker.start();
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        broker.deleteAllMessages();
+        broker.stop();
+    }
+
+    @Test
+    public void testWithFeedOutputPaths() throws Exception {
+        String[] args = new String[]{"-" + ARG.entityName.getArgName(), "agg-coord",
+                                     "-" + ARG.feedNames.getArgName(), "click-logs,raw-logs",
+                                     "-" + ARG.feedInstancePaths.getArgName(),
+                                     "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
+                                     "-" + ARG.workflowId.getArgName(), "workflow-01-00",
+                                     "-" + ARG.runId.getArgName(), "1",
+                                     "-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
+                                     "-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
+                                     "-" + ARG.brokerUrl.getArgName(), BROKER_URL,
+                                     "-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
+                                     "-" + ARG.entityType.getArgName(), ("process"),
+                                     "-" + ARG.operation.getArgName(), ("GENERATE"),
+                                     "-" + ARG.logFile.getArgName(), ("/logFile"),
+                                     "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
+                                     "-" + ARG.status.getArgName(), ("SUCCEEDED"),
+                                     "-" + ARG.brokerTTL.getArgName(), "10",
+                                     "-" + ARG.cluster.getArgName(), "corp"};
+        testProcessMessageCreator(args);
+    }
+
+    @Test
+    public void testWithEmptyFeedOutputPaths() throws Exception {
+        String[] args = new String[]{"-" + ARG.entityName.getArgName(), "agg-coord",
+                                     "-" + ARG.feedNames.getArgName(), "null",
+                                     "-" + ARG.feedInstancePaths.getArgName(),
+                                     "null",
+                                     "-" + ARG.workflowId.getArgName(), "workflow-01-00",
+                                     "-" + ARG.runId.getArgName(), "1",
+                                     "-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
+                                     "-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
+                                     "-" + ARG.brokerUrl.getArgName(), BROKER_URL,
+                                     "-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
+                                     "-" + ARG.entityType.getArgName(), ("process"),
+                                     "-" + ARG.operation.getArgName(), ("GENERATE"),
+                                     "-" + ARG.logFile.getArgName(), ("/logFile"),
+                                     "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
+                                     "-" + ARG.status.getArgName(), ("SUCCEEDED"),
+                                     "-" + ARG.brokerTTL.getArgName(), "10",
+                                     "-" + ARG.cluster.getArgName(), "corp"};
+        testProcessMessageCreator(args);
+    }
+
+    private void testProcessMessageCreator(String[] args) throws Exception {
+
+        Thread t = new Thread() {
+            @Override
+            public void run() {
+                try {
+                    consumer();
+                } catch (AssertionError e) {
+                    error = e;
+                } catch (JMSException ignore) {
+
+                }
+            }
+        };
+        t.start();
+        Thread.sleep(1500);
+        new MessageProducer().run(args);
+        t.join();
+        if (error != null) {
+            throw error;
+        }
+    }
+
+    private void consumer() throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                BROKER_URL);
+        Connection connection = connectionFactory.createConnection();
+        connection.start();
+
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Destination destination = session.createTopic(TOPIC_NAME);
+        MessageConsumer consumer = session.createConsumer(destination);
+
+        // wait till you get atleast one message
+        MapMessage m;
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+
+        assertMessage(m);
+        Assert.assertTrue((m.getString(ARG.feedNames.getArgName())
+                .equals("click-logs,raw-logs"))
+                || (m.getString(ARG.feedNames.getArgName()).equals("null")));
+        Assert.assertTrue(m.getString(ARG.feedInstancePaths.getArgName())
+                .equals("/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20")
+                || (m.getString(ARG.feedInstancePaths.getArgName()).equals("null")));
+
+        connection.close();
+    }
+
+    private void assertMessage(MapMessage m) throws JMSException {
+        Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
+                "agg-coord");
+        Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
+                "workflow-01-00");
+        Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
+        Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
+                "2011-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
+                "2012-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 5607493..45252dd 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
@@ -21,12 +21,12 @@ package org.apache.falcon.messaging;
 import org.apache.activemq.ActiveMQConnectionFactory;
 import org.apache.activemq.broker.BrokerService;
 import org.apache.activemq.util.ByteArrayInputStream;
+import org.apache.falcon.cluster.util.EmbeddedCluster;
+import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.falcon.cluster.util.EmbeddedCluster;
-import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -38,162 +38,166 @@ import java.io.OutputStream;
 
 public class FeedProducerTest {
 
-	private String[] args;
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
-	private static final String TOPIC_NAME = "Falcon.process1.click-logs";
-	private BrokerService broker;
+    private String[] args;
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String TOPIC_NAME = "Falcon.process1.click-logs";
+    private BrokerService broker;
 
-	private Path logFile;
+    private Path logFile;
 
-	private volatile AssertionError error;
-	private EmbeddedCluster dfsCluster;
-	private Configuration conf ;
+    private volatile AssertionError error;
+    private EmbeddedCluster dfsCluster;
+    private Configuration conf;
 
-	@BeforeClass
-	public void setup() throws Exception {
+    @BeforeClass
+    public void setup() throws Exception {
 
-		this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
         conf = dfsCluster.getConf();
-		logFile = new Path(conf.get("fs.default.name"),
-				"/falcon/feed/agg-logs/instance-2012-01-01-10-00.csv");
-
-		args = new String[] { "-" + ARG.entityName.getArgName(), TOPIC_NAME,
-				"-" + ARG.feedNames.getArgName(), "click-logs",
-				"-" + ARG.feedInstancePaths.getArgName(),
-				"/click-logs/10/05/05/00/20",
-				"-" + ARG.workflowId.getArgName(), "workflow-01-00",
-				"-" + ARG.runId.getArgName(), "1",
-				"-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
-				"-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
-				"-" + ARG.brokerUrl.getArgName(), BROKER_URL,
-				"-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
-				"-" + ARG.entityType.getArgName(), ("FEED"),
-				"-" + ARG.operation.getArgName(), ("DELETE"),
-				"-" + ARG.logFile.getArgName(), (logFile.toString()),
-				"-" + ARG.topicName.getArgName(), (TOPIC_NAME),
-				"-" + ARG.status.getArgName(), ("SUCCEEDED"),
-				"-" + ARG.brokerTTL.getArgName(), "10",
-				"-" + ARG.cluster.getArgName(), "corp" };
-
-		broker = new BrokerService();
-		broker.addConnector(BROKER_URL);
-		broker.setDataDirectory("target/activemq");
-		broker.start();
-	}
-
-	@AfterClass
-	public void tearDown() throws Exception {
-		broker.deleteAllMessages();
-		broker.stop();
-		this.dfsCluster.shutdown();
-	}
-
-	@Test
-	public void testLogFile() throws Exception {
-		FileSystem fs = dfsCluster.getFileSystem();
-		OutputStream out = fs.create(logFile);
-		InputStream in = new ByteArrayInputStream(
-				("instancePaths=/falcon/feed/agg-logs/path1/2010/10/10/20,"
-						+ "/falcon/feed/agg-logs/path1/2010/10/10/21,"
-						+ "/falcon/feed/agg-logs/path1/2010/10/10/22,"
-						+ "/falcon/feed/agg-logs/path1/2010/10/10/23")
-						.getBytes());
-		IOUtils.copyBytes(in, out, conf);
-		testProcessMessageCreator();
-	}
-
-	@Test
-	public void testEmptyLogFile() throws Exception {
-		FileSystem fs = dfsCluster.getFileSystem();
-		OutputStream out = fs.create(logFile);
-		InputStream in = new ByteArrayInputStream(("instancePaths=").getBytes());
-		IOUtils.copyBytes(in, out, conf);
-
-		new MessageProducer().run(this.args);
-	}
-
-	private void testProcessMessageCreator() throws Exception {
-
-		Thread t = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumer();
-				} catch (AssertionError e) {
-					error = e;
-				} catch (JMSException ignore) {
-
-				}
-			}
-		};
-		t.start();
-		Thread.sleep(1500);
-		new MessageProducer().run(this.args);
-		t.join();
-		if (error != null) {
-			throw error;
-		}
-	}
-
-	private void consumer() throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				BROKER_URL);
-		Connection connection = connectionFactory.createConnection();
-		connection.start();
-
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Destination destination = session.createTopic(TOPIC_NAME);
-		MessageConsumer consumer = session.createConsumer(destination);
-
-		// wait till you get atleast one message
-		MapMessage m;
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/falcon/feed/agg-logs/path1/2010/10/10/20");
-
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/falcon/feed/agg-logs/path1/2010/10/10/21");
-
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/falcon/feed/agg-logs/path1/2010/10/10/22");
-
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/falcon/feed/agg-logs/path1/2010/10/10/23");
-
-		connection.close();
-	}
-
-	private void assertMessage(MapMessage m) throws JMSException {
-		Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
-				TOPIC_NAME);
-		Assert.assertEquals(m.getString(ARG.operation.getArgName()), "DELETE");
-		Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
-				"workflow-01-00");
-		Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
-		Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
-				"2011-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
-				"2012-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
-	}
+        logFile = new Path(conf.get("fs.default.name"),
+                "/falcon/feed/agg-logs/instance-2012-01-01-10-00.csv");
+
+        args = new String[]{"-" + ARG.entityName.getArgName(), TOPIC_NAME,
+                            "-" + ARG.feedNames.getArgName(), "click-logs",
+                            "-" + ARG.feedInstancePaths.getArgName(),
+                            "/click-logs/10/05/05/00/20",
+                            "-" + ARG.workflowId.getArgName(), "workflow-01-00",
+                            "-" + ARG.runId.getArgName(), "1",
+                            "-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
+                            "-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
+                            "-" + ARG.brokerUrl.getArgName(), BROKER_URL,
+                            "-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
+                            "-" + ARG.entityType.getArgName(), ("FEED"),
+                            "-" + ARG.operation.getArgName(), ("DELETE"),
+                            "-" + ARG.logFile.getArgName(), (logFile.toString()),
+                            "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
+                            "-" + ARG.status.getArgName(), ("SUCCEEDED"),
+                            "-" + ARG.brokerTTL.getArgName(), "10",
+                            "-" + ARG.cluster.getArgName(), "corp"};
+
+        broker = new BrokerService();
+        broker.addConnector(BROKER_URL);
+        broker.setDataDirectory("target/activemq");
+        broker.start();
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        broker.deleteAllMessages();
+        broker.stop();
+        this.dfsCluster.shutdown();
+    }
+
+    @Test
+    public void testLogFile() throws Exception {
+        FileSystem fs = dfsCluster.getFileSystem();
+        OutputStream out = fs.create(logFile);
+        InputStream in = new ByteArrayInputStream(
+                ("instancePaths=/falcon/feed/agg-logs/path1/2010/10/10/20,"
+                        + "/falcon/feed/agg-logs/path1/2010/10/10/21,"
+                        + "/falcon/feed/agg-logs/path1/2010/10/10/22,"
+                        + "/falcon/feed/agg-logs/path1/2010/10/10/23")
+                        .getBytes());
+        IOUtils.copyBytes(in, out, conf);
+        testProcessMessageCreator();
+    }
+
+    @Test
+    public void testEmptyLogFile() throws Exception {
+        FileSystem fs = dfsCluster.getFileSystem();
+        OutputStream out = fs.create(logFile);
+        InputStream in = new ByteArrayInputStream(("instancePaths=").getBytes());
+        IOUtils.copyBytes(in, out, conf);
+
+        new MessageProducer().run(this.args);
+    }
+
+    private void testProcessMessageCreator() throws Exception {
+
+        Thread t = new Thread() {
+            @Override
+            public void run() {
+                try {
+                    consumer();
+                } catch (AssertionError e) {
+                    error = e;
+                } catch (JMSException ignore) {
+
+                }
+            }
+        };
+        t.start();
+        Thread.sleep(1500);
+        new MessageProducer().run(this.args);
+        t.join();
+        if (error != null) {
+            throw error;
+        }
+    }
+
+    private void consumer() throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                BROKER_URL);
+        Connection connection = connectionFactory.createConnection();
+        connection.start();
+
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Destination destination = session.createTopic(TOPIC_NAME);
+        MessageConsumer consumer = session.createConsumer(destination);
+
+        // wait till you get atleast one message
+        MapMessage m;
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/falcon/feed/agg-logs/path1/2010/10/10/20");
+
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/falcon/feed/agg-logs/path1/2010/10/10/21");
+
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/falcon/feed/agg-logs/path1/2010/10/10/22");
+
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/falcon/feed/agg-logs/path1/2010/10/10/23");
+
+        connection.close();
+    }
+
+    private void assertMessage(MapMessage m) throws JMSException {
+        Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
+                TOPIC_NAME);
+        Assert.assertEquals(m.getString(ARG.operation.getArgName()), "DELETE");
+        Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
+                "workflow-01-00");
+        Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
+        Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
+                "2011-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
+                "2012-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
index d2554ce..55a5fc0 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
@@ -18,14 +18,6 @@
 
 package org.apache.falcon.messaging;
 
-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.falcon.messaging.EntityInstanceMessage.ARG;
@@ -34,117 +26,121 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import javax.jms.*;
+
 public class ProcessProducerTest {
 
-	private String[] args;
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
-	private static final String TOPIC_NAME = "FALCON.PROCESS";
-	private BrokerService broker;
-
-	private volatile AssertionError error;
-
-	@BeforeClass
-	public void setup() throws Exception {
-		args = new String[] { "-" + ARG.entityName.getArgName(), TOPIC_NAME,
-				"-" + ARG.feedNames.getArgName(), "click-logs,raw-logs",
-				"-" + ARG.feedInstancePaths.getArgName(),
-				"/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
-				"-" + ARG.workflowId.getArgName(), "workflow-01-00",
-				"-" + ARG.runId.getArgName(), "1",
-				"-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
-				"-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
-				"-" + ARG.brokerUrl.getArgName(), BROKER_URL,
-				"-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
-				"-" + ARG.entityType.getArgName(), ("process"),
-				"-" + ARG.operation.getArgName(), ("GENERATE"),
-				"-" + ARG.logFile.getArgName(), ("/logFile"),
-				"-" + ARG.topicName.getArgName(), (TOPIC_NAME),
-				"-" + ARG.status.getArgName(), ("SUCCEEDED"),
-				"-" + ARG.brokerTTL.getArgName(), "10",
-				"-" + ARG.cluster.getArgName(), "corp" };
-		broker = new BrokerService();
-		broker.addConnector(BROKER_URL);
-		broker.setDataDirectory("target/activemq");
-		broker.setBrokerName("localhost");
-		broker.setSchedulerSupport(true);
-		broker.start();
-	}
-
-	@AfterClass
-	public void tearDown() throws Exception {
-		broker.deleteAllMessages();
-		broker.stop();
-	}
-
-	@Test
-	public void testProcessMessageCreator() throws Exception {
-
-		Thread t = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumer();
-				} catch (AssertionError e) {
-					error = e;
-				} catch (JMSException ignore) {
-
-				}
-			}
-		};
-		t.start();
-		Thread.sleep(1500);
-		new MessageProducer().run(this.args);
-		t.join();
-		if (error != null) {
-			throw error;
-		}
-	}
-
-	private void consumer() throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				BROKER_URL);
-		Connection connection = connectionFactory.createConnection();
-		connection.start();
-
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Destination destination = session.createTopic(TOPIC_NAME);
-		MessageConsumer consumer = session.createConsumer(destination);
-
-		// wait till you get atleast one message
-		MapMessage m;
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedNames.getArgName()),
-				"click-logs");
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/click-logs/10/05/05/00/20");
-
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-		assertMessage(m);
-		Assert.assertEquals(m.getString(ARG.feedNames.getArgName()), "raw-logs");
-		Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
-				"/raw-logs/10/05/05/00/20");
-		connection.close();
-	}
-
-	private void assertMessage(MapMessage m) throws JMSException {
-		Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
-				TOPIC_NAME);
-		Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
-				"workflow-01-00");
-		Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
-		Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
-				"2011-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
-				"2012-01-01T01:00Z");
-		Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
-	}
+    private String[] args;
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String TOPIC_NAME = "FALCON.PROCESS";
+    private BrokerService broker;
+
+    private volatile AssertionError error;
+
+    @BeforeClass
+    public void setup() throws Exception {
+        args = new String[]{"-" + ARG.entityName.getArgName(), TOPIC_NAME,
+                            "-" + ARG.feedNames.getArgName(), "click-logs,raw-logs",
+                            "-" + ARG.feedInstancePaths.getArgName(),
+                            "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
+                            "-" + ARG.workflowId.getArgName(), "workflow-01-00",
+                            "-" + ARG.runId.getArgName(), "1",
+                            "-" + ARG.nominalTime.getArgName(), "2011-01-01-01-00",
+                            "-" + ARG.timeStamp.getArgName(), "2012-01-01-01-00",
+                            "-" + ARG.brokerUrl.getArgName(), BROKER_URL,
+                            "-" + ARG.brokerImplClass.getArgName(), (BROKER_IMPL_CLASS),
+                            "-" + ARG.entityType.getArgName(), ("process"),
+                            "-" + ARG.operation.getArgName(), ("GENERATE"),
+                            "-" + ARG.logFile.getArgName(), ("/logFile"),
+                            "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
+                            "-" + ARG.status.getArgName(), ("SUCCEEDED"),
+                            "-" + ARG.brokerTTL.getArgName(), "10",
+                            "-" + ARG.cluster.getArgName(), "corp"};
+        broker = new BrokerService();
+        broker.addConnector(BROKER_URL);
+        broker.setDataDirectory("target/activemq");
+        broker.setBrokerName("localhost");
+        broker.setSchedulerSupport(true);
+        broker.start();
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        broker.deleteAllMessages();
+        broker.stop();
+    }
+
+    @Test
+    public void testProcessMessageCreator() throws Exception {
+
+        Thread t = new Thread() {
+            @Override
+            public void run() {
+                try {
+                    consumer();
+                } catch (AssertionError e) {
+                    error = e;
+                } catch (JMSException ignore) {
+
+                }
+            }
+        };
+        t.start();
+        Thread.sleep(1500);
+        new MessageProducer().run(this.args);
+        t.join();
+        if (error != null) {
+            throw error;
+        }
+    }
+
+    private void consumer() throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                BROKER_URL);
+        Connection connection = connectionFactory.createConnection();
+        connection.start();
+
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Destination destination = session.createTopic(TOPIC_NAME);
+        MessageConsumer consumer = session.createConsumer(destination);
+
+        // wait till you get atleast one message
+        MapMessage m;
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedNames.getArgName()),
+                "click-logs");
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/click-logs/10/05/05/00/20");
+
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+        assertMessage(m);
+        Assert.assertEquals(m.getString(ARG.feedNames.getArgName()), "raw-logs");
+        Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
+                "/raw-logs/10/05/05/00/20");
+        connection.close();
+    }
+
+    private void assertMessage(MapMessage m) throws JMSException {
+        Assert.assertEquals(m.getString(ARG.entityName.getArgName()),
+                TOPIC_NAME);
+        Assert.assertEquals(m.getString(ARG.workflowId.getArgName()),
+                "workflow-01-00");
+        Assert.assertEquals(m.getString(ARG.runId.getArgName()), "1");
+        Assert.assertEquals(m.getString(ARG.nominalTime.getArgName()),
+                "2011-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.timeStamp.getArgName()),
+                "2012-01-01T01:00Z");
+        Assert.assertEquals(m.getString(ARG.status.getArgName()), "SUCCEEDED");
+    }
 }


[35/47] git commit: More check style fixes relating to oozie module

Posted by sr...@apache.org.
More check style fixes relating to oozie module


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

Branch: refs/heads/master
Commit: 9b4d845f018c4f6a0a659cde63131ed08f0fa811
Parents: 223d8f0
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:47:08 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:47:08 2013 +0530

----------------------------------------------------------------------
 .../converter/AbstractOozieEntityMapper.java       |   42 ++-
 .../java/org/apache/falcon/logging/LogMover.java   |   32 ++-
 .../org/apache/falcon/logging/LogProvider.java     |   10 +-
 .../apache/falcon/service/FalconPathFilter.java    |    3 +
 .../service/SharedLibraryHostingService.java       |   12 +-
 .../java/org/apache/falcon/util/OozieUtils.java    |    5 +
 .../falcon/workflow/FalconPostProcessing.java      |    9 +-
 .../falcon/workflow/OozieWorkflowBuilder.java      |    8 +-
 .../falcon/workflow/engine/NullBundleJob.java      |    3 +
 .../falcon/workflow/engine/NullCoordJob.java       |    3 +
 .../falcon/workflow/engine/OozieClientFactory.java |   27 +-
 .../workflow/engine/OozieHouseKeepingService.java  |    5 +-
 .../workflow/engine/OozieWorkflowEngine.java       |  259 ++++++++-------
 .../org/apache/oozie/client/CustomOozieClient.java |    7 +-
 .../oozie/bundle/BundleUnmarshallingTest.java      |    3 +
 .../coordinator/CoordinatorUnmarshallingTest.java  |    2 +-
 .../oozie/workflow/FalconPostProcessingTest.java   |    9 +-
 .../oozie/workflow/WorkflowUnmarshallingTest.java  |    3 +
 .../apache/oozie/client/CustomOozieClientTest.java |    3 +
 19 files changed, 274 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java b/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
index ac3e76e..f3ddb99 100644
--- a/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
+++ b/oozie/src/main/java/org/apache/falcon/converter/AbstractOozieEntityMapper.java
@@ -52,9 +52,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+/**
+ * Entity mapper base class that allows an entity to be mapped to oozie bundle.
+ * @param <T>
+ */
 public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
-    private static Logger LOG = Logger.getLogger(AbstractOozieEntityMapper.class);
+    private static final Logger LOG = Logger.getLogger(AbstractOozieEntityMapper.class);
 
     protected static final String NOMINAL_TIME_EL = "${coord:formatTime(coord:nominalTime(), 'yyyy-MM-dd-HH-mm')}";
 
@@ -63,11 +67,11 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
     protected static final String MR_QUEUE_NAME = "queueName";
     protected static final String MR_JOB_PRIORITY = "jobPriority";
 
-    protected static final JAXBContext workflowJaxbContext;
-    protected static final JAXBContext coordJaxbContext;
-    protected static final JAXBContext bundleJaxbContext;
+    protected static final JAXBContext WORKFLOW_JAXB_CONTEXT;
+    protected static final JAXBContext COORD_JAXB_CONTEXT;
+    protected static final JAXBContext BUNDLE_JAXB_CONTEXT;
 
-    protected static final FalconPathFilter falconJarFilter = new FalconPathFilter() {
+    protected static final FalconPathFilter FALCON_JAR_FILTER = new FalconPathFilter() {
         @Override
         public boolean accept(Path path) {
             if (path.getName().startsWith("falcon")) {
@@ -88,9 +92,9 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     static {
         try {
-            workflowJaxbContext = JAXBContext.newInstance(WORKFLOWAPP.class);
-            coordJaxbContext = JAXBContext.newInstance(COORDINATORAPP.class);
-            bundleJaxbContext = JAXBContext.newInstance(BUNDLEAPP.class);
+            WORKFLOW_JAXB_CONTEXT = JAXBContext.newInstance(WORKFLOWAPP.class);
+            COORD_JAXB_CONTEXT = JAXBContext.newInstance(COORDINATORAPP.class);
+            BUNDLE_JAXB_CONTEXT = JAXBContext.newInstance(BUNDLEAPP.class);
         } catch (JAXBException e) {
             throw new RuntimeException("Unable to create JAXB context", e);
         }
@@ -147,7 +151,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
                 fs.mkdirs(libPath);
             }
 
-            SharedLibraryHostingService.pushLibsToHDFS(libPath.toString(), cluster, falconJarFilter);
+            SharedLibraryHostingService.pushLibsToHDFS(libPath.toString(), cluster, FALCON_JAR_FILTER);
         } catch (IOException e) {
             LOG.error("Failed to copy shared libs on cluster " + cluster.getName(), e);
             throw new FalconException("Failed to copy shared libs on cluster " + cluster.getName(), e);
@@ -158,7 +162,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected org.apache.falcon.oozie.coordinator.CONFIGURATION getCoordConfig(Map<String, String> propMap) {
         org.apache.falcon.oozie.coordinator.CONFIGURATION conf
-                = new org.apache.falcon.oozie.coordinator.CONFIGURATION();
+            = new org.apache.falcon.oozie.coordinator.CONFIGURATION();
         List<org.apache.falcon.oozie.coordinator.CONFIGURATION.Property> props = conf.getProperty();
         for (Entry<String, String> prop : propMap.entrySet()) {
             props.add(createCoordProperty(prop.getKey(), prop.getValue()));
@@ -219,7 +223,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
     protected org.apache.falcon.oozie.coordinator.CONFIGURATION.Property createCoordProperty(String name,
                                                                                              String value) {
         org.apache.falcon.oozie.coordinator.CONFIGURATION.Property prop
-                = new org.apache.falcon.oozie.coordinator.CONFIGURATION.Property();
+            = new org.apache.falcon.oozie.coordinator.CONFIGURATION.Property();
         prop.setName(name);
         prop.setValue(value);
         return prop;
@@ -227,14 +231,15 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected org.apache.falcon.oozie.bundle.CONFIGURATION.Property createBundleProperty(String name, String value) {
         org.apache.falcon.oozie.bundle.CONFIGURATION.Property prop
-                = new org.apache.falcon.oozie.bundle.CONFIGURATION.Property();
+            = new org.apache.falcon.oozie.bundle.CONFIGURATION.Property();
         prop.setName(name);
         prop.setValue(value);
         return prop;
     }
 
     protected void marshal(Cluster cluster, JAXBElement<?> jaxbElement, JAXBContext jaxbContext, Path outPath)
-            throws FalconException {
+        throws FalconException {
+
         try {
             Marshaller marshaller = jaxbContext.createMarshaller();
             marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
@@ -274,13 +279,14 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
             name = "coordinator";
         }
         name = name + ".xml";
-        marshal(cluster, new ObjectFactory().createCoordinatorApp(coord), coordJaxbContext, new Path(outPath, name));
+        marshal(cluster, new ObjectFactory().createCoordinatorApp(coord), COORD_JAXB_CONTEXT, new Path(outPath, name));
         return name;
     }
 
     protected void marshal(Cluster cluster, BUNDLEAPP bundle, Path outPath) throws FalconException {
 
-        marshal(cluster, new org.apache.falcon.oozie.bundle.ObjectFactory().createBundleApp(bundle), bundleJaxbContext,
+        marshal(cluster, new org.apache.falcon.oozie.bundle.ObjectFactory().createBundleApp(bundle),
+                BUNDLE_JAXB_CONTEXT,
                 new Path(
                         outPath, "bundle.xml"));
     }
@@ -288,7 +294,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
     protected void marshal(Cluster cluster, WORKFLOWAPP workflow, Path outPath) throws FalconException {
 
         marshal(cluster, new org.apache.falcon.oozie.workflow.ObjectFactory().createWorkflowApp(workflow),
-                workflowJaxbContext,
+                WORKFLOW_JAXB_CONTEXT,
                 new Path(outPath, "workflow.xml"));
     }
 
@@ -310,7 +316,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected WORKFLOWAPP getWorkflowTemplate(String template) throws FalconException {
         try {
-            Unmarshaller unmarshaller = workflowJaxbContext.createUnmarshaller();
+            Unmarshaller unmarshaller = WORKFLOW_JAXB_CONTEXT.createUnmarshaller();
             @SuppressWarnings("unchecked")
             JAXBElement<WORKFLOWAPP> jaxbElement = (JAXBElement<WORKFLOWAPP>) unmarshaller.unmarshal(this.getClass()
                     .getResourceAsStream(template));
@@ -322,7 +328,7 @@ public abstract class AbstractOozieEntityMapper<T extends Entity> {
 
     protected COORDINATORAPP getCoordinatorTemplate(String template) throws FalconException {
         try {
-            Unmarshaller unmarshaller = coordJaxbContext.createUnmarshaller();
+            Unmarshaller unmarshaller = COORD_JAXB_CONTEXT.createUnmarshaller();
             @SuppressWarnings("unchecked")
             JAXBElement<COORDINATORAPP> jaxbElement = (JAXBElement<COORDINATORAPP>) unmarshaller
                     .unmarshal(AbstractOozieEntityMapper.class.getResourceAsStream(template));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/logging/LogMover.java b/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
index 261fc53..a1b0c32 100644
--- a/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
+++ b/oozie/src/main/java/org/apache/falcon/logging/LogMover.java
@@ -17,13 +17,21 @@
  */
 package org.apache.falcon.logging;
 
-import org.apache.commons.cli.*;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapred.TaskCompletionEvent;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Logger;
@@ -40,17 +48,23 @@ import java.net.URL;
 import java.net.URLConnection;
 import java.util.List;
 
+/**
+ * Utitlity called in the post process of oozie workflow to move oozie action executor log.
+ */
 public class LogMover extends Configured implements Tool {
 
     private static final Logger LOG = Logger.getLogger(LogMover.class);
 
+    /**
+     * Args to the command.
+     */
     private static class ARGS {
-        String oozieUrl;
-        String subflowId;
-        String runId;
-        String logDir;
-        String status;
-        String entityType;
+        private String oozieUrl;
+        private String subflowId;
+        private String runId;
+        private String logDir;
+        private String status;
+        private String entityType;
     }
 
     public static void main(String[] args) throws Exception {
@@ -63,7 +77,7 @@ public class LogMover extends Configured implements Tool {
             ARGS args = new ARGS();
             setupArgs(arguments, args);
             OozieClient client = new OozieClient(args.oozieUrl);
-            WorkflowJob jobInfo = null;
+            WorkflowJob jobInfo;
             try {
                 jobInfo = client.getJobInfo(args.subflowId);
             } catch (OozieClientException e) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
index 8eec0d4..48d4589 100644
--- a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
+++ b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
@@ -37,13 +37,16 @@ import org.mortbay.log.Log;
 
 import java.io.IOException;
 
+/**
+ * Get oozie action execution logs corresponding to a run as saved by the log mover.
+ */
 public final class LogProvider {
     private static final Logger LOG = Logger.getLogger(LogProvider.class);
 
     public Instance populateLogUrls(Entity entity, Instance instance,
                                     String runId) throws FalconException {
 
-        Cluster clusterObj = (Cluster) ConfigurationStore.get().get(
+        Cluster clusterObj = ConfigurationStore.get().get(
                 EntityType.CLUSTER, instance.cluster);
         String resolvedRunId = "-";
         try {
@@ -72,7 +75,8 @@ public final class LogProvider {
 
     public String getResolvedRunId(FileSystem fs, Cluster cluster,
                                    Entity entity, Instance instance, String runId)
-            throws FalconException, IOException {
+        throws FalconException, IOException {
+
         if (StringUtils.isEmpty(runId)) {
             Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
                     EntityUtil.getLogPath(cluster, entity) + "/job-"
@@ -104,7 +108,7 @@ public final class LogProvider {
 
     private Instance populateActionLogUrls(FileSystem fs, Cluster cluster,
                                            Entity entity, Instance instance, String formatedRunId)
-            throws FalconException, OozieClientException, IOException {
+        throws FalconException, OozieClientException, IOException {
 
         Path actionPaths = new Path(ClusterHelper.getStorageUrl(cluster),
                 EntityUtil.getLogPath(cluster, entity) + "/job-"

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java b/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
index 03bcea6..51bbfd4 100644
--- a/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
+++ b/oozie/src/main/java/org/apache/falcon/service/FalconPathFilter.java
@@ -21,6 +21,9 @@ package org.apache.falcon.service;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 
+/**
+ * Path filter for considering jars for inclusion / exclusion while staging lib in oozie.
+ */
 public interface FalconPathFilter extends PathFilter {
 
     String getJarName(Path path);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java b/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
index 841177a..b6c4f25 100644
--- a/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
+++ b/oozie/src/main/java/org/apache/falcon/service/SharedLibraryHostingService.java
@@ -35,12 +35,15 @@ import org.apache.log4j.Logger;
 import java.io.File;
 import java.io.IOException;
 
+/**
+ * Host shared libraries in oozie shared lib dir upon creation or modification of cluster.
+ */
 public class SharedLibraryHostingService implements ConfigurationChangeListener {
-    private static Logger LOG = Logger.getLogger(SharedLibraryHostingService.class);
+    private static final Logger LOG = Logger.getLogger(SharedLibraryHostingService.class);
 
     private static final String[] LIBS = StartupProperties.get().getProperty("shared.libs").split(",");
 
-    private static final FalconPathFilter nonFalconJarFilter = new FalconPathFilter() {
+    private static final FalconPathFilter NON_FALCON_JAR_FILTER = new FalconPathFilter() {
         @Override
         public boolean accept(Path path) {
             for (String jarName : LIBS) {
@@ -65,14 +68,15 @@ public class SharedLibraryHostingService implements ConfigurationChangeListener
     private void addLibsTo(Cluster cluster) throws FalconException {
         String libLocation = ClusterHelper.getLocation(cluster, "working") + "/lib";
         try {
-            pushLibsToHDFS(libLocation, cluster, nonFalconJarFilter);
+            pushLibsToHDFS(libLocation, cluster, NON_FALCON_JAR_FILTER);
         } catch (IOException e) {
             LOG.error("Failed to copy shared libs to cluster " + cluster.getName(), e);
         }
     }
 
     public static void pushLibsToHDFS(String path, Cluster cluster, FalconPathFilter pathFilter)
-            throws IOException, FalconException {
+        throws IOException, FalconException {
+
         String localPaths = StartupProperties.get().getProperty("system.lib.location");
         assert localPaths != null && !localPaths.isEmpty() : "Invalid value for system.lib.location";
         if (!new File(localPaths).isDirectory()) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java b/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
index 20ef6bf..2f53370 100644
--- a/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
+++ b/oozie/src/main/java/org/apache/falcon/util/OozieUtils.java
@@ -23,8 +23,13 @@ import java.io.ByteArrayInputStream;
 import java.util.Map;
 import java.util.Properties;
 
+/**
+ * Help methods relating to oozie configuration.
+ */
 public final class OozieUtils {
 
+    private OozieUtils() {}
+
     public static Properties toProperties(String properties) {
         Configuration conf = new Configuration(false);
         conf.addResource(new ByteArrayInputStream(properties.getBytes()));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
index 4c99bfc..392f145 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/FalconPostProcessing.java
@@ -29,9 +29,15 @@ import org.apache.log4j.Logger;
 import java.util.ArrayList;
 import java.util.List;
 
+/**
+ * Utility called by oozie workflow engine post workflow execution in parent workflow.
+ */
 public class FalconPostProcessing extends Configured implements Tool {
     private static final Logger LOG = Logger.getLogger(FalconPostProcessing.class);
 
+    /**
+     * Args that the utility understands.
+     */
     public enum Arg {
         CLUSTER("cluster", "name of the current cluster"),
         ENTITY_TYPE("entityType", "type of the entity"),
@@ -170,7 +176,8 @@ public class FalconPostProcessing extends Configured implements Tool {
     }
 
     private static CommandLine getCommand(String[] arguments)
-            throws ParseException {
+        throws ParseException {
+
         Options options = new Options();
         addOption(options, Arg.CLUSTER);
         addOption(options, Arg.ENTITY_TYPE);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java b/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
index 8d67a9b..1978c53 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/OozieWorkflowBuilder.java
@@ -35,10 +35,14 @@ import java.util.Date;
 import java.util.List;
 import java.util.Properties;
 
+/**
+ * Base workflow builder for falcon entities.
+ * @param <T>
+ */
 public abstract class OozieWorkflowBuilder<T extends Entity> extends WorkflowBuilder<T> {
 
-    private static Logger LOG = Logger.getLogger(OozieWorkflowBuilder.class);
-    protected static final ConfigurationStore configStore = ConfigurationStore.get();
+    private static final Logger LOG = Logger.getLogger(OozieWorkflowBuilder.class);
+    protected static final ConfigurationStore CONFIG_STORE = ConfigurationStore.get();
 
     protected Properties createAppProperties(String clusterName, Path bundlePath, String user) throws FalconException {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
index f2c9ac3..aead7eb 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullBundleJob.java
@@ -24,6 +24,9 @@ import org.apache.oozie.client.CoordinatorJob;
 import java.util.Date;
 import java.util.List;
 
+/**
+ * Default Bundle Job.
+ */
 public class NullBundleJob implements BundleJob {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/engine/NullCoordJob.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullCoordJob.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullCoordJob.java
index 0d0c816..c93f543 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/NullCoordJob.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/NullCoordJob.java
@@ -24,6 +24,9 @@ import org.apache.oozie.client.CoordinatorJob;
 import java.util.Date;
 import java.util.List;
 
+/**
+ * Default coord job.
+ */
 public class NullCoordJob implements CoordinatorJob {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
index a335418..175b832 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
@@ -29,28 +29,32 @@ import org.apache.oozie.client.OozieClient;
 
 import java.util.concurrent.ConcurrentHashMap;
 
-//import org.apache.oozie.local.LocalOozie;
-
-public class OozieClientFactory {
+/**
+ * Factory for providing appropriate oozie client.
+ */
+public final class OozieClientFactory {
 
     private static final Logger LOG = Logger.getLogger(OozieClientFactory.class);
 
-    private static final ConcurrentHashMap<String, OozieClient> cache =
-            new ConcurrentHashMap<String, OozieClient>();
+    private static final ConcurrentHashMap<String, OozieClient> CACHE =
+        new ConcurrentHashMap<String, OozieClient>();
     private static final String LOCAL_OOZIE = "local";
     private static volatile boolean localInitialized = false;
 
-    public synchronized static OozieClient get(Cluster cluster)
-            throws FalconException {
+    private OozieClientFactory() {}
+
+    public static synchronized OozieClient get(Cluster cluster)
+        throws FalconException {
+
         assert cluster != null : "Cluster cant be null";
         String oozieUrl = ClusterHelper.getOozieUrl(cluster);
-        if (!cache.containsKey(oozieUrl)) {
+        if (!CACHE.containsKey(oozieUrl)) {
             OozieClient ref = getClientRef(oozieUrl);
             LOG.info("Caching Oozie client object for " + oozieUrl);
-            cache.putIfAbsent(oozieUrl, ref);
+            CACHE.putIfAbsent(oozieUrl, ref);
             return ref;
         } else {
-            return cache.get(oozieUrl);
+            return CACHE.get(oozieUrl);
         }
     }
 
@@ -59,7 +63,8 @@ public class OozieClientFactory {
     }
 
     private static OozieClient getClientRef(String oozieUrl)
-            throws FalconException {
+        throws FalconException {
+
         if (LOCAL_OOZIE.equals(oozieUrl)) {
             return getLocalOozieClient();
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
index dd18f9f..7e2f8a4 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
@@ -29,9 +29,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+/**
+ * Service that cleans up artifacts that falcon dropped on hdfs for oozie's use.
+ */
 public class OozieHouseKeepingService implements WorkflowEngineActionListener {
 
-    private static Logger LOG = Logger.getLogger(OozieHouseKeepingService.class);
+    private static final Logger LOG = Logger.getLogger(OozieHouseKeepingService.class);
 
     @Override
     public void beforeSchedule(Entity entity, String cluster) throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
index f8da808..a75ad74 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
@@ -41,7 +41,7 @@ import java.util.*;
 import java.util.Map.Entry;
 
 /**
- * Workflow engine which uses oozies APIs
+ * Workflow engine which uses oozies APIs.
  */
 public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
@@ -76,7 +76,7 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     private static final String FALCON_INSTANCE_SOURCE_CLUSTERS = "falcon.instance.source.clusters";
 
     private static final String[] BUNDLE_UPDATEABLE_PROPS = new String[]{
-            "parallel", "clusters.clusters[\\d+].validity.end"};
+        "parallel", "clusters.clusters[\\d+].validity.end", };
 
     public OozieWorkflowEngine() {
         registerListener(new OozieHouseKeepingService());
@@ -126,39 +126,41 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private boolean isBundleInState(Entity entity, BundleStatus status)
-            throws FalconException {
+        throws FalconException {
+
         Map<String, BundleJob> bundles = findLatestBundle(entity);
         for (BundleJob bundle : bundles.values()) {
-            if (bundle == MISSING) // There is no active bundle
-            {
+            if (bundle == MISSING) {// There is no active bundle
                 return false;
             }
 
             switch (status) {
-                case ACTIVE:
-                    if (!BUNDLE_ACTIVE_STATUS.contains(bundle.getStatus())) {
-                        return false;
-                    }
-                    break;
+            case ACTIVE:
+                if (!BUNDLE_ACTIVE_STATUS.contains(bundle.getStatus())) {
+                    return false;
+                }
+                break;
 
-                case RUNNING:
-                    if (!BUNDLE_RUNNING_STATUS.contains(bundle.getStatus())) {
-                        return false;
-                    }
-                    break;
+            case RUNNING:
+                if (!BUNDLE_RUNNING_STATUS.contains(bundle.getStatus())) {
+                    return false;
+                }
+                break;
 
-                case SUSPENDED:
-                    if (!BUNDLE_SUSPENDED_STATUS.contains(bundle.getStatus())) {
-                        return false;
-                    }
-                    break;
+            case SUSPENDED:
+                if (!BUNDLE_SUSPENDED_STATUS.contains(bundle.getStatus())) {
+                    return false;
+                }
+                break;
+            default:
             }
         }
         return true;
     }
 
     private BundleJob findBundle(Entity entity, String cluster)
-            throws FalconException {
+        throws FalconException {
+
         String stPath = EntityUtil.getStagingPath(entity);
         LOG.info("Staging path for entity " + stPath);
         List<BundleJob> bundles = findBundles(entity, cluster);
@@ -171,7 +173,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private List<BundleJob> findBundles(Entity entity, String cluster)
-            throws FalconException {
+        throws FalconException {
+
         try {
             OozieClient client = OozieClientFactory.get(cluster);
             List<BundleJob> jobs = client.getBundleJobsInfo(
@@ -194,7 +197,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private Map<String, List<BundleJob>> findBundles(Entity entity)
-            throws FalconException {
+        throws FalconException {
+
         Set<String> clusters = EntityUtil.getClustersDefinedInColos(entity);
         Map<String, List<BundleJob>> jobMap = new HashMap<String, List<BundleJob>>();
         for (String cluster : clusters) {
@@ -208,7 +212,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     // In this case, there will not be a bundle with the latest entity md5
     // So, pick last created bundle
     private Map<String, BundleJob> findLatestBundle(Entity entity)
-            throws FalconException {
+        throws FalconException {
+
         Map<String, List<BundleJob>> bundlesMap = findBundles(entity);
         Map<String, BundleJob> bundleMap = new HashMap<String, BundleJob>();
         for (String cluster : bundlesMap.keySet()) {
@@ -271,7 +276,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private String doBundleAction(Entity entity, BundleAction action, String cluster)
-            throws FalconException {
+        throws FalconException {
+
         boolean success = true;
         List<BundleJob> jobs = findBundles(entity, cluster);
         if (jobs.isEmpty()) {
@@ -282,29 +288,30 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         beforeAction(entity, action, cluster);
         for (BundleJob job : jobs) {
             switch (action) {
-                case SUSPEND:
-                    // not already suspended and preconditions are true
-                    if (!BUNDLE_SUSPENDED_STATUS.contains(job.getStatus())
-                            && BUNDLE_SUSPEND_PRECOND.contains(job.getStatus())) {
-                        suspend(cluster, job.getId());
-                        success = true;
-                    }
-                    break;
-
-                case RESUME:
-                    // not already running and preconditions are true
-                    if (!BUNDLE_RUNNING_STATUS.contains(job.getStatus())
-                            && BUNDLE_RESUME_PRECOND.contains(job.getStatus())) {
-                        resume(cluster, job.getId());
-                        success = true;
-                    }
-                    break;
+            case SUSPEND:
+                // not already suspended and preconditions are true
+                if (!BUNDLE_SUSPENDED_STATUS.contains(job.getStatus())
+                        && BUNDLE_SUSPEND_PRECOND.contains(job.getStatus())) {
+                    suspend(cluster, job.getId());
+                    success = true;
+                }
+                break;
 
-                case KILL:
-                    // not already killed and preconditions are true
-                    killBundle(cluster, job);
+            case RESUME:
+                // not already running and preconditions are true
+                if (!BUNDLE_RUNNING_STATUS.contains(job.getStatus())
+                        && BUNDLE_RESUME_PRECOND.contains(job.getStatus())) {
+                    resume(cluster, job.getId());
                     success = true;
-                    break;
+                }
+                break;
+
+            case KILL:
+                // not already killed and preconditions are true
+                killBundle(cluster, job);
+                success = true;
+                break;
+            default:
             }
             afterAction(entity, action, cluster);
         }
@@ -333,46 +340,51 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private void beforeAction(Entity entity, BundleAction action, String cluster)
-            throws FalconException {
+        throws FalconException {
+
         for (WorkflowEngineActionListener listener : listeners) {
             switch (action) {
-                case SUSPEND:
-                    listener.beforeSuspend(entity, cluster);
-                    break;
+            case SUSPEND:
+                listener.beforeSuspend(entity, cluster);
+                break;
 
-                case RESUME:
-                    listener.beforeResume(entity, cluster);
-                    break;
+            case RESUME:
+                listener.beforeResume(entity, cluster);
+                break;
 
-                case KILL:
-                    listener.beforeDelete(entity, cluster);
-                    break;
+            case KILL:
+                listener.beforeDelete(entity, cluster);
+                break;
+            default:
             }
         }
     }
 
     private void afterAction(Entity entity, BundleAction action, String cluster)
-            throws FalconException {
+        throws FalconException {
+
         for (WorkflowEngineActionListener listener : listeners) {
             switch (action) {
-                case SUSPEND:
-                    listener.afterSuspend(entity, cluster);
-                    break;
+            case SUSPEND:
+                listener.afterSuspend(entity, cluster);
+                break;
 
-                case RESUME:
-                    listener.afterResume(entity, cluster);
-                    break;
+            case RESUME:
+                listener.afterResume(entity, cluster);
+                break;
 
-                case KILL:
-                    listener.afterDelete(entity, cluster);
-                    break;
+            case KILL:
+                listener.afterDelete(entity, cluster);
+                break;
+            default:
             }
         }
     }
 
     @Override
     public InstancesResult getRunningInstances(Entity entity)
-            throws FalconException {
+        throws FalconException {
+
         try {
             WorkflowBuilder<Entity> builder = WorkflowBuilder.getBuilder(
                     ENGINE, entity);
@@ -447,7 +459,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     @Override
     public InstancesResult getStatus(Entity entity, Date start, Date end)
-            throws FalconException {
+        throws FalconException {
+
         return doJobAction(JobAction.STATUS, entity, start, end, null);
     }
 
@@ -456,7 +469,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private WorkflowJob getWorkflowInfo(String cluster, String wfId)
-            throws FalconException {
+        throws FalconException {
+
         OozieClient client = OozieClientFactory.get(cluster);
         try {
             return client.getJobInfo(wfId);
@@ -546,53 +560,55 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private String performAction(JobAction action, Properties props,
                                  String cluster, String status, WorkflowJob jobInfo)
-            throws FalconException {
-        switch (action) {
-            case KILL:
-                if (!WF_KILL_PRECOND.contains(jobInfo.getStatus())) {
-                    break;
-                }
+        throws FalconException {
 
-                kill(cluster, jobInfo.getId());
-                status = Status.KILLED.name();
+        switch (action) {
+        case KILL:
+            if (!WF_KILL_PRECOND.contains(jobInfo.getStatus())) {
                 break;
+            }
 
-            case SUSPEND:
-                if (!WF_SUSPEND_PRECOND.contains(jobInfo.getStatus())) {
-                    break;
-                }
+            kill(cluster, jobInfo.getId());
+            status = Status.KILLED.name();
+            break;
 
-                suspend(cluster, jobInfo.getId());
-                status = Status.SUSPENDED.name();
+        case SUSPEND:
+            if (!WF_SUSPEND_PRECOND.contains(jobInfo.getStatus())) {
                 break;
+            }
 
-            case RESUME:
-                if (!WF_RESUME_PRECOND.contains(jobInfo.getStatus())) {
-                    break;
-                }
+            suspend(cluster, jobInfo.getId());
+            status = Status.SUSPENDED.name();
+            break;
 
-                resume(cluster, jobInfo.getId());
-                status = Status.RUNNING.name();
+        case RESUME:
+            if (!WF_RESUME_PRECOND.contains(jobInfo.getStatus())) {
                 break;
+            }
 
-            case RERUN:
-                if (!WF_RERUN_PRECOND.contains(jobInfo.getStatus())) {
-                    break;
-                }
+            resume(cluster, jobInfo.getId());
+            status = Status.RUNNING.name();
+            break;
 
-                reRun(cluster, jobInfo.getId(), props);
-                status = Status.RUNNING.name();
+        case RERUN:
+            if (!WF_RERUN_PRECOND.contains(jobInfo.getStatus())) {
                 break;
+            }
 
-            case STATUS:
-                break;
+            reRun(cluster, jobInfo.getId(), props);
+            status = Status.RUNNING.name();
+            break;
+
+        case STATUS:
+            break;
+        default:
         }
         return status;
     }
 
     private String getSourceCluster(String cluster,
                                     CoordinatorAction coordinatorAction, Entity entity)
-            throws FalconException {
+        throws FalconException {
 
         OozieClient client = OozieClientFactory.get(cluster);
         CoordinatorJob coordJob;
@@ -656,8 +672,7 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 TimeZone tz = EntityUtil.getTimeZone(coord.getTimeZone());
                 Date iterStart = EntityUtil.getNextStartTime(
                         coord.getStartTime(), freq, tz, start);
-                final Date iterEnd = (coord.getNextMaterializedTime().before(end) ? coord.getNextMaterializedTime() :
-                        end);
+                Date iterEnd = (coord.getNextMaterializedTime().before(end) ? coord.getNextMaterializedTime() : end);
                 while (!iterStart.after(iterEnd)) {
                     int sequence = EntityUtil.getInstanceSequence(
                             coord.getStartTime(), freq, tz, iterStart);
@@ -690,6 +705,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 .getFalconTimeUnit());
     }
 
+    /**
+     * TimeUnit as understood by Oozie.
+     */
     private enum OozieTimeUnit {
         MINUTE(TimeUnit.minutes), HOUR(TimeUnit.hours), DAY(TimeUnit.days), WEEK(
                 null), MONTH(TimeUnit.months), END_OF_DAY(null), END_OF_MONTH(
@@ -712,7 +730,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private List<CoordinatorJob> getApplicableCoords(Entity entity,
                                                      OozieClient client, Date start, Date end, List<BundleJob> bundles)
-            throws FalconException {
+        throws FalconException {
+
         List<CoordinatorJob> applicableCoords = new ArrayList<CoordinatorJob>();
         try {
             for (BundleJob bundle : bundles) {
@@ -751,7 +770,7 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private boolean canUpdateBundle(Entity oldEntity, Entity newEntity)
-            throws FalconException {
+        throws FalconException {
         return EntityUtil.equals(oldEntity, newEntity, BUNDLE_UPDATEABLE_PROPS);
     }
 
@@ -894,7 +913,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private void suspend(String cluster, BundleJob bundle)
-            throws FalconException {
+        throws FalconException {
+
         bundle = getBundleInfo(cluster, bundle.getId());
         for (CoordinatorJob coord : bundle.getCoordinators()) {
             suspend(cluster, coord.getId());
@@ -909,7 +929,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private void updateInternal(Entity oldEntity, Entity newEntity,
                                 String cluster, BundleJob bundle, boolean alreadyCreated)
-            throws FalconException {
+        throws FalconException {
+
         OozieWorkflowBuilder<Entity> builder = (OozieWorkflowBuilder<Entity>) WorkflowBuilder
                 .getBuilder(ENGINE, oldEntity);
 
@@ -949,7 +970,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private void scheduleForUpdate(Entity entity, String cluster, Date startDate, String user)
-            throws FalconException {
+        throws FalconException {
+
         WorkflowBuilder<Entity> builder = WorkflowBuilder.getBuilder(ENGINE,
                 entity);
         Properties bundleProps = builder.newWorkflowSchedule(entity, startDate,
@@ -977,7 +999,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private BundleJob getBundleInfo(String cluster, String bundleId)
-            throws FalconException {
+        throws FalconException {
+
         OozieClient client = OozieClientFactory.get(cluster);
         try {
             return client.getBundleJobInfo(bundleId);
@@ -1006,7 +1029,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     @Override
     public void reRun(String cluster, String jobId, Properties props)
-            throws FalconException {
+        throws FalconException {
+
         OozieClient client = OozieClientFactory.get(cluster);
         try {
             WorkflowJob jobInfo = client.getJobInfo(jobId);
@@ -1030,13 +1054,15 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private void assertStatus(String cluster, String jobId, Status... statuses)
-            throws FalconException {
+        throws FalconException {
+
         String actualStatus = getWorkflowStatus(cluster, jobId);
         for (int counter = 0; counter < 3; counter++) {
             if (!statusEquals(actualStatus, statuses)) {
                 try {
                     Thread.sleep(100);
                 } catch (InterruptedException ignore) {
+                    //ignore
                 }
             } else {
                 return;
@@ -1059,7 +1085,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     @Override
     public String getWorkflowStatus(String cluster, String jobId)
-            throws FalconException {
+        throws FalconException {
+
         OozieClient client = OozieClientFactory.get(cluster);
         try {
             if (jobId.endsWith("-W")) {
@@ -1141,7 +1168,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     }
 
     private void change(String cluster, String jobId, String changeValue)
-            throws FalconException {
+        throws FalconException {
+
         try {
             OozieClient client = OozieClientFactory.get(cluster);
             client.change(jobId, changeValue);
@@ -1183,13 +1211,12 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 Date intendedPauseTime = (StringUtils.isEmpty(pauseTime) ? null
                         : SchemaHelper.parseDateUTC(pauseTime));
                 if (coord.getConcurrency() != concurrency
-                        || (endTime != null && !coord.getEndTime().equals(
-                        endTime))
-                        || (intendedPauseTime != null && !intendedPauseTime
-                        .equals(coord.getPauseTime()))) {
+                        || (endTime != null && !coord.getEndTime().equals(endTime))
+                        || (intendedPauseTime != null && !intendedPauseTime.equals(coord.getPauseTime()))) {
                     try {
                         Thread.sleep(100);
                     } catch (InterruptedException ignore) {
+                        //ignore
                     }
                 } else {
                     return;
@@ -1223,7 +1250,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     @Override
     public InstancesResult getJobDetails(String cluster, String jobId)
-            throws FalconException {
+        throws FalconException {
+
         OozieClient client = OozieClientFactory.get(cluster);
         Instance[] instances = new Instance[1];
         Instance instance = new Instance();
@@ -1244,5 +1272,4 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         }
 
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java b/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
index 4998b0d..7634984 100644
--- a/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
+++ b/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
@@ -32,9 +32,12 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
+/**
+ * Wrapped Oozie Client.
+ */
 public class CustomOozieClient extends OozieClient {
 
-    private static final Map<String, String> none = new HashMap<String, String>();
+    private static final Map<String, String> NONE = new HashMap<String, String>();
 
     public CustomOozieClient(String oozieUrl) {
         super(oozieUrl);
@@ -64,7 +67,7 @@ public class CustomOozieClient extends OozieClient {
     private class OozieConfiguration extends ClientCallable<Properties> {
 
         public OozieConfiguration(String resource) {
-            super("GET", RestConstants.ADMIN, resource, none);
+            super("GET", RestConstants.ADMIN, resource, NONE);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
index 97dca12..8e9d37c 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
@@ -28,6 +28,9 @@ import javax.xml.transform.stream.StreamSource;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
+/**
+ * Test to verify if the bundle xml is geting unmarshalled.
+ */
 public class BundleUnmarshallingTest {
     @Test
     public void testValidBundleUnamrashalling() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
index da8b626..f3deac7 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
@@ -29,7 +29,7 @@ import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
 /**
- * Class to test if generated coordinator.xml is valid
+ * Class to test if generated coordinator.xml is valid.
  */
 public class CoordinatorUnmarshallingTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
index 169c5d3..5ac8006 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
@@ -28,6 +28,9 @@ import org.testng.annotations.Test;
 
 import javax.jms.*;
 
+/**
+ * Test for validating the falcon post processing utility.
+ */
 public class FalconPostProcessingTest {
 
     private String[] args;
@@ -65,7 +68,7 @@ public class FalconPostProcessingTest {
                             "-" + Arg.WF_ENGINE_URL.getOptionName(),
                             "http://localhost:11000/oozie/",
                             "-" + Arg.LOG_DIR.getOptionName(), "target/log",
-                            "-" + Arg.USER_SUBFLOW_ID.getOptionName(), "userflow@wf-id" + "test"};
+                            "-" + Arg.USER_SUBFLOW_ID.getOptionName(), "userflow@wf-id" + "test", };
         broker = new BrokerService();
         broker.addConnector(BROKER_URL);
         broker.setDataDirectory("target/activemq");
@@ -91,7 +94,7 @@ public class FalconPostProcessingTest {
                 } catch (AssertionError e) {
                     error = e;
                 } catch (JMSException ignore) {
-
+                    error = null;
                 }
             }
         };
@@ -117,7 +120,7 @@ public class FalconPostProcessingTest {
 
         // wait till you get atleast one message
         MapMessage m;
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
index e17d377..54bba62 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
@@ -29,6 +29,9 @@ import javax.xml.bind.Unmarshaller;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
+/**
+ * Test to verify workflow xml unmarshalling.
+ */
 public class WorkflowUnmarshallingTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9b4d845f/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java b/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
index 897a86d..cf0b385 100644
--- a/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
+++ b/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
@@ -22,6 +22,9 @@ import org.testng.annotations.Test;
 
 import java.util.Properties;
 
+/**
+ * Test to verify if the oozie client provided via CustomOozieClient is valid.
+ */
 public class CustomOozieClientTest {
 
     @Test(enabled = false)


[12/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
index ad05a77..a335418 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieClientFactory.java
@@ -26,13 +26,14 @@ import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.log4j.Logger;
 import org.apache.oozie.client.CustomOozieClient;
 import org.apache.oozie.client.OozieClient;
-//import org.apache.oozie.local.LocalOozie;
 
 import java.util.concurrent.ConcurrentHashMap;
 
+//import org.apache.oozie.local.LocalOozie;
+
 public class OozieClientFactory {
 
-	private static final Logger LOG = Logger.getLogger(OozieClientFactory.class);
+    private static final Logger LOG = Logger.getLogger(OozieClientFactory.class);
 
     private static final ConcurrentHashMap<String, OozieClient> cache =
             new ConcurrentHashMap<String, OozieClient>();
@@ -56,6 +57,7 @@ public class OozieClientFactory {
     public static OozieClient get(String cluster) throws FalconException {
         return get((Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, cluster));
     }
+
     private static OozieClient getClientRef(String oozieUrl)
             throws FalconException {
         if (LOCAL_OOZIE.equals(oozieUrl)) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
index 8e10353..dd18f9f 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieHouseKeepingService.java
@@ -18,15 +18,15 @@
 
 package org.apache.falcon.workflow.engine;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 public class OozieHouseKeepingService implements WorkflowEngineActionListener {
@@ -49,16 +49,18 @@ public class OozieHouseKeepingService implements WorkflowEngineActionListener {
     public void afterDelete(Entity entity, String clusterName) throws FalconException {
         try {
             Cluster cluster = EntityUtil.getEntity(EntityType.CLUSTER, clusterName);
-            Path entityPath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(entity)).getParent();
+            Path entityPath = new Path(ClusterHelper.getLocation(cluster, "staging"),
+                    EntityUtil.getStagingPath(entity)).getParent();
             LOG.info("Deleting entity path " + entityPath + " on cluster " + clusterName);
-            
+
             Configuration conf = ClusterHelper.getConfiguration(cluster);
             FileSystem fs = FileSystem.get(conf);
             if (fs.exists(entityPath) && !fs.delete(entityPath, true)) {
                 throw new FalconException("Unable to cleanup entity path: " + entityPath);
             }
         } catch (Exception e) {
-            throw new FalconException("Failed to cleanup entity path for " + entity.toShortString() + " on cluster " + clusterName, e);
+            throw new FalconException(
+                    "Failed to cleanup entity path for " + entity.toShortString() + " on cluster " + clusterName, e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
index 8f6901a..f8da808 100644
--- a/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
+++ b/oozie/src/main/java/org/apache/falcon/workflow/engine/OozieWorkflowEngine.java
@@ -18,30 +18,12 @@
 
 package org.apache.falcon.workflow.engine;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TimeZone;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.EntityUtil;
-import org.apache.falcon.entity.v0.Entity;
-import org.apache.falcon.entity.v0.EntityGraph;
-import org.apache.falcon.entity.v0.EntityType;
-import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.entity.v0.*;
 import org.apache.falcon.entity.v0.Frequency.TimeUnit;
-import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.InstancesResult;
 import org.apache.falcon.resource.InstancesResult.Instance;
@@ -55,9 +37,11 @@ import org.apache.oozie.client.*;
 import org.apache.oozie.client.CoordinatorJob.Timeunit;
 import org.apache.oozie.client.WorkflowJob.Status;
 
+import java.util.*;
+import java.util.Map.Entry;
+
 /**
  * Workflow engine which uses oozies APIs
- *
  */
 public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
@@ -91,8 +75,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     private static final String FALCON_INSTANCE_ACTION_CLUSTERS = "falcon.instance.action.clusters";
     private static final String FALCON_INSTANCE_SOURCE_CLUSTERS = "falcon.instance.source.clusters";
 
-    private static final String[] BUNDLE_UPDATEABLE_PROPS = new String[] {
-            "parallel", "clusters.clusters[\\d+].validity.end" };
+    private static final String[] BUNDLE_UPDATEABLE_PROPS = new String[]{
+            "parallel", "clusters.clusters[\\d+].validity.end"};
 
     public OozieWorkflowEngine() {
         registerListener(new OozieHouseKeepingService());
@@ -146,22 +130,27 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         Map<String, BundleJob> bundles = findLatestBundle(entity);
         for (BundleJob bundle : bundles.values()) {
             if (bundle == MISSING) // There is no active bundle
+            {
                 return false;
+            }
 
             switch (status) {
                 case ACTIVE:
-                    if (!BUNDLE_ACTIVE_STATUS.contains(bundle.getStatus()))
+                    if (!BUNDLE_ACTIVE_STATUS.contains(bundle.getStatus())) {
                         return false;
+                    }
                     break;
 
                 case RUNNING:
-                    if (!BUNDLE_RUNNING_STATUS.contains(bundle.getStatus()))
+                    if (!BUNDLE_RUNNING_STATUS.contains(bundle.getStatus())) {
                         return false;
+                    }
                     break;
 
                 case SUSPENDED:
-                    if (!BUNDLE_SUSPENDED_STATUS.contains(bundle.getStatus()))
+                    if (!BUNDLE_SUSPENDED_STATUS.contains(bundle.getStatus())) {
                         return false;
+                    }
                     break;
             }
         }
@@ -190,11 +179,12 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                             + EntityUtil.getWorkflowName(entity) + ";", 0, 256);
             if (jobs != null) {
                 List<BundleJob> filteredJobs = new ArrayList<BundleJob>();
-                for(BundleJob job : jobs)
-                    if(job.getStatus() != Job.Status.KILLED || job.getEndTime() == null) {
+                for (BundleJob job : jobs) {
+                    if (job.getStatus() != Job.Status.KILLED || job.getEndTime() == null) {
                         filteredJobs.add(job);
                         LOG.debug("Found bundle " + job.getId());
                     }
+                }
                 return filteredJobs;
             }
             return new ArrayList<BundleJob>();
@@ -224,11 +214,12 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         for (String cluster : bundlesMap.keySet()) {
             Date latest = null;
             bundleMap.put(cluster, MISSING);
-            for (BundleJob job : bundlesMap.get(cluster))
+            for (BundleJob job : bundlesMap.get(cluster)) {
                 if (latest == null || latest.before(job.getCreatedTime())) {
                     bundleMap.put(cluster, job);
                     latest = job.getCreatedTime();
                 }
+            }
         }
         return bundleMap;
     }
@@ -237,11 +228,12 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         List<BundleJob> bundles = findBundles(entity, cluster);
         Date latest = null;
         BundleJob bundle = MISSING;
-        for (BundleJob job : bundles)
+        for (BundleJob job : bundles) {
             if (latest == null || latest.before(job.getCreatedTime())) {
                 bundle = job;
                 latest = job.getCreatedTime();
             }
+        }
         return bundle;
     }
 
@@ -272,8 +264,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
     private String doBundleAction(Entity entity, BundleAction action) throws FalconException {
         Set<String> clusters = EntityUtil.getClustersDefinedInColos(entity);
         String result = null;
-        for(String cluster:clusters)
+        for (String cluster : clusters) {
             result = doBundleAction(entity, action, cluster);
+        }
         return result;
     }
 
@@ -322,7 +315,7 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         OozieClient client = OozieClientFactory.get(cluster);
         try {
             //kill all coords
-            for(CoordinatorJob coord:job.getCoordinators()) {
+            for (CoordinatorJob coord : job.getCoordinators()) {
                 client.kill(coord.getId());
                 LOG.debug("Killed coord " + coord.getId() + " on cluster " + cluster);
             }
@@ -389,8 +382,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
             List<String> coordNames = new ArrayList<String>();
             for (String wfName : wfNames) {
                 if (EntityUtil.getWorkflowName(Tag.RETENTION, entity)
-                        .toString().equals(wfName))
+                        .toString().equals(wfName)) {
                     continue;
+                }
                 coordNames.add(wfName);
             }
 
@@ -400,8 +394,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 if (wfs != null) {
                     for (WorkflowJob job : wfs) {
                         WorkflowJob wf = client.getJobInfo(job.getId());
-                        if (StringUtils.isEmpty(wf.getParentId()))
+                        if (StringUtils.isEmpty(wf.getParentId())) {
                             continue;
+                        }
 
                         CoordinatorAction action = client.getCoordActionInfo(wf
                                 .getParentId());
@@ -484,8 +479,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
         List<Instance> instances = new ArrayList<Instance>();
         for (String cluster : actionsMap.keySet()) {
-            if (clusterList.size() != 0 && !clusterList.contains(cluster))
+            if (clusterList.size() != 0 && !clusterList.contains(cluster)) {
                 continue;
+            }
 
             List<CoordinatorAction> actions = actionsMap.get(cluster);
             String sourceCluster = null;
@@ -494,8 +490,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                     sourceCluster = getSourceCluster(cluster,
                             coordinatorAction, entity);
                     if (sourceClusterList.size() != 0
-                            && !sourceClusterList.contains(sourceCluster))
+                            && !sourceClusterList.contains(sourceCluster)) {
                         continue;
+                    }
                 }
                 String status = mapActionStatus(coordinatorAction.getStatus());
                 WorkflowJob jobInfo = null;
@@ -552,32 +549,36 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
             throws FalconException {
         switch (action) {
             case KILL:
-                if (!WF_KILL_PRECOND.contains(jobInfo.getStatus()))
+                if (!WF_KILL_PRECOND.contains(jobInfo.getStatus())) {
                     break;
+                }
 
                 kill(cluster, jobInfo.getId());
                 status = Status.KILLED.name();
                 break;
 
             case SUSPEND:
-                if (!WF_SUSPEND_PRECOND.contains(jobInfo.getStatus()))
+                if (!WF_SUSPEND_PRECOND.contains(jobInfo.getStatus())) {
                     break;
+                }
 
                 suspend(cluster, jobInfo.getId());
                 status = Status.SUSPENDED.name();
                 break;
 
             case RESUME:
-                if (!WF_RESUME_PRECOND.contains(jobInfo.getStatus()))
+                if (!WF_RESUME_PRECOND.contains(jobInfo.getStatus())) {
                     break;
+                }
 
                 resume(cluster, jobInfo.getId());
                 status = Status.RUNNING.name();
                 break;
 
             case RERUN:
-                if (!WF_RERUN_PRECOND.contains(jobInfo.getStatus()))
+                if (!WF_RERUN_PRECOND.contains(jobInfo.getStatus())) {
                     break;
+                }
 
                 reRun(cluster, jobInfo.getId(), props);
                 status = Status.RUNNING.name();
@@ -655,7 +656,8 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 TimeZone tz = EntityUtil.getTimeZone(coord.getTimeZone());
                 Date iterStart = EntityUtil.getNextStartTime(
                         coord.getStartTime(), freq, tz, start);
-                final Date iterEnd = (coord.getNextMaterializedTime().before(end) ? coord.getNextMaterializedTime() : end);
+                final Date iterEnd = (coord.getNextMaterializedTime().before(end) ? coord.getNextMaterializedTime() :
+                        end);
                 while (!iterStart.after(iterEnd)) {
                     int sequence = EntityUtil.getInstanceSequence(
                             coord.getStartTime(), freq, tz, iterStart);
@@ -700,9 +702,10 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         }
 
         public TimeUnit getFalconTimeUnit() {
-            if (falconTimeUnit == null)
+            if (falconTimeUnit == null) {
                 throw new IllegalStateException("Invalid coord frequency: "
                         + name());
+            }
             return falconTimeUnit;
         }
     }
@@ -718,8 +721,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
                 for (CoordinatorJob coord : coords) {
                     String coordName = EntityUtil.getWorkflowName(
                             Tag.RETENTION, entity).toString();
-                    if (coordName.equals(coord.getAppName()))
+                    if (coordName.equals(coord.getAppName())) {
                         continue;
+                    }
                     // if end time is before coord-start time or start time is
                     // after coord-end time ignore.
                     if (!(end.compareTo(coord.getStartTime()) <= 0 || start
@@ -787,16 +791,19 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         //Update affected entities
         Set<Entity> affectedEntities = EntityGraph.get().getDependents(oldEntity);
         for (Entity affectedEntity : affectedEntities) {
-            if (affectedEntity.getEntityType() != EntityType.PROCESS)
+            if (affectedEntity.getEntityType() != EntityType.PROCESS) {
                 continue;
+            }
 
             LOG.info("Dependent entities need to be updated " + affectedEntity.toShortString());
-            if (!UpdateHelper.shouldUpdate(oldEntity, newEntity, affectedEntity))
+            if (!UpdateHelper.shouldUpdate(oldEntity, newEntity, affectedEntity)) {
                 continue;
+            }
 
             BundleJob affectedProcBundle = findLatestBundle(affectedEntity, cluster);
-            if (affectedProcBundle == MISSING)
+            if (affectedProcBundle == MISSING) {
                 continue;
+            }
 
             LOG.info("Triggering update for " + cluster + ", " + affectedProcBundle.getId());
 
@@ -843,10 +850,11 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private void updateCoords(String cluster, String bundleId, int concurrency,
                               Date endTime) throws FalconException {
-        if (endTime.compareTo(now()) <= 0)
+        if (endTime.compareTo(now()) <= 0) {
             throw new FalconException("End time "
                     + SchemaHelper.formatDateUTC(endTime)
                     + " can't be in the past");
+        }
 
         BundleJob bundle = getBundleInfo(cluster, bundleId);
         // change coords
@@ -929,9 +937,10 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
             LOG.info("Will set old coord end time to "
                     + SchemaHelper.formatDateUTC(endTime));
         }
-        if (endTime != null)
+        if (endTime != null) {
             updateCoords(cluster, bundle.getId(),
                     EntityUtil.getParallel(oldEntity), endTime);
+        }
 
         if (oldBundleStatus != Job.Status.SUSPENDED
                 && oldBundleStatus != Job.Status.PREPSUSPENDED) {
@@ -957,10 +966,13 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private Date getMinStartTime(BundleJob bundle) {
         Date startTime = null;
-        if (bundle.getCoordinators() != null)
-            for (CoordinatorJob coord : bundle.getCoordinators())
-                if (startTime == null || startTime.after(coord.getStartTime()))
+        if (bundle.getCoordinators() != null) {
+            for (CoordinatorJob coord : bundle.getCoordinators()) {
+                if (startTime == null || startTime.after(coord.getStartTime())) {
                     startTime = coord.getStartTime();
+                }
+            }
+        }
         return startTime;
     }
 
@@ -979,9 +991,10 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         StringBuilder filter = new StringBuilder();
         filter.append(OozieClient.FILTER_STATUS).append('=')
                 .append(Job.Status.RUNNING.name());
-        for (String wfName : wfNames)
+        for (String wfName : wfNames) {
             filter.append(';').append(OozieClient.FILTER_NAME).append('=')
                     .append(wfName);
+        }
 
         OozieClient client = OozieClientFactory.get(cluster);
         try {
@@ -998,12 +1011,13 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
         try {
             WorkflowJob jobInfo = client.getJobInfo(jobId);
             Properties jobprops = OozieUtils.toProperties(jobInfo.getConf());
-            if (props == null || props.isEmpty())
+            if (props == null || props.isEmpty()) {
                 jobprops.put(OozieClient.RERUN_FAIL_NODES, "false");
-            else
+            } else {
                 for (Entry<Object, Object> entry : props.entrySet()) {
                     jobprops.put(entry.getKey(), entry.getValue());
                 }
+            }
             jobprops.remove(OozieClient.COORDINATOR_APP_PATH);
             jobprops.remove(OozieClient.BUNDLE_APP_PATH);
             client.reRun(jobId, jobprops);
@@ -1036,8 +1050,9 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
 
     private boolean statusEquals(String left, Status... right) {
         for (Status rightElement : right) {
-            if (left.equals(rightElement.name()))
+            if (left.equals(rightElement.name())) {
                 return true;
+            }
         }
         return false;
     }
@@ -1047,13 +1062,13 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
             throws FalconException {
         OozieClient client = OozieClientFactory.get(cluster);
         try {
-            if(jobId.endsWith("-W")) {
+            if (jobId.endsWith("-W")) {
                 WorkflowJob jobInfo = client.getJobInfo(jobId);
                 return jobInfo.getStatus().name();
-            } else if(jobId.endsWith("-C")) {
+            } else if (jobId.endsWith("-C")) {
                 CoordinatorJob coord = client.getCoordJobInfo(jobId);
                 return coord.getStatus().name();
-            } else if(jobId.endsWith("-B")) {
+            } else if (jobId.endsWith("-B")) {
                 BundleJob bundle = client.getBundleJobInfo(jobId);
                 return bundle.getStatus().name();
             }
@@ -1147,14 +1162,16 @@ public class OozieWorkflowEngine extends AbstractWorkflowEngine {
             changeValue.append(OozieClient.CHANGE_VALUE_ENDTIME).append("=")
                     .append(endTimeStr).append(";");
         }
-        if (pauseTime != null)
+        if (pauseTime != null) {
             changeValue.append(OozieClient.CHANGE_VALUE_PAUSETIME).append("=")
                     .append(pauseTime);
+        }
 
         String changeValueStr = changeValue.toString();
-        if (changeValue.toString().endsWith(";"))
+        if (changeValue.toString().endsWith(";")) {
             changeValueStr = changeValue.substring(0,
                     changeValueStr.length() - 1);
+        }
 
         change(cluster, id, changeValueStr);
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java b/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
index 5895d51..4998b0d 100644
--- a/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
+++ b/oozie/src/main/java/org/apache/oozie/client/CustomOozieClient.java
@@ -51,16 +51,16 @@ public class CustomOozieClient extends OozieClient {
     @Override
     protected HttpURLConnection createConnection(URL url, String method) throws IOException, OozieClientException {
         HttpURLConnection conn = super.createConnection(url, method);
-        
+
         int connectTimeout = Integer.valueOf(RuntimeProperties.get().getProperty("oozie.connect.timeout", "1000"));
         conn.setConnectTimeout(connectTimeout);
 
         int readTimeout = Integer.valueOf(RuntimeProperties.get().getProperty("oozie.read.timeout", "45000"));
         conn.setReadTimeout(readTimeout);
-        
+
         return conn;
     }
-    
+
     private class OozieConfiguration extends ClientCallable<Properties> {
 
         public OozieConfiguration(String resource) {
@@ -78,8 +78,7 @@ public class CustomOozieClient extends OozieClient {
                     props.put(key, json.get(key));
                 }
                 return props;
-            }
-            else {
+            } else {
                 handleError(conn);
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/resources/oozie-bundle-0.1.xsd
----------------------------------------------------------------------
diff --git a/oozie/src/main/resources/oozie-bundle-0.1.xsd b/oozie/src/main/resources/oozie-bundle-0.1.xsd
index fbcd41c..41534b3 100644
--- a/oozie/src/main/resources/oozie-bundle-0.1.xsd
+++ b/oozie/src/main/resources/oozie-bundle-0.1.xsd
@@ -38,7 +38,7 @@
         </xs:sequence>
     </xs:complexType>
     <xs:complexType name="COORDINATOR">
-        <xs:sequence  minOccurs="1" maxOccurs="1">
+        <xs:sequence minOccurs="1" maxOccurs="1">
             <xs:element name="app-path" type="xs:string" minOccurs="1" maxOccurs="1"/>
             <xs:element name="configuration" type="bundle:CONFIGURATION" minOccurs="0" maxOccurs="1"/>
         </xs:sequence>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/main/resources/oozie-workflow-0.3.xsd
----------------------------------------------------------------------
diff --git a/oozie/src/main/resources/oozie-workflow-0.3.xsd b/oozie/src/main/resources/oozie-workflow-0.3.xsd
index 06d2013..8cda3d9 100644
--- a/oozie/src/main/resources/oozie-workflow-0.3.xsd
+++ b/oozie/src/main/resources/oozie-workflow-0.3.xsd
@@ -126,7 +126,7 @@
                 <xs:element name="sub-workflow" type="workflow:SUB-WORKFLOW" minOccurs="1" maxOccurs="1"/>
                 <xs:element name="fs" type="workflow:FS" minOccurs="1" maxOccurs="1"/>
                 <xs:element name="java" type="workflow:JAVA" minOccurs="1" maxOccurs="1"/>
-               <!-- <xs:any namespace="##other" minOccurs="1" maxOccurs="1"/>-->
+                <!-- <xs:any namespace="##other" minOccurs="1" maxOccurs="1"/>-->
             </xs:choice>
             <xs:element name="ok" type="workflow:ACTION_TRANSITION" minOccurs="1" maxOccurs="1"/>
             <xs:element name="error" type="workflow:ACTION_TRANSITION" minOccurs="1" maxOccurs="1"/>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
index 6c6a5e5..97dca12 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/bundle/BundleUnmarshallingTest.java
@@ -18,6 +18,9 @@
 
 package org.apache.falcon.oozie.bundle;
 
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBElement;
 import javax.xml.bind.Unmarshaller;
@@ -25,17 +28,16 @@ import javax.xml.transform.stream.StreamSource;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
 public class BundleUnmarshallingTest {
     @Test
     public void testValidBundleUnamrashalling() throws Exception {
-        Unmarshaller unmarshaller = JAXBContext.newInstance(org.apache.falcon.oozie.bundle.BUNDLEAPP.class).createUnmarshaller();
+        Unmarshaller unmarshaller = JAXBContext.newInstance(
+                org.apache.falcon.oozie.bundle.BUNDLEAPP.class).createUnmarshaller();
         SchemaFactory schemaFactory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
         Schema schema = schemaFactory.newSchema(this.getClass().getResource("/oozie-bundle-0.1.xsd"));
         unmarshaller.setSchema(schema);
-        Object bundle = unmarshaller.unmarshal(new StreamSource(BundleUnmarshallingTest.class.getResourceAsStream("/oozie/xmls/bundle.xml")),
+        Object bundle = unmarshaller.unmarshal(
+                new StreamSource(BundleUnmarshallingTest.class.getResourceAsStream("/oozie/xmls/bundle.xml")),
                 BUNDLEAPP.class);
         BUNDLEAPP bundleApp = ((JAXBElement<BUNDLEAPP>) bundle).getValue();
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
index b7b4440..da8b626 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/coordinator/CoordinatorUnmarshallingTest.java
@@ -17,6 +17,10 @@
  */
 package org.apache.falcon.oozie.coordinator;
 
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import org.xml.sax.SAXException;
+
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBElement;
 import javax.xml.bind.JAXBException;
@@ -24,14 +28,8 @@ import javax.xml.bind.Unmarshaller;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
-import org.testng.Assert;
-import org.testng.annotations.Test;
-import org.xml.sax.SAXException;
-
 /**
- * 
  * Class to test if generated coordinator.xml is valid
- * 
  */
 public class CoordinatorUnmarshallingTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
index b2cf821..169c5d3 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/FalconPostProcessingTest.java
@@ -17,14 +17,6 @@
  */
 package org.apache.falcon.oozie.workflow;
 
-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.falcon.workflow.FalconPostProcessing;
@@ -34,129 +26,132 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import javax.jms.*;
+
 public class FalconPostProcessingTest {
 
-	private String[] args;
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
-	private static final String FALCON_TOPIC_NAME = "FALCON.ENTITY.TOPIC";
-	private static final String ENTITY_NAME = "agg-coord";
-	private BrokerService broker;
-
-	private volatile AssertionError error;
-
-	@BeforeClass
-	public void setup() throws Exception {
-		args = new String[] { "-" + Arg.ENTITY_NAME.getOptionName(),
-				ENTITY_NAME, "-" + Arg.FEED_NAMES.getOptionName(),
-				"click-logs,raw-logs",
-				"-" + Arg.FEED_INSTANCE_PATHS.getOptionName(),
-				"/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
-				"-" + Arg.WORKFLOW_ID.getOptionName(), "workflow-01-00",
-				"-" + Arg.RUN_ID.getOptionName(), "1",
-				"-" + Arg.NOMINAL_TIME.getOptionName(), "2011-01-01-01-00",
-				"-" + Arg.TIMESTAMP.getOptionName(), "2012-01-01-01-00",
-				"-" + Arg.BRKR_URL.getOptionName(), BROKER_URL,
-				"-" + Arg.BRKR_IMPL_CLASS.getOptionName(), (BROKER_IMPL_CLASS),
-				"-" + Arg.USER_BRKR_URL.getOptionName(), BROKER_URL,
-				"-" + Arg.USER_BRKR_IMPL_CLASS.getOptionName(),
-				(BROKER_IMPL_CLASS), "-" + Arg.ENTITY_TYPE.getOptionName(),
-				("process"), "-" + Arg.OPERATION.getOptionName(), ("GENERATE"),
-				"-" + Arg.LOG_FILE.getOptionName(), ("/logFile"),
-				"-" + Arg.STATUS.getOptionName(), ("SUCCEEDED"),
-				"-" + Arg.BRKR_TTL.getOptionName(), "10",
-				"-" + Arg.CLUSTER.getOptionName(), "corp",
-				"-" + Arg.WF_ENGINE_URL.getOptionName(),
-				"http://localhost:11000/oozie/",
-				"-" + Arg.LOG_DIR.getOptionName(), "target/log",
-				"-" + Arg.USER_SUBFLOW_ID.getOptionName(), "userflow@wf-id"+"test" };
-		broker = new BrokerService();
-		broker.addConnector(BROKER_URL);
-		broker.setDataDirectory("target/activemq");
-		broker.setBrokerName("localhost");
-		broker.start();
-	}
-
-	@AfterClass
-	public void tearDown() throws Exception {
-		broker.deleteAllMessages();
-		broker.stop();
-	}
-
-	@Test
-	public void testProcessMessageCreator() throws Exception {
-
-		Thread t = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumer(BROKER_URL, "FALCON." + ENTITY_NAME);
-					consumer(BROKER_URL, FALCON_TOPIC_NAME);
-				} catch (AssertionError e) {
-					error = e;
-				} catch (JMSException ignore) {
-
-				}
-			}
-		};
-		t.start();
-		Thread.sleep(1500);
-		new FalconPostProcessing().run(this.args);
-		t.join();
-		if (error != null) {
-			throw error;
-		}
-	}
-
-	private void consumer(String brokerUrl, String topic) throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				brokerUrl);
-		Connection connection = connectionFactory.createConnection();
-		connection.start();
-
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Destination destination = session.createTopic(topic);
-		MessageConsumer consumer = session.createConsumer(destination);
-
-		// wait till you get atleast one message
-		MapMessage m;
-		for (m = null; m == null;)
-			m = (MapMessage) consumer.receive();
-		System.out.println("Consumed: " + m.toString());
-
-		assertMessage(m);
-		if (topic.equals(FALCON_TOPIC_NAME)) {
-			Assert.assertEquals(m.getString(Arg.FEED_NAMES.getOptionName()),
-					"click-logs,raw-logs");
-			Assert.assertEquals(
-					m.getString(Arg.FEED_INSTANCE_PATHS.getOptionName()),
-					"/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20");
-		} else {
-			Assert.assertEquals(m.getString(Arg.FEED_NAMES.getOptionName()),
-					"click-logs");
-			Assert.assertEquals(
-					m.getString(Arg.FEED_INSTANCE_PATHS.getOptionName()),
-					"/click-logs/10/05/05/00/20");
-
-		}
-
-		connection.close();
-	}
-
-	private void assertMessage(MapMessage m) throws JMSException {
-		Assert.assertEquals(m.getString(Arg.ENTITY_NAME.getOptionName()),
-				"agg-coord");
-		Assert.assertEquals(m.getString(Arg.WORKFLOW_ID.getOptionName()),
-				"workflow-01-00");
-		Assert.assertEquals(m.getString(Arg.RUN_ID.getOptionName()), "1");
-		Assert.assertEquals(m.getString(Arg.NOMINAL_TIME.getOptionName()),
-				"2011-01-01T01:00Z");
-		Assert.assertEquals(m.getString(Arg.TIMESTAMP.getOptionName()),
-				"2012-01-01T01:00Z");
-		Assert.assertEquals(m.getString(Arg.STATUS.getOptionName()),
-				"SUCCEEDED");
-	}
+    private String[] args;
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String FALCON_TOPIC_NAME = "FALCON.ENTITY.TOPIC";
+    private static final String ENTITY_NAME = "agg-coord";
+    private BrokerService broker;
+
+    private volatile AssertionError error;
+
+    @BeforeClass
+    public void setup() throws Exception {
+        args = new String[]{"-" + Arg.ENTITY_NAME.getOptionName(),
+                            ENTITY_NAME, "-" + Arg.FEED_NAMES.getOptionName(),
+                            "click-logs,raw-logs",
+                            "-" + Arg.FEED_INSTANCE_PATHS.getOptionName(),
+                            "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20",
+                            "-" + Arg.WORKFLOW_ID.getOptionName(), "workflow-01-00",
+                            "-" + Arg.RUN_ID.getOptionName(), "1",
+                            "-" + Arg.NOMINAL_TIME.getOptionName(), "2011-01-01-01-00",
+                            "-" + Arg.TIMESTAMP.getOptionName(), "2012-01-01-01-00",
+                            "-" + Arg.BRKR_URL.getOptionName(), BROKER_URL,
+                            "-" + Arg.BRKR_IMPL_CLASS.getOptionName(), (BROKER_IMPL_CLASS),
+                            "-" + Arg.USER_BRKR_URL.getOptionName(), BROKER_URL,
+                            "-" + Arg.USER_BRKR_IMPL_CLASS.getOptionName(),
+                            (BROKER_IMPL_CLASS), "-" + Arg.ENTITY_TYPE.getOptionName(),
+                            ("process"), "-" + Arg.OPERATION.getOptionName(), ("GENERATE"),
+                            "-" + Arg.LOG_FILE.getOptionName(), ("/logFile"),
+                            "-" + Arg.STATUS.getOptionName(), ("SUCCEEDED"),
+                            "-" + Arg.BRKR_TTL.getOptionName(), "10",
+                            "-" + Arg.CLUSTER.getOptionName(), "corp",
+                            "-" + Arg.WF_ENGINE_URL.getOptionName(),
+                            "http://localhost:11000/oozie/",
+                            "-" + Arg.LOG_DIR.getOptionName(), "target/log",
+                            "-" + Arg.USER_SUBFLOW_ID.getOptionName(), "userflow@wf-id" + "test"};
+        broker = new BrokerService();
+        broker.addConnector(BROKER_URL);
+        broker.setDataDirectory("target/activemq");
+        broker.setBrokerName("localhost");
+        broker.start();
+    }
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        broker.deleteAllMessages();
+        broker.stop();
+    }
+
+    @Test
+    public void testProcessMessageCreator() throws Exception {
+
+        Thread t = new Thread() {
+            @Override
+            public void run() {
+                try {
+                    consumer(BROKER_URL, "FALCON." + ENTITY_NAME);
+                    consumer(BROKER_URL, FALCON_TOPIC_NAME);
+                } catch (AssertionError e) {
+                    error = e;
+                } catch (JMSException ignore) {
+
+                }
+            }
+        };
+        t.start();
+        Thread.sleep(1500);
+        new FalconPostProcessing().run(this.args);
+        t.join();
+        if (error != null) {
+            throw error;
+        }
+    }
+
+    private void consumer(String brokerUrl, String topic) throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                brokerUrl);
+        Connection connection = connectionFactory.createConnection();
+        connection.start();
+
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Destination destination = session.createTopic(topic);
+        MessageConsumer consumer = session.createConsumer(destination);
+
+        // wait till you get atleast one message
+        MapMessage m;
+        for (m = null; m == null; ) {
+            m = (MapMessage) consumer.receive();
+        }
+        System.out.println("Consumed: " + m.toString());
+
+        assertMessage(m);
+        if (topic.equals(FALCON_TOPIC_NAME)) {
+            Assert.assertEquals(m.getString(Arg.FEED_NAMES.getOptionName()),
+                    "click-logs,raw-logs");
+            Assert.assertEquals(
+                    m.getString(Arg.FEED_INSTANCE_PATHS.getOptionName()),
+                    "/click-logs/10/05/05/00/20,/raw-logs/10/05/05/00/20");
+        } else {
+            Assert.assertEquals(m.getString(Arg.FEED_NAMES.getOptionName()),
+                    "click-logs");
+            Assert.assertEquals(
+                    m.getString(Arg.FEED_INSTANCE_PATHS.getOptionName()),
+                    "/click-logs/10/05/05/00/20");
+
+        }
+
+        connection.close();
+    }
+
+    private void assertMessage(MapMessage m) throws JMSException {
+        Assert.assertEquals(m.getString(Arg.ENTITY_NAME.getOptionName()),
+                "agg-coord");
+        Assert.assertEquals(m.getString(Arg.WORKFLOW_ID.getOptionName()),
+                "workflow-01-00");
+        Assert.assertEquals(m.getString(Arg.RUN_ID.getOptionName()), "1");
+        Assert.assertEquals(m.getString(Arg.NOMINAL_TIME.getOptionName()),
+                "2011-01-01T01:00Z");
+        Assert.assertEquals(m.getString(Arg.TIMESTAMP.getOptionName()),
+                "2012-01-01T01:00Z");
+        Assert.assertEquals(m.getString(Arg.STATUS.getOptionName()),
+                "SUCCEEDED");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java b/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
index e86e177..e17d377 100644
--- a/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
+++ b/oozie/src/test/java/org/apache/falcon/oozie/workflow/WorkflowUnmarshallingTest.java
@@ -18,6 +18,10 @@
 
 package org.apache.falcon.oozie.workflow;
 
+import org.testng.Assert;
+import org.testng.annotations.Test;
+import org.xml.sax.SAXException;
+
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBElement;
 import javax.xml.bind.JAXBException;
@@ -25,20 +29,18 @@ import javax.xml.bind.Unmarshaller;
 import javax.xml.validation.Schema;
 import javax.xml.validation.SchemaFactory;
 
-import org.testng.Assert;
-import org.testng.annotations.Test;
-import org.xml.sax.SAXException;
-
 public class WorkflowUnmarshallingTest {
 
     @Test
     public void testValidWorkflowUnamrashalling() throws JAXBException, SAXException {
-        Unmarshaller unmarshaller = JAXBContext.newInstance(org.apache.falcon.oozie.workflow.WORKFLOWAPP.class).createUnmarshaller();
+        Unmarshaller unmarshaller = JAXBContext.newInstance(
+                org.apache.falcon.oozie.workflow.WORKFLOWAPP.class).createUnmarshaller();
         SchemaFactory schemaFactory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
         Schema schema = schemaFactory.newSchema(this.getClass().getResource("/oozie-workflow-0.3.xsd"));
         unmarshaller.setSchema(schema);
-        JAXBElement<WORKFLOWAPP> workflowApp = (JAXBElement<WORKFLOWAPP>) unmarshaller.unmarshal(WorkflowUnmarshallingTest.class
-                .getResourceAsStream("/oozie/xmls/workflow.xml"));
+        JAXBElement<WORKFLOWAPP> workflowApp = (JAXBElement<WORKFLOWAPP>) unmarshaller.unmarshal(
+                WorkflowUnmarshallingTest.class
+                        .getResourceAsStream("/oozie/xmls/workflow.xml"));
         WORKFLOWAPP app = workflowApp.getValue();
         Assert.assertEquals(app.getName(), "java-main-wf");
         Assert.assertEquals(((ACTION) app.getDecisionOrForkOrJoin().get(0)).getName(), "java-node");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
----------------------------------------------------------------------
diff --git a/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java b/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
index 327d95c..897a86d 100644
--- a/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
+++ b/oozie/src/test/java/org/apache/oozie/client/CustomOozieClientTest.java
@@ -24,7 +24,7 @@ import java.util.Properties;
 
 public class CustomOozieClientTest {
 
-    @Test (enabled = false)
+    @Test(enabled = false)
     public void testGetConfiguration() throws Exception {
         CustomOozieClient client = new CustomOozieClient("http://localhost:11000/oozie");
         Properties props = client.getConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/resources/oozie/xmls/bundle.xml
----------------------------------------------------------------------
diff --git a/oozie/src/test/resources/oozie/xmls/bundle.xml b/oozie/src/test/resources/oozie/xmls/bundle.xml
index 3d75e3d..18f6215 100644
--- a/oozie/src/test/resources/oozie/xmls/bundle.xml
+++ b/oozie/src/test/resources/oozie/xmls/bundle.xml
@@ -15,18 +15,18 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<bundle-app name='bundle-app' xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance' xmlns='uri:oozie:bundle:0.1'>
-          <coordinator name='coord-1'>
-                 <app-path>${nameNode}/user/${userName}/${examplesRoot}/apps/aggregator/coordinator.xml</app-path>
-                 <configuration>
-                     <property>
-                         <name>start</name>
-                         <value>${start}</value>
-                     </property>
-                     <property>
-                         <name>end</name>
-                         <value>${end}</value>
-                     </property>
-                 </configuration>
-          </coordinator>
+<bundle-app name='bundle-app' xmlns='uri:oozie:bundle:0.1'>
+    <coordinator name='coord-1'>
+        <app-path>${nameNode}/user/${userName}/${examplesRoot}/apps/aggregator/coordinator.xml</app-path>
+        <configuration>
+            <property>
+                <name>start</name>
+                <value>${start}</value>
+            </property>
+            <property>
+                <name>end</name>
+                <value>${end}</value>
+            </property>
+        </configuration>
+    </coordinator>
 </bundle-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/resources/oozie/xmls/coordinator.xml
----------------------------------------------------------------------
diff --git a/oozie/src/test/resources/oozie/xmls/coordinator.xml b/oozie/src/test/resources/oozie/xmls/coordinator.xml
index f975432..64829c8 100644
--- a/oozie/src/test/resources/oozie/xmls/coordinator.xml
+++ b/oozie/src/test/resources/oozie/xmls/coordinator.xml
@@ -17,40 +17,42 @@
   limitations under the License.
   -->
 
-<coordinator-app end="" frequency="" name="test" start="" timezone="" xmlns="uri:oozie:coordinator:0.3" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="uri:oozie:coordinator:0.2 ../../main/resources/coordinator.xsd ">
-  <controls>
-    <timeout>timeout</timeout>
-    <concurrency>concurrency</concurrency>
-    <execution>execution</execution>
-    <throttle>throttle</throttle>
-  </controls>
-  <datasets>
-    <include>include</include>
-    <dataset frequency="" initial-instance="" name="dataset1" timezone="">
-      <uri-template>uri-template</uri-template>
-      <done-flag>done-flag</done-flag>
-    </dataset>
-  </datasets>
-  <input-events>
-    <data-in dataset="dataset1" name="data-in">
-      <instance>instance</instance>
-    </data-in>
-  </input-events>
-  <output-events>
-    <data-out dataset="dataset1" name="data-out1">
-      <instance>instance</instance>
-    </data-out>
-  </output-events>
-  <action>
-    <workflow>
-      <app-path>app-path</app-path>
-      <configuration>
-        <property>
-          <name>name</name>
-          <value>value</value>
-          <description>description</description>
-        </property>
-      </configuration>
-    </workflow>
-  </action>
+<coordinator-app end="" frequency="" name="test" start="" timezone="" xmlns="uri:oozie:coordinator:0.3"
+                 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+                 xsi:schemaLocation="/../main/resources/coordinator.xsd">
+    <controls>
+        <timeout>timeout</timeout>
+        <concurrency>concurrency</concurrency>
+        <execution>execution</execution>
+        <throttle>throttle</throttle>
+    </controls>
+    <datasets>
+        <include>include</include>
+        <dataset frequency="" initial-instance="" name="dataset1" timezone="">
+            <uri-template>uri-template</uri-template>
+            <done-flag>done-flag</done-flag>
+        </dataset>
+    </datasets>
+    <input-events>
+        <data-in dataset="dataset1" name="data-in">
+            <instance>instance</instance>
+        </data-in>
+    </input-events>
+    <output-events>
+        <data-out dataset="dataset1" name="data-out1">
+            <instance>instance</instance>
+        </data-out>
+    </output-events>
+    <action>
+        <workflow>
+            <app-path>app-path</app-path>
+            <configuration>
+                <property>
+                    <name>name</name>
+                    <value>value</value>
+                    <description>description</description>
+                </property>
+            </configuration>
+        </workflow>
+    </action>
 </coordinator-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/oozie/src/test/resources/oozie/xmls/workflow.xml
----------------------------------------------------------------------
diff --git a/oozie/src/test/resources/oozie/xmls/workflow.xml b/oozie/src/test/resources/oozie/xmls/workflow.xml
index 1ef7bf5..5d14413 100644
--- a/oozie/src/test/resources/oozie/xmls/workflow.xml
+++ b/oozie/src/test/resources/oozie/xmls/workflow.xml
@@ -25,7 +25,7 @@
             <main-class>org.apache.falcon.messaging.MessageProducer</main-class>
             <arg>${wf:name()}</arg>
             <arg>${wf:appPath()}</arg>
-	    <arg>${timestamp()}</arg>
+            <arg>${timestamp()}</arg>
 
         </java>
         <ok to="end"/>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/FalconWebException.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/FalconWebException.java b/prism/src/main/java/org/apache/falcon/FalconWebException.java
index 50e6772..c0d41ca 100644
--- a/prism/src/main/java/org/apache/falcon/FalconWebException.java
+++ b/prism/src/main/java/org/apache/falcon/FalconWebException.java
@@ -32,7 +32,7 @@ public class FalconWebException extends WebApplicationException {
     private static final Logger LOG = Logger.getLogger(FalconWebException.class);
 
     public static FalconWebException newException(Throwable e,
-                                                 Response.Status status) {
+                                                  Response.Status status) {
         LOG.error("Failure reason", e);
         return newException(e.getMessage() + "\n" + getAddnInfo(e), status);
     }
@@ -44,14 +44,14 @@ public class FalconWebException extends WebApplicationException {
 
 
     public static FalconWebException newException(APIResult result,
-                                                 Response.Status status) {
+                                                  Response.Status status) {
         LOG.error("Action failed: " + status + "\nError:" + result.getMessage());
         return new FalconWebException(Response.status(status).
                 entity(result).type(MediaType.TEXT_XML_TYPE).build());
     }
 
     public static FalconWebException newException(String message,
-                                                 Response.Status status) {
+                                                  Response.Status status) {
         LOG.error("Action failed: " + status + "\nError:" + message);
         APIResult result = new APIResult(APIResult.Status.FAILED, message);
         return new FalconWebException(Response.status(status).
@@ -65,8 +65,9 @@ public class FalconWebException extends WebApplicationException {
     }
 
     private static String getMessage(Throwable e) {
-        if(StringUtils.isEmpty(e.getMessage()))
+        if (StringUtils.isEmpty(e.getMessage())) {
             return e.getClass().getName();
+        }
         return e.getMessage();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java b/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
index 3b75caa..54a9b4a 100644
--- a/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
+++ b/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
@@ -18,12 +18,6 @@
 
 package org.apache.falcon.listener;
 
-import java.util.Map;
-import java.util.Properties;
-
-import javax.servlet.ServletContextEvent;
-import javax.servlet.ServletContextListener;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.service.ServiceInitializer;
@@ -32,6 +26,11 @@ import org.apache.falcon.util.RuntimeProperties;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.log4j.Logger;
 
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+import java.util.Map;
+import java.util.Properties;
+
 public class ContextStartupListener implements ServletContextListener {
 
     private static Logger LOG = Logger.getLogger(ContextStartupListener.class);
@@ -48,7 +47,7 @@ public class ContextStartupListener implements ServletContextListener {
 
         LOG.info("Initializing runtime properties ...");
         RuntimeProperties.get();
-        
+
         try {
             startupServices.initialize();
             ConfigurationStore.get();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java b/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
index c98bf53..2b99e70 100644
--- a/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
+++ b/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
@@ -46,7 +46,7 @@ public class ChainableMonitoringPlugin extends AbstractFalconAspect implements M
                 LOG.info("Registered Monitoring Plugin " + pluginClass);
             }
         } catch (FalconException e) {
-            plugins = Arrays.asList((MonitoringPlugin)new LoggingPlugin());
+            plugins = Arrays.asList((MonitoringPlugin) new LoggingPlugin());
             LOG.error("Unable to initialize monitoring plugins: " + pluginClasses, e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index ff18864..a272957 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -18,20 +18,8 @@
 
 package org.apache.falcon.resource;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.NoSuchElementException;
-import java.util.Set;
-
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.core.Response;
-
 import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.lang.ObjectUtils;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconRuntimException;
 import org.apache.falcon.FalconWebException;
@@ -54,8 +42,15 @@ import org.apache.falcon.util.DeploymentUtil;
 import org.apache.falcon.util.RuntimeProperties;
 import org.apache.falcon.workflow.WorkflowEngineFactory;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.log4j.Logger;
 
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.*;
+
 public abstract class AbstractEntityManager {
     private static final Logger LOG = Logger.getLogger(AbstractEntityManager.class);
     private static final Logger AUDIT = Logger.getLogger("AUDIT");
@@ -74,14 +69,16 @@ public abstract class AbstractEntityManager {
 
     protected void checkColo(String colo) throws FalconWebException {
         if (!DeploymentUtil.getCurrentColo().equals(colo)) {
-            throw FalconWebException.newException("Current colo (" + DeploymentUtil.getCurrentColo() + ") is not " + colo,
+            throw FalconWebException.newException(
+                    "Current colo (" + DeploymentUtil.getCurrentColo() + ") is not " + colo,
                     Response.Status.BAD_REQUEST);
         }
     }
 
     protected Set<String> getAllColos() {
-        if (DeploymentUtil.isEmbeddedMode())
+        if (DeploymentUtil.isEmbeddedMode()) {
             return DeploymentUtil.getDefaultColos();
+        }
         String[] colos = RuntimeProperties.get().getProperty("all.colos", DeploymentUtil.getDefaultColo()).split(",");
         return new HashSet<String>(Arrays.asList(colos));
     }
@@ -95,28 +92,32 @@ public abstract class AbstractEntityManager {
         }
         return colos;
     }
-    
+
     protected Set<String> getApplicableColos(String type, String name) throws FalconWebException {
         try {
-            if (DeploymentUtil.isEmbeddedMode())
+            if (DeploymentUtil.isEmbeddedMode()) {
                 return DeploymentUtil.getDefaultColos();
+            }
 
-            if (EntityType.valueOf(type.toUpperCase()) == EntityType.CLUSTER)
+            if (EntityType.valueOf(type.toUpperCase()) == EntityType.CLUSTER) {
                 return getAllColos();
+            }
 
             return getApplicableColos(type, EntityUtil.getEntity(type, name));
         } catch (FalconException e) {
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
         }
     }
-    
+
     protected Set<String> getApplicableColos(String type, Entity entity) throws FalconWebException {
         try {
-            if (DeploymentUtil.isEmbeddedMode())
+            if (DeploymentUtil.isEmbeddedMode()) {
                 return DeploymentUtil.getDefaultColos();
+            }
 
-            if (EntityType.valueOf(type.toUpperCase()) == EntityType.CLUSTER)
+            if (EntityType.valueOf(type.toUpperCase()) == EntityType.CLUSTER) {
                 return getAllColos();
+            }
 
             Set<String> clusters = EntityUtil.getClustersDefined(entity);
             Set<String> colos = new HashSet<String>();
@@ -134,16 +135,13 @@ public abstract class AbstractEntityManager {
      * Submit a new entity. Entities can be of type feed, process or data end
      * points. Entity definitions are validated structurally against schema and
      * subsequently for other rules before they are admitted into the system
-     * 
+     * <p/>
      * Entity name acts as the key and an entity once added, can't be added
      * again unless deleted.
-     * 
-     * @param request
-     *            - Servlet Request
-     * @param type
-     *            - entity type - feed, process or data end point
-     * @param colo
-     *            - applicable colo
+     *
+     * @param request - Servlet Request
+     * @param type    - entity type - feed, process or data end point
+     * @param colo    - applicable colo
      * @return result of the operation
      */
     public APIResult submit(HttpServletRequest request, String type, String colo) {
@@ -162,7 +160,7 @@ public abstract class AbstractEntityManager {
     /**
      * Post an entity XML with entity type. Validates the XML which can be
      * Process, Feed or Dataendpoint
-     * 
+     *
      * @param type
      * @return APIResule -Succeeded or Failed
      */
@@ -171,7 +169,8 @@ public abstract class AbstractEntityManager {
             EntityType entityType = EntityType.valueOf(type.toUpperCase());
             Entity entity = deserializeEntity(request, entityType);
             validate(entity);
-            return new APIResult(APIResult.Status.SUCCEEDED, "Validated successfully (" + entityType + ") " + entity.getName());
+            return new APIResult(APIResult.Status.SUCCEEDED,
+                    "Validated successfully (" + entityType + ") " + entity.getName());
         } catch (Throwable e) {
             LOG.error("Validation failed for entity (" + type + ") ", e);
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
@@ -181,7 +180,7 @@ public abstract class AbstractEntityManager {
     /**
      * Deletes a scheduled entity, a deleted entity is removed completely from
      * execution pool.
-     * 
+     *
      * @param type
      * @param entity
      * @return APIResult
@@ -204,10 +203,12 @@ public abstract class AbstractEntityManager {
 
                 configStore.remove(entityType, entity);
             } catch (EntityNotRegisteredException e) { // already deleted
-                return new APIResult(APIResult.Status.SUCCEEDED, entity + "(" + type + ") doesn't exist. Nothing to do");
+                return new APIResult(APIResult.Status.SUCCEEDED,
+                        entity + "(" + type + ") doesn't exist. Nothing to do");
             }
 
-            return new APIResult(APIResult.Status.SUCCEEDED, entity + "(" + type + ") removed successfully " + removedFromEngine);
+            return new APIResult(APIResult.Status.SUCCEEDED,
+                    entity + "(" + type + ") removed successfully " + removedFromEngine);
         } catch (Throwable e) {
             LOG.error("Unable to reach workflow engine for deletion or " + "deletion failed", e);
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
@@ -229,7 +230,7 @@ public abstract class AbstractEntityManager {
             if (!EntityUtil.equals(oldEntity, newEntity)) {
                 configStore.initiateUpdate(newEntity);
                 //Update in workflow engine
-            	if(! DeploymentUtil.isPrism()) {
+                if (!DeploymentUtil.isPrism()) {
                     Set<String> oldClusters = EntityUtil.getClustersDefinedInColos(oldEntity);
                     Set<String> newClusters = EntityUtil.getClustersDefinedInColos(newEntity);
                     newClusters.retainAll(oldClusters); //common clusters for update
@@ -238,11 +239,11 @@ public abstract class AbstractEntityManager {
                     for (String cluster : newClusters) {
                         getWorkflowEngine().update(oldEntity, newEntity, cluster);
                     }
-                    for(String cluster:oldClusters) {
+                    for (String cluster : oldClusters) {
                         getWorkflowEngine().delete(oldEntity, cluster);
                     }
-            	}
-            	
+                }
+
                 configStore.update(entityType, newEntity);
             }
 
@@ -256,11 +257,14 @@ public abstract class AbstractEntityManager {
     }
 
     private void validateUpdate(Entity oldEntity, Entity newEntity) throws FalconException {
-        if (oldEntity.getEntityType() != newEntity.getEntityType() || !oldEntity.equals(newEntity))
-            throw new FalconException(oldEntity.toShortString() + " can't be updated with " + newEntity.toShortString());
+        if (oldEntity.getEntityType() != newEntity.getEntityType() || !oldEntity.equals(newEntity)) {
+            throw new FalconException(
+                    oldEntity.toShortString() + " can't be updated with " + newEntity.toShortString());
+        }
 
-        if (oldEntity.getEntityType() == EntityType.CLUSTER)
+        if (oldEntity.getEntityType() == EntityType.CLUSTER) {
             throw new FalconException("Update not supported for clusters");
+        }
 
         String[] props = oldEntity.getEntityType().getImmutableProperties();
         for (String prop : props) {
@@ -271,8 +275,9 @@ public abstract class AbstractEntityManager {
             } catch (Exception e) {
                 throw new FalconException(e);
             }
-            if (!ObjectUtils.equals(oldProp, newProp))
+            if (!ObjectUtils.equals(oldProp, newProp)) {
                 throw new ValidationException(oldEntity.toShortString() + ": " + prop + " can't be changed");
+            }
         }
     }
 
@@ -283,23 +288,27 @@ public abstract class AbstractEntityManager {
             for (Pair<String, EntityType> ref : referencedBy) {
                 messages.append(ref).append("\n");
             }
-            throw new FalconException(entity.getName() + "(" + entity.getEntityType() + ") cant " + "be removed as it is referred by "
-                    + messages);
+            throw new FalconException(
+                    entity.getName() + "(" + entity.getEntityType() + ") cant " + "be removed as it is referred by "
+                            + messages);
         }
     }
 
-    protected synchronized Entity submitInternal(HttpServletRequest request, String type) throws IOException, FalconException {
+    protected synchronized Entity submitInternal(HttpServletRequest request, String type)
+            throws IOException, FalconException {
 
         EntityType entityType = EntityType.valueOf(type.toUpperCase());
         Entity entity = deserializeEntity(request, entityType);
 
         Entity existingEntity = configStore.get(entityType, entity.getName());
         if (existingEntity != null) {
-            if (EntityUtil.equals(existingEntity, entity))
+            if (EntityUtil.equals(existingEntity, entity)) {
                 return existingEntity;
+            }
 
-            throw new EntityAlreadyExistsException(entity.toShortString() + " already registered with configuration store. "
-                    + "Can't be submitted again. Try removing before submitting.");
+            throw new EntityAlreadyExistsException(
+                    entity.toShortString() + " already registered with configuration store. "
+                            + "Can't be submitted again. Try removing before submitting.");
         }
 
         validate(entity);
@@ -308,7 +317,8 @@ public abstract class AbstractEntityManager {
         return entity;
     }
 
-    protected Entity deserializeEntity(HttpServletRequest request, EntityType entityType) throws IOException, FalconException {
+    protected Entity deserializeEntity(HttpServletRequest request, EntityType entityType)
+            throws IOException, FalconException {
 
         EntityParser<?> entityParser = EntityParserFactory.getParser(entityType);
         InputStream xmlStream = request.getInputStream();
@@ -330,7 +340,7 @@ public abstract class AbstractEntityManager {
         }
     }
 
-    @SuppressWarnings({ "unchecked", "rawtypes" })
+    @SuppressWarnings({"unchecked", "rawtypes"})
     private void validate(Entity entity) throws FalconException {
         EntityParser entityParser = EntityParserFactory.getParser(entity.getEntityType());
         entityParser.validate(entity);
@@ -356,7 +366,7 @@ public abstract class AbstractEntityManager {
 
     /**
      * Returns the status of requested entity.
-     * 
+     *
      * @param type
      * @param entity
      * @return String
@@ -395,7 +405,7 @@ public abstract class AbstractEntityManager {
 
     /**
      * Returns dependencies.
-     * 
+     *
      * @param type
      * @param entity
      * @return EntityList
@@ -406,7 +416,7 @@ public abstract class AbstractEntityManager {
             Entity entityObj = EntityUtil.getEntity(type, entity);
             Set<Entity> dependents = EntityGraph.get().getDependents(entityObj);
             Entity[] entities = dependents.toArray(new Entity[dependents.size()]);
-            return new EntityList(entities == null ? new Entity[] {} : entities);
+            return new EntityList(entities == null ? new Entity[]{} : entities);
         } catch (Exception e) {
             LOG.error("Unable to get dependencies for entity " + entity + "(" + type + ")", e);
             throw FalconWebException.newException(e, Response.Status.BAD_REQUEST);
@@ -415,7 +425,7 @@ public abstract class AbstractEntityManager {
 
     /**
      * Returns the list of entities registered of a given type.
-     * 
+     *
      * @param type
      * @return String
      */
@@ -424,7 +434,7 @@ public abstract class AbstractEntityManager {
             EntityType entityType = EntityType.valueOf(type.toUpperCase());
             Collection<String> entityNames = configStore.getEntities(entityType);
             if (entityNames == null || entityNames.equals("")) {
-                return new EntityList(new Entity[] {});
+                return new EntityList(new Entity[]{});
             }
             Entity[] entities = new Entity[entityNames.size()];
             int index = 0;
@@ -440,7 +450,7 @@ public abstract class AbstractEntityManager {
 
     /**
      * Returns the entity definition as an XML based on name
-     * 
+     *
      * @param type
      * @param entityName
      * @return String

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index 6b3de49..8097c43 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -18,15 +18,6 @@
 
 package org.apache.falcon.resource;
 
-import java.io.IOException;
-import java.util.Date;
-import java.util.Properties;
-import java.util.Set;
-
-import javax.servlet.ServletInputStream;
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.core.Response;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.FalconWebException;
@@ -41,9 +32,17 @@ import org.apache.falcon.resource.InstancesResult.Instance;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
 import org.apache.log4j.Logger;
 
+import javax.servlet.ServletInputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+import java.util.Date;
+import java.util.Properties;
+import java.util.Set;
+
 public abstract class AbstractInstanceManager extends AbstractEntityManager {
     private static final Logger LOG = Logger.getLogger(AbstractInstanceManager.class);
-	
+
     protected void checkType(String type) {
         if (StringUtils.isEmpty(type)) {
             throw FalconWebException.newInstanceException("entity type is empty",
@@ -51,7 +50,8 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         } else {
             EntityType entityType = EntityType.valueOf(type.toUpperCase());
             if (entityType == EntityType.CLUSTER) {
-                throw FalconWebException.newInstanceException("Instance management functions don't apply to Cluster entities",
+                throw FalconWebException.newInstanceException(
+                        "Instance management functions don't apply to Cluster entities",
                         Response.Status.BAD_REQUEST);
             }
         }
@@ -73,45 +73,45 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
 
 
     public InstancesResult getStatus(String type, String entity, String startStr, String endStr,
-                                           String colo) {
+                                     String colo) {
         checkColo(colo);
         checkType(type);
         try {
-			validateParams(type, entity, startStr, endStr);
-
-			Date start = EntityUtil.parseDateUTC(startStr);
-			Date end = getEndDate(start, endStr);
-			Entity entityObject = EntityUtil.getEntity(type, entity);
-
-			AbstractWorkflowEngine wfEngine = getWorkflowEngine();
-			return wfEngine.getStatus(
-					entityObject, start, end);
-		} catch (Throwable e) {
-			LOG.error("Failed to get instances status", e);
-			throw FalconWebException
-					.newInstanceException(e, Response.Status.BAD_REQUEST);
-		}
-	}
-
-	public InstancesResult getLogs(String type, String entity, String startStr,
-			String endStr, String colo, String runId){
-
-		try {
-			// TODO getStatus does all validations and filters clusters
-			InstancesResult result = getStatus(type, entity, startStr, endStr,
-					colo);
-			LogProvider logProvider = new LogProvider();
-			Entity entityObject = EntityUtil.getEntity(type, entity);
-			for (Instance instance : result.getInstances()) {
-				logProvider.populateLogUrls(entityObject, instance, runId);
-			}
-			return result;
-		} catch (Exception e) {
-			LOG.error("Failed to get logs for instances", e);
-			throw FalconWebException.newInstanceException(e,
-					Response.Status.BAD_REQUEST);
-		}
-	}
+            validateParams(type, entity, startStr, endStr);
+
+            Date start = EntityUtil.parseDateUTC(startStr);
+            Date end = getEndDate(start, endStr);
+            Entity entityObject = EntityUtil.getEntity(type, entity);
+
+            AbstractWorkflowEngine wfEngine = getWorkflowEngine();
+            return wfEngine.getStatus(
+                    entityObject, start, end);
+        } catch (Throwable e) {
+            LOG.error("Failed to get instances status", e);
+            throw FalconWebException
+                    .newInstanceException(e, Response.Status.BAD_REQUEST);
+        }
+    }
+
+    public InstancesResult getLogs(String type, String entity, String startStr,
+                                   String endStr, String colo, String runId) {
+
+        try {
+            // TODO getStatus does all validations and filters clusters
+            InstancesResult result = getStatus(type, entity, startStr, endStr,
+                    colo);
+            LogProvider logProvider = new LogProvider();
+            Entity entityObject = EntityUtil.getEntity(type, entity);
+            for (Instance instance : result.getInstances()) {
+                logProvider.populateLogUrls(entityObject, instance, runId);
+            }
+            return result;
+        } catch (Exception e) {
+            LOG.error("Failed to get logs for instances", e);
+            throw FalconWebException.newInstanceException(e,
+                    Response.Status.BAD_REQUEST);
+        }
+    }
 
     public InstancesResult killInstance(HttpServletRequest request,
                                         String type, String entity, String startStr, String endStr, String colo) {
@@ -121,11 +121,11 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             audit(request, entity, type, "INSTANCE_KILL");
             validateParams(type, entity, startStr, endStr);
-            
+
             Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);            
+            Date end = getEndDate(start, endStr);
             Entity entityObject = EntityUtil.getEntity(type, entity);
-            
+
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
             return wfEngine.killInstances(entityObject, start, end, props);
@@ -143,11 +143,11 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             audit(request, entity, type, "INSTANCE_SUSPEND");
             validateParams(type, entity, startStr, endStr);
-            
+
             Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);            
+            Date end = getEndDate(start, endStr);
             Entity entityObject = EntityUtil.getEntity(type, entity);
-            
+
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
             return wfEngine.suspendInstances(entityObject, start, end, props);
@@ -165,11 +165,11 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         try {
             audit(request, entity, type, "INSTANCE_RESUME");
             validateParams(type, entity, startStr, endStr);
-            
+
             Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);            
+            Date end = getEndDate(start, endStr);
             Entity entityObject = EntityUtil.getEntity(type, entity);
-            
+
             Properties props = getProperties(request);
             AbstractWorkflowEngine wfEngine = getWorkflowEngine();
             return wfEngine.resumeInstances(entityObject, start, end, props);
@@ -180,16 +180,16 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
     }
 
     public InstancesResult reRunInstance(String type, String entity, String startStr, String endStr,
-                                                HttpServletRequest request, String colo) {
+                                         HttpServletRequest request, String colo) {
 
         checkColo(colo);
         checkType(type);
         try {
             audit(request, entity, type, "INSTANCE_RERUN");
             validateParams(type, entity, startStr, endStr);
-            
+
             Date start = EntityUtil.parseDateUTC(startStr);
-            Date end = getEndDate(start, endStr);            
+            Date end = getEndDate(start, endStr);
             Entity entityObject = EntityUtil.getEntity(type, entity);
 
             Properties props = getProperties(request);
@@ -203,7 +203,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
 
     private Properties getProperties(HttpServletRequest request) throws IOException {
         Properties props = new Properties();
-        ServletInputStream xmlStream = request==null?null:request.getInputStream();
+        ServletInputStream xmlStream = request == null ? null : request.getInputStream();
         if (xmlStream != null) {
             if (xmlStream.markSupported()) {
                 xmlStream.mark(XML_DEBUG_LEN); // mark up to debug len
@@ -217,11 +217,12 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         Date end;
         if (StringUtils.isEmpty(endStr)) {
             end = new Date(start.getTime() + 1000); // next sec
-        } else
+        } else {
             end = EntityUtil.parseDateUTC(endStr);
+        }
         return end;
     }
-    
+
     private void validateParams(String type, String entity, String startStr, String endStr) throws FalconException {
         validateNotEmpty("entityType", type);
         validateNotEmpty("entityName", entity);
@@ -231,13 +232,14 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         validateDateRange(entityObject, startStr, endStr);
     }
 
-	private void validateDateRange(Entity entity, String start, String end)
-			throws FalconException {
+    private void validateDateRange(Entity entity, String start, String end)
+            throws FalconException {
         Set<String> clusters = EntityUtil.getClustersDefined(entity);
         Pair<Date, String> clusterMinStartDate = null;
         Pair<Date, String> clusterMaxEndDate = null;
         for (String cluster : clusters) {
-            if (clusterMinStartDate == null || clusterMinStartDate.first.after(EntityUtil.getStartTime(entity, cluster))) {
+            if (clusterMinStartDate == null || clusterMinStartDate.first.after(
+                    EntityUtil.getStartTime(entity, cluster))) {
                 clusterMinStartDate = Pair.of(EntityUtil.getStartTime(entity, cluster), cluster);
             }
             if (clusterMaxEndDate == null || clusterMaxEndDate.first.before(EntityUtil.getEndTime(entity, cluster))) {
@@ -245,40 +247,46 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
             }
         }
 
-		validateDateRangeFor(entity, clusterMinStartDate, clusterMaxEndDate,
-				start, end);
-	}
-    
-    private void validateDateRangeFor(Entity entity, Pair<Date,String> clusterMinStart,
-    		Pair<Date,String> clusterMaxEnd, String start, String end) throws FalconException {
-
-    	Date instStart = EntityUtil.parseDateUTC(start);
-    	if (instStart.before(clusterMinStart.first))
-    		throw new ValidationException("Start date " + start + " is before "
-    				+ entity.getEntityType() + "'s  start "
-    				+ SchemaHelper.formatDateUTC(clusterMinStart.first)
-    				+ " for cluster " + clusterMinStart.second);
-
-		if (StringUtils.isNotEmpty(end)) {
-			Date instEnd = EntityUtil.parseDateUTC(end);
-			if (instStart.after(instEnd))
-				throw new ValidationException("Start date " + start
-						+ " is after end date " + end);
-
-			if (instEnd.after(clusterMaxEnd.first))
-				throw new ValidationException("End date " + end + " is after "
-						+ entity.getEntityType() + "'s end "
-						+ SchemaHelper.formatDateUTC(clusterMaxEnd.first)
-						+ " for cluster " + clusterMaxEnd.second);
-		} else if (instStart.after(clusterMaxEnd.first))
-			throw new ValidationException("Start date " + start + " is after "
-					+ entity.getEntityType() + "'s end "
-					+ SchemaHelper.formatDateUTC(clusterMaxEnd.first)
-					+ " for cluster " + clusterMaxEnd.second);
+        validateDateRangeFor(entity, clusterMinStartDate, clusterMaxEndDate,
+                start, end);
+    }
+
+    private void validateDateRangeFor(Entity entity, Pair<Date, String> clusterMinStart,
+                                      Pair<Date, String> clusterMaxEnd, String start, String end)
+            throws FalconException {
+
+        Date instStart = EntityUtil.parseDateUTC(start);
+        if (instStart.before(clusterMinStart.first)) {
+            throw new ValidationException("Start date " + start + " is before "
+                    + entity.getEntityType() + "'s  start "
+                    + SchemaHelper.formatDateUTC(clusterMinStart.first)
+                    + " for cluster " + clusterMinStart.second);
+        }
+
+        if (StringUtils.isNotEmpty(end)) {
+            Date instEnd = EntityUtil.parseDateUTC(end);
+            if (instStart.after(instEnd)) {
+                throw new ValidationException("Start date " + start
+                        + " is after end date " + end);
+            }
+
+            if (instEnd.after(clusterMaxEnd.first)) {
+                throw new ValidationException("End date " + end + " is after "
+                        + entity.getEntityType() + "'s end "
+                        + SchemaHelper.formatDateUTC(clusterMaxEnd.first)
+                        + " for cluster " + clusterMaxEnd.second);
+            }
+        } else if (instStart.after(clusterMaxEnd.first)) {
+            throw new ValidationException("Start date " + start + " is after "
+                    + entity.getEntityType() + "'s end "
+                    + SchemaHelper.formatDateUTC(clusterMaxEnd.first)
+                    + " for cluster " + clusterMaxEnd.second);
+        }
     }
 
     private void validateNotEmpty(String field, String param) throws ValidationException {
-        if (StringUtils.isEmpty(param))
+        if (StringUtils.isEmpty(param)) {
             throw new ValidationException("Parameter " + field + " is empty");
-    }    
+        }
+    }
 }


[17/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/ProcessEntityParser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/ProcessEntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/ProcessEntityParser.java
index d8751a2..ba618ef 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/ProcessEntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/ProcessEntityParser.java
@@ -18,35 +18,23 @@
 
 package org.apache.falcon.entity.parser;
 
-import java.net.ConnectException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TimeZone;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.Input;
-import org.apache.falcon.entity.v0.process.Inputs;
-import org.apache.falcon.entity.v0.process.LateInput;
-import org.apache.falcon.entity.v0.process.Output;
-import org.apache.falcon.entity.v0.process.Outputs;
+import org.apache.falcon.entity.v0.process.*;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.net.ConnectException;
+import java.util.*;
 
 /**
  * Concrete Parser which has XML parsing and validation logic for Process XML.
- * 
  */
 public class ProcessEntityParser extends EntityParser<Process> {
 
@@ -56,17 +44,18 @@ public class ProcessEntityParser extends EntityParser<Process> {
 
     @Override
     public void validate(Process process) throws FalconException {
-        if(process.getTimezone() == null)
+        if (process.getTimezone() == null) {
             process.setTimezone(TimeZone.getTimeZone("UTC"));
+        }
 
         // check if dependent entities exists
         Set<String> clusters = new HashSet<String>();
         for (org.apache.falcon.entity.v0.process.Cluster cluster : process.getClusters().getClusters()) {
             String clusterName = cluster.getName();
-			if (!clusters.add(cluster.getName())) {
-				throw new ValidationException("Cluster: " + cluster.getName()
-						+ " is defined more than once for process: "+process.getName());
-			}
+            if (!clusters.add(cluster.getName())) {
+                throw new ValidationException("Cluster: " + cluster.getName()
+                        + " is defined more than once for process: " + process.getName());
+            }
             validateEntityExists(EntityType.CLUSTER, clusterName);
             validateProcessValidity(cluster.getValidity().getStart(), cluster.getValidity().getEnd());
             validateHDFSpaths(process, clusterName);
@@ -98,25 +87,28 @@ public class ProcessEntityParser extends EntityParser<Process> {
     }
 
     private void validateHDFSpaths(Process process, String clusterName) throws FalconException {
-        org.apache.falcon.entity.v0.cluster.Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER, clusterName);
+        org.apache.falcon.entity.v0.cluster.Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER,
+                clusterName);
         String workflowPath = process.getWorkflow().getPath();
-        String libPath=process.getWorkflow().getLib();
+        String libPath = process.getWorkflow().getLib();
         String nameNode = getNameNode(cluster, clusterName);
         try {
             Configuration configuration = new Configuration();
             configuration.set("fs.default.name", nameNode);
             FileSystem fs = FileSystem.get(configuration);
             if (!fs.exists(new Path(workflowPath))) {
-                throw new ValidationException("Workflow path: " + workflowPath + " does not exists in HDFS: " + nameNode);
+                throw new ValidationException(
+                        "Workflow path: " + workflowPath + " does not exists in HDFS: " + nameNode);
             }
-            
-            if (libPath!=null && !fs.exists(new Path(libPath))) {
+
+            if (libPath != null && !fs.exists(new Path(libPath))) {
                 throw new ValidationException("Lib path: " + libPath + " does not exists in HDFS: " + nameNode);
             }
         } catch (ValidationException e) {
             throw new ValidationException(e);
         } catch (ConnectException e) {
-            throw new ValidationException("Unable to connect to Namenode: " + nameNode + " referenced in cluster: " + clusterName);
+            throw new ValidationException(
+                    "Unable to connect to Namenode: " + nameNode + " referenced in cluster: " + clusterName);
         } catch (Exception e) {
             throw new FalconException(e);
         }
@@ -125,8 +117,9 @@ public class ProcessEntityParser extends EntityParser<Process> {
     private String getNameNode(Cluster cluster, String clusterName) throws ValidationException {
         // cluster should never be null as it is validated while submitting
         // feeds.
-        if (new Path(ClusterHelper.getStorageUrl(cluster)).toUri().getScheme()==null) {
-            throw new ValidationException("Cannot get valid nameNode scheme from write interface of cluster: " + clusterName);
+        if (new Path(ClusterHelper.getStorageUrl(cluster)).toUri().getScheme() == null) {
+            throw new ValidationException(
+                    "Cannot get valid nameNode scheme from write interface of cluster: " + clusterName);
         }
         return ClusterHelper.getStorageUrl(cluster);
     }
@@ -134,7 +127,8 @@ public class ProcessEntityParser extends EntityParser<Process> {
     private void validateProcessValidity(Date start, Date end) throws FalconException {
         try {
             if (!start.before(end)) {
-                throw new ValidationException("Process start time: " + start + " should be before process end time: " + end);
+                throw new ValidationException(
+                        "Process start time: " + start + " should be before process end time: " + end);
             }
         } catch (ValidationException e) {
             throw new ValidationException(e);
@@ -145,15 +139,15 @@ public class ProcessEntityParser extends EntityParser<Process> {
 
     private void validateDatasetName(Inputs inputs, Outputs outputs) throws ValidationException {
         Set<String> datasetNames = new HashSet<String>();
-        if(inputs != null) {
+        if (inputs != null) {
             for (Input input : inputs.getInputs()) {
                 if (!datasetNames.add(input.getName())) {
                     throw new ValidationException("Input name: " + input.getName() + " is already used");
                 }
             }
         }
-        
-        if(outputs != null) {
+
+        if (outputs != null) {
             for (Output output : outputs.getOutputs()) {
                 if (!datasetNames.add(output.getName())) {
                     throw new ValidationException("Output name: " + output.getName() + " is already used");
@@ -163,27 +157,28 @@ public class ProcessEntityParser extends EntityParser<Process> {
     }
 
     private void validateLateInputs(Process process) throws ValidationException {
-        Map<String,String> feeds = new HashMap<String,String>();
-        if(process.getInputs() != null) {
+        Map<String, String> feeds = new HashMap<String, String>();
+        if (process.getInputs() != null) {
             for (Input in : process.getInputs().getInputs()) {
-            	feeds.put(in.getName(),in.getFeed());
+                feeds.put(in.getName(), in.getFeed());
             }
         }
 
         if (process.getLateProcess() != null) {
-        	for (LateInput lp : process.getLateProcess().getLateInputs()) {
-        		if (!feeds.keySet().contains(lp.getInput())){
-        			throw new ValidationException("Late Input: " + lp.getInput() + " is not specified in the inputs");
-				}
-				try {
-        			Feed feed = (Feed) ConfigurationStore.get().get(EntityType.FEED, feeds.get(lp.getInput()));
-        			if(feed.getLateArrival()==null){
-        				throw new ValidationException("Late Input feed: "+lp.getInput()+" is not configured with late arrival cut-off" );
-        			}
-				} catch (FalconException e) {
-					throw new ValidationException(e);
-				}
-        	}
+            for (LateInput lp : process.getLateProcess().getLateInputs()) {
+                if (!feeds.keySet().contains(lp.getInput())) {
+                    throw new ValidationException("Late Input: " + lp.getInput() + " is not specified in the inputs");
+                }
+                try {
+                    Feed feed = (Feed) ConfigurationStore.get().get(EntityType.FEED, feeds.get(lp.getInput()));
+                    if (feed.getLateArrival() == null) {
+                        throw new ValidationException(
+                                "Late Input feed: " + lp.getInput() + " is not configured with late arrival cut-off");
+                    }
+                } catch (FalconException e) {
+                    throw new ValidationException(e);
+                }
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java b/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
index fc9ccdf..e01a378 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/ValidationException.java
@@ -21,22 +21,20 @@ package org.apache.falcon.entity.parser;
 import org.apache.falcon.FalconException;
 
 /**
- * 
  * ValidationException during parsing
- * 
  */
 public class ValidationException extends FalconException {
 
-	public ValidationException(String message) {
-		super(message);
-	}
+    public ValidationException(String message) {
+        super(message);
+    }
 
-	public ValidationException(Exception e) {
-	    super(e);
+    public ValidationException(Exception e) {
+        super(e);
     }
 
-	public ValidationException(String message, Exception e) {
-	    super(message, e);
+    public ValidationException(String message, Exception e) {
+        super(message, e);
     }
 
     private static final long serialVersionUID = -4502166408759507355L;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
index c882453..8fd3775 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/ConfigurationStore.java
@@ -18,25 +18,6 @@
 
 package org.apache.falcon.entity.store;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.xml.bind.JAXBException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
@@ -44,8 +25,21 @@ import org.apache.falcon.service.ConfigurationChangeListener;
 import org.apache.falcon.service.FalconService;
 import org.apache.falcon.util.ReflectionUtils;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
 public class ConfigurationStore implements FalconService {
 
     private static final Logger LOG = Logger.getLogger(ConfigurationStore.class);
@@ -62,7 +56,8 @@ public class ConfigurationStore implements FalconService {
         return store;
     }
 
-    private final Map<EntityType, ConcurrentHashMap<String, Entity>> dictionary = new HashMap<EntityType, ConcurrentHashMap<String, Entity>>();
+    private final Map<EntityType, ConcurrentHashMap<String, Entity>> dictionary
+            = new HashMap<EntityType, ConcurrentHashMap<String, Entity>>();
 
     private final FileSystem fs;
     private final Path storePath;
@@ -98,13 +93,15 @@ public class ConfigurationStore implements FalconService {
     public void init() throws FalconException {
         String listenerClassNames = StartupProperties.get().
                 getProperty("configstore.listeners", "org.apache.falcon.entity.v0.EntityGraph");
-        for(String listenerClassName:listenerClassNames.split(",")) {
+        for (String listenerClassName : listenerClassNames.split(",")) {
             listenerClassName = listenerClassName.trim();
-            if (listenerClassName.isEmpty()) continue;
+            if (listenerClassName.isEmpty()) {
+                continue;
+            }
             ConfigurationChangeListener listener = ReflectionUtils.getInstanceByClassName(listenerClassName);
             registerListener(listener);
         }
-        
+
         try {
             for (EntityType type : EntityType.values()) {
                 ConcurrentHashMap<String, Entity> entityMap = dictionary.get(type);
@@ -113,7 +110,7 @@ public class ConfigurationStore implements FalconService {
                     for (FileStatus file : files) {
                         String fileName = file.getPath().getName();
                         String encodedEntityName = fileName.substring(0, fileName.length() - 4); // drop
-                                                                                                 // ".xml"
+                        // ".xml"
                         String entityName = URLDecoder.decode(encodedEntityName, UTF_8);
                         Entity entity = restore(type, entityName);
                         entityMap.put(entityName, entity);
@@ -131,11 +128,8 @@ public class ConfigurationStore implements FalconService {
     }
 
     /**
-     * 
-     * @param type
-     *            - EntityType that need to be published
-     * @param entity
-     *            - Reference to the Entity Object
+     * @param type   - EntityType that need to be published
+     * @param entity - Reference to the Entity Object
      * @throws FalconException
      */
     public synchronized void publish(EntityType type, Entity entity) throws FalconException {
@@ -145,8 +139,9 @@ public class ConfigurationStore implements FalconService {
                 dictionary.get(type).put(entity.getName(), entity);
                 onAdd(entity);
             } else {
-                throw new EntityAlreadyExistsException(entity.toShortString() + " already registered with configuration store. "
-                        + "Can't be submitted again. Try removing before submitting.");
+                throw new EntityAlreadyExistsException(
+                        entity.toShortString() + " already registered with configuration store. "
+                                + "Can't be submitted again. Try removing before submitting.");
             }
         } catch (IOException e) {
             throw new StoreAccessException(e);
@@ -193,25 +188,22 @@ public class ConfigurationStore implements FalconService {
 
     private void onChange(Entity oldEntity, Entity newEntity) throws FalconException {
         for (ConfigurationChangeListener listener : listeners) {
-			listener.onChange(oldEntity, newEntity);
+            listener.onChange(oldEntity, newEntity);
         }
     }
 
     public synchronized void initiateUpdate(Entity entity) throws FalconException {
         if (get(entity.getEntityType(), entity.getName()) == null || updatesInProgress.get() != null) {
-            throw new FalconException("An update for " + entity.toShortString() + " is already in progress or doesn't exist");
+            throw new FalconException(
+                    "An update for " + entity.toShortString() + " is already in progress or doesn't exist");
         }
         updatesInProgress.set(entity);
     }
 
     /**
-     * 
-     * @param type
-     *            - Entity type that is being retrieved
-     * @param name
-     *            - Name as it appears in the entity xml definition
-     * @param <T>
-     *            - Actual Entity object type
+     * @param type - Entity type that is being retrieved
+     * @param name - Name as it appears in the entity xml definition
+     * @param <T>  - Actual Entity object type
      * @return - Entity object from internal dictionary, If the object is not
      *         loaded in memory yet, it will retrieve it from persistent store
      *         just in time. On startup all the entities will be added to the
@@ -249,11 +241,9 @@ public class ConfigurationStore implements FalconService {
 
     /**
      * Remove an entity which is already stored in the config store
-     * 
-     * @param type
-     *            - Entity type being removed
-     * @param name
-     *            - Name of the entity object being removed
+     *
+     * @param type - Entity type being removed
+     * @param name - Name of the entity object being removed
      * @return - True is remove is successful, false if request entity doesn't
      *         exist
      * @throws FalconException
@@ -279,19 +269,17 @@ public class ConfigurationStore implements FalconService {
                 listener.onRemove(entity);
             } catch (Throwable e) {
                 LOG.warn(
-                        "Encountered exception while notifying " + listener + "(" + entity.getEntityType() + ") " + entity.getName(),
+                        "Encountered exception while notifying " + listener + "(" + entity.getEntityType() + ") "
+                                + entity.getName(),
                         e);
             }
         }
     }
 
     /**
-     * 
-     * @param type
-     *            - Entity type that needs to be searched
-     * @param keywords
-     *            - List of keywords to search for. only entities that have all
-     *            the keywords being searched would be returned
+     * @param type     - Entity type that needs to be searched
+     * @param keywords - List of keywords to search for. only entities that have all
+     *                 the keywords being searched would be returned
      * @return - Array of entity types
      */
     public Entity[] search(EntityType type, String... keywords) {
@@ -299,20 +287,17 @@ public class ConfigurationStore implements FalconService {
     }
 
     /**
-     * 
-     * @param type
-     *            - Entity type that is to be stored into persistent storage
-     * @param entity
-     *            - entity to persist. JAXB Annotated entity will be marshalled
-     *            to the persistent store. The convention used for storing the
-     *            object:: PROP(config.store.uri)/{entitytype}/{entityname}.xml
-     * @throws java.io.IOException
-     *             If any error in accessing the storage
+     * @param type   - Entity type that is to be stored into persistent storage
+     * @param entity - entity to persist. JAXB Annotated entity will be marshalled
+     *               to the persistent store. The convention used for storing the
+     *               object:: PROP(config.store.uri)/{entitytype}/{entityname}.xml
+     * @throws java.io.IOException If any error in accessing the storage
      * @throws FalconException
      */
     private void persist(EntityType type, Entity entity) throws IOException, FalconException {
         OutputStream out = fs
-                .create(new Path(storePath, type + Path.SEPARATOR + URLEncoder.encode(entity.getName(), UTF_8) + ".xml"));
+                .create(new Path(storePath,
+                        type + Path.SEPARATOR + URLEncoder.encode(entity.getName(), UTF_8) + ".xml"));
         try {
             type.getMarshaller().marshal(entity, out);
             LOG.info("Persisted configuration " + type + "/" + entity.getName());
@@ -326,37 +311,31 @@ public class ConfigurationStore implements FalconService {
 
     /**
      * Archive removed configuration in the persistent store
-     * 
-     * @param type
-     *            - Entity type to archive
-     * @param name
-     *            - name
-     * @throws IOException
-     *             If any error in accessing the storage
+     *
+     * @param type - Entity type to archive
+     * @param name - name
+     * @throws IOException If any error in accessing the storage
      */
     private void archive(EntityType type, String name) throws IOException {
         Path archivePath = new Path(storePath, "archive" + Path.SEPARATOR + type);
         fs.mkdirs(archivePath);
-        fs.rename(new Path(storePath, type + Path.SEPARATOR + URLEncoder.encode(name, UTF_8) + ".xml"), new Path(archivePath,
-                URLEncoder.encode(name, UTF_8) + "." + System.currentTimeMillis()));
+        fs.rename(new Path(storePath, type + Path.SEPARATOR + URLEncoder.encode(name, UTF_8) + ".xml"),
+                new Path(archivePath,
+                        URLEncoder.encode(name, UTF_8) + "." + System.currentTimeMillis()));
         LOG.info("Archived configuration " + type + "/" + name);
     }
 
     /**
-     * 
-     * @param type
-     *            - Entity type to restore from persistent store
-     * @param name
-     *            - Name of the entity to restore.
-     * @param <T>
-     *            - Actual entity object type
+     * @param type - Entity type to restore from persistent store
+     * @param name - Name of the entity to restore.
+     * @param <T>  - Actual entity object type
      * @return - De-serialized entity object restored from persistent store
-     * @throws IOException
-     *             If any error in accessing the storage
+     * @throws IOException     If any error in accessing the storage
      * @throws FalconException
      */
     @SuppressWarnings("unchecked")
-    private synchronized <T extends Entity> T restore(EntityType type, String name) throws IOException, FalconException {
+    private synchronized <T extends Entity> T restore(EntityType type, String name)
+            throws IOException, FalconException {
 
         InputStream in = fs.open(new Path(storePath, type + Path.SEPARATOR + URLEncoder.encode(name, UTF_8) + ".xml"));
         try {
@@ -379,5 +358,6 @@ public class ConfigurationStore implements FalconService {
     }
 
     @Override
-    public void destroy() { }
+    public void destroy() {
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java b/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
index b9b504d..a231242 100644
--- a/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
+++ b/common/src/main/java/org/apache/falcon/entity/store/StoreAccessException.java
@@ -22,14 +22,14 @@ import org.apache.falcon.FalconException;
 
 public class StoreAccessException extends FalconException {
 
-  /**
-   * @param e Exception
-   */
-  public StoreAccessException(String message, Exception e) {
-    super(message, e);
-  }
+    /**
+     * @param e Exception
+     */
+    public StoreAccessException(String message, Exception e) {
+        super(message, e);
+    }
 
-  public StoreAccessException(Exception e) {
-    super(e);
-  }
+    public StoreAccessException(Exception e) {
+        super(e);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
index 75d6d8b..f2b66e5 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/EntityGraph.java
@@ -18,12 +18,6 @@
 
 package org.apache.falcon.entity.v0;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.feed.Feed;
@@ -34,6 +28,12 @@ import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.service.ConfigurationChangeListener;
 import org.apache.log4j.Logger;
 
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 public class EntityGraph implements ConfigurationChangeListener {
 
     private static Logger LOG = Logger.getLogger(EntityGraph.class);
@@ -76,8 +76,9 @@ public class EntityGraph implements ConfigurationChangeListener {
                 nodeEdges = getEdgesFor((Feed) entity);
                 break;
         }
-        if (nodeEdges == null)
+        if (nodeEdges == null) {
             return;
+        }
         LOG.trace("Adding edges for " + entity.getName() + ": " + nodeEdges);
 
         for (Map.Entry<Node, Set<Node>> entry : nodeEdges.entrySet()) {
@@ -101,8 +102,9 @@ public class EntityGraph implements ConfigurationChangeListener {
                 nodeEdges = getEdgesFor((Feed) entity);
                 break;
         }
-        if (nodeEdges == null)
+        if (nodeEdges == null) {
             return;
+        }
 
         for (Map.Entry<Node, Set<Node>> entry : nodeEdges.entrySet()) {
             if (graph.containsKey(entry.getKey())) {
@@ -154,7 +156,7 @@ public class EntityGraph implements ConfigurationChangeListener {
             nodeEdges.put(clusterNode, new HashSet<Node>());
             nodeEdges.get(clusterNode).add(processNode);
         }
-        
+
         return nodeEdges;
     }
 
@@ -188,19 +190,23 @@ public class EntityGraph implements ConfigurationChangeListener {
 
         @Override
         public boolean equals(Object o) {
-            if (this == o)
+            if (this == o) {
                 return true;
-            if (o == null || getClass() != o.getClass())
+            }
+            if (o == null || getClass() != o.getClass()) {
                 return false;
+            }
 
             Node node = (Node) o;
 
             boolean nameEqual = name != null ? !name.equals(node.name) : node.name != null;
 
-            if (nameEqual)
+            if (nameEqual) {
                 return false;
-            if (type != node.type)
+            }
+            if (type != node.type) {
                 return false;
+            }
 
             return true;
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java b/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
index 1a9febc..b523c8b 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/EntityIntegrityChecker.java
@@ -18,21 +18,22 @@
 
 package org.apache.falcon.entity.v0;
 
+import org.apache.falcon.FalconException;
+import org.apache.falcon.Pair;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.falcon.FalconException;
-import org.apache.falcon.Pair;
-
 public class EntityIntegrityChecker {
 
     public static Pair<String, EntityType>[] referencedBy(Entity entity) throws FalconException {
         Set<Entity> deps = EntityGraph.get().getDependents(entity);
-        if(deps == null)
+        if (deps == null) {
             return null;
-        
+        }
+
         switch (entity.getEntityType()) {
             case CLUSTER:
                 return filter(deps, EntityType.FEED, EntityType.PROCESS);
@@ -46,12 +47,14 @@ public class EntityIntegrityChecker {
     }
 
     @SuppressWarnings("unchecked")
-    private static Pair<String, EntityType>[] filter(Set<Entity> deps, EntityType ... types) {
-        List<Pair<String, EntityType>> filteredSet = new ArrayList<Pair<String,EntityType>>();
+    private static Pair<String, EntityType>[] filter(Set<Entity> deps, EntityType... types) {
+        List<Pair<String, EntityType>> filteredSet = new ArrayList<Pair<String, EntityType>>();
         List<EntityType> validTypes = Arrays.asList(types);
-        for(Entity dep:deps)
-            if(validTypes.contains(dep.getEntityType()))
+        for (Entity dep : deps) {
+            if (validTypes.contains(dep.getEntityType())) {
                 filteredSet.add(Pair.of(dep.getName(), dep.getEntityType()));
+            }
+        }
         return filteredSet.toArray(new Pair[0]);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/v0/UnschedulableEntityException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/v0/UnschedulableEntityException.java b/common/src/main/java/org/apache/falcon/entity/v0/UnschedulableEntityException.java
index bbf2749..cad196b 100644
--- a/common/src/main/java/org/apache/falcon/entity/v0/UnschedulableEntityException.java
+++ b/common/src/main/java/org/apache/falcon/entity/v0/UnschedulableEntityException.java
@@ -24,7 +24,6 @@ import org.apache.falcon.FalconException;
  * This exception is thrown when Unschedulable entity
  * like CLUSTER is tried with actions like Schedule, Suspend,
  * Resume.
- *
  */
 public class UnschedulableEntityException extends FalconException {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java b/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
index 02e76a7..f57ef95 100644
--- a/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
+++ b/common/src/main/java/org/apache/falcon/expression/ExpressionHelper.java
@@ -49,12 +49,13 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
         return instance;
     }
 
-    private ExpressionHelper() {}
+    private ExpressionHelper() {
+    }
 
     public <T> T evaluate(String expression, Class<T> clazz) throws FalconException {
         return evaluateFullExpression("${" + expression + "}", clazz);
     }
-    
+
     @SuppressWarnings("unchecked")
     public <T> T evaluateFullExpression(String expression, Class<T> clazz) throws FalconException {
         try {
@@ -67,8 +68,9 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
     @Override
     public Method resolveFunction(String prefix, String name) {
         for (Method method : ExpressionHelper.class.getDeclaredMethods()) {
-            if (method.getName().equals(name))
+            if (method.getName().equals(name)) {
                 return method;
+            }
         }
         throw new UnsupportedOperationException("Not found " + prefix + ":" + name);
     }
@@ -146,15 +148,15 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
     }
 
     public static Date latest(int n) {
-    	//by pass Falcon validations
-        return  referenceDate.get();
+        //by pass Falcon validations
+        return referenceDate.get();
     }
-    
+
     public static Date future(int n, int limit) {
-    	//by pass Falcon validations
-        return  referenceDate.get();
+        //by pass Falcon validations
+        return referenceDate.get();
     }
-    
+
     public static long hours(int val) {
         return TimeUnit.HOURS.toMillis(val);
     }
@@ -180,19 +182,19 @@ public final class ExpressionHelper implements FunctionMapper, VariableResolver
     }
 
     public static String substitute(String originalValue, Properties properties) {
-      Matcher envVarMatcher = sysPropertyPattern.matcher(originalValue);
-      while (envVarMatcher.find()) {
-        String envVar = originalValue.substring(envVarMatcher.start() + 2,
-            envVarMatcher.end() - 1);
-        String envVal = properties.getProperty(envVar, System.getenv(envVar));
-
-        envVar = "\\$\\{" + envVar + "\\}";
-        if (envVal != null) {
-          originalValue = originalValue.replaceAll(envVar, Matcher.quoteReplacement(envVal));
-          envVarMatcher = sysPropertyPattern.matcher(originalValue);
+        Matcher envVarMatcher = sysPropertyPattern.matcher(originalValue);
+        while (envVarMatcher.find()) {
+            String envVar = originalValue.substring(envVarMatcher.start() + 2,
+                    envVarMatcher.end() - 1);
+            String envVal = properties.getProperty(envVar, System.getenv(envVar));
+
+            envVar = "\\$\\{" + envVar + "\\}";
+            if (envVal != null) {
+                originalValue = originalValue.replaceAll(envVar, Matcher.quoteReplacement(envVal));
+                envVarMatcher = sysPropertyPattern.matcher(originalValue);
+            }
         }
-      }
-      return originalValue;
+        return originalValue;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/group/FeedGroup.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/group/FeedGroup.java b/common/src/main/java/org/apache/falcon/group/FeedGroup.java
index 4dade88..e40ab13 100644
--- a/common/src/main/java/org/apache/falcon/group/FeedGroup.java
+++ b/common/src/main/java/org/apache/falcon/group/FeedGroup.java
@@ -17,6 +17,11 @@
  */
 package org.apache.falcon.group;
 
+import org.apache.falcon.entity.FeedHelper;
+import org.apache.falcon.entity.common.FeedDataPath;
+import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.entity.v0.feed.LocationType;
+
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -24,80 +29,75 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Matcher;
 
-import org.apache.falcon.entity.FeedHelper;
-import org.apache.falcon.entity.common.FeedDataPath;
-import org.apache.falcon.entity.v0.Frequency;
-import org.apache.falcon.entity.v0.feed.LocationType;
-
 /**
  * Group, which represents a logical group of feeds which can belong to this
  * group.
  */
 public class FeedGroup {
-	
-	public FeedGroup(String group, Frequency frequency, String path) {
-		this.name = group;
-		this.frequency = frequency;
-		this.datePattern = getDatePattern(path);
-		this.feeds = Collections
-				.newSetFromMap(new ConcurrentHashMap<String, Boolean>());
-	}
 
-	public static String getDatePattern(String path) {
-		Matcher matcher = FeedDataPath.PATTERN.matcher(path);
-		List<String> fields = new ArrayList<String>();
-		while (matcher.find()) {
-			String var = path.substring(matcher.start(), matcher.end());
-			fields.add(var);
-		}
-		Collections.sort(fields);
-		return fields.toString();
-	}
+    public FeedGroup(String group, Frequency frequency, String path) {
+        this.name = group;
+        this.frequency = frequency;
+        this.datePattern = getDatePattern(path);
+        this.feeds = Collections
+                .newSetFromMap(new ConcurrentHashMap<String, Boolean>());
+    }
+
+    public static String getDatePattern(String path) {
+        Matcher matcher = FeedDataPath.PATTERN.matcher(path);
+        List<String> fields = new ArrayList<String>();
+        while (matcher.find()) {
+            String var = path.substring(matcher.start(), matcher.end());
+            fields.add(var);
+        }
+        Collections.sort(fields);
+        return fields.toString();
+    }
 
-	private String name;
-	private Frequency frequency;
-	private String datePattern;
-	private Set<String> feeds;
+    private String name;
+    private Frequency frequency;
+    private String datePattern;
+    private Set<String> feeds;
 
-	public Set<String> getFeeds() {
-		return feeds;
-	}
+    public Set<String> getFeeds() {
+        return feeds;
+    }
 
-	@Override
-	public boolean equals(Object obj) {
-		if (!(obj instanceof FeedGroup) || obj == null) {
-			return false;
-		}
-		FeedGroup group = (FeedGroup) obj;
-		return (this.name.equals(group.getName())
-				&& this.frequency.equals(group.frequency)
-				&& this.datePattern
-					.equals(group.datePattern));
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof FeedGroup) || obj == null) {
+            return false;
+        }
+        FeedGroup group = (FeedGroup) obj;
+        return (this.name.equals(group.getName())
+                && this.frequency.equals(group.frequency)
+                && this.datePattern
+                .equals(group.datePattern));
 
-	}
+    }
 
-	@Override
-	public int hashCode() {
-		return 127 * name.hashCode() + 31 * frequency.hashCode() + datePattern.hashCode();
-	}
+    @Override
+    public int hashCode() {
+        return 127 * name.hashCode() + 31 * frequency.hashCode() + datePattern.hashCode();
+    }
 
-	public String getName() {
-		return name;
-	}
+    public String getName() {
+        return name;
+    }
 
-	public Frequency getFrequency() {
-		return frequency;
-	}
+    public Frequency getFrequency() {
+        return frequency;
+    }
 
-	public String getDatePattern() {
-		return datePattern;
-	}
+    public String getDatePattern() {
+        return datePattern;
+    }
 
-	public boolean canContainFeed(org.apache.falcon.entity.v0.feed.Feed feed) {
-		if (this.frequency.equals(feed.getFrequency())
-				&& this.datePattern.equals(getDatePattern(FeedHelper.getLocation(feed, LocationType.DATA).getPath()))) {
-			return true;
-		}
-		return false;
-	}
+    public boolean canContainFeed(org.apache.falcon.entity.v0.feed.Feed feed) {
+        if (this.frequency.equals(feed.getFrequency())
+                && this.datePattern.equals(getDatePattern(FeedHelper.getLocation(feed, LocationType.DATA).getPath()))) {
+            return true;
+        }
+        return false;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java b/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
index e154a14..532392f 100644
--- a/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
+++ b/common/src/main/java/org/apache/falcon/group/FeedGroupMap.java
@@ -17,12 +17,6 @@
  */
 package org.apache.falcon.group;
 
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.FeedHelper;
@@ -33,88 +27,94 @@ import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.feed.LocationType;
 import org.apache.falcon.service.ConfigurationChangeListener;
 
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 /**
  * Has 2 way mappings from feed to group and group to feed
  */
 public class FeedGroupMap implements ConfigurationChangeListener {
 
-	private static final FeedGroupMap instance = new FeedGroupMap();
-	private Map<String, FeedGroup> groupsMapping = new ConcurrentHashMap<String, FeedGroup>();
-
-	private FeedGroupMap() {
-		// singleton
-	}
-
-	public static FeedGroupMap get() {
-		return instance;
-	}
-
-	public Map<String, FeedGroup> getGroupsMapping() {
-		return Collections.unmodifiableMap(groupsMapping);
-	}
-
-	@Override
-	public void onAdd(Entity entity) throws FalconException {
-
-		if (entity.getEntityType().equals(EntityType.FEED)) {
-			Feed feed = (Feed) entity;
-			if (feed.getGroups() == null || feed.getGroups().equals("")) {
-				return;
-			}
-			Set<FeedGroup> groupSet = getGroups(feed);
-			addGroups(feed.getName(), groupSet);
-		}
-
-	}
-
-	@Override
-	public void onRemove(Entity entity) throws FalconException {
-		if (entity.getEntityType().equals(EntityType.FEED)) {
-			Feed feed = (Feed) entity;
-			if (StringUtils.isEmpty(feed.getGroups())) {
-				return;
-			}
-			String[] groups = feed.getGroups().split(",");
-			for (String group : groups) {
-				groupsMapping.get(group).getFeeds().remove(entity.getName());
-				if (groupsMapping.get(group).getFeeds().size() == 0) {
-					groupsMapping.remove(group);
-				}
-			}
-
-		}
-
-	}
-
-	@Override
-	public void onChange(Entity oldEntity, Entity newEntity)
-			throws FalconException {
-		onRemove(oldEntity);
-		onAdd(newEntity);
-	}
-
-	private void addGroups(String feed, Set<FeedGroup> groups) {
-		for (FeedGroup group : groups) {
-			if (groupsMapping.containsKey(group.getName())) {
-				groupsMapping.get(group.getName()).getFeeds().add(feed);
-			} else {
-				group.getFeeds().add(feed);
-				groupsMapping.put(group.getName(), group);
-			}
-		}
-	}
-
-	public Set<FeedGroup> getGroups(String groups, Frequency frequency, String path) {
-		Set<FeedGroup> groupSet = new HashSet<FeedGroup>();
-		String[] groupArray = groups.split(",");
-		for (String group : groupArray) {
-			groupSet.add(new FeedGroup(group, frequency, path));
-		}
-		return groupSet;
-	}
-
-	public Set<FeedGroup> getGroups(org.apache.falcon.entity.v0.feed.Feed feed) {
-		return getGroups(feed.getGroups(), feed.getFrequency(),
-				FeedHelper.getLocation(feed, LocationType.DATA).getPath());
-	}
+    private static final FeedGroupMap instance = new FeedGroupMap();
+    private Map<String, FeedGroup> groupsMapping = new ConcurrentHashMap<String, FeedGroup>();
+
+    private FeedGroupMap() {
+        // singleton
+    }
+
+    public static FeedGroupMap get() {
+        return instance;
+    }
+
+    public Map<String, FeedGroup> getGroupsMapping() {
+        return Collections.unmodifiableMap(groupsMapping);
+    }
+
+    @Override
+    public void onAdd(Entity entity) throws FalconException {
+
+        if (entity.getEntityType().equals(EntityType.FEED)) {
+            Feed feed = (Feed) entity;
+            if (feed.getGroups() == null || feed.getGroups().equals("")) {
+                return;
+            }
+            Set<FeedGroup> groupSet = getGroups(feed);
+            addGroups(feed.getName(), groupSet);
+        }
+
+    }
+
+    @Override
+    public void onRemove(Entity entity) throws FalconException {
+        if (entity.getEntityType().equals(EntityType.FEED)) {
+            Feed feed = (Feed) entity;
+            if (StringUtils.isEmpty(feed.getGroups())) {
+                return;
+            }
+            String[] groups = feed.getGroups().split(",");
+            for (String group : groups) {
+                groupsMapping.get(group).getFeeds().remove(entity.getName());
+                if (groupsMapping.get(group).getFeeds().size() == 0) {
+                    groupsMapping.remove(group);
+                }
+            }
+
+        }
+
+    }
+
+    @Override
+    public void onChange(Entity oldEntity, Entity newEntity)
+            throws FalconException {
+        onRemove(oldEntity);
+        onAdd(newEntity);
+    }
+
+    private void addGroups(String feed, Set<FeedGroup> groups) {
+        for (FeedGroup group : groups) {
+            if (groupsMapping.containsKey(group.getName())) {
+                groupsMapping.get(group.getName()).getFeeds().add(feed);
+            } else {
+                group.getFeeds().add(feed);
+                groupsMapping.put(group.getName(), group);
+            }
+        }
+    }
+
+    public Set<FeedGroup> getGroups(String groups, Frequency frequency, String path) {
+        Set<FeedGroup> groupSet = new HashSet<FeedGroup>();
+        String[] groupArray = groups.split(",");
+        for (String group : groupArray) {
+            groupSet.add(new FeedGroup(group, frequency, path));
+        }
+        return groupSet;
+    }
+
+    public Set<FeedGroup> getGroups(org.apache.falcon.entity.v0.feed.Feed feed) {
+        return getGroups(feed.getGroups(), feed.getFrequency(),
+                FeedHelper.getLocation(feed, LocationType.DATA).getPath());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/security/CurrentUser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/CurrentUser.java b/common/src/main/java/org/apache/falcon/security/CurrentUser.java
index 80c4de9..9a3086c 100644
--- a/common/src/main/java/org/apache/falcon/security/CurrentUser.java
+++ b/common/src/main/java/org/apache/falcon/security/CurrentUser.java
@@ -40,7 +40,9 @@ public final class CurrentUser {
             throw new IllegalStateException
                     ("Bad user name sent for authentication");
         }
-        if (user.equals(getUserInternal())) return;
+        if (user.equals(getUserInternal())) {
+            return;
+        }
 
         Subject subject = new Subject();
         subject.getPrincipals().add(new FalconPrincipal(user));
@@ -66,7 +68,7 @@ public final class CurrentUser {
         if (subject == null) {
             return null;
         } else {
-            for(FalconPrincipal principal: subject.
+            for (FalconPrincipal principal : subject.
                     getPrincipals(FalconPrincipal.class)) {
                 return principal.getName();
             }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
index d6fa26d..a27a342 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
@@ -40,7 +40,7 @@ public class FalconLoginModule implements LoginModule {
     }
 
     private <T extends Principal> T getCanonicalUser(Class<T> cls) {
-        for(T user: subject.getPrincipals(cls)) {
+        for (T user : subject.getPrincipals(cls)) {
             return user;
         }
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java b/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
index b88f27a..20ec8df 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconSecurityConfiguration.java
@@ -27,7 +27,7 @@ public class FalconSecurityConfiguration extends Configuration {
     private static final AppConfigurationEntry OS_SPECIFIC_LOGIN =
             new AppConfigurationEntry(SecurityConstants.OS_LOGIN_MODULE_NAME,
                     AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                    new HashMap<String,String>());
+                    new HashMap<String, String>());
 
     private static final AppConfigurationEntry[] SIMPLE_CONF =
             new AppConfigurationEntry[]{OS_SPECIFIC_LOGIN};

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/LogCleanupService.java b/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
index c1fe265..17d5926 100644
--- a/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
+++ b/common/src/main/java/org/apache/falcon/service/LogCleanupService.java
@@ -17,13 +17,6 @@
  */
 package org.apache.falcon.service;
 
-import java.util.Date;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import javax.servlet.jsp.el.ELException;
-import javax.servlet.jsp.el.ExpressionEvaluator;
-
 import org.apache.commons.el.ExpressionEvaluatorImpl;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.aspect.GenericAlert;
@@ -34,58 +27,64 @@ import org.apache.falcon.expression.ExpressionHelper;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.log4j.Logger;
 
+import javax.servlet.jsp.el.ELException;
+import javax.servlet.jsp.el.ExpressionEvaluator;
+import java.util.Date;
+import java.util.Timer;
+import java.util.TimerTask;
+
 public class LogCleanupService implements FalconService {
 
-	private static final Logger LOG = Logger.getLogger(LogCleanupService.class);
-	private final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
-	private final ExpressionHelper resolver = ExpressionHelper.get();
+    private static final Logger LOG = Logger.getLogger(LogCleanupService.class);
+    private final ExpressionEvaluator EVALUATOR = new ExpressionEvaluatorImpl();
+    private final ExpressionHelper resolver = ExpressionHelper.get();
 
-	@Override
-	public String getName() {
-		return "Falcon Log cleanup service";
-	}
+    @Override
+    public String getName() {
+        return "Falcon Log cleanup service";
+    }
 
-	@Override
-	public void init() throws FalconException {
-		Timer timer = new Timer();
-		timer.schedule(new CleanupThread(), 0, getDelay());
-		LOG.info("Falcon log cleanup service initialized");
+    @Override
+    public void init() throws FalconException {
+        Timer timer = new Timer();
+        timer.schedule(new CleanupThread(), 0, getDelay());
+        LOG.info("Falcon log cleanup service initialized");
 
-	}
+    }
 
-	private class CleanupThread extends TimerTask {
+    private class CleanupThread extends TimerTask {
 
-		private AbstractCleanupHandler processCleanupHandler = new ProcessCleanupHandler();
-		private AbstractCleanupHandler feedCleanupHandler = new FeedCleanupHandler();
+        private AbstractCleanupHandler processCleanupHandler = new ProcessCleanupHandler();
+        private AbstractCleanupHandler feedCleanupHandler = new FeedCleanupHandler();
 
-		@Override
-		public void run() {
-			try {
-				LOG.info("Cleaning up logs at: " + new Date());
-				processCleanupHandler.cleanup();
-				feedCleanupHandler.cleanup();
-			} catch (Throwable t) {
-				LOG.error("Error in cleanup task: ", t);
-				GenericAlert.alertLogCleanupServiceFailed(
-						"Exception in log cleanup service", t);
-			}
-		}
-	}
+        @Override
+        public void run() {
+            try {
+                LOG.info("Cleaning up logs at: " + new Date());
+                processCleanupHandler.cleanup();
+                feedCleanupHandler.cleanup();
+            } catch (Throwable t) {
+                LOG.error("Error in cleanup task: ", t);
+                GenericAlert.alertLogCleanupServiceFailed(
+                        "Exception in log cleanup service", t);
+            }
+        }
+    }
 
-	@Override
-	public void destroy() throws FalconException {
-		LOG.info("Falcon log cleanup service destroyed");
-	}
+    @Override
+    public void destroy() throws FalconException {
+        LOG.info("Falcon log cleanup service destroyed");
+    }
 
-	private long getDelay() throws FalconException {
-		String delay = StartupProperties.get().getProperty(
-				"falcon.cleanup.service.frequency", "days(1)");
-		try {
-			return (Long) EVALUATOR.evaluate("${" + delay + "}", Long.class,
-					resolver, resolver);
-		} catch (ELException e) {
-			throw new FalconException("Exception in EL evaluation", e);
-		}
-	}
+    private long getDelay() throws FalconException {
+        String delay = StartupProperties.get().getProperty(
+                "falcon.cleanup.service.frequency", "days(1)");
+        try {
+            return (Long) EVALUATOR.evaluate("${" + delay + "}", Long.class,
+                    resolver, resolver);
+        } catch (ELException e) {
+            throw new FalconException("Exception in EL evaluation", e);
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java b/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
index f44ad12..466cb81 100644
--- a/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
+++ b/common/src/main/java/org/apache/falcon/service/ServiceInitializer.java
@@ -33,13 +33,15 @@ public class ServiceInitializer {
                 getProperty("application.services", "org.apache.falcon.entity.store.ConfigurationStore");
         for (String serviceClassName : serviceClassNames.split(",")) {
             serviceClassName = serviceClassName.trim();
-            if (serviceClassName.isEmpty()) continue;
+            if (serviceClassName.isEmpty()) {
+                continue;
+            }
             FalconService service = ReflectionUtils.getInstanceByClassName(serviceClassName);
             services.register(service);
             LOG.info("Initializing service : " + serviceClassName);
             try {
                 service.init();
-            } catch(Throwable t) {
+            } catch (Throwable t) {
                 LOG.fatal("Failed to initialize service " + serviceClassName, t);
                 throw new FalconException(t);
             }
@@ -52,7 +54,7 @@ public class ServiceInitializer {
             LOG.info("Destroying service : " + service.getClass().getName());
             try {
                 service.destroy();
-            } catch(Throwable t) {
+            } catch (Throwable t) {
                 LOG.fatal("Failed to destroy service " + service.getClass().getName(), t);
                 throw new FalconException(t);
             }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/service/Services.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/service/Services.java b/common/src/main/java/org/apache/falcon/service/Services.java
index dc41d5d..955e906 100644
--- a/common/src/main/java/org/apache/falcon/service/Services.java
+++ b/common/src/main/java/org/apache/falcon/service/Services.java
@@ -18,22 +18,23 @@
 
 package org.apache.falcon.service;
 
+import org.apache.falcon.FalconException;
+import org.apache.falcon.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
-import org.apache.falcon.FalconException;
-import org.apache.falcon.util.ReflectionUtils;
-import org.apache.log4j.Logger;
-
 
 public final class Services implements Iterable<FalconService> {
     private static final Logger LOG = Logger.getLogger(Services.class);
 
     private static Services instance = new Services();
 
-    private Services() { }
+    private Services() {
+    }
 
     public static Services get() {
         return instance;
@@ -80,8 +81,8 @@ public final class Services implements Iterable<FalconService> {
         register(service);
         return service;
     }
-    
-    public void reset(){
-    	services.clear();
+
+    public void reset() {
+        services.clear();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/update/UpdateHelper.java b/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
index 7f36e12..4e199da 100644
--- a/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
+++ b/common/src/main/java/org/apache/falcon/update/UpdateHelper.java
@@ -18,9 +18,6 @@
 
 package org.apache.falcon.update;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.FeedHelper;
@@ -36,31 +33,40 @@ import org.apache.falcon.entity.v0.process.Inputs;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.log4j.Logger;
 
+import java.util.ArrayList;
+import java.util.List;
+
 public final class UpdateHelper {
     private static final Logger LOG = Logger.getLogger(UpdateHelper.class);
-    private static final String[] FEED_FIELDS = new String[] { "partitions", "groups", "lateArrival.cutOff", "schema.location", "schema.provider",
-        "ACL.group", "ACL.owner", "ACL.permission"};
-    private static final String[] PROCESS_FIELDS = new String[] { "retry.policy", "retry.delay", "retry.attempts", 
-        "lateProcess.policy", "lateProcess.delay", "lateProcess.lateInputs[\\d+].input", "lateProcess.lateInputs[\\d+].workflowPath"};
-    
+    private static final String[] FEED_FIELDS = new String[]{"partitions", "groups", "lateArrival.cutOff",
+                                                             "schema.location", "schema.provider",
+                                                             "ACL.group", "ACL.owner", "ACL.permission"};
+    private static final String[] PROCESS_FIELDS = new String[]{"retry.policy", "retry.delay", "retry.attempts",
+                                                                "lateProcess.policy", "lateProcess.delay",
+                                                                "lateProcess.lateInputs[\\d+].input",
+                                                                "lateProcess.lateInputs[\\d+].workflowPath"};
+
     public static boolean shouldUpdate(Entity oldEntity, Entity newEntity, String cluster) throws FalconException {
         Entity oldView = EntityUtil.getClusterView(oldEntity, cluster);
         Entity newView = EntityUtil.getClusterView(newEntity, cluster);
-        switch(oldEntity.getEntityType()) {
+        switch (oldEntity.getEntityType()) {
             case FEED:
-                if(EntityUtil.equals(oldView, newView, FEED_FIELDS))
+                if (EntityUtil.equals(oldView, newView, FEED_FIELDS)) {
                     return false;
+                }
                 return true;
-                
+
             case PROCESS:
-                if(EntityUtil.equals(oldView, newView, PROCESS_FIELDS))
+                if (EntityUtil.equals(oldView, newView, PROCESS_FIELDS)) {
                     return false;
+                }
                 return true;
         }
         throw new IllegalArgumentException("Unhandled entity type " + oldEntity.getEntityType());
     }
 
-    public static boolean shouldUpdate(Entity oldEntity, Entity newEntity, Entity affectedEntity) throws FalconException {
+    public static boolean shouldUpdate(Entity oldEntity, Entity newEntity, Entity affectedEntity)
+            throws FalconException {
         if (oldEntity.getEntityType() == EntityType.FEED && affectedEntity.getEntityType() == EntityType.PROCESS) {
             return shouldUpdate((Feed) oldEntity, (Feed) newEntity, (Process) affectedEntity);
         } else {
@@ -71,31 +77,33 @@ public final class UpdateHelper {
     }
 
     public static boolean shouldUpdate(Feed oldFeed, Feed newFeed, Process affectedProcess) {
-		if (!FeedHelper
-				.getLocation(oldFeed.getLocations(), LocationType.DATA)
-				.getPath()
-				.equals(FeedHelper.getLocation(newFeed.getLocations(),
-						LocationType.DATA).getPath())
-				|| !FeedHelper
-						.getLocation(oldFeed.getLocations(), LocationType.META)
-						.getPath()
-						.equals(FeedHelper.getLocation(newFeed.getLocations(),
-								LocationType.META).getPath())
-				|| !FeedHelper
-						.getLocation(oldFeed.getLocations(), LocationType.STATS)
-						.getPath()
-						.equals(FeedHelper.getLocation(newFeed.getLocations(),
-								LocationType.STATS).getPath())
-				|| !FeedHelper
-						.getLocation(oldFeed.getLocations(), LocationType.TMP)
-						.getPath()
-						.equals(FeedHelper.getLocation(newFeed.getLocations(),
-								LocationType.TMP).getPath()))
+        if (!FeedHelper
+                .getLocation(oldFeed.getLocations(), LocationType.DATA)
+                .getPath()
+                .equals(FeedHelper.getLocation(newFeed.getLocations(),
+                        LocationType.DATA).getPath())
+                || !FeedHelper
+                .getLocation(oldFeed.getLocations(), LocationType.META)
+                .getPath()
+                .equals(FeedHelper.getLocation(newFeed.getLocations(),
+                        LocationType.META).getPath())
+                || !FeedHelper
+                .getLocation(oldFeed.getLocations(), LocationType.STATS)
+                .getPath()
+                .equals(FeedHelper.getLocation(newFeed.getLocations(),
+                        LocationType.STATS).getPath())
+                || !FeedHelper
+                .getLocation(oldFeed.getLocations(), LocationType.TMP)
+                .getPath()
+                .equals(FeedHelper.getLocation(newFeed.getLocations(),
+                        LocationType.TMP).getPath())) {
             return true;
+        }
         LOG.debug(oldFeed.toShortString() + ": Location identical. Ignoring...");
 
-        if (!oldFeed.getFrequency().equals(newFeed.getFrequency()))
+        if (!oldFeed.getFrequency().equals(newFeed.getFrequency())) {
             return true;
+        }
         LOG.debug(oldFeed.toShortString() + ": Frequency identical. Ignoring...");
 
         // it is not possible to have oldFeed partitions as non empty and
@@ -118,39 +126,42 @@ public final class UpdateHelper {
                 if (newFeed.getPartitions() != null && oldFeed.getPartitions() != null) {
                     List<String> newParts = getPartitions(newFeed.getPartitions());
                     List<String> oldParts = getPartitions(oldFeed.getPartitions());
-                    if (newParts.size() != oldParts.size())
+                    if (newParts.size() != oldParts.size()) {
                         return true;
-                    if (!newParts.containsAll(oldParts))
+                    }
+                    if (!newParts.containsAll(oldParts)) {
                         return true;
+                    }
                 }
                 LOG.debug(oldFeed.toShortString() + ": Partitions identical. Ignoring...");
             }
         }
 
         for (Cluster cluster : affectedProcess.getClusters().getClusters()) {
-			if (!FeedHelper
-					.getCluster(oldFeed, cluster.getName())
-					.getValidity()
-					.getStart()
-					.equals(FeedHelper.getCluster(newFeed, cluster.getName())
-							.getValidity().getStart())
-					|| !FeedHelper.getLocation(oldFeed, LocationType.DATA,
-							cluster.getName()).getPath().equals(
-							FeedHelper.getLocation(newFeed, LocationType.DATA,
-									cluster.getName()).getPath())
-					|| !FeedHelper.getLocation(oldFeed, LocationType.META,
-							cluster.getName()).getPath().equals(
-							FeedHelper.getLocation(newFeed, LocationType.META,
-									cluster.getName()).getPath())
-					|| !FeedHelper.getLocation(oldFeed, LocationType.STATS,
-							cluster.getName()).getPath().equals(
-							FeedHelper.getLocation(newFeed, LocationType.STATS,
-									cluster.getName()).getPath())
-					|| !FeedHelper.getLocation(oldFeed, LocationType.TMP,
-							cluster.getName()).getPath().equals(
-							FeedHelper.getLocation(newFeed, LocationType.TMP,
-									cluster.getName()).getPath()))
-				return true;
+            if (!FeedHelper
+                    .getCluster(oldFeed, cluster.getName())
+                    .getValidity()
+                    .getStart()
+                    .equals(FeedHelper.getCluster(newFeed, cluster.getName())
+                            .getValidity().getStart())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.DATA,
+                    cluster.getName()).getPath().equals(
+                    FeedHelper.getLocation(newFeed, LocationType.DATA,
+                            cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.META,
+                    cluster.getName()).getPath().equals(
+                    FeedHelper.getLocation(newFeed, LocationType.META,
+                            cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.STATS,
+                    cluster.getName()).getPath().equals(
+                    FeedHelper.getLocation(newFeed, LocationType.STATS,
+                            cluster.getName()).getPath())
+                    || !FeedHelper.getLocation(oldFeed, LocationType.TMP,
+                    cluster.getName()).getPath().equals(
+                    FeedHelper.getLocation(newFeed, LocationType.TMP,
+                            cluster.getName()).getPath())) {
+                return true;
+            }
             LOG.debug(oldFeed.toShortString() + ": Feed on cluster" + cluster.getName() + " identical. Ignoring...");
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java b/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
index f618876..f5dbc83 100644
--- a/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/ApplicationProperties.java
@@ -18,6 +18,10 @@
 
 package org.apache.falcon.util;
 
+import org.apache.falcon.FalconException;
+import org.apache.falcon.expression.ExpressionHelper;
+import org.apache.log4j.Logger;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
@@ -26,10 +30,6 @@ import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
-import org.apache.falcon.FalconException;
-import org.apache.falcon.expression.ExpressionHelper;
-import org.apache.log4j.Logger;
-
 public abstract class ApplicationProperties extends Properties {
 
     private static Logger LOG = Logger.getLogger(ApplicationProperties.class);
@@ -52,7 +52,7 @@ public abstract class ApplicationProperties extends Properties {
     public String getDomain() {
         return domain;
     }
-    
+
     protected void initialize() {
         String propFile = getPropertyFile();
         String userHome = System.getProperty("user.home");
@@ -68,7 +68,8 @@ public abstract class ApplicationProperties extends Properties {
             location = LocationType.FILE;
             propertyFile = new File(confDir, propFile).getAbsolutePath();
         } else {
-            LOG.info("config.location is not set, properties file not present in " + "user home dir, falling back to classpath for "
+            LOG.info("config.location is not set, properties file not present in "
+                    + "user home dir, falling back to classpath for "
                     + propFile);
             location = LocationType.CLASSPATH;
             propertyFile = propFile;
@@ -99,15 +100,16 @@ public abstract class ApplicationProperties extends Properties {
                     LOG.info("Loading properties from " + propertyFile);
                     Properties origProps = new Properties();
                     origProps.load(resource);
-                    if(domain == null) {
+                    if (domain == null) {
                         domain = origProps.getProperty("*.domain");
-                        if(domain == null)
+                        if (domain == null) {
                             throw new FalconException("Domain is not set!");
+                        }
                     }
                     LOG.info("Initializing properties with domain " + domain);
-                    
+
                     Set<String> keys = getKeys(origProps.keySet());
-                    for(String key:keys) {
+                    for (String key : keys) {
                         String value = origProps.getProperty(domain + "." + key, origProps.getProperty("*." + key));
                         value = ExpressionHelper.substitute(value);
                         LOG.debug(key + "=" + value);
@@ -124,7 +126,7 @@ public abstract class ApplicationProperties extends Properties {
 
     private Set<String> getKeys(Set<Object> keySet) {
         Set<String> keys = new HashSet<String>();
-        for(Object keyObj:keySet) {
+        for (Object keyObj : keySet) {
             String key = (String) keyObj;
             keys.add(key.substring(key.indexOf('.') + 1));
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/BuildProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/BuildProperties.java b/common/src/main/java/org/apache/falcon/util/BuildProperties.java
index e91a647..898daee 100644
--- a/common/src/main/java/org/apache/falcon/util/BuildProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/BuildProperties.java
@@ -27,26 +27,26 @@ public class BuildProperties extends ApplicationProperties {
     private static final String PROPERTY_FILE = "falcon-buildinfo.properties";
 
     private static final AtomicReference<BuildProperties> instance =
-        new AtomicReference<BuildProperties>();
+            new AtomicReference<BuildProperties>();
 
     private BuildProperties() throws FalconException {
-      super();
+        super();
     }
 
     @Override
     protected String getPropertyFile() {
-      return PROPERTY_FILE;
+        return PROPERTY_FILE;
     }
 
     public static Properties get() {
-      try {
-        if (instance.get() == null) {
-          instance.compareAndSet(null, new BuildProperties());
+        try {
+            if (instance.get() == null) {
+                instance.compareAndSet(null, new BuildProperties());
+            }
+            return instance.get();
+        } catch (FalconException e) {
+            throw new RuntimeException("Unable to read application " +
+                    "falcon build information properties", e);
         }
-        return instance.get();
-      } catch (FalconException e) {
-        throw new RuntimeException("Unable to read application " +
-            "falcon build information properties", e);
-      }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java b/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
index d13b817..4e2f7db 100644
--- a/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
@@ -27,26 +27,26 @@ public class DeploymentProperties extends ApplicationProperties {
     private static final String PROPERTY_FILE = "deploy.properties";
 
     private static final AtomicReference<DeploymentProperties> instance =
-        new AtomicReference<DeploymentProperties>();
+            new AtomicReference<DeploymentProperties>();
 
     private DeploymentProperties() throws FalconException {
-      super();
+        super();
     }
 
     @Override
     protected String getPropertyFile() {
-      return PROPERTY_FILE;
+        return PROPERTY_FILE;
     }
 
     public static Properties get() {
-      try {
-        if (instance.get() == null) {
-          instance.compareAndSet(null, new DeploymentProperties());
+        try {
+            if (instance.get() == null) {
+                instance.compareAndSet(null, new DeploymentProperties());
+            }
+            return instance.get();
+        } catch (FalconException e) {
+            throw new RuntimeException("Unable to read application " +
+                    "startup properties", e);
         }
-        return instance.get();
-      } catch (FalconException e) {
-        throw new RuntimeException("Unable to read application " +
-            "startup properties", e);
-      }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java b/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
index 86acb81..9aeb3ab 100644
--- a/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
+++ b/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
@@ -18,12 +18,12 @@
 
 package org.apache.falcon.util;
 
-import java.util.HashSet;
-import java.util.Set;
-
 import org.apache.falcon.entity.ColoClusterRelation;
 import org.apache.log4j.Logger;
 
+import java.util.HashSet;
+import java.util.Set;
+
 public class DeploymentUtil {
     private static final Logger LOG = Logger.getLogger(DeploymentUtil.class);
 
@@ -35,7 +35,7 @@ public class DeploymentUtil {
     protected final static String currentColo;
     protected final static boolean embeddedMode;
     protected static boolean prism = false;
-    
+
     static {
         DEFAULT_ALL_COLOS.add(DEFAULT_COLO);
         embeddedMode = DeploymentProperties.get().
@@ -44,33 +44,33 @@ public class DeploymentUtil {
             currentColo = DEFAULT_COLO;
         } else {
             currentColo = StartupProperties.get().
-                getProperty("current.colo", DEFAULT_COLO);
+                    getProperty("current.colo", DEFAULT_COLO);
         }
         LOG.info("Running in embedded mode? " + embeddedMode);
         LOG.info("Current colo: " + currentColo);
     }
-    
+
     public static void setPrismMode() {
-    	prism = true;
+        prism = true;
     }
-    
+
     public static boolean isPrism() {
-    	return !embeddedMode && prism;
+        return !embeddedMode && prism;
     }
-    
+
     public static String getCurrentColo() {
         return currentColo;
     }
-    
+
     public static Set<String> getCurrentClusters() {
         String colo = getCurrentColo();
         return ColoClusterRelation.get().getClusters(colo);
     }
-    
+
     public static boolean isEmbeddedMode() {
         return embeddedMode;
     }
-    
+
     public static String getDefaultColo() {
         return DEFAULT_COLO;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java b/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
index 8f255fd..d1bed8e 100644
--- a/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
+++ b/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
@@ -18,10 +18,10 @@
 
 package org.apache.falcon.util;
 
-import java.lang.reflect.Method;
-
 import org.apache.falcon.FalconException;
 
+import java.lang.reflect.Method;
+
 public final class ReflectionUtils {
 
     public static <T> T getInstance(String classKey) throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java b/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
index 6ecf33f..86a54f8 100644
--- a/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
@@ -18,72 +18,72 @@
 
 package org.apache.falcon.util;
 
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.falcon.FalconException;
 import org.apache.log4j.Logger;
 
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
 public class RuntimeProperties extends ApplicationProperties {
 
-  private static Logger LOG = Logger.getLogger(RuntimeProperties.class);
+    private static Logger LOG = Logger.getLogger(RuntimeProperties.class);
 
-  private static final String PROPERTY_FILE = "runtime.properties";
+    private static final String PROPERTY_FILE = "runtime.properties";
 
-  private static final AtomicReference<RuntimeProperties> instance =
-      new AtomicReference<RuntimeProperties>();
+    private static final AtomicReference<RuntimeProperties> instance =
+            new AtomicReference<RuntimeProperties>();
 
-  private RuntimeProperties() throws FalconException {
-    super();
-    Thread refreshThread = new Thread(new DynamicLoader(this));
-    refreshThread.start();
-  }
+    private RuntimeProperties() throws FalconException {
+        super();
+        Thread refreshThread = new Thread(new DynamicLoader(this));
+        refreshThread.start();
+    }
 
-  @Override
-  protected String getPropertyFile() {
-    return PROPERTY_FILE;
-  }
+    @Override
+    protected String getPropertyFile() {
+        return PROPERTY_FILE;
+    }
 
-  public static Properties get() {
-    try {
-      if (instance.get() == null) {
-        instance.compareAndSet(null, new RuntimeProperties());
-      }
-      return instance.get();
-    } catch (FalconException e) {
-      throw new RuntimeException("Unable to read application " +
-          "runtime properties", e);
+    public static Properties get() {
+        try {
+            if (instance.get() == null) {
+                instance.compareAndSet(null, new RuntimeProperties());
+            }
+            return instance.get();
+        } catch (FalconException e) {
+            throw new RuntimeException("Unable to read application " +
+                    "runtime properties", e);
+        }
     }
-  }
 
-  private class DynamicLoader implements Runnable {
+    private class DynamicLoader implements Runnable {
 
-    private static final long REFRESH_DELAY = 300000L;
-    private static final int MAX_ITER = 20;  //1hr
-    private final ApplicationProperties applicationProperties;
+        private static final long REFRESH_DELAY = 300000L;
+        private static final int MAX_ITER = 20;  //1hr
+        private final ApplicationProperties applicationProperties;
 
-    private DynamicLoader(ApplicationProperties applicationProperties) {
-      this.applicationProperties = applicationProperties;
-    }
+        private DynamicLoader(ApplicationProperties applicationProperties) {
+            this.applicationProperties = applicationProperties;
+        }
 
-    @Override
-    public void run() {
-      long backOffDelay = REFRESH_DELAY;
-      while (true) {
-        try {
-          try {
-            applicationProperties.loadProperties();
-            backOffDelay = REFRESH_DELAY;
-          } catch (FalconException e) {
-            LOG.warn("Error refreshing runtime properties", e);
-            backOffDelay += REFRESH_DELAY;
-          }
-          Thread.sleep(Math.min(MAX_ITER * REFRESH_DELAY, backOffDelay));
-        } catch (InterruptedException e) {
-          LOG.info("Application is stopping. Aborting...");
-          break;
+        @Override
+        public void run() {
+            long backOffDelay = REFRESH_DELAY;
+            while (true) {
+                try {
+                    try {
+                        applicationProperties.loadProperties();
+                        backOffDelay = REFRESH_DELAY;
+                    } catch (FalconException e) {
+                        LOG.warn("Error refreshing runtime properties", e);
+                        backOffDelay += REFRESH_DELAY;
+                    }
+                    Thread.sleep(Math.min(MAX_ITER * REFRESH_DELAY, backOffDelay));
+                } catch (InterruptedException e) {
+                    LOG.info("Application is stopping. Aborting...");
+                    break;
+                }
+            }
         }
-      }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/util/StartupProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/StartupProperties.java b/common/src/main/java/org/apache/falcon/util/StartupProperties.java
index e264758..4a19df4 100644
--- a/common/src/main/java/org/apache/falcon/util/StartupProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/StartupProperties.java
@@ -18,36 +18,36 @@
 
 package org.apache.falcon.util;
 
+import org.apache.falcon.FalconException;
+
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.falcon.FalconException;
-
 public class StartupProperties extends ApplicationProperties {
 
-  private static final String PROPERTY_FILE = "startup.properties";
-
-  private static final AtomicReference<StartupProperties> instance =
-      new AtomicReference<StartupProperties>();
-
-  private StartupProperties() throws FalconException {
-    super();
-  }
-
-  @Override
-  protected String getPropertyFile() {
-    return PROPERTY_FILE;
-  }
-
-  public static Properties get() {
-    try {
-      if (instance.get() == null) {
-        instance.compareAndSet(null, new StartupProperties());
-      }
-      return instance.get();
-    } catch (FalconException e) {
-      throw new RuntimeException("Unable to read application " +
-          "startup properties", e);
+    private static final String PROPERTY_FILE = "startup.properties";
+
+    private static final AtomicReference<StartupProperties> instance =
+            new AtomicReference<StartupProperties>();
+
+    private StartupProperties() throws FalconException {
+        super();
+    }
+
+    @Override
+    protected String getPropertyFile() {
+        return PROPERTY_FILE;
+    }
+
+    public static Properties get() {
+        try {
+            if (instance.get() == null) {
+                instance.compareAndSet(null, new StartupProperties());
+            }
+            return instance.get();
+        } catch (FalconException e) {
+            throw new RuntimeException("Unable to read application " +
+                    "startup properties", e);
+        }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
index 9b75327..76a9edc 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
@@ -18,15 +18,15 @@
 
 package org.apache.falcon.workflow;
 
+import org.apache.falcon.FalconException;
+import org.apache.falcon.entity.v0.Entity;
+import org.apache.falcon.util.ReflectionUtils;
+
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.falcon.FalconException;
-import org.apache.falcon.entity.v0.Entity;
-import org.apache.falcon.util.ReflectionUtils;
-
 public abstract class WorkflowBuilder<T extends Entity> {
 
     public static WorkflowBuilder<Entity> getBuilder(String engine, Entity entity) throws FalconException {
@@ -35,8 +35,9 @@ public abstract class WorkflowBuilder<T extends Entity> {
     }
 
     public abstract Map<String, Properties> newWorkflowSchedule(T entity, List<String> clusters) throws FalconException;
-    
-    public abstract Properties newWorkflowSchedule(T entity, Date startDate, String clusterName, String user) throws FalconException;
+
+    public abstract Properties newWorkflowSchedule(T entity, Date startDate, String clusterName, String user)
+            throws FalconException;
 
     public abstract String[] getWorkflowNames(T entity);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
index 5880f68..a267e39 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
@@ -27,11 +27,12 @@ public class WorkflowEngineFactory {
 
     private static final String WORKFLOW_ENGINE = "workflow.engine.impl";
 
-    private WorkflowEngineFactory() { }
+    private WorkflowEngineFactory() {
+    }
 
-	public static AbstractWorkflowEngine getWorkflowEngine()
+    public static AbstractWorkflowEngine getWorkflowEngine()
             throws FalconException {
         return ReflectionUtils.getInstance(WORKFLOW_ENGINE);
-	}
+    }
 
 }


[10/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java b/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
index 6f246a5..50122fe 100644
--- a/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
+++ b/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
@@ -17,15 +17,6 @@
  */
 package org.apache.falcon.service;
 
-import javax.jms.Connection;
-import javax.jms.ConnectionFactory;
-import javax.jms.DeliveryMode;
-import javax.jms.Destination;
-import javax.jms.JMSException;
-import javax.jms.MapMessage;
-import javax.jms.Session;
-import javax.jms.TextMessage;
-
 import org.apache.activemq.ActiveMQConnectionFactory;
 import org.apache.activemq.broker.BrokerService;
 import org.apache.falcon.FalconException;
@@ -36,87 +27,89 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import javax.jms.*;
+
 public class FalconTopicSubscriberTest {
 
-	private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
-	// private static final String BROKER_URL =
-	// "tcp://localhost:61616?daemon=true";
-	private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
-	private static final String TOPIC_NAME = "FALCON.ENTITY.TOPIC";
-	private BrokerService broker;
+    private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
+    // private static final String BROKER_URL =
+    // "tcp://localhost:61616?daemon=true";
+    private static final String BROKER_IMPL_CLASS = "org.apache.activemq.ActiveMQConnectionFactory";
+    private static final String TOPIC_NAME = "FALCON.ENTITY.TOPIC";
+    private BrokerService broker;
 
-	@BeforeClass
-	public void setup() throws Exception {
-		broker = new BrokerService();
-		broker.setUseJmx(true);
-		broker.addConnector(BROKER_URL);
-		broker.start();
-	}
+    @BeforeClass
+    public void setup() throws Exception {
+        broker = new BrokerService();
+        broker.setUseJmx(true);
+        broker.addConnector(BROKER_URL);
+        broker.start();
+    }
 
-	public void sendMessages() throws JMSException {
-		ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
-				BROKER_URL);
-		Connection connection = connectionFactory.createConnection();
-		connection.start();
+    public void sendMessages() throws JMSException {
+        ConnectionFactory connectionFactory = new ActiveMQConnectionFactory(
+                BROKER_URL);
+        Connection connection = connectionFactory.createConnection();
+        connection.start();
 
-		Session session = connection.createSession(false,
-				Session.AUTO_ACKNOWLEDGE);
-		Destination destination = session.createTopic(TOPIC_NAME);
-		javax.jms.MessageProducer producer = session
-				.createProducer(destination);
-		producer.setDeliveryMode(DeliveryMode.PERSISTENT);
-		for (int i = 0; i < 10; i++) {
-			EntityInstanceMessage falconMessage = getMockFalconMessage(i);
-			MapMessage message = session.createMapMessage();
-			for (ARG arg : ARG.values()) {
-				message.setString(arg.getPropName(), falconMessage
-						.getKeyValueMap().get(arg));
-			}
-			Log.debug("Sending:" + message);
-			producer.send(message);
-		}
+        Session session = connection.createSession(false,
+                Session.AUTO_ACKNOWLEDGE);
+        Destination destination = session.createTopic(TOPIC_NAME);
+        javax.jms.MessageProducer producer = session
+                .createProducer(destination);
+        producer.setDeliveryMode(DeliveryMode.PERSISTENT);
+        for (int i = 0; i < 10; i++) {
+            EntityInstanceMessage falconMessage = getMockFalconMessage(i);
+            MapMessage message = session.createMapMessage();
+            for (ARG arg : ARG.values()) {
+                message.setString(arg.getPropName(), falconMessage
+                        .getKeyValueMap().get(arg));
+            }
+            Log.debug("Sending:" + message);
+            producer.send(message);
+        }
 
-		EntityInstanceMessage message = getMockFalconMessage(15);
-		message.getKeyValueMap().put(ARG.status, "FAILED");
-		TextMessage textMessage = session.createTextMessage(message.toString());
-		producer.send(textMessage);
+        EntityInstanceMessage message = getMockFalconMessage(15);
+        message.getKeyValueMap().put(ARG.status, "FAILED");
+        TextMessage textMessage = session.createTextMessage(message.toString());
+        producer.send(textMessage);
 
-	}
+    }
 
-	private EntityInstanceMessage getMockFalconMessage(int i) {
-		EntityInstanceMessage message = new EntityInstanceMessage();
-		message.getKeyValueMap().put(ARG.brokerImplClass, BROKER_IMPL_CLASS);
-		message.getKeyValueMap().put(ARG.brokerUrl, BROKER_URL);
-		message.getKeyValueMap().put(ARG.entityName, "process1");
-		message.getKeyValueMap().put(ARG.entityType, "PROCESS");
-		message.getKeyValueMap().put(ARG.feedInstancePaths,
-				"/clicks/hour/00/0" + i);
-		message.getKeyValueMap().put(ARG.feedNames, "clicks");
-		message.getKeyValueMap().put(ARG.logFile, "/logfile");
-		message.getKeyValueMap().put(ARG.nominalTime, "2012-10-10-10-10");
-		message.getKeyValueMap().put(ARG.operation, "GENERATE");
-		message.getKeyValueMap().put(ARG.runId, "0");
-		message.getKeyValueMap().put(ARG.timeStamp, "2012-10-10-10-1" + i);
-		message.getKeyValueMap().put(ARG.workflowId, "workflow-" + i);
-		message.getKeyValueMap().put(ARG.topicName, TOPIC_NAME);
-		message.getKeyValueMap().put(ARG.status, "SUCCEEDED");
-		return message;
-	}
+    private EntityInstanceMessage getMockFalconMessage(int i) {
+        EntityInstanceMessage message = new EntityInstanceMessage();
+        message.getKeyValueMap().put(ARG.brokerImplClass, BROKER_IMPL_CLASS);
+        message.getKeyValueMap().put(ARG.brokerUrl, BROKER_URL);
+        message.getKeyValueMap().put(ARG.entityName, "process1");
+        message.getKeyValueMap().put(ARG.entityType, "PROCESS");
+        message.getKeyValueMap().put(ARG.feedInstancePaths,
+                "/clicks/hour/00/0" + i);
+        message.getKeyValueMap().put(ARG.feedNames, "clicks");
+        message.getKeyValueMap().put(ARG.logFile, "/logfile");
+        message.getKeyValueMap().put(ARG.nominalTime, "2012-10-10-10-10");
+        message.getKeyValueMap().put(ARG.operation, "GENERATE");
+        message.getKeyValueMap().put(ARG.runId, "0");
+        message.getKeyValueMap().put(ARG.timeStamp, "2012-10-10-10-1" + i);
+        message.getKeyValueMap().put(ARG.workflowId, "workflow-" + i);
+        message.getKeyValueMap().put(ARG.topicName, TOPIC_NAME);
+        message.getKeyValueMap().put(ARG.status, "SUCCEEDED");
+        return message;
+    }
 
-	@Test
-	public void testSubscriber() throws FalconException, JMSException {
-		FalconTopicSubscriber subscriber1 = new FalconTopicSubscriber(
-				BROKER_IMPL_CLASS, "", "", BROKER_URL, TOPIC_NAME);
+    @Test
+    public void testSubscriber() throws FalconException, JMSException {
+        FalconTopicSubscriber subscriber1 = new FalconTopicSubscriber(
+                BROKER_IMPL_CLASS, "", "", BROKER_URL, TOPIC_NAME);
 
-		subscriber1.startSubscriber();
-		sendMessages();
-		subscriber1.closeSubscriber();
+        subscriber1.startSubscriber();
+        sendMessages();
+        subscriber1.closeSubscriber();
 
-	}
+    }
 
-	@AfterClass
-	public void tearDown() throws Exception {
-		broker.stop();
-	}
+    @AfterClass
+    public void tearDown() throws Exception {
+        broker.stop();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/pom.xml
----------------------------------------------------------------------
diff --git a/process/pom.xml b/process/pom.xml
index 4a3a872..5fa95c9 100644
--- a/process/pom.xml
+++ b/process/pom.xml
@@ -12,18 +12,18 @@
 	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>
-		<groupId>org.apache.falcon</groupId>
-		<artifactId>falcon-main</artifactId>
-		<version>0.2-SNAPSHOT</version>
-	</parent>
-	<artifactId>falcon-process</artifactId>
-	<description>Apache Falcon Process Module</description>
-	<name>Apache Falcon Process</name>
-	<packaging>jar</packaging>
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.falcon</groupId>
+        <artifactId>falcon-main</artifactId>
+        <version>0.2-SNAPSHOT</version>
+    </parent>
+    <artifactId>falcon-process</artifactId>
+    <description>Apache Falcon Process Module</description>
+    <name>Apache Falcon Process</name>
+    <packaging>jar</packaging>
 
     <dependencies>
         <dependency>
@@ -32,53 +32,53 @@
         </dependency>
 
         <dependency>
-			<groupId>org.apache.falcon</groupId>
-			<artifactId>falcon-oozie-adaptor</artifactId>
-		</dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-oozie-adaptor</artifactId>
+        </dependency>
 
         <dependency>
-			<groupId>org.apache.falcon</groupId>
-			<artifactId>falcon-common</artifactId>
-		</dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-common</artifactId>
+        </dependency>
 
         <dependency>
-			<groupId>org.apache.falcon</groupId>
-			<artifactId>falcon-feed</artifactId>
-		</dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-feed</artifactId>
+        </dependency>
 
         <dependency>
-			<groupId>org.apache.falcon</groupId>
-			<artifactId>falcon-test-util</artifactId>
-		</dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-test-util</artifactId>
+        </dependency>
 
-		<dependency>
-			<groupId>org.apache.falcon</groupId>
-			<artifactId>falcon-messaging</artifactId>
-		</dependency>
+        <dependency>
+            <groupId>org.apache.falcon</groupId>
+            <artifactId>falcon-messaging</artifactId>
+        </dependency>
 
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-core</artifactId>
-		</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.apache.hadoop</groupId>
+            <artifactId>hadoop-test</artifactId>
+        </dependency>
 
-		<dependency>
-			<groupId>org.springframework</groupId>
-			<artifactId>spring-jms</artifactId>
-		</dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jms</artifactId>
+        </dependency>
 
-		<dependency>
-			<groupId>org.springframework</groupId>
-			<artifactId>spring-beans</artifactId>
-		</dependency>
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+        </dependency>
 
         <dependency>
             <groupId>org.testng</groupId>
             <artifactId>testng</artifactId>
         </dependency>
-	</dependencies>
+    </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java b/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
index 7286512..4e90792 100644
--- a/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
+++ b/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
@@ -18,14 +18,7 @@
 
 package org.apache.falcon.converter;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.EntityUtil;
@@ -43,18 +36,13 @@ import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.entity.v0.process.Property;
 import org.apache.falcon.expression.ExpressionHelper;
 import org.apache.falcon.messaging.EntityInstanceMessage.ARG;
-import org.apache.falcon.oozie.coordinator.CONTROLS;
-import org.apache.falcon.oozie.coordinator.COORDINATORAPP;
-import org.apache.falcon.oozie.coordinator.DATAIN;
-import org.apache.falcon.oozie.coordinator.DATAOUT;
-import org.apache.falcon.oozie.coordinator.DATASETS;
-import org.apache.falcon.oozie.coordinator.INPUTEVENTS;
-import org.apache.falcon.oozie.coordinator.OUTPUTEVENTS;
-import org.apache.falcon.oozie.coordinator.SYNCDATASET;
-import org.apache.falcon.oozie.coordinator.WORKFLOW;
+import org.apache.falcon.oozie.coordinator.*;
 import org.apache.falcon.oozie.workflow.ACTION;
 import org.apache.falcon.oozie.workflow.SUBWORKFLOW;
 import org.apache.falcon.oozie.workflow.WORKFLOWAPP;
+import org.apache.hadoop.fs.Path;
+
+import java.util.*;
 
 public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
 
@@ -69,7 +57,7 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
     protected List<COORDINATORAPP> getCoordinators(Cluster cluster, Path bundlePath) throws FalconException {
         List<COORDINATORAPP> apps = new ArrayList<COORDINATORAPP>();
         apps.add(createDefaultCoordinator(cluster, bundlePath));
-        
+
         return apps;
     }
 
@@ -89,27 +77,26 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
 
     /**
      * Creates default oozie coordinator
-     * 
-     * @param cluster
-     *            - Cluster for which the coordiantor app need to be created
-     * @param bundlePath
-     *            - bundle path
+     *
+     * @param cluster    - Cluster for which the coordiantor app need to be created
+     * @param bundlePath - bundle path
      * @return COORDINATORAPP
-     * @throws FalconException
-     *             on Error
+     * @throws FalconException on Error
      */
     public COORDINATORAPP createDefaultCoordinator(Cluster cluster, Path bundlePath) throws FalconException {
         Process process = getEntity();
-        if (process == null)
+        if (process == null) {
             return null;
+        }
 
         COORDINATORAPP coord = new COORDINATORAPP();
-        String coordName = EntityUtil.getWorkflowName(Tag.DEFAULT,process).toString();
+        String coordName = EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString();
         Path coordPath = getCoordPath(bundlePath, coordName);
 
         // coord attributes
         coord.setName(coordName);
-        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster.getName());
+        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process,
+                cluster.getName());
         coord.setStart(SchemaHelper.formatDateUTC(processCluster.getValidity().getStart()));
         coord.setEnd(SchemaHelper.formatDateUTC(processCluster.getValidity().getEnd()));
         coord.setTimezone(process.getTimezone().getID());
@@ -129,7 +116,9 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
                     evaluate(process.getTimeout().toString(), Long.class);
         } else {
             timeout_ms = frequency_ms * 6;
-            if (timeout_ms < THIRTY_MINUTES) timeout_ms = THIRTY_MINUTES;
+            if (timeout_ms < THIRTY_MINUTES) {
+                timeout_ms = THIRTY_MINUTES;
+            }
         }
         controls.setTimeout(String.valueOf(timeout_ms / (1000 * 60)));
         if (timeout_ms / frequency_ms * 2 > 0) {
@@ -143,17 +132,20 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         List<String> inputFeeds = new ArrayList<String>();
         List<String> inputPaths = new ArrayList<String>();
         // inputs
-        if (process.getInputs() != null) {            
+        if (process.getInputs() != null) {
             for (Input input : process.getInputs().getInputs()) {
-                if(!input.isOptional()) {
-                    if (coord.getDatasets() == null)
+                if (!input.isOptional()) {
+                    if (coord.getDatasets() == null) {
                         coord.setDatasets(new DATASETS());
-                    if (coord.getInputEvents() == null)
+                    }
+                    if (coord.getInputEvents() == null) {
                         coord.setInputEvents(new INPUTEVENTS());
+                    }
 
-                    SYNCDATASET syncdataset = createDataSet(input.getFeed(), cluster, input.getName(), LocationType.DATA);
+                    SYNCDATASET syncdataset = createDataSet(input.getFeed(), cluster, input.getName(),
+                            LocationType.DATA);
                     coord.getDatasets().getDatasetOrAsyncDataset().add(syncdataset);
-    
+
                     DATAIN datain = createDataIn(input);
                     coord.getInputEvents().getDataIn().add(datain);
                 }
@@ -172,13 +164,15 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         List<String> outputFeeds = new ArrayList<String>();
         List<String> outputPaths = new ArrayList<String>();
         if (process.getOutputs() != null) {
-            if (coord.getDatasets() == null)
+            if (coord.getDatasets() == null) {
                 coord.setDatasets(new DATASETS());
-            if (coord.getOutputEvents() == null)
+            }
+            if (coord.getOutputEvents() == null) {
                 coord.setOutputEvents(new OUTPUTEVENTS());
-            
+            }
+
             for (Output output : process.getOutputs().getOutputs()) {
-                SYNCDATASET syncdataset = createDataSet(output.getFeed(), cluster, output.getName(),LocationType.DATA);
+                SYNCDATASET syncdataset = createDataSet(output.getFeed(), cluster, output.getName(), LocationType.DATA);
                 coord.getDatasets().getDatasetOrAsyncDataset().add(syncdataset);
 
                 DATAOUT dataout = createDataOut(output);
@@ -188,14 +182,14 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
                 props.put(output.getName(), outputExpr);
                 outputFeeds.add(output.getName());
                 outputPaths.add(outputExpr);
-                
-				// stats and meta paths
-				createOutputEvent(output.getFeed(),output.getName(), cluster, "stats",
-						LocationType.STATS, coord, props, output.getInstance());
-				createOutputEvent(output.getFeed(),output.getName(), cluster, "meta",
-						LocationType.META, coord, props,output.getInstance());
-				createOutputEvent(output.getFeed(),output.getName(), cluster, "tmp",
-						LocationType.TMP, coord, props,output.getInstance());
+
+                // stats and meta paths
+                createOutputEvent(output.getFeed(), output.getName(), cluster, "stats",
+                        LocationType.STATS, coord, props, output.getInstance());
+                createOutputEvent(output.getFeed(), output.getName(), cluster, "meta",
+                        LocationType.META, coord, props, output.getInstance());
+                createOutputEvent(output.getFeed(), output.getName(), cluster, "tmp",
+                        LocationType.TMP, coord, props, output.getInstance());
 
             }
         }
@@ -235,50 +229,53 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         return datain;
     }
 
-	private void createOutputEvent(String feed, String name, Cluster cluster,
-			String type, LocationType locType, COORDINATORAPP coord,
-			Map<String, String> props, String instance)
-			throws FalconException {
-		SYNCDATASET dataset = createDataSet(feed, cluster,name+type,
-				locType);
-		coord.getDatasets().getDatasetOrAsyncDataset().add(dataset);
-		DATAOUT dataout = new DATAOUT();
-        if (coord.getOutputEvents() == null)
+    private void createOutputEvent(String feed, String name, Cluster cluster,
+                                   String type, LocationType locType, COORDINATORAPP coord,
+                                   Map<String, String> props, String instance)
+            throws FalconException {
+        SYNCDATASET dataset = createDataSet(feed, cluster, name + type,
+                locType);
+        coord.getDatasets().getDatasetOrAsyncDataset().add(dataset);
+        DATAOUT dataout = new DATAOUT();
+        if (coord.getOutputEvents() == null) {
             coord.setOutputEvents(new OUTPUTEVENTS());
-		dataout.setName(name+type);
-		dataout.setDataset(name+type);
-		dataout.setInstance(getELExpression(instance));
-		coord.getOutputEvents().getDataOut().add(dataout);
-        String outputExpr = "${coord:dataOut('" + name+type+ "')}";
-        props.put(name+"."+type, outputExpr);
-	}
-	
+        }
+        dataout.setName(name + type);
+        dataout.setDataset(name + type);
+        dataout.setInstance(getELExpression(instance));
+        coord.getOutputEvents().getDataOut().add(dataout);
+        String outputExpr = "${coord:dataOut('" + name + type + "')}";
+        props.put(name + "." + type, outputExpr);
+    }
+
     private String join(Iterator<String> itr, char sep) {
         String joinedStr = StringUtils.join(itr, sep);
-        if(joinedStr.isEmpty())
+        if (joinedStr.isEmpty()) {
             joinedStr = "null";
+        }
         return joinedStr;
     }
 
-    private SYNCDATASET createDataSet(String feedName, Cluster cluster, String datasetName, LocationType locationType) throws FalconException {
+    private SYNCDATASET createDataSet(String feedName, Cluster cluster, String datasetName, LocationType locationType)
+            throws FalconException {
         Feed feed = (Feed) EntityUtil.getEntity(EntityType.FEED, feedName);
 
         SYNCDATASET syncdataset = new SYNCDATASET();
         syncdataset.setName(datasetName);
-		String locPath = FeedHelper.getLocation(feed, locationType,
-				cluster.getName()).getPath();
-		syncdataset.setUriTemplate(new Path(locPath).toUri().getScheme()!=null?locPath:"${nameNode}"
-				+ locPath);
+        String locPath = FeedHelper.getLocation(feed, locationType,
+                cluster.getName()).getPath();
+        syncdataset.setUriTemplate(new Path(locPath).toUri().getScheme() != null ? locPath : "${nameNode}"
+                + locPath);
         syncdataset.setFrequency("${coord:" + feed.getFrequency().toString() + "}");
 
         org.apache.falcon.entity.v0.feed.Cluster feedCluster = FeedHelper.getCluster(feed, cluster.getName());
         syncdataset.setInitialInstance(SchemaHelper.formatDateUTC(feedCluster.getValidity().getStart()));
         syncdataset.setTimezone(feed.getTimezone().getID());
-		if (feed.getAvailabilityFlag() == null) {
-			syncdataset.setDoneFlag("");
-		} else {
-			syncdataset.setDoneFlag(feed.getAvailabilityFlag());
-		}
+        if (feed.getAvailabilityFlag() == null) {
+            syncdataset.setDoneFlag("");
+        } else {
+            syncdataset.setDoneFlag(feed.getAvailabilityFlag());
+        }
         return syncdataset;
     }
 
@@ -294,8 +291,9 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         Process process = getEntity();
         Map<String, String> props = new HashMap<String, String>();
         if (process.getProperties() != null) {
-            for (Property prop : process.getProperties().getProperties())
+            for (Property prop : process.getProperties().getProperties()) {
                 props.put(prop.getName(), prop.getValue());
+            }
         }
         return props;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
index f958651..d5d097a 100644
--- a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
+++ b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
@@ -18,17 +18,9 @@
 
 package org.apache.falcon.workflow;
 
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
-
 import org.apache.falcon.converter.OozieProcessMapper;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.EntityUtil;
@@ -43,26 +35,31 @@ import org.apache.falcon.entity.v0.feed.LocationType;
 import org.apache.falcon.entity.v0.process.Input;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.security.CurrentUser;
+import org.apache.hadoop.fs.Path;
 import org.apache.oozie.client.CoordinatorJob.Timeunit;
 import org.apache.oozie.client.OozieClient;
 
+import java.util.*;
+
 public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
 
     @Override
     public Map<String, Properties> newWorkflowSchedule(Process process, List<String> clusters) throws FalconException {
         Map<String, Properties> propertiesMap = new HashMap<String, Properties>();
 
-        for (String clusterName: clusters) {
+        for (String clusterName : clusters) {
             org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, clusterName);
-            Properties properties = newWorkflowSchedule(process, processCluster.getValidity().getStart(), clusterName, 
+            Properties properties = newWorkflowSchedule(process, processCluster.getValidity().getStart(), clusterName,
                     CurrentUser.getUser());
-            if (properties != null)
+            if (properties != null) {
                 propertiesMap.put(clusterName, properties);
+            }
         }
         return propertiesMap;
     }
 
-    private void addOptionalInputProperties(Properties properties, Input in, String clusterName) throws FalconException {
+    private void addOptionalInputProperties(Properties properties, Input in, String clusterName)
+            throws FalconException {
         Feed feed = EntityUtil.getEntity(EntityType.FEED, in.getFeed());
         org.apache.falcon.entity.v0.feed.Cluster cluster = FeedHelper.getCluster(feed, clusterName);
         String inName = in.getName();
@@ -72,37 +69,41 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
         properties.put(inName + ".end_of_duration", Timeunit.NONE.name());
         properties.put(inName + ".initial-instance", SchemaHelper.formatDateUTC(cluster.getValidity().getStart()));
         properties.put(inName + ".done-flag", "notused");
-        
+
         String locPath = FeedHelper.getLocation(feed, LocationType.DATA, clusterName).getPath().replace('$', '%');
-		properties.put(inName + ".uri-template", new Path(locPath).toUri().getScheme()!=null?locPath:"${nameNode}"+locPath);
+        properties.put(inName + ".uri-template",
+                new Path(locPath).toUri().getScheme() != null ? locPath : "${nameNode}" + locPath);
 
         properties.put(inName + ".start-instance", in.getStart());
         properties.put(inName + ".end-instance", in.getEnd());
     }
 
     private Timeunit mapToCoordTimeUnit(TimeUnit tu) {
-        switch(tu) {
-        case days:
-            return Timeunit.DAY;
-            
-        case hours:
-            return Timeunit.HOUR;
-            
-        case minutes:
-            return Timeunit.MINUTE;
-            
-        case months:
-            return Timeunit.MONTH;
+        switch (tu) {
+            case days:
+                return Timeunit.DAY;
+
+            case hours:
+                return Timeunit.HOUR;
+
+            case minutes:
+                return Timeunit.MINUTE;
+
+            case months:
+                return Timeunit.MONTH;
         }
         throw new IllegalArgumentException("Unhandled time unit " + tu);
     }
-    
+
     @Override
-    public Properties newWorkflowSchedule(Process process, Date startDate, String clusterName, String user) throws FalconException {
+    public Properties newWorkflowSchedule(Process process, Date startDate, String clusterName, String user)
+            throws FalconException {
         org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, clusterName);
         if (!startDate.before(processCluster.getValidity().getEnd()))
-            // start time >= end time
+        // start time >= end time
+        {
             return null;
+        }
 
         Cluster cluster = configStore.get(EntityType.CLUSTER, processCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(process));
@@ -110,23 +111,25 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
         EntityUtil.setStartDate(processClone, clusterName, startDate);
 
         OozieProcessMapper mapper = new OozieProcessMapper(processClone);
-        if(!mapper.map(cluster, bundlePath)){
+        if (!mapper.map(cluster, bundlePath)) {
             return null;
         }
-        
+
         Properties properties = createAppProperties(clusterName, bundlePath, user);
-        
+
         //Add libpath
         String libPath = process.getWorkflow().getLib();
         if (!StringUtils.isEmpty(libPath)) {
             String path = libPath.replace("${nameNode}", "");
             properties.put(OozieClient.LIBPATH, "${nameNode}" + path);
         }
-        
-        if(process.getInputs() != null) {
-            for(Input in:process.getInputs().getInputs())
-                if(in.isOptional())
+
+        if (process.getInputs() != null) {
+            for (Input in : process.getInputs().getInputs()) {
+                if (in.isOptional()) {
                     addOptionalInputProperties(properties, in, clusterName);
+                }
+            }
         }
         return properties;
     }
@@ -140,6 +143,6 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
 
     @Override
     public String[] getWorkflowNames(Process process) {
-        return new String[] { EntityUtil.getWorkflowName(Tag.DEFAULT,process).toString() };
+        return new String[]{EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString()};
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/main/resources/config/workflow/process-parent-workflow.xml
----------------------------------------------------------------------
diff --git a/process/src/main/resources/config/workflow/process-parent-workflow.xml b/process/src/main/resources/config/workflow/process-parent-workflow.xml
index 5bbdf05..8c800eb 100644
--- a/process/src/main/resources/config/workflow/process-parent-workflow.xml
+++ b/process/src/main/resources/config/workflow/process-parent-workflow.xml
@@ -16,83 +16,106 @@
   ~ limitations under the License.
   -->
 <workflow-app xmlns='uri:oozie:workflow:0.3' name='falcon-process-parent-workflow'>
-	<start to='should-record' />
-	<decision name='should-record'>
+    <start to='should-record'/>
+    <decision name='should-record'>
         <switch>
-            <case to="recordsize"> 
-              ${shouldRecord=="true"}
+            <case to="recordsize">
+                ${shouldRecord=="true"}
             </case>
             <default to="user-workflow"/>
         </switch>
     </decision>
-	<action name='recordsize'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.latedata.LateDataHandler</main-class>
-			<arg>-out</arg><arg>${logDir}/latedata/${nominalTime}</arg>
-			<arg>-paths</arg><arg>${falconInPaths}</arg>
-			<arg>-falconInputFeeds</arg><arg>${falconInputFeeds}</arg>
-			<capture-output />
-		</java>
-		<ok to="user-workflow" />
-		<error to="fail" />
-	</action>
+    <action name='recordsize'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.latedata.LateDataHandler</main-class>
+            <arg>-out</arg>
+            <arg>${logDir}/latedata/${nominalTime}</arg>
+            <arg>-paths</arg>
+            <arg>${falconInPaths}</arg>
+            <arg>-falconInputFeeds</arg>
+            <arg>${falconInputFeeds}</arg>
+            <capture-output/>
+        </java>
+        <ok to="user-workflow"/>
+        <error to="fail"/>
+    </action>
 
-	<action name='user-workflow'>
-		<sub-workflow>
-			<app-path>#USER_WF_PATH#</app-path>
-			<propagate-configuration />
-		</sub-workflow>
-		<ok to="succeeded-post-processing" />
-		<error to="failed-post-processing" />
-	</action>
+    <action name='user-workflow'>
+        <sub-workflow>
+            <app-path>#USER_WF_PATH#</app-path>
+            <propagate-configuration/>
+        </sub-workflow>
+        <ok to="succeeded-post-processing"/>
+        <error to="failed-post-processing"/>
+    </action>
 
-	<action name='succeeded-post-processing'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
-			<arg>-cluster</arg><arg>${cluster}</arg>
-			<arg>-entityType</arg><arg>${entityType}</arg>
-			<arg>-entityName</arg><arg>${entityName}</arg>
-			<arg>-nominalTime</arg><arg>${nominalTime}</arg>
-			<arg>-operation</arg><arg>GENERATE</arg>
-			<arg>-workflowId</arg><arg>${wf:id()}</arg>
-			<arg>-runId</arg><arg>${wf:run()}</arg>
-			<arg>-status</arg><arg>SUCCEEDED</arg>
-			<arg>-timeStamp</arg><arg>${timeStamp}</arg>			
-			<arg>-brokerImplClass</arg><arg>${wf:conf("broker.impl.class")}</arg>
-			<arg>-brokerUrl</arg><arg>${wf:conf("broker.url")}</arg>
-			<arg>-userBrokerImplClass</arg><arg>${userBrokerImplClass}</arg>
-			<arg>-userBrokerUrl</arg><arg>${userBrokerUrl}</arg>
-			<arg>-brokerTTL</arg><arg>${wf:conf("broker.ttlInMins")}</arg>
-			<arg>-feedNames</arg><arg>${feedNames}</arg>
-			<arg>-feedInstancePaths</arg><arg>${feedInstancePaths}</arg>			
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
-			<arg>-workflowEngineUrl</arg> <arg>${workflowEngineUrl}</arg>
-			<arg>-subflowId</arg> <arg>${wf:id()}@user-workflow</arg>
-			<arg>-logDir</arg> <arg>${logDir}/job-${nominalTime}/</arg>
+    <action name='succeeded-post-processing'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
+            <arg>-cluster</arg>
+            <arg>${cluster}</arg>
+            <arg>-entityType</arg>
+            <arg>${entityType}</arg>
+            <arg>-entityName</arg>
+            <arg>${entityName}</arg>
+            <arg>-nominalTime</arg>
+            <arg>${nominalTime}</arg>
+            <arg>-operation</arg>
+            <arg>GENERATE</arg>
+            <arg>-workflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-runId</arg>
+            <arg>${wf:run()}</arg>
+            <arg>-status</arg>
+            <arg>SUCCEEDED</arg>
+            <arg>-timeStamp</arg>
+            <arg>${timeStamp}</arg>
+            <arg>-brokerImplClass</arg>
+            <arg>${wf:conf("broker.impl.class")}</arg>
+            <arg>-brokerUrl</arg>
+            <arg>${wf:conf("broker.url")}</arg>
+            <arg>-userBrokerImplClass</arg>
+            <arg>${userBrokerImplClass}</arg>
+            <arg>-userBrokerUrl</arg>
+            <arg>${userBrokerUrl}</arg>
+            <arg>-brokerTTL</arg>
+            <arg>${wf:conf("broker.ttlInMins")}</arg>
+            <arg>-feedNames</arg>
+            <arg>${feedNames}</arg>
+            <arg>-feedInstancePaths</arg>
+            <arg>${feedInstancePaths}</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
+            <arg>-workflowEngineUrl</arg>
+            <arg>${workflowEngineUrl}</arg>
+            <arg>-subflowId</arg>
+            <arg>${wf:id()}@user-workflow</arg>
+            <arg>-logDir</arg>
+            <arg>${logDir}/job-${nominalTime}/</arg>
             <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
             <file>${wf:conf("falcon.libpath")}/ant.jar</file>
             <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>
@@ -100,45 +123,65 @@
             <file>${wf:conf("falcon.libpath")}/json-simple.jar</file>
             <file>${wf:conf("falcon.libpath")}/oozie-client.jar</file>
             <file>${wf:conf("falcon.libpath")}/spring-jms.jar</file>
-		</java>
-		<ok to="end" />
-		<error to="fail" />
-	</action>
-		<action name='failed-post-processing'>
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<configuration>
-				<property>
-					<name>mapred.job.queue.name</name>
-					<value>${queueName}</value>
-				</property>
-				<property>
-					<name>oozie.launcher.mapred.job.priority</name>
-					<value>${jobPriority}</value>
-				</property>
-			</configuration>
-			<main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
-			<arg>-cluster</arg><arg>${cluster}</arg>
-			<arg>-entityType</arg><arg>${entityType}</arg>
-			<arg>-entityName</arg><arg>${entityName}</arg>
-			<arg>-nominalTime</arg><arg>${nominalTime}</arg>
-			<arg>-operation</arg><arg>GENERATE</arg>
-			<arg>-workflowId</arg><arg>${wf:id()}</arg>
-			<arg>-runId</arg><arg>${wf:run()}</arg>
-			<arg>-status</arg><arg>FAILED</arg>
-			<arg>-timeStamp</arg><arg>${timeStamp}</arg>			
-			<arg>-brokerImplClass</arg><arg>${wf:conf("broker.impl.class")}</arg>
-			<arg>-brokerUrl</arg><arg>${wf:conf("broker.url")}</arg>
-			<arg>-userBrokerImplClass</arg><arg>${userBrokerImplClass}</arg>
-			<arg>-userBrokerUrl</arg><arg>${userBrokerUrl}</arg>
-			<arg>-brokerTTL</arg><arg>${wf:conf("broker.ttlInMins")}</arg>
-			<arg>-feedNames</arg><arg>${feedNames}</arg>
-			<arg>-feedInstancePaths</arg><arg>${feedInstancePaths}</arg>			
-			<arg>-logFile</arg><arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
-			<arg>-workflowEngineUrl</arg> <arg>${workflowEngineUrl}</arg>
-			<arg>-subflowId</arg> <arg>${wf:id()}@user-workflow</arg>
-			<arg>-logDir</arg> <arg>${logDir}/job-${nominalTime}/</arg>
+        </java>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    <action name='failed-post-processing'>
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.job.priority</name>
+                    <value>${jobPriority}</value>
+                </property>
+            </configuration>
+            <main-class>org.apache.falcon.workflow.FalconPostProcessing</main-class>
+            <arg>-cluster</arg>
+            <arg>${cluster}</arg>
+            <arg>-entityType</arg>
+            <arg>${entityType}</arg>
+            <arg>-entityName</arg>
+            <arg>${entityName}</arg>
+            <arg>-nominalTime</arg>
+            <arg>${nominalTime}</arg>
+            <arg>-operation</arg>
+            <arg>GENERATE</arg>
+            <arg>-workflowId</arg>
+            <arg>${wf:id()}</arg>
+            <arg>-runId</arg>
+            <arg>${wf:run()}</arg>
+            <arg>-status</arg>
+            <arg>FAILED</arg>
+            <arg>-timeStamp</arg>
+            <arg>${timeStamp}</arg>
+            <arg>-brokerImplClass</arg>
+            <arg>${wf:conf("broker.impl.class")}</arg>
+            <arg>-brokerUrl</arg>
+            <arg>${wf:conf("broker.url")}</arg>
+            <arg>-userBrokerImplClass</arg>
+            <arg>${userBrokerImplClass}</arg>
+            <arg>-userBrokerUrl</arg>
+            <arg>${userBrokerUrl}</arg>
+            <arg>-brokerTTL</arg>
+            <arg>${wf:conf("broker.ttlInMins")}</arg>
+            <arg>-feedNames</arg>
+            <arg>${feedNames}</arg>
+            <arg>-feedInstancePaths</arg>
+            <arg>${feedInstancePaths}</arg>
+            <arg>-logFile</arg>
+            <arg>${logDir}/instancePaths-${nominalTime}.csv</arg>
+            <arg>-workflowEngineUrl</arg>
+            <arg>${workflowEngineUrl}</arg>
+            <arg>-subflowId</arg>
+            <arg>${wf:id()}@user-workflow</arg>
+            <arg>-logDir</arg>
+            <arg>${logDir}/job-${nominalTime}/</arg>
             <file>${wf:conf("falcon.libpath")}/activemq-core.jar</file>
             <file>${wf:conf("falcon.libpath")}/ant.jar</file>
             <file>${wf:conf("falcon.libpath")}/geronimo-j2ee-management.jar</file>
@@ -146,14 +189,14 @@
             <file>${wf:conf("falcon.libpath")}/json-simple.jar</file>
             <file>${wf:conf("falcon.libpath")}/oozie-client.jar</file>
             <file>${wf:conf("falcon.libpath")}/spring-jms.jar</file>
-		</java>
-		<ok to="fail" />
-		<error to="fail" />
-	</action>
-	<kill name="fail">
-		<message>Workflow failed, error
-			message[${wf:errorMessage(wf:lastErrorNode())}]
-		</message>
-	</kill>
-	<end name='end' />
+        </java>
+        <ok to="fail"/>
+        <error to="fail"/>
+    </action>
+    <kill name="fail">
+        <message>Workflow failed, error
+            message[${wf:errorMessage(wf:lastErrorNode())}]
+        </message>
+    </kill>
+    <end name='end'/>
 </workflow-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
index 81b54bc..bd2d6ac 100644
--- a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
+++ b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
@@ -18,37 +18,37 @@
 
 package org.apache.falcon.converter;
 
-import javax.xml.bind.Unmarshaller;
-
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
 
+import javax.xml.bind.Unmarshaller;
+
 public class AbstractTestBase {
     private static final String PROCESS_XML = "/config/process/process-0.1.xml";
     private static final String FEED_XML = "/config/feed/feed-0.1.xml";
     private static final String CLUSTER_XML = "/config/cluster/cluster-0.1.xml";
-    
-    
+
+
     private void storeEntity(EntityType type, String name) throws Exception {
         Unmarshaller unmarshaller = type.getUnmarshaller();
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
-        switch(type) {
+        switch (type) {
             case CLUSTER:
                 Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
                 cluster.setName(name);
                 store.publish(type, cluster);
                 break;
-                
+
             case FEED:
                 Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
                 feed.setName(name);
                 store.publish(type, feed);
                 break;
-                
+
             case PROCESS:
                 Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
                 process.setName(name);
@@ -56,15 +56,15 @@ public class AbstractTestBase {
                 break;
         }
     }
-    
+
     public void setup() throws Exception {
         storeEntity(EntityType.CLUSTER, "corp");
         storeEntity(EntityType.FEED, "clicks");
         storeEntity(EntityType.FEED, "impressions");
         storeEntity(EntityType.FEED, "clicksummary");
         storeEntity(EntityType.PROCESS, "clicksummary");
-    }    
-    
+    }
+
     public void cleanup() throws Exception {
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(EntityType.PROCESS, "clicksummary");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 c730b1a..d7fb4b0 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.falcon.converter;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -27,75 +26,73 @@ import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.cluster.Interfacetype;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.hadoop.conf.Configuration;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 
 public class OozieProcessMapperLateProcessTest {
 
-	private static String hdfsUrl;
-	private static final String CLUSTER_XML = "/config/late/late-cluster.xml";
-	private static final String FEED1_XML = "/config/late/late-feed1.xml";
-	private static final String FEED2_XML = "/config/late/late-feed2.xml";
-	private static final String FEED3_XML = "/config/late/late-feed3.xml";
-	private static final String PROCESS1_XML = "/config/late/late-process1.xml";
-	private static final String PROCESS2_XML = "/config/late/late-process2.xml";
-	private static final ConfigurationStore store = ConfigurationStore.get();
-	private static EmbeddedCluster dfsCluster;
-	private static 	Configuration conf;
+    private static String hdfsUrl;
+    private static final String CLUSTER_XML = "/config/late/late-cluster.xml";
+    private static final String FEED1_XML = "/config/late/late-feed1.xml";
+    private static final String FEED2_XML = "/config/late/late-feed2.xml";
+    private static final String FEED3_XML = "/config/late/late-feed3.xml";
+    private static final String PROCESS1_XML = "/config/late/late-process1.xml";
+    private static final String PROCESS2_XML = "/config/late/late-process2.xml";
+    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static EmbeddedCluster dfsCluster;
+    private static Configuration conf;
 
-	@BeforeClass
-	public void setUpDFS() throws Exception {
+    @BeforeClass
+    public void setUpDFS() throws Exception {
 
-		cleanupStore();
+        cleanupStore();
 
-		dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+        dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
         conf = dfsCluster.getConf();
-		hdfsUrl = conf.get("fs.default.name");
-
-		Cluster cluster = (Cluster) EntityType.CLUSTER.getUnmarshaller()
-				.unmarshal(this.getClass().getResource(CLUSTER_XML));
-		ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(hdfsUrl);
-
-		store.publish(EntityType.CLUSTER, cluster);
-
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				this.getClass().getResource(FEED1_XML));
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				this.getClass().getResource(FEED2_XML));
-		Feed feed3 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				this.getClass().getResource(FEED3_XML));
-
-		store.publish(EntityType.FEED, feed1);
-		store.publish(EntityType.FEED, feed2);
-		store.publish(EntityType.FEED, feed3);
-
-		Process process1 = (Process) EntityType.PROCESS.getUnmarshaller()
-				.unmarshal(this.getClass().getResource(PROCESS1_XML));
-		store.publish(EntityType.PROCESS, process1);
-		Process process2 = (Process) EntityType.PROCESS.getUnmarshaller()
-				.unmarshal(this.getClass().getResource(PROCESS2_XML));
-		store.publish(EntityType.PROCESS, process2);
-
-	}
-
-	private void cleanupStore() throws FalconException {
-		store.remove(EntityType.PROCESS, "late-process1");
-		store.remove(EntityType.PROCESS, "late-process2");
-		store.remove(EntityType.FEED, "late-feed1");
-		store.remove(EntityType.FEED, "late-feed2");
-		store.remove(EntityType.FEED, "late-feed3");
-		store.remove(EntityType.CLUSTER, "late-cluster");
-
-	}
-
-	
-	
-	
-
-	@AfterClass
-	public void tearDown() throws Exception {
-		cleanupStore();
-		dfsCluster.shutdown();
-	}
+        hdfsUrl = conf.get("fs.default.name");
+
+        Cluster cluster = (Cluster) EntityType.CLUSTER.getUnmarshaller()
+                .unmarshal(this.getClass().getResource(CLUSTER_XML));
+        ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(hdfsUrl);
+
+        store.publish(EntityType.CLUSTER, cluster);
+
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                this.getClass().getResource(FEED1_XML));
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                this.getClass().getResource(FEED2_XML));
+        Feed feed3 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                this.getClass().getResource(FEED3_XML));
+
+        store.publish(EntityType.FEED, feed1);
+        store.publish(EntityType.FEED, feed2);
+        store.publish(EntityType.FEED, feed3);
+
+        Process process1 = (Process) EntityType.PROCESS.getUnmarshaller()
+                .unmarshal(this.getClass().getResource(PROCESS1_XML));
+        store.publish(EntityType.PROCESS, process1);
+        Process process2 = (Process) EntityType.PROCESS.getUnmarshaller()
+                .unmarshal(this.getClass().getResource(PROCESS2_XML));
+        store.publish(EntityType.PROCESS, process2);
+
+    }
+
+    private void cleanupStore() throws FalconException {
+        store.remove(EntityType.PROCESS, "late-process1");
+        store.remove(EntityType.PROCESS, "late-process2");
+        store.remove(EntityType.FEED, "late-feed1");
+        store.remove(EntityType.FEED, "late-feed2");
+        store.remove(EntityType.FEED, "late-feed3");
+        store.remove(EntityType.CLUSTER, "late-cluster");
+
+    }
+
+
+    @AfterClass
+    public void tearDown() throws Exception {
+        cleanupStore();
+        dfsCluster.shutdown();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 3372e6e..3106920 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
@@ -19,9 +19,6 @@
 package org.apache.falcon.converter;
 
 import junit.framework.Assert;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.Tag;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.ClusterHelper;
@@ -44,6 +41,9 @@ import org.apache.falcon.oozie.coordinator.SYNCDATASET;
 import org.apache.falcon.oozie.workflow.ACTION;
 import org.apache.falcon.oozie.workflow.DECISION;
 import org.apache.falcon.oozie.workflow.WORKFLOWAPP;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
@@ -62,7 +62,7 @@ import java.io.InputStreamReader;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertTrue;
 
-public class OozieProcessMapperTest extends AbstractTestBase{
+public class OozieProcessMapperTest extends AbstractTestBase {
 
     private String hdfsUrl;
 
@@ -76,7 +76,7 @@ public class OozieProcessMapperTest extends AbstractTestBase{
     @BeforeMethod
     public void setUp() throws Exception {
         super.setup();
-        
+
         ConfigurationStore store = ConfigurationStore.get();
         Cluster cluster = store.get(EntityType.CLUSTER, "corp");
         ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(hdfsUrl);
@@ -85,54 +85,70 @@ public class OozieProcessMapperTest extends AbstractTestBase{
         Path wfpath = new Path(process.getWorkflow().getPath());
         assert new Path(hdfsUrl).getFileSystem(new Configuration()).mkdirs(wfpath);
     }
-    
+
     public void testDefCoordMap(Process process, COORDINATORAPP coord) throws Exception {
         assertEquals("FALCON_PROCESS_DEFAULT_" + process.getName(), coord.getName());
         Validity processValidity = process.getClusters().getClusters().get(0).getValidity();
         assertEquals(SchemaHelper.formatDateUTC(processValidity.getStart()), coord.getStart());
         assertEquals(SchemaHelper.formatDateUTC(processValidity.getEnd()), coord.getEnd());
-        assertEquals("${coord:"+process.getFrequency().toString()+"}", coord.getFrequency());
+        assertEquals("${coord:" + process.getFrequency().toString() + "}", coord.getFrequency());
         assertEquals(process.getTimezone().getID(), coord.getTimezone());
 
-        assertEquals(process.getParallel()+"", coord.getControls().getConcurrency());
+        assertEquals(process.getParallel() + "", coord.getControls().getConcurrency());
         assertEquals(process.getOrder().name(), coord.getControls().getExecution());
-        
-        assertEquals(process.getInputs().getInputs().get(0).getName(), coord.getInputEvents().getDataIn().get(0).getName());
-        assertEquals(process.getInputs().getInputs().get(0).getName(), coord.getInputEvents().getDataIn().get(0).getDataset());
-        assertEquals("${"+process.getInputs().getInputs().get(0).getStart()+"}", coord.getInputEvents().getDataIn().get(0).getStartInstance());
-        assertEquals("${"+process.getInputs().getInputs().get(0).getEnd()+"}", coord.getInputEvents().getDataIn().get(0).getEndInstance());
-        
-        assertEquals(process.getInputs().getInputs().get(1).getName(), coord.getInputEvents().getDataIn().get(1).getName());
-        assertEquals(process.getInputs().getInputs().get(1).getName(), coord.getInputEvents().getDataIn().get(1).getDataset());
-        assertEquals("${"+process.getInputs().getInputs().get(1).getStart()+"}", coord.getInputEvents().getDataIn().get(1).getStartInstance());
-        assertEquals("${" + process.getInputs().getInputs().get(1).getEnd()+"}", coord.getInputEvents().getDataIn().get(1).getEndInstance());
-        
-        assertEquals(process.getOutputs().getOutputs().get(0).getName()+"stats", coord.getOutputEvents().getDataOut().get(1).getName());
-        assertEquals(process.getOutputs().getOutputs().get(0).getName()+"meta", coord.getOutputEvents().getDataOut().get(2).getName());
-        assertEquals(process.getOutputs().getOutputs().get(0).getName()+"tmp", coord.getOutputEvents().getDataOut().get(3).getName());
-        
-        assertEquals(process.getOutputs().getOutputs().get(0).getName(), coord.getOutputEvents().getDataOut().get(0).getName());
-        assertEquals("${"+process.getOutputs().getOutputs().get(0).getInstance()+"}", coord.getOutputEvents().getDataOut().get(0).getInstance());
-        assertEquals(process.getOutputs().getOutputs().get(0).getName(), coord.getOutputEvents().getDataOut().get(0).getDataset());
+
+        assertEquals(process.getInputs().getInputs().get(0).getName(),
+                coord.getInputEvents().getDataIn().get(0).getName());
+        assertEquals(process.getInputs().getInputs().get(0).getName(),
+                coord.getInputEvents().getDataIn().get(0).getDataset());
+        assertEquals("${" + process.getInputs().getInputs().get(0).getStart() + "}",
+                coord.getInputEvents().getDataIn().get(0).getStartInstance());
+        assertEquals("${" + process.getInputs().getInputs().get(0).getEnd() + "}",
+                coord.getInputEvents().getDataIn().get(0).getEndInstance());
+
+        assertEquals(process.getInputs().getInputs().get(1).getName(),
+                coord.getInputEvents().getDataIn().get(1).getName());
+        assertEquals(process.getInputs().getInputs().get(1).getName(),
+                coord.getInputEvents().getDataIn().get(1).getDataset());
+        assertEquals("${" + process.getInputs().getInputs().get(1).getStart() + "}",
+                coord.getInputEvents().getDataIn().get(1).getStartInstance());
+        assertEquals("${" + process.getInputs().getInputs().get(1).getEnd() + "}",
+                coord.getInputEvents().getDataIn().get(1).getEndInstance());
+
+        assertEquals(process.getOutputs().getOutputs().get(0).getName() + "stats",
+                coord.getOutputEvents().getDataOut().get(1).getName());
+        assertEquals(process.getOutputs().getOutputs().get(0).getName() + "meta",
+                coord.getOutputEvents().getDataOut().get(2).getName());
+        assertEquals(process.getOutputs().getOutputs().get(0).getName() + "tmp",
+                coord.getOutputEvents().getDataOut().get(3).getName());
+
+        assertEquals(process.getOutputs().getOutputs().get(0).getName(),
+                coord.getOutputEvents().getDataOut().get(0).getName());
+        assertEquals("${" + process.getOutputs().getOutputs().get(0).getInstance() + "}",
+                coord.getOutputEvents().getDataOut().get(0).getInstance());
+        assertEquals(process.getOutputs().getOutputs().get(0).getName(),
+                coord.getOutputEvents().getDataOut().get(0).getDataset());
 
         assertEquals(6, coord.getDatasets().getDatasetOrAsyncDataset().size());
-        
+
         ConfigurationStore store = ConfigurationStore.get();
         Feed feed = store.get(EntityType.FEED, process.getInputs().getInputs().get(0).getFeed());
         SYNCDATASET ds = (SYNCDATASET) coord.getDatasets().getDatasetOrAsyncDataset().get(0);
-        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity().getStart()), ds.getInitialInstance());
+        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity().getStart()),
+                ds.getInitialInstance());
         assertEquals(feed.getTimezone().getID(), ds.getTimezone());
-        assertEquals("${coord:"+feed.getFrequency().toString()+"}", ds.getFrequency());
+        assertEquals("${coord:" + feed.getFrequency().toString() + "}", ds.getFrequency());
         assertEquals("", ds.getDoneFlag());
-        assertEquals(ds.getUriTemplate(),"${nameNode}" + FeedHelper.getLocation(feed, LocationType.DATA,feed.getClusters().getClusters().get(0).getName()).getPath());   
-        for(Property prop:coord.getAction().getWorkflow().getConfiguration().getProperty()){
-        	if(prop.getName().equals("mapred.job.priority")){
-        		assertEquals(prop.getValue(), "LOW");
-        		break;
-        	}
+        assertEquals(ds.getUriTemplate(), "${nameNode}" + FeedHelper.getLocation(feed, LocationType.DATA,
+                feed.getClusters().getClusters().get(0).getName()).getPath());
+        for (Property prop : coord.getAction().getWorkflow().getConfiguration().getProperty()) {
+            if (prop.getName().equals("mapred.job.priority")) {
+                assertEquals(prop.getValue(), "LOW");
+                break;
+            }
         }
     }
-    
+
     @Test
     public void testBundle() throws Exception {
         Process process = ConfigurationStore.get().get(EntityType.PROCESS, "clicksummary");
@@ -140,16 +156,17 @@ public class OozieProcessMapperTest extends AbstractTestBase{
         OozieProcessMapper mapper = new OozieProcessMapper(process);
         Path bundlePath = new Path("/", EntityUtil.getStagingPath(process));
         mapper.map(cluster, bundlePath);
-        
+
         FileSystem fs = new Path(hdfsUrl).getFileSystem(new Configuration());
         assertTrue(fs.exists(bundlePath));
 
         BUNDLEAPP bundle = getBundle(fs, bundlePath);
         assertEquals(EntityUtil.getWorkflowName(process).toString(), bundle.getName());
         assertEquals(1, bundle.getCoordinator().size());
-        assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT,process).toString(), bundle.getCoordinator().get(0).getName());
+        assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString(),
+                bundle.getCoordinator().get(0).getName());
         String coordPath = bundle.getCoordinator().get(0).getAppPath().replace("${nameNode}", "");
-        
+
         COORDINATORAPP coord = getCoordinator(fs, new Path(coordPath));
         testDefCoordMap(process, coord);
         assertEquals(coord.getControls().getThrottle(), "12");
@@ -157,7 +174,7 @@ public class OozieProcessMapperTest extends AbstractTestBase{
 
         String wfPath = coord.getAction().getWorkflow().getAppPath().replace("${nameNode}", "");
         WORKFLOWAPP parentWorkflow = getParentWorkflow(fs, new Path(wfPath));
-        testParentWorkflow(process,parentWorkflow);
+        testParentWorkflow(process, parentWorkflow);
     }
 
     @Test
@@ -176,7 +193,8 @@ public class OozieProcessMapperTest extends AbstractTestBase{
         BUNDLEAPP bundle = getBundle(fs, bundlePath);
         assertEquals(EntityUtil.getWorkflowName(process).toString(), bundle.getName());
         assertEquals(1, bundle.getCoordinator().size());
-        assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT,process).toString(), bundle.getCoordinator().get(0).getName());
+        assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString(),
+                bundle.getCoordinator().get(0).getName());
         String coordPath = bundle.getCoordinator().get(0).getAppPath().replace("${nameNode}", "");
 
         COORDINATORAPP coord = getCoordinator(fs, new Path(coordPath));
@@ -186,63 +204,68 @@ public class OozieProcessMapperTest extends AbstractTestBase{
 
         String wfPath = coord.getAction().getWorkflow().getAppPath().replace("${nameNode}", "");
         WORKFLOWAPP parentWorkflow = getParentWorkflow(fs, new Path(wfPath));
-        testParentWorkflow(process,parentWorkflow);
+        testParentWorkflow(process, parentWorkflow);
     }
 
-    public void testParentWorkflow(Process process, WORKFLOWAPP parentWorkflow){
-    		Assert.assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT,process).toString(), parentWorkflow.getName());
-    		Assert.assertEquals("should-record", ((DECISION) parentWorkflow.getDecisionOrForkOrJoin().get(0)).getName());
-    		Assert.assertEquals("recordsize", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(1)).getName());
-    		Assert.assertEquals("user-workflow", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(2)).getName());
-    		Assert.assertEquals("succeeded-post-processing", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(3)).getName());
-    		Assert.assertEquals("failed-post-processing", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(4)).getName());
+    public void testParentWorkflow(Process process, WORKFLOWAPP parentWorkflow) {
+        Assert.assertEquals(EntityUtil.getWorkflowName(Tag.DEFAULT, process).toString(), parentWorkflow.getName());
+        Assert.assertEquals("should-record", ((DECISION) parentWorkflow.getDecisionOrForkOrJoin().get(0)).getName());
+        Assert.assertEquals("recordsize", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(1)).getName());
+        Assert.assertEquals("user-workflow", ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(2)).getName());
+        Assert.assertEquals("succeeded-post-processing",
+                ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(3)).getName());
+        Assert.assertEquals("failed-post-processing",
+                ((ACTION) parentWorkflow.getDecisionOrForkOrJoin().get(4)).getName());
     }
-    
+
     private COORDINATORAPP getCoordinator(FileSystem fs, Path path) throws Exception {
         String bundleStr = readFile(fs, path);
-        
+
         Unmarshaller unmarshaller = JAXBContext.newInstance(COORDINATORAPP.class).createUnmarshaller();
         SchemaFactory schemaFactory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
         Schema schema = schemaFactory.newSchema(this.getClass().getResource("/oozie-coordinator-0.3.xsd"));
         unmarshaller.setSchema(schema);
-        JAXBElement<COORDINATORAPP> jaxbBundle = unmarshaller.unmarshal(new StreamSource(new ByteArrayInputStream(bundleStr.trim().getBytes())), COORDINATORAPP.class);
-        return jaxbBundle.getValue();                
+        JAXBElement<COORDINATORAPP> jaxbBundle = unmarshaller.unmarshal(
+                new StreamSource(new ByteArrayInputStream(bundleStr.trim().getBytes())), COORDINATORAPP.class);
+        return jaxbBundle.getValue();
     }
-    
+
     private WORKFLOWAPP getParentWorkflow(FileSystem fs, Path path) throws Exception {
         String workflow = readFile(fs, new Path(path, "workflow.xml"));
-        
+
         Unmarshaller unmarshaller = JAXBContext.newInstance(WORKFLOWAPP.class).createUnmarshaller();
         SchemaFactory schemaFactory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
         Schema schema = schemaFactory.newSchema(this.getClass().getResource("/oozie-workflow-0.3.xsd"));
         unmarshaller.setSchema(schema);
-        JAXBElement<WORKFLOWAPP> jaxbWorkflow = unmarshaller.unmarshal(new StreamSource(new ByteArrayInputStream(workflow.trim().getBytes())), WORKFLOWAPP.class);
-        return jaxbWorkflow.getValue();                
+        JAXBElement<WORKFLOWAPP> jaxbWorkflow = unmarshaller.unmarshal(
+                new StreamSource(new ByteArrayInputStream(workflow.trim().getBytes())), WORKFLOWAPP.class);
+        return jaxbWorkflow.getValue();
     }
-    
+
     private BUNDLEAPP getBundle(FileSystem fs, Path path) throws Exception {
         String bundleStr = readFile(fs, new Path(path, "bundle.xml"));
-        
+
         Unmarshaller unmarshaller = JAXBContext.newInstance(BUNDLEAPP.class).createUnmarshaller();
         SchemaFactory schemaFactory = SchemaFactory.newInstance("http://www.w3.org/2001/XMLSchema");
         Schema schema = schemaFactory.newSchema(this.getClass().getResource("/oozie-bundle-0.1.xsd"));
         unmarshaller.setSchema(schema);
-        JAXBElement<BUNDLEAPP> jaxbBundle = unmarshaller.unmarshal(new StreamSource(new ByteArrayInputStream(bundleStr.trim().getBytes())), BUNDLEAPP.class);
-        return jaxbBundle.getValue();        
+        JAXBElement<BUNDLEAPP> jaxbBundle = unmarshaller.unmarshal(
+                new StreamSource(new ByteArrayInputStream(bundleStr.trim().getBytes())), BUNDLEAPP.class);
+        return jaxbBundle.getValue();
     }
-    
+
     private String readFile(FileSystem fs, Path path) throws Exception {
         BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(path)));
         String line;
         StringBuffer contents = new StringBuffer();
-        while((line=reader.readLine()) != null) {
+        while ((line = reader.readLine()) != null) {
             contents.append(line);
         }
         return contents.toString();
     }
-    
+
     @AfterClass
-    public void cleanup() throws Exception{
+    public void cleanup() throws Exception {
         super.cleanup();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/cluster/cluster-0.1.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/cluster/cluster-0.1.xml b/process/src/test/resources/config/cluster/cluster-0.1.xml
index 6463043..3780b3f 100644
--- a/process/src/test/resources/config/cluster/cluster-0.1.xml
+++ b/process/src/test/resources/config/cluster/cluster-0.1.xml
@@ -17,26 +17,27 @@
   ~ limitations under the License.
   -->
 
-<cluster colo="gs" description="" name="corp" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="hftp://localhost:50010"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="staging" path="/projects/falcon/staging" />
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/falcon/working" />
-	</locations>
-	<properties>
-		<property name="field1" value="value1" />
-		<property name="field2" value="value2" />
-	</properties>
+<cluster colo="gs" description="" name="corp" xmlns="uri:falcon:cluster:0.1"
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:50010"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="staging" path="/projects/falcon/staging"/>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+    </locations>
+    <properties>
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/feed/feed-0.1.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/feed/feed-0.1.xml b/process/src/test/resources/config/feed/feed-0.1.xml
index f724281..fb9b707 100644
--- a/process/src/test/resources/config/feed/feed-0.1.xml
+++ b/process/src/test/resources/config/feed/feed-0.1.xml
@@ -16,45 +16,48 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="country" />
+<feed description="clicks log" name="clicks" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="country"/>
     </partitions>
 
     <groups>online,bi</groups>
 
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    <late-arrival cut-off="hours(6)" />
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="corp" type="source">
             <validity start="2011-11-01T00:00Z" end="2099-12-31T23:59Z"/>
-            <retention limit="hours(6)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
-        	<locations>
-        		<location type="data" path="/projects/falcon/clicks/${YY}/${MM}" />
-        		<location type="stats" path="/projects/falcon/clicksStats" />
-        		<location type="meta" path="/projects/falcon/clicksMetaData" />
-    		</locations>
+            <retention limit="hours(6)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <locations>
+                <location type="data" path="/projects/falcon/clicks/${YY}/${MM}"/>
+                <location type="stats" path="/projects/falcon/clicksStats"/>
+                <location type="meta" path="/projects/falcon/clicksMetaData"/>
+            </locations>
         </cluster>
         <cluster name="backupCluster" type="target">
             <validity start="2011-11-01T00:00Z" end="2099-12-31T23:59Z"/>
-            <retention limit="hours(6)" action="archive" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="archive"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 
     <properties>
-        <property name="field1" value="value1" />
-        <property name="field2" value="value2" />
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
     </properties>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-cluster.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-cluster.xml b/process/src/test/resources/config/late/late-cluster.xml
index 4a0b34d..ac0817f 100644
--- a/process/src/test/resources/config/late/late-cluster.xml
+++ b/process/src/test/resources/config/late/late-cluster.xml
@@ -17,26 +17,27 @@
   ~ limitations under the License.
   -->
 
-<cluster colo="gs" description="" name="late-cluster" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="hftp://localhost:50010"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="staging" path="/projects/falcon/staging" />
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/projects/falcon/working" />
-	</locations>
-	<properties>
-		<property name="field1" value="value1" />
-		<property name="field2" value="value2" />
-	</properties>
+<cluster colo="gs" description="" name="late-cluster" xmlns="uri:falcon:cluster:0.1"
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:50010"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://localhost:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="staging" path="/projects/falcon/staging"/>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/projects/falcon/working"/>
+    </locations>
+    <properties>
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-feed1.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-feed1.xml b/process/src/test/resources/config/late/late-feed1.xml
index 10c1085..c500c4c 100644
--- a/process/src/test/resources/config/late/late-feed1.xml
+++ b/process/src/test/resources/config/late/late-feed1.xml
@@ -16,10 +16,11 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<feed description="clicks log" name="late-feed1" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="country" />
+<feed description="clicks log" name="late-feed1" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="country"/>
     </partitions>
 
     <groups>online,bi</groups>
@@ -31,21 +32,22 @@
     <clusters>
         <cluster name="late-cluster" type="source">
             <validity start="2011-11-01T00:00Z" end="2099-12-31T23:59Z"/>
-            <retention limit="hours(6)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 
     <properties>
-        <property name="field1" value="value1" />
-        <property name="field2" value="value2" />
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
     </properties>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-feed2.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-feed2.xml b/process/src/test/resources/config/late/late-feed2.xml
index 1456289..6ccffe2 100644
--- a/process/src/test/resources/config/late/late-feed2.xml
+++ b/process/src/test/resources/config/late/late-feed2.xml
@@ -16,10 +16,11 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<feed description="clicks log" name="late-feed2" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="country" />
+<feed description="clicks log" name="late-feed2" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="country"/>
     </partitions>
 
     <groups>online,bi</groups>
@@ -31,21 +32,22 @@
     <clusters>
         <cluster name="late-cluster" type="source">
             <validity start="2011-11-01T00:00Z" end="2099-12-31T23:59Z"/>
-            <retention limit="hours(6)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 
     <properties>
-        <property name="field1" value="value1" />
-        <property name="field2" value="value2" />
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
     </properties>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-feed3.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-feed3.xml b/process/src/test/resources/config/late/late-feed3.xml
index d33738a..239f140 100644
--- a/process/src/test/resources/config/late/late-feed3.xml
+++ b/process/src/test/resources/config/late/late-feed3.xml
@@ -16,36 +16,38 @@
   ~ See the License for the specific language governing permissions and
   ~ limitations under the License.
   -->
-<feed description="clicks log" name="late-feed3" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-   <partitions>
-        <partition name="fraud" />
-        <partition name="country" />
+<feed description="clicks log" name="late-feed3" xmlns="uri:falcon:feed:0.1"
+        >
+    <partitions>
+        <partition name="fraud"/>
+        <partition name="country"/>
     </partitions>
 
     <groups>online,bi</groups>
 
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    <late-arrival cut-off="hours(6)" />
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="late-cluster" type="source">
             <validity start="2011-11-01T00:00Z" end="2099-12-31T23:59Z"/>
-            <retention limit="hours(6)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/projects/falcon/clicks" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/projects/falcon/clicks"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 
     <properties>
-        <property name="field1" value="value1" />
-        <property name="field2" value="value2" />
+        <property name="field1" value="value1"/>
+        <property name="field2" value="value2"/>
     </properties>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/process/src/test/resources/config/late/late-process1.xml
----------------------------------------------------------------------
diff --git a/process/src/test/resources/config/late/late-process1.xml b/process/src/test/resources/config/late/late-process1.xml
index 9995164..aba5525 100644
--- a/process/src/test/resources/config/late/late-process1.xml
+++ b/process/src/test/resources/config/late/late-process1.xml
@@ -4,7 +4,7 @@
     License. You may obtain a copy of the License at ~ ~ http://www.apache.org/licenses/LICENSE-2.0 ~ ~ Unless required by applicable law or agreed to in writing, software ~ distributed 
     under the License is distributed on an "AS IS" BASIS, ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ~ See the License for the specific language governing 
     permissions and ~ limitations under the License. -->
-<process name="late-process1" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="late-process1" xmlns="uri:falcon:process:0.1">
     <!-- where -->
     <clusters>
         <cluster name="late-cluster">
@@ -17,24 +17,25 @@
     <execution>LIFO</execution>
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    
+
     <!-- what -->
     <inputs>
-        <input name="impression" feed="late-feed1" start-instance="today(0,0)" end-instance="today(0,2)" />
-        <input name="clicks" feed="late-feed2" start-instance="yesterday(0,0)" end-instance="today(0,0)" partition="*/US" />
+        <input name="impression" feed="late-feed1" start-instance="today(0,0)" end-instance="today(0,2)"/>
+        <input name="clicks" feed="late-feed2" start-instance="yesterday(0,0)" end-instance="today(0,0)"
+               partition="*/US"/>
     </inputs>
 
     <outputs>
-        <output name="clicksummary" feed="late-feed3" instance="today(0,0)" />
+        <output name="clicksummary" feed="late-feed3" instance="today(0,0)"/>
     </outputs>
 
     <!-- how -->
     <properties>
-        <property name="procprop" value="procprop" />
+        <property name="procprop" value="procprop"/>
     </properties>
 
-    <workflow engine="oozie" path="/user/guest/workflow" />
+    <workflow engine="oozie" path="/user/guest/workflow"/>
 
-    <retry policy="periodic" delay="hours(10)" attempts="3" />
+    <retry policy="periodic" delay="hours(10)" attempts="3"/>
 
 </process>


[24/47] git commit: Further fixes to client module for checkstyle

Posted by sr...@apache.org.
Further fixes to client module for checkstyle


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

Branch: refs/heads/master
Commit: b1fcaacccdc2574704c0631c8ab625ec590c04f2
Parents: 74a1c65
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 14:48:52 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 14:48:52 2013 +0530

----------------------------------------------------------------------
 .../src/main/resources/falcon/checkstyle.xml       |    7 ++++++-
 client/src/main/java/org/apache/falcon/Pair.java   |    3 +++
 .../org/apache/falcon/client/FalconClient.java     |    9 +++++++--
 .../java/org/apache/falcon/entity/v0/Entity.java   |    3 +--
 .../org/apache/falcon/resource/EntityList.java     |    2 ++
 .../apache/falcon/resource/InstancesResult.java    |    2 ++
 .../falcon/workflow/OozieFeedWorkflowBuilder.java  |    2 +-
 .../workflow/OozieProcessWorkflowBuilder.java      |    2 +-
 webapp/src/main/java/org/apache/falcon/Debug.java  |    2 +-
 9 files changed, 24 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/build-tools/src/main/resources/falcon/checkstyle.xml
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle.xml b/build-tools/src/main/resources/falcon/checkstyle.xml
index 9854d6c..641c5ff 100644
--- a/build-tools/src/main/resources/falcon/checkstyle.xml
+++ b/build-tools/src/main/resources/falcon/checkstyle.xml
@@ -222,5 +222,10 @@
     </module>
 
     <!-- allow warnings to be suppressed -->
-    <module name="SuppressionCommentFilter"/>
+    <module name="SuppressionCommentFilter">
+        <property name="offCommentFormat" value="SUSPEND CHECKSTYLE CHECK"/>
+        <property name="onCommentFormat" value="RESUME CHECKSTYLE CHECK"/>
+        <property name="checkFormat" value="ParameterNumberCheck|VisibilityModifierCheck|HiddenFieldCheck|EmptyBlockCheck"/>
+    </module>
+
 </module>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/client/src/main/java/org/apache/falcon/Pair.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/Pair.java b/client/src/main/java/org/apache/falcon/Pair.java
index 4c60feb..e6b44e9 100644
--- a/client/src/main/java/org/apache/falcon/Pair.java
+++ b/client/src/main/java/org/apache/falcon/Pair.java
@@ -24,8 +24,11 @@ package org.apache.falcon;
  * @param <B> - Second element in pair
  */
 public class Pair<A, B> {
+
+    //SUSPEND CHECKSTYLE CHECK
     public final A first;
     public final B second;
+    //RESUME CHECKSTYLE CHECK
 
     public Pair(A fst, B snd) {
         this.first = fst;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/client/src/main/java/org/apache/falcon/client/FalconClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/client/FalconClient.java b/client/src/main/java/org/apache/falcon/client/FalconClient.java
index 44e9ccd..30c0ec6 100644
--- a/client/src/main/java/org/apache/falcon/client/FalconClient.java
+++ b/client/src/main/java/org/apache/falcon/client/FalconClient.java
@@ -308,6 +308,7 @@ public class FalconClient {
                 getServletInputStream(clusters, sourceClusters, null), null, colo);
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     public String rerunInstances(String type, String entity, String start,
                                  String end, String filePath, String colo,
                                  String clusters, String sourceClusters)
@@ -326,6 +327,7 @@ public class FalconClient {
         return sendInstanceRequest(Instances.RERUN, type, entity, start, end,
                 getServletInputStream(clusters, sourceClusters, temp), null, colo);
     }
+    //RESUME CHECKSTYLE CHECK
 
     public String rerunInstances(String type, String entity, String start,
                                  String end, String colo, String clusters, String sourceClusters)
@@ -446,7 +448,7 @@ public class FalconClient {
     }
 
     private String sendListRequest(Entities entities, String entityType)
-            throws FalconCLIException {
+        throws FalconCLIException {
 
         ClientResponse clientResponse = service.path(entities.path)
                 .path(entityType).header(REMOTE_USER, USER)
@@ -501,6 +503,7 @@ public class FalconClient {
                 .method(instances.method, InstancesResult.class);
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     private String sendInstanceRequest(Instances instances, String type,
                                        String entity, String start, String end, InputStream props,
                                        String runid, String colo) throws FalconCLIException {
@@ -538,6 +541,7 @@ public class FalconClient {
         }
 
     }
+    //RESUME CHECKSTYLE CHECK
 
     private String sendAdminRequest(AdminOperations job)
         throws FalconCLIException {
@@ -669,7 +673,8 @@ public class FalconClient {
     }
 
     private void checkIfSuccessfull(ClientResponse clientResponse)
-            throws FalconCLIException {
+        throws FalconCLIException {
+
         if (clientResponse.getStatus() == Response.Status.BAD_REQUEST
                 .getStatusCode()) {
             throw FalconCLIException.fromReponse(clientResponse);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
index 1929429..252e860 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
@@ -88,8 +88,7 @@ public abstract class Entity {
         return "(" + getEntityType().name().toLowerCase() + ") " + getName();
     }
 
-    @Override
-    public Entity clone() {
+    public Entity copy() {
         return fromString(getEntityType(), toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/client/src/main/java/org/apache/falcon/resource/EntityList.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/EntityList.java b/client/src/main/java/org/apache/falcon/resource/EntityList.java
index 61110b2..933a737 100644
--- a/client/src/main/java/org/apache/falcon/resource/EntityList.java
+++ b/client/src/main/java/org/apache/falcon/resource/EntityList.java
@@ -39,10 +39,12 @@ public class EntityList {
      * Element within an entity.
      */
     public static class EntityElement {
+        //SUSPEND CHECKSTYLE CHECK
         @XmlElement
         public String type;
         @XmlElement
         public String name;
+        //RESUME CHECKSTYLE CHECK
 
         public EntityElement() {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
index cc737a5..65a8b65 100644
--- a/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
+++ b/client/src/main/java/org/apache/falcon/resource/InstancesResult.java
@@ -25,6 +25,7 @@ import java.util.Date;
 /**
  * Pojo for JAXB marshalling / unmarshalling.
  */
+//SUSPEND CHECKSTYLE CHECK
 @XmlRootElement
 public class InstancesResult extends APIResult {
 
@@ -198,3 +199,4 @@ public class InstancesResult extends APIResult {
         }
     }
 }
+//RESUME CHECKSTYLE CHECK

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
index 02cb740..14f39e9 100644
--- a/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
+++ b/feed/src/main/java/org/apache/falcon/workflow/OozieFeedWorkflowBuilder.java
@@ -63,7 +63,7 @@ public class OozieFeedWorkflowBuilder extends OozieWorkflowBuilder<Feed> {
 
         Cluster cluster = configStore.get(EntityType.CLUSTER, feedCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(feed));
-        Feed feedClone = (Feed) feed.clone();
+        Feed feedClone = (Feed) feed.copy();
         EntityUtil.setStartDate(feedClone, clusterName, startDate);
 
         AbstractOozieEntityMapper<Feed> mapper = new OozieFeedMapper(feedClone);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
index d5d097a..76b5cb4 100644
--- a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
+++ b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
@@ -107,7 +107,7 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
 
         Cluster cluster = configStore.get(EntityType.CLUSTER, processCluster.getName());
         Path bundlePath = new Path(ClusterHelper.getLocation(cluster, "staging"), EntityUtil.getStagingPath(process));
-        Process processClone = (Process) process.clone();
+        Process processClone = (Process) process.copy();
         EntityUtil.setStartDate(processClone, clusterName, startDate);
 
         OozieProcessMapper mapper = new OozieProcessMapper(processClone);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/b1fcaacc/webapp/src/main/java/org/apache/falcon/Debug.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/Debug.java b/webapp/src/main/java/org/apache/falcon/Debug.java
index 73c99ed..70aaa40 100644
--- a/webapp/src/main/java/org/apache/falcon/Debug.java
+++ b/webapp/src/main/java/org/apache/falcon/Debug.java
@@ -71,7 +71,7 @@ public class Debug {
 
         entity = args[2];
         Entity obj = EntityUtil.getEntity(type, entity);
-        Process newEntity = (Process) obj.clone();
+        Process newEntity = (Process) obj.copy();
         newEntity.setFrequency(Frequency.fromString("minutes(5)"));
         System.out.println("##############OLD ENTITY " + EntityUtil.md5(obj));
         System.out.println("##############NEW ENTITY " + EntityUtil.md5(newEntity));


[47/47] git commit: checkstyle related fixes for webapp module.

Posted by sr...@apache.org.
checkstyle related fixes for webapp module.


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

Branch: refs/heads/master
Commit: f15ef92a8d70d968f727a6b50b681d33ec58d78a
Parents: e997931
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:28:18 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:28:18 2013 -0700

----------------------------------------------------------------------
 webapp/src/main/java/org/apache/falcon/Debug.java  |   19 +++--
 webapp/src/main/java/org/apache/falcon/Main.java   |   11 +++-
 .../apache/falcon/resource/ConfigSyncService.java  |    4 +-
 .../apache/falcon/resource/InstanceManager.java    |    3 +
 .../falcon/resource/SchedulableEntityManager.java  |    7 +-
 .../java/org/apache/falcon/cli/FalconCLITest.java  |   54 ++++++++-------
 .../org/apache/falcon/logging/LogMoverTest.java    |   24 +++---
 .../org/apache/falcon/logging/LogProviderTest.java |   41 ++++++-----
 .../apache/falcon/resource/AbstractTestBase.java   |   28 ++++---
 .../falcon/resource/EntityManagerJerseyTest.java   |   47 ++++++-------
 .../resource/ProcessInstanceManagerTest.java       |    3 +
 .../falcon/security/BasicAuthFilterTest.java       |    3 +
 .../falcon/util/ResourcesReflectionUtilTest.java   |    3 +
 13 files changed, 141 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/main/java/org/apache/falcon/Debug.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/Debug.java b/webapp/src/main/java/org/apache/falcon/Debug.java
index 70aaa40..2710b27 100644
--- a/webapp/src/main/java/org/apache/falcon/Debug.java
+++ b/webapp/src/main/java/org/apache/falcon/Debug.java
@@ -30,17 +30,22 @@ import org.apache.falcon.service.Services;
 import org.apache.falcon.util.DeploymentProperties;
 import org.apache.falcon.util.StartupProperties;
 import org.apache.falcon.workflow.engine.OozieWorkflowEngine;
-import org.apache.log4j.Logger;
 
 import javax.xml.bind.JAXBException;
 import javax.xml.bind.Unmarshaller;
 import java.io.ByteArrayInputStream;
-import java.text.SimpleDateFormat;
 
-public class Debug {
-    private static final Logger LOG = Logger.getLogger(Debug.class);
+/**
+ * A driver for debugging purposes.
+ */
+public final class Debug {
+    // private static final SimpleDateFormat FORMATTER = new SimpleDateFormat("yyyy-MM-dd HH:mm Z");
 
-    private static final SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm Z");
+    /**
+     * Prevent users from constructing this.
+     */
+    private Debug() {
+    }
 
     public static void main(String[] args) throws Exception {
         String falconUrl = args[0];
@@ -78,8 +83,8 @@ public class Debug {
 
 
 //        OozieWorkflowEngine engine = new OozieWorkflowEngine();
-//        Date start = formatter.parse("2010-01-02 01:05 UTC");
-//        Date end = formatter.parse("2010-01-02 01:21 UTC");
+//        Date start = FORMATTER.parse("2010-01-02 01:05 UTC");
+//        Date end = FORMATTER.parse("2010-01-02 01:21 UTC");
 //        InstancesResult status = engine.suspendInstances(obj, start, end, new Properties());
 //        System.out.println(Arrays.toString(status.getInstances()));
 //        AbstractInstanceManager manager = new InstanceManager();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/main/java/org/apache/falcon/Main.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/Main.java b/webapp/src/main/java/org/apache/falcon/Main.java
index 3726559..609cbfc 100644
--- a/webapp/src/main/java/org/apache/falcon/Main.java
+++ b/webapp/src/main/java/org/apache/falcon/Main.java
@@ -21,7 +21,16 @@ package org.apache.falcon;
 import org.apache.activemq.broker.BrokerService;
 import org.apache.falcon.util.EmbeddedServer;
 
-public class Main {
+/**
+ * Driver for running Falcon as a standalone server with embedded jetty server.
+ */
+public final class Main {
+
+    /**
+     * Prevent users from constructing this.
+     */
+    private Main() {
+    }
 
     public static void main(String[] args) throws Exception {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java b/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
index 248c2a0..9722116 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
@@ -26,6 +26,9 @@ import javax.ws.rs.*;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 
+/**
+ * This class provides RESTful API for Entity Configurations.
+ */
 @Path("sync")
 public class ConfigSyncService extends AbstractEntityManager {
 
@@ -64,5 +67,4 @@ public class ConfigSyncService extends AbstractEntityManager {
                             @Dimension("colo") @QueryParam("colo") String colo) {
         return super.update(request, type, entityName, colo);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
index 9f789a8..104dfef 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
@@ -26,6 +26,9 @@ import javax.ws.rs.*;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 
+/**
+ * This class provides RESTful API for the lifecycle management of the entity instances.
+ */
 @Path("instance")
 public class InstanceManager extends AbstractInstanceManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
index 02e6614..96f5db4 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
@@ -18,7 +18,6 @@
 
 package org.apache.falcon.resource;
 
-import org.apache.falcon.FalconWebException;
 import org.apache.falcon.monitors.Dimension;
 import org.apache.falcon.monitors.Monitored;
 
@@ -27,6 +26,9 @@ import javax.ws.rs.*;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 
+/**
+ * Entity management operations as REST API for feed and process.
+ */
 @Path("entities")
 public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
@@ -37,8 +39,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
     @Override
     public APIResult getStatus(@Dimension("entityType") @PathParam("type") String type,
                                @Dimension("entityName") @PathParam("entity") String entity,
-                               @Dimension("colo") @QueryParam("colo") final String colo)
-            throws FalconWebException {
+                               @Dimension("colo") @QueryParam("colo") final String colo) {
         return super.getStatus(type, entity, colo);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java b/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
index a59994a..64f0171 100644
--- a/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
+++ b/webapp/src/test/java/org/apache/falcon/cli/FalconCLITest.java
@@ -25,15 +25,19 @@ import org.testng.annotations.Test;
 import java.io.*;
 import java.util.Map;
 
-//Refactor both the classes to move this methods to helper;
+/**
+ * Test for Falcon CLI.
+ *
+ * todo: Refactor both the classes to move this methods to helper;
+ */
 public class FalconCLITest extends AbstractTestBase {
 
     private InMemoryWriter stream = new InMemoryWriter(System.out);
     // private static final String BROKER_URL =
     // "tcp://localhost:61616?daemon=true";
-    private static final boolean enableTest = true;
+    private static final boolean TEST_ENABLED = true;
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testSubmitEntityValidCommands() throws Exception {
 
         FalconCLI.OUT.set(stream);
@@ -41,7 +45,7 @@ public class FalconCLITest extends AbstractTestBase {
         String filePath;
         Map<String, String> overlay = getUniqueOverlay();
 
-        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        filePath = overlayParametersOverTemplate(clusterFileTemplate, overlay);
         Assert.assertEquals(
                 0,
                 executeWithURL("entity -submit -type cluster -file " + filePath));
@@ -74,21 +78,21 @@ public class FalconCLITest extends AbstractTestBase {
                         + overlay.get("processName"));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testListWithEmptyConfigStore() throws Exception {
         Assert.assertEquals(
                 0,
                 executeWithURL("entity -list -type process "));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testSubmitAndScheduleEntityValidCommands() throws Exception {
 
         Thread.sleep(5000);
         String filePath;
         Map<String, String> overlay = getUniqueOverlay();
 
-        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        filePath = overlayParametersOverTemplate(clusterFileTemplate, overlay);
         Assert.assertEquals(-1,
                 executeWithURL("entity -submitAndSchedule -type cluster -file "
                         + filePath));
@@ -117,13 +121,13 @@ public class FalconCLITest extends AbstractTestBase {
         Thread.sleep(5000);
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testValidateValidCommands() throws Exception {
 
         String filePath;
         Map<String, String> overlay = getUniqueOverlay();
 
-        filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+        filePath = overlayParametersOverTemplate(clusterFileTemplate, overlay);
         Assert.assertEquals(0,
                 executeWithURL("entity -validate -type cluster -file "
                         + filePath));
@@ -153,7 +157,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testDefinitionEntityValidCommands() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -178,7 +182,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testScheduleEntityValidCommands() throws Exception {
 
         Map<String, String> overlay = getUniqueOverlay();
@@ -199,7 +203,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testSuspendResumeStatusEntityValidCommands() throws Exception {
 
         Thread.sleep(5000);
@@ -270,12 +274,12 @@ public class FalconCLITest extends AbstractTestBase {
         Thread.sleep(5000);
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testSubCommandPresence() throws Exception {
         Assert.assertEquals(-1, executeWithURL("entity -type cluster "));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testDeleteEntityValidCommands() throws Exception {
 
         Map<String, String> overlay = getUniqueOverlay();
@@ -313,7 +317,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInvalidCLIEntitycommands() throws Exception {
 
         Map<String, String> overlay = getUniqueOverlay();
@@ -325,7 +329,7 @@ public class FalconCLITest extends AbstractTestBase {
                 executeWithURL("entity -schedule -type feed -file " + "name"));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInstanceRunningAndStatusCommands() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -354,7 +358,7 @@ public class FalconCLITest extends AbstractTestBase {
                         + " -start " + START_INSTANCE));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInstanceSuspendAndResume() throws Exception {
         Thread.sleep(5000);
         Map<String, String> overlay = getUniqueOverlay();
@@ -379,7 +383,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     private static final String START_INSTANCE = "2012-04-20T00:00Z";
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInstanceKillAndRerun() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -403,7 +407,7 @@ public class FalconCLITest extends AbstractTestBase {
                         + createTempJobPropertiesFile()));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testContinue() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -426,7 +430,7 @@ public class FalconCLITest extends AbstractTestBase {
                         + " -start " + START_INSTANCE));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInvalidCLIInstanceCommands() throws Exception {
         // no command
         Assert.assertEquals(-1, executeWithURL(" -kill -type process -name "
@@ -445,7 +449,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testFalconURL() throws Exception {
         Assert.assertEquals(-1, new FalconCLI()
                 .run(("instance -status -type process -name " + "processName"
@@ -461,7 +465,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testClientProperties() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -477,7 +481,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testGetVersion() throws Exception {
         Assert.assertEquals(0,
                 new FalconCLI().run("admin -version".split("\\s")));
@@ -486,7 +490,7 @@ public class FalconCLITest extends AbstractTestBase {
                 new FalconCLI().run("admin -stack".split("\\s")));
     }
 
-    @Test(enabled = enableTest)
+    @Test(enabled = TEST_ENABLED)
     public void testInstanceGetLogs() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         submitTestFiles(overlay);
@@ -522,7 +526,7 @@ public class FalconCLITest extends AbstractTestBase {
 
     public void submitTestFiles(Map<String, String> overlay) throws Exception {
 
-        String filePath = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE,
+        String filePath = overlayParametersOverTemplate(clusterFileTemplate,
                 overlay);
         Assert.assertEquals(
                 0,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java b/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
index 15b07fe..bd6dbe2 100644
--- a/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
+++ b/webapp/src/test/java/org/apache/falcon/logging/LogMoverTest.java
@@ -41,22 +41,22 @@ import java.util.Collection;
 import java.util.List;
 
 /**
- * Requires Oozie to be running on localhost
+ * Test for LogMover.
+ * Requires Oozie to be running on localhost.
  */
 public class LogMoverTest {
 
-    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static final ConfigurationStore STORE = ConfigurationStore.get();
+    private static final String PROCESS_NAME = "testProcess" + System.currentTimeMillis();
     private static EmbeddedCluster testCluster = null;
     private static Process testProcess = null;
-    private static String processName = "testProcess"
-            + System.currentTimeMillis();
-    FileSystem fs;
+    private static FileSystem fs;
 
     @BeforeClass
     public void setup() throws Exception {
         cleanupStore();
         testCluster = EmbeddedCluster.newCluster("testCluster", true);
-        store.publish(EntityType.CLUSTER, testCluster.getCluster());
+        STORE.publish(EntityType.CLUSTER, testCluster.getCluster());
         SharedLibraryHostingService listener = new SharedLibraryHostingService();
         listener.onAdd(testCluster.getCluster());
         fs = FileSystem.get(testCluster.getConf());
@@ -73,8 +73,8 @@ public class LogMoverTest {
 
         testProcess = new ProcessEntityParser().parse(LogMoverTest.class
                 .getResourceAsStream("/org/apache/falcon/logging/process.xml"));
-        testProcess.setName(processName);
-        store.publish(EntityType.PROCESS, testProcess);
+        testProcess.setName(PROCESS_NAME);
+        STORE.publish(EntityType.PROCESS, testProcess);
     }
 
     @AfterClass
@@ -84,9 +84,9 @@ public class LogMoverTest {
 
     private void cleanupStore() throws FalconException {
         for (EntityType type : EntityType.values()) {
-            Collection<String> entities = store.getEntities(type);
+            Collection<String> entities = STORE.getEntities(type);
             for (String entity : entities) {
-                store.remove(type, entity);
+                STORE.remove(type, entity);
             }
         }
     }
@@ -102,7 +102,7 @@ public class LogMoverTest {
         List<WorkflowJob> jobs;
         while (true) {
             jobs = client.getJobsInfo(OozieClient.FILTER_NAME + "="
-                    + "FALCON_PROCESS_DEFAULT_" + processName);
+                    + "FALCON_PROCESS_DEFAULT_" + PROCESS_NAME);
             if (jobs.size() > 0) {
                 break;
             } else {
@@ -143,7 +143,7 @@ public class LogMoverTest {
                                    ClusterHelper.getOozieUrl(testCluster.getCluster()),
                                    "-subflowId", jobId + "@user-workflow", "-runId", "1",
                                    "-logDir", getLogPath().toString() + "/job-2010-01-01-01-00",
-                                   "-status", "SUCCEEDED", "-entityType", "process"});
+                                   "-status", "SUCCEEDED", "-entityType", "process", });
 
         Path oozieLogPath = new Path(getLogPath(),
                 "job-2010-01-01-01-00/001/oozie.log");

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java b/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
index ecb14ed..a3ebaf0 100644
--- a/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
+++ b/webapp/src/test/java/org/apache/falcon/logging/LogProviderTest.java
@@ -35,12 +35,15 @@ import org.testng.annotations.Test;
 
 import java.util.Collection;
 
+/**
+ * Test for LogProvider.
+ */
 public class LogProviderTest {
 
-    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static final ConfigurationStore STORE = ConfigurationStore.get();
     private static EmbeddedCluster testCluster = null;
     private static Process testProcess = null;
-    private static String processName = "testProcess";
+    private static final String PROCESS_NAME = "testProcess";
     private static FileSystem fs;
     private Instance instance;
 
@@ -48,10 +51,10 @@ public class LogProviderTest {
     public void setup() throws Exception {
         testCluster = EmbeddedCluster.newCluster("testCluster", false);
         cleanupStore();
-        store.publish(EntityType.CLUSTER, testCluster.getCluster());
+        STORE.publish(EntityType.CLUSTER, testCluster.getCluster());
         fs = FileSystem.get(testCluster.getConf());
         Path instanceLogPath = new Path(
-                "/workflow/staging/falcon/workflows/process/" + processName
+                "/workflow/staging/falcon/workflows/process/" + PROCESS_NAME
                         + "/logs/job-2010-01-01-01-00/000");
         fs.mkdirs(instanceLogPath);
         fs.createNewFile(new Path(instanceLogPath, "oozie.log"));
@@ -60,18 +63,18 @@ public class LogProviderTest {
         fs.createNewFile(new Path(instanceLogPath, "mr_Action2_SUCCEEDED.log"));
 
         fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
-                + processName + "/logs/job-2010-01-01-01-00/001"));
+                + PROCESS_NAME + "/logs/job-2010-01-01-01-00/001"));
         fs.mkdirs(new Path("/workflow/staging/falcon/workflows/process/"
-                + processName + "/logs/job-2010-01-01-01-00/002"));
+                + PROCESS_NAME + "/logs/job-2010-01-01-01-00/002"));
         Path run3 = new Path("/workflow/staging/falcon/workflows/process/"
-                + processName + "/logs/job-2010-01-01-01-00/003");
+                + PROCESS_NAME + "/logs/job-2010-01-01-01-00/003");
         fs.mkdirs(run3);
         fs.createNewFile(new Path(run3, "oozie.log"));
 
         testProcess = new ProcessEntityParser().parse(LogMoverTest.class
                 .getResourceAsStream("/org/apache/falcon/logging/process.xml"));
-        testProcess.setName(processName);
-        store.publish(EntityType.PROCESS, testProcess);
+        testProcess.setName(PROCESS_NAME);
+        STORE.publish(EntityType.PROCESS, testProcess);
     }
 
     @BeforeMethod
@@ -85,9 +88,9 @@ public class LogProviderTest {
 
     private void cleanupStore() throws FalconException {
         for (EntityType type : EntityType.values()) {
-            Collection<String> entities = store.getEntities(type);
+            Collection<String> entities = STORE.getEntities(type);
             for (String entity : entities) {
-                store.remove(type, entity);
+                STORE.remove(type, entity);
             }
         }
     }
@@ -99,25 +102,24 @@ public class LogProviderTest {
                 instance, "0");
         Assert.assertEquals(
                 instance.logFile,
-                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
-                        "-01-01-00/000/oozie.log");
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01"
+                        + "-01-01-00/000/oozie.log");
 
         InstanceAction action = instanceWithLog.actions[0];
         Assert.assertEquals(action.action, "mr_Action2");
         Assert.assertEquals(action.status, "SUCCEEDED");
         Assert.assertEquals(
                 action.logFile,
-                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
-                        "-01-01-00/000/mr_Action2_SUCCEEDED.log");
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01"
+                        + "-01-01-00/000/mr_Action2_SUCCEEDED.log");
 
         action = instanceWithLog.actions[1];
         Assert.assertEquals(action.action, "mr_Action");
         Assert.assertEquals(action.status, "FAILED");
         Assert.assertEquals(
                 action.logFile,
-                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01" +
-                        "-01-01-00/000/mr_Action_FAILED.log");
-
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01"
+                        + "-01-01-00/000/mr_Action_FAILED.log");
     }
 
     @Test
@@ -143,6 +145,7 @@ public class LogProviderTest {
         provider.populateLogUrls(testProcess, instance, null);
         Assert.assertEquals(
                 instance.logFile,
-                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/job-2010-01-01-01-00/003/oozie.log");
+                "http://localhost:50070/data/workflow/staging/falcon/workflows/process/testProcess/logs/"
+                        + "job-2010-01-01-01-00/003/oozie.log");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java b/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
index de9958d..3ddd282 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/AbstractTestBase.java
@@ -58,10 +58,14 @@ import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+/**
+ * Base test class for CLI, Entity and Process Instances.
+ */
 public class AbstractTestBase {
     protected static final String FEED_TEMPLATE1 = "/feed-template1.xml";
     protected static final String FEED_TEMPLATE2 = "/feed-template2.xml";
-    protected String CLUSTER_FILE_TEMPLATE = "/cluster-template.xml";
+
+    protected String clusterFileTemplate = "/cluster-template.xml";
 
     protected static final String SAMPLE_PROCESS_XML = "/process-version-0.xml";
     protected static final String PROCESS_TEMPLATE = "/process-template.xml";
@@ -80,10 +84,10 @@ public class AbstractTestBase {
     protected String processName;
     protected String outputFeedName;
 
-    private static final Pattern varPattern = Pattern.compile("##[A-Za-z0-9_]*##");
+    private static final Pattern VAR_PATTERN = Pattern.compile("##[A-Za-z0-9_]*##");
 
     protected void scheduleProcess(String processTemplate, Map<String, String> overlay) throws Exception {
-        ClientResponse response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        ClientResponse response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -156,7 +160,7 @@ public class AbstractTestBase {
                         done = true;
                     }
                 }
-                if (done == true) {
+                if (done) {
                     return;
                 }
             }
@@ -198,16 +202,16 @@ public class AbstractTestBase {
         this.server.start();
 
         if (System.getProperty("falcon.test.hadoop.embedded", "true").equals("true")) {
-            CLUSTER_FILE_TEMPLATE = "target/cluster-template.xml";
+            clusterFileTemplate = "target/cluster-template.xml";
             this.cluster = EmbeddedCluster.newCluster("##cluster##", true);
             Cluster clusterEntity = this.cluster.getCluster();
-            FileOutputStream out = new FileOutputStream(CLUSTER_FILE_TEMPLATE);
+            FileOutputStream out = new FileOutputStream(clusterFileTemplate);
             marshaller.marshal(clusterEntity, out);
             out.close();
         } else {
             Map<String, String> overlay = new HashMap<String, String>();
             overlay.put("cluster", RandomStringUtils.randomAlphabetic(5));
-            String file = overlayParametersOverTemplate(CLUSTER_FILE_TEMPLATE, overlay);
+            String file = overlayParametersOverTemplate(clusterFileTemplate, overlay);
             this.cluster = StandAloneCluster.newCluster(file);
             clusterName = cluster.getCluster().getName();
         }
@@ -232,7 +236,7 @@ public class AbstractTestBase {
     }
 
     /**
-     * Converts a InputStream into ServletInputStream
+     * Converts a InputStream into ServletInputStream.
      *
      * @param fileName
      * @return ServletInputStream
@@ -266,7 +270,7 @@ public class AbstractTestBase {
     }
 
     protected ClientResponse submitAndSchedule(String template, Map<String, String> overlay, EntityType entityType)
-            throws Exception {
+        throws Exception {
         String tmpFile = overlayParametersOverTemplate(template, overlay);
         ServletInputStream rawlogStream = getServletInputStream(tmpFile);
 
@@ -276,7 +280,7 @@ public class AbstractTestBase {
     }
 
     protected ClientResponse submitToFalcon(String template, Map<String, String> overlay, EntityType entityType)
-            throws IOException {
+        throws IOException {
         String tmpFile = overlayParametersOverTemplate(template, overlay);
         return submitFileToFalcon(entityType, tmpFile);
     }
@@ -333,11 +337,11 @@ public class AbstractTestBase {
         BufferedReader reader = new BufferedReader(in);
         String line;
         while ((line = reader.readLine()) != null) {
-            Matcher matcher = varPattern.matcher(line);
+            Matcher matcher = VAR_PATTERN.matcher(line);
             while (matcher.find()) {
                 String variable = line.substring(matcher.start(), matcher.end());
                 line = line.replace(variable, overlay.get(variable.substring(2, variable.length() - 2)));
-                matcher = varPattern.matcher(line);
+                matcher = VAR_PATTERN.matcher(line);
             }
             out.write(line.getBytes());
             out.write("\n".getBytes());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
index 39fa823..b2d234d 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
@@ -18,7 +18,6 @@
 package org.apache.falcon.resource;
 
 import com.sun.jersey.api.client.ClientResponse;
-import org.apache.falcon.FalconWebException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Input;
@@ -27,7 +26,6 @@ import org.apache.falcon.entity.v0.process.Property;
 import org.apache.falcon.entity.v0.process.Validity;
 import org.apache.falcon.util.BuildProperties;
 import org.apache.falcon.util.DeploymentProperties;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
@@ -48,11 +46,12 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.*;
 
+/**
+ * Test class for Entity REST APIs.
+ *
+ * Tests should be enabled only in local environments as they need running instance of the web server.
+ */
 public class EntityManagerJerseyTest extends AbstractTestBase {
-    /**
-     * Tests should be enabled only in local environments as they need running
-     * instance of webserver
-     */
 
     @Test
     public void testUpdateCheckUser() throws Exception {
@@ -216,7 +215,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -238,15 +237,15 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
     }
 
     @Test
-    public void testNotFoundStatus() throws FalconWebException {
+    public void testNotFoundStatus() {
         ClientResponse response;
         String feed1 = "f1" + System.currentTimeMillis();
         response = this.service
@@ -255,21 +254,19 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
         String status = response.getEntity(String.class);
         Assert.assertEquals(response.getStatus(), Response.Status.BAD_REQUEST.getStatusCode());
-
     }
 
     @Test
-    public void testVersion() throws FalconWebException {
+    public void testVersion() {
         ClientResponse response;
         response = this.service
                 .path("api/admin/version")
                 .header("Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
         String status = response.getEntity(String.class);
-        Assert.assertEquals(status, "{Version:\"" +
-                BuildProperties.get().getProperty("build.version") + "\",Mode:\"" +
-                DeploymentProperties.get().getProperty("deploy.mode") + "\"}");
-
+        Assert.assertEquals(status, "{Version:\""
+                + BuildProperties.get().getProperty("build.version") + "\",Mode:\""
+                + DeploymentProperties.get().getProperty("deploy.mode") + "\"}");
     }
 
     @Test
@@ -293,7 +290,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         Map<String, String> overlay = getUniqueOverlay();
 
         InputStream stream = getServletInputStream(overlayParametersOverTemplate(
-                CLUSTER_FILE_TEMPLATE, overlay));
+                clusterFileTemplate, overlay));
 
         clientRepsonse = this.service.path("api/entities/validate/cluster")
                 .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
@@ -307,7 +304,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse clientRepsonse;
         Map<String, String> overlay = getUniqueOverlay();
 
-        clientRepsonse = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay,
+        clientRepsonse = submitToFalcon(clusterFileTemplate, overlay,
                 EntityType.CLUSTER);
         assertSuccessful(clientRepsonse);
 
@@ -345,7 +342,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -363,7 +360,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -415,7 +412,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -484,7 +481,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -503,7 +500,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = submitToFalcon(FEED_TEMPLATE1, overlay, EntityType.FEED);
@@ -551,7 +548,6 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
                 .header("Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).delete(ClientResponse.class);
         assertSuccessful(response);
-
     }
 
     @Test
@@ -565,7 +561,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
 
         Map<String, String> overlay = getUniqueOverlay();
 
-        response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
+        response = submitToFalcon(clusterFileTemplate, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
 
         response = this.service
@@ -573,6 +569,5 @@ public class EntityManagerJerseyTest extends AbstractTestBase {
                 .header("Remote-User", REMOTE_USER).type(MediaType.TEXT_XML)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
         Assert.assertEquals(response.getStatus(), 200);
-
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
index 6ecc063..c2f6b51 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
@@ -34,6 +34,9 @@ import org.testng.annotations.Test;
 
 import javax.ws.rs.core.MediaType;
 
+/**
+ * Test class for Process Instance REST API.
+ */
 @Test(enabled = false)
 public class ProcessInstanceManagerTest extends AbstractTestBase {
     private static final String START_INSTANCE = "2012-04-20T00:00Z";

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java b/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
index 238bfc4..0ff993b 100644
--- a/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
+++ b/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
@@ -34,6 +34,9 @@ import javax.ws.rs.core.Response;
 
 import static org.mockito.Mockito.*;
 
+/**
+ * Test for BasicAuthFilter using mock objects.
+ */
 public class BasicAuthFilterTest {
 
     @Mock

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f15ef92a/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java b/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
index 96d5c53..f2ce0c2 100644
--- a/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
+++ b/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
@@ -21,6 +21,9 @@ package org.apache.falcon.util;
 import junit.framework.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Test class for org.apache.falcon.util.ResourcesReflectionUtil.
+ */
 public class ResourcesReflectionUtilTest {
 
     @Test


[18/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
index 65849f3..f0ef515 100644
--- a/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
+++ b/common/src/main/java/org/apache/falcon/entity/EntityUtil.java
@@ -18,26 +18,9 @@
 
 package org.apache.falcon.entity;
 
-import java.lang.reflect.Method;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TimeZone;
-
 import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.WorkflowNameBuilder.WorkflowName;
@@ -49,57 +32,61 @@ import org.apache.falcon.entity.v0.SchemaHelper;
 import org.apache.falcon.entity.v0.feed.Cluster;
 import org.apache.falcon.entity.v0.feed.ClusterType;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.LateInput;
-import org.apache.falcon.entity.v0.process.LateProcess;
-import org.apache.falcon.entity.v0.process.PolicyType;
+import org.apache.falcon.entity.v0.process.*;
 import org.apache.falcon.entity.v0.process.Process;
-import org.apache.falcon.entity.v0.process.Retry;
 import org.apache.falcon.util.DeploymentUtil;
 import org.apache.falcon.util.RuntimeProperties;
+import org.apache.hadoop.fs.Path;
+
+import java.lang.reflect.Method;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.*;
 
 public class EntityUtil {
-	private static final long MINUTE_IN_MS = 60000L;
-	private static final long HOUR_IN_MS = 3600000L;
-	private static final long DAY_IN_MS = 86400000L;
-	private static final long MONTH_IN_MS = 2592000000L;
-
-	public static <T extends Entity> T getEntity(EntityType type, String entityName) throws FalconException {
-		ConfigurationStore configStore = ConfigurationStore.get();
-		T entity = configStore.get(type, entityName);
-		if (entity == null) {
-			throw new EntityNotRegisteredException(entityName + " (" + type + ") not found");
-		}
-		return entity;        
-	}
-
-	public static <T extends Entity> T getEntity(String type, String entityName) throws FalconException {
-		EntityType entityType;
-		try {
-			entityType = EntityType.valueOf(type.toUpperCase());
-		} catch (IllegalArgumentException e) {
-			throw new FalconException("Invalid entity type: " + type, e);
-		}
-		return getEntity(entityType, entityName);
-	}
-
-	public static TimeZone getTimeZone(String tzId) {
-		if (tzId == null) {
-			throw new IllegalArgumentException("Invalid TimeZone: " + tzId);
-		}
-		TimeZone tz = TimeZone.getTimeZone(tzId);
-		if (!tzId.equals("GMT") && tz.getID().equals("GMT")) {
-			throw new IllegalArgumentException("Invalid TimeZone: " + tzId);
-		}
-		return tz;
-	}
-
-	public static Date getEndTime(Entity entity, String cluster) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			return getEndTime((Process) entity, cluster);
-		} else {
-			return getEndTime((Feed) entity, cluster);
-		}
-	}
+    private static final long MINUTE_IN_MS = 60000L;
+    private static final long HOUR_IN_MS = 3600000L;
+    private static final long DAY_IN_MS = 86400000L;
+    private static final long MONTH_IN_MS = 2592000000L;
+
+    public static <T extends Entity> T getEntity(EntityType type, String entityName) throws FalconException {
+        ConfigurationStore configStore = ConfigurationStore.get();
+        T entity = configStore.get(type, entityName);
+        if (entity == null) {
+            throw new EntityNotRegisteredException(entityName + " (" + type + ") not found");
+        }
+        return entity;
+    }
+
+    public static <T extends Entity> T getEntity(String type, String entityName) throws FalconException {
+        EntityType entityType;
+        try {
+            entityType = EntityType.valueOf(type.toUpperCase());
+        } catch (IllegalArgumentException e) {
+            throw new FalconException("Invalid entity type: " + type, e);
+        }
+        return getEntity(entityType, entityName);
+    }
+
+    public static TimeZone getTimeZone(String tzId) {
+        if (tzId == null) {
+            throw new IllegalArgumentException("Invalid TimeZone: " + tzId);
+        }
+        TimeZone tz = TimeZone.getTimeZone(tzId);
+        if (!tzId.equals("GMT") && tz.getID().equals("GMT")) {
+            throw new IllegalArgumentException("Invalid TimeZone: " + tzId);
+        }
+        return tz;
+    }
+
+    public static Date getEndTime(Entity entity, String cluster) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            return getEndTime((Process) entity, cluster);
+        } else {
+            return getEndTime((Feed) entity, cluster);
+        }
+    }
 
     public static Date parseDateUTC(String dateStr) throws FalconException {
         try {
@@ -109,100 +96,100 @@ public class EntityUtil {
         }
     }
 
-	public static Date getStartTime(Entity entity, String cluster) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			return getStartTime((Process) entity, cluster);
-		} else {
-			return getStartTime((Feed) entity, cluster);
-		}
-	}
-
-	public static Date getEndTime(Process process, String cluster) {
-		org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
-		return processCluster.getValidity().getEnd();
-	}
-
-	public static Date getStartTime(Process process, String cluster) {
-		org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
-		return processCluster.getValidity().getStart();
-	}
-
-	public static Date getEndTime(Feed feed, String cluster) {
-		org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
-		return clusterDef.getValidity().getEnd();
-	}
-
-	public static Date getStartTime(Feed feed, String cluster) {
-		org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
-		return clusterDef.getValidity().getStart();
-	}
-
-	public static int getParallel(Entity entity) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			return getParallel((Process) entity);
-		} else {
-			return getParallel((Feed) entity);
-		}
-	}
-
-	public static void setStartDate(Entity entity, String cluster, Date startDate) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			setStartDate((Process) entity, cluster, startDate);
-		} else {
-			setStartDate((Feed) entity, cluster, startDate);
-		}
-	}
-
-	public static void setEndTime(Entity entity, String cluster, Date endDate) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			setEndTime((Process) entity, cluster, endDate);
-		} else {
-			setEndTime((Feed) entity, cluster, endDate);
-		}
-	}
-
-	public static void setParallel(Entity entity, int parallel) {
-		if (entity.getEntityType() == EntityType.PROCESS) {
-			setParallel((Process) entity, parallel);
-		} else {
-			setParallel((Feed) entity, parallel);
-		}
-	}
-
-	public static int getParallel(Process process) {
-		return process.getParallel();
-	}
-
-	public static void setStartDate(Process process, String cluster, Date startDate) {
-		org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
-		processCluster.getValidity().setStart(startDate);
-	}
-
-	public static void setParallel(Process process, int parallel) {
-		process.setParallel(parallel);
-	}
-
-	public static void setEndTime(Process process, String cluster, Date endDate) {
-		org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
-		processCluster.getValidity().setEnd(endDate);
-	}
-
-	public static int getParallel(Feed feed) {
-		return 1;
-	}
-
-	public static void setStartDate(Feed feed, String cluster, Date startDate) {
-		org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
-		clusterDef.getValidity().setStart(startDate);
-	}
-
-	public static void setEndTime(Feed feed, String cluster, Date endDate) {
-		org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
-		clusterDef.getValidity().setStart(endDate);
-	}
-
-	public static void setParallel(Feed feed, int parallel) {
-	}
+    public static Date getStartTime(Entity entity, String cluster) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            return getStartTime((Process) entity, cluster);
+        } else {
+            return getStartTime((Feed) entity, cluster);
+        }
+    }
+
+    public static Date getEndTime(Process process, String cluster) {
+        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
+        return processCluster.getValidity().getEnd();
+    }
+
+    public static Date getStartTime(Process process, String cluster) {
+        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
+        return processCluster.getValidity().getStart();
+    }
+
+    public static Date getEndTime(Feed feed, String cluster) {
+        org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
+        return clusterDef.getValidity().getEnd();
+    }
+
+    public static Date getStartTime(Feed feed, String cluster) {
+        org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
+        return clusterDef.getValidity().getStart();
+    }
+
+    public static int getParallel(Entity entity) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            return getParallel((Process) entity);
+        } else {
+            return getParallel((Feed) entity);
+        }
+    }
+
+    public static void setStartDate(Entity entity, String cluster, Date startDate) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            setStartDate((Process) entity, cluster, startDate);
+        } else {
+            setStartDate((Feed) entity, cluster, startDate);
+        }
+    }
+
+    public static void setEndTime(Entity entity, String cluster, Date endDate) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            setEndTime((Process) entity, cluster, endDate);
+        } else {
+            setEndTime((Feed) entity, cluster, endDate);
+        }
+    }
+
+    public static void setParallel(Entity entity, int parallel) {
+        if (entity.getEntityType() == EntityType.PROCESS) {
+            setParallel((Process) entity, parallel);
+        } else {
+            setParallel((Feed) entity, parallel);
+        }
+    }
+
+    public static int getParallel(Process process) {
+        return process.getParallel();
+    }
+
+    public static void setStartDate(Process process, String cluster, Date startDate) {
+        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
+        processCluster.getValidity().setStart(startDate);
+    }
+
+    public static void setParallel(Process process, int parallel) {
+        process.setParallel(parallel);
+    }
+
+    public static void setEndTime(Process process, String cluster, Date endDate) {
+        org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, cluster);
+        processCluster.getValidity().setEnd(endDate);
+    }
+
+    public static int getParallel(Feed feed) {
+        return 1;
+    }
+
+    public static void setStartDate(Feed feed, String cluster, Date startDate) {
+        org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
+        clusterDef.getValidity().setStart(startDate);
+    }
+
+    public static void setEndTime(Feed feed, String cluster, Date endDate) {
+        org.apache.falcon.entity.v0.feed.Cluster clusterDef = FeedHelper.getCluster(feed, cluster);
+        clusterDef.getValidity().setStart(endDate);
+    }
+
+    public static void setParallel(Feed feed, int parallel) {
+    }
 
     public static Frequency getFrequency(Entity entity) {
         if (entity.getEntityType() == EntityType.PROCESS) {
@@ -236,75 +223,79 @@ public class EntityUtil {
         return feed.getTimezone();
     }
 
-	public static Date getNextStartTime(Date startTime, Frequency frequency, TimeZone timezone, Date now) {
-		if (startTime.after(now))
-			return startTime;
-
-		Calendar startCal = Calendar.getInstance(timezone);
-		startCal.setTime(startTime);
-
-		int count = 0;
-		switch (frequency.getTimeUnit()) {
-		case months:
-			count = (int) ((now.getTime() - startTime.getTime()) / MONTH_IN_MS);
-			break;
-		case days:
-			count = (int) ((now.getTime() - startTime.getTime()) / DAY_IN_MS);
-			break;
-		case hours:
-			count = (int) ((now.getTime() - startTime.getTime()) / HOUR_IN_MS);
-			break;
-		case minutes:
-			count = (int) ((now.getTime() - startTime.getTime()) / MINUTE_IN_MS);
-			break;
-		default:
-		}
-
-		if (count > 2) {
-			startCal.add(frequency.getTimeUnit().getCalendarUnit(), ((count - 2) / frequency.getFrequency()) * frequency.getFrequency());
-		}
-		while (startCal.getTime().before(now)) {
-			startCal.add(frequency.getTimeUnit().getCalendarUnit(), frequency.getFrequency());
-		}
-		return startCal.getTime();
-	}
-
-	public static int getInstanceSequence(Date startTime, Frequency frequency, TimeZone tz, Date instanceTime) {
-		if (startTime.after(instanceTime))
-			return -1;
-
-		Calendar startCal = Calendar.getInstance(tz);
-		startCal.setTime(startTime);
-
-		int count = 0;
-		switch (frequency.getTimeUnit()) {
-		case months:
-			count = (int) ((instanceTime.getTime() - startTime.getTime()) / MONTH_IN_MS);
-			break;
-		case days:
-			count = (int) ((instanceTime.getTime() - startTime.getTime()) / DAY_IN_MS);
-			break;
-		case hours:
-			count = (int) ((instanceTime.getTime() - startTime.getTime()) / HOUR_IN_MS);
-			break;
-		case minutes:
-			count = (int) ((instanceTime.getTime() - startTime.getTime()) / MINUTE_IN_MS);
-			break;
-		default:
-		}
-
-		if (count > 2) {
-			startCal.add(frequency.getTimeUnit().getCalendarUnit(), (count / frequency.getFrequency()) * frequency.getFrequency());
-			count = (count / frequency.getFrequency());
-		} else {
+    public static Date getNextStartTime(Date startTime, Frequency frequency, TimeZone timezone, Date now) {
+        if (startTime.after(now)) {
+            return startTime;
+        }
+
+        Calendar startCal = Calendar.getInstance(timezone);
+        startCal.setTime(startTime);
+
+        int count = 0;
+        switch (frequency.getTimeUnit()) {
+            case months:
+                count = (int) ((now.getTime() - startTime.getTime()) / MONTH_IN_MS);
+                break;
+            case days:
+                count = (int) ((now.getTime() - startTime.getTime()) / DAY_IN_MS);
+                break;
+            case hours:
+                count = (int) ((now.getTime() - startTime.getTime()) / HOUR_IN_MS);
+                break;
+            case minutes:
+                count = (int) ((now.getTime() - startTime.getTime()) / MINUTE_IN_MS);
+                break;
+            default:
+        }
+
+        if (count > 2) {
+            startCal.add(frequency.getTimeUnit().getCalendarUnit(),
+                    ((count - 2) / frequency.getFrequency()) * frequency.getFrequency());
+        }
+        while (startCal.getTime().before(now)) {
+            startCal.add(frequency.getTimeUnit().getCalendarUnit(), frequency.getFrequency());
+        }
+        return startCal.getTime();
+    }
+
+    public static int getInstanceSequence(Date startTime, Frequency frequency, TimeZone tz, Date instanceTime) {
+        if (startTime.after(instanceTime)) {
+            return -1;
+        }
+
+        Calendar startCal = Calendar.getInstance(tz);
+        startCal.setTime(startTime);
+
+        int count = 0;
+        switch (frequency.getTimeUnit()) {
+            case months:
+                count = (int) ((instanceTime.getTime() - startTime.getTime()) / MONTH_IN_MS);
+                break;
+            case days:
+                count = (int) ((instanceTime.getTime() - startTime.getTime()) / DAY_IN_MS);
+                break;
+            case hours:
+                count = (int) ((instanceTime.getTime() - startTime.getTime()) / HOUR_IN_MS);
+                break;
+            case minutes:
+                count = (int) ((instanceTime.getTime() - startTime.getTime()) / MINUTE_IN_MS);
+                break;
+            default:
+        }
+
+        if (count > 2) {
+            startCal.add(frequency.getTimeUnit().getCalendarUnit(),
+                    (count / frequency.getFrequency()) * frequency.getFrequency());
+            count = (count / frequency.getFrequency());
+        } else {
             count = 0;
         }
-		while (startCal.getTime().before(instanceTime)) {
-			startCal.add(frequency.getTimeUnit().getCalendarUnit(), frequency.getFrequency());
-			count++;
-		}
-		return count + 1;
-	}
+        while (startCal.getTime().before(instanceTime)) {
+            startCal.add(frequency.getTimeUnit().getCalendarUnit(), frequency.getFrequency());
+            count++;
+        }
+        return count + 1;
+    }
 
     public static String md5(Entity entity) throws FalconException {
         return new String(Hex.encodeHex(DigestUtils.md5(stringOf(entity))));
@@ -315,10 +306,12 @@ public class EntityUtil {
     }
 
     public static boolean equals(Entity lhs, Entity rhs, String[] filterProps) throws FalconException {
-        if (lhs == null && rhs == null)
+        if (lhs == null && rhs == null) {
             return true;
-        if (lhs == null || rhs == null)
+        }
+        if (lhs == null || rhs == null) {
             return false;
+        }
 
         if (lhs.equals(rhs)) {
             String lhsString = stringOf(lhs, filterProps);
@@ -332,38 +325,43 @@ public class EntityUtil {
     public static String stringOf(Entity entity) throws FalconException {
         return stringOf(entity, null);
     }
-    
+
     private static String stringOf(Entity entity, String[] filterProps) throws FalconException {
         Map<String, String> map = new HashMap<String, String>();
         mapToProperties(entity, null, map, filterProps);
         List<String> keyList = new ArrayList<String>(map.keySet());
         Collections.sort(keyList);
         StringBuilder builer = new StringBuilder();
-        for (String key : keyList)
+        for (String key : keyList) {
             builer.append(key).append('=').append(map.get(key)).append('\n');
+        }
         return builer.toString();
     }
 
     @SuppressWarnings("rawtypes")
-    private static void mapToProperties(Object obj, String name, Map<String, String> propMap, String[] filterProps) throws FalconException {
-        if (obj == null)
+    private static void mapToProperties(Object obj, String name, Map<String, String> propMap, String[] filterProps)
+            throws FalconException {
+        if (obj == null) {
             return;
+        }
 
-        if (filterProps != null && name != null)
+        if (filterProps != null && name != null) {
             for (String filter : filterProps) {
-                if (name.matches(filter.replace(".", "\\.").replace("[", "\\[").replace("]", "\\]")))
+                if (name.matches(filter.replace(".", "\\.").replace("[", "\\[").replace("]", "\\]"))) {
                     return;
+                }
             }
+        }
 
-        if (Date.class.isAssignableFrom(obj.getClass()))
-            propMap.put(name, SchemaHelper.formatDateUTC((Date)obj));
-        else if (obj.getClass().getPackage().getName().equals("java.lang"))
+        if (Date.class.isAssignableFrom(obj.getClass())) {
+            propMap.put(name, SchemaHelper.formatDateUTC((Date) obj));
+        } else if (obj.getClass().getPackage().getName().equals("java.lang")) {
             propMap.put(name, String.valueOf(obj));
-        else if (TimeZone.class.isAssignableFrom(obj.getClass()))
+        } else if (TimeZone.class.isAssignableFrom(obj.getClass())) {
             propMap.put(name, ((TimeZone) obj).getID());
-        else if (Enum.class.isAssignableFrom(obj.getClass()))
+        } else if (Enum.class.isAssignableFrom(obj.getClass())) {
             propMap.put(name, ((Enum) obj).name());
-        else if (List.class.isAssignableFrom(obj.getClass())) {
+        } else if (List.class.isAssignableFrom(obj.getClass())) {
             List list = (List) obj;
             for (int index = 0; index < list.size(); index++) {
                 mapToProperties(list.get(index), name + "[" + index + "]", propMap, filterProps);
@@ -376,206 +374,217 @@ public class EntityUtil {
                 try {
                     Map map = PropertyUtils.describe(obj);
                     for (Object key : map.keySet()) {
-                        if (!key.equals("class"))
-                            mapToProperties(map.get(key), name != null ? name + "." + key : (String)key, propMap, filterProps);
+                        if (!key.equals("class")) {
+                            mapToProperties(map.get(key), name != null ? name + "." + key : (String) key, propMap,
+                                    filterProps);
+                        }
                     }
                 } catch (Exception e1) {
                     throw new FalconException(e1);
                 }
-            } catch(Exception e) {
+            } catch (Exception e) {
                 throw new FalconException(e);
             }
         }
     }
 
     public static String getStagingPath(Entity entity) throws FalconException {
-		try {
-			return "falcon/workflows/" + entity.getEntityType().name().toLowerCase() + "/" + entity.getName() + "/"
-			+ md5(entity);
-		} catch (Exception e) {
-			throw new FalconException(e);
-		}
-	}
-
-	public static WorkflowName getWorkflowName(Tag tag, List<String> suffixes,
-			Entity entity) {
-		WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
-				entity);
-		builder.setTag(tag);
-		builder.setSuffixes(suffixes);
-		return builder.getWorkflowName();
-	}
-
-	public static WorkflowName getWorkflowName(Tag tag, Entity entity) {
-		return getWorkflowName(tag, null, entity);
-	}
-
-	public static WorkflowName getWorkflowName(Entity entity) {
-		return getWorkflowName(null, null, entity);
-	}
-
-	public static String getWorkflowNameSuffix(String workflowName,
-			Entity entity) throws FalconException {
-		WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
-				entity);
-		return builder.getWorkflowSuffixes(workflowName).replaceAll("_", "");
-	}
-
-	public static Tag getWorkflowNameTag(String workflowName, Entity entity) {
-		WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
-				entity);
-		return builder.getWorkflowTag(workflowName);
-	}
-
-	public static <T extends Entity> T getClusterView(T entity, String clusterName) {
-	    switch(entity.getEntityType()) {
-	    case CLUSTER:
-	        return entity;
-	        
-	    case FEED:
-	        Feed feed = (Feed) entity.clone();
-	        Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
-	        Iterator<Cluster> itr = feed.getClusters().getClusters().iterator();
-	        while(itr.hasNext()) {
-	            Cluster cluster = itr.next();
-	            //In addition to retaining the required clster, retain the sources clusters if this is the target cluster
-	            //1. Retain cluster if cluster n
-	            if(!(cluster.getName().equals(clusterName) || 
-	                    (feedCluster.getType() == ClusterType.TARGET && cluster.getType() == ClusterType.SOURCE)))
-	                itr.remove();
-	        }
-	        return (T) feed;            
-	        
-	    case PROCESS:
-	        Process process = (Process) entity.clone();
-	        Iterator<org.apache.falcon.entity.v0.process.Cluster> procItr = process.getClusters().getClusters().iterator();
-	        while(procItr.hasNext()) {
-	            org.apache.falcon.entity.v0.process.Cluster cluster = procItr.next();
-	            if(!cluster.getName().equals(clusterName))
-	                procItr.remove();
-	        }
-	        return (T) process;
-	    }
-	    throw new UnsupportedOperationException("Not supported for entity type " + entity.getEntityType());
-	}
-	
-	public static Set<String> getClustersDefined(Entity entity) {
-	    Set<String> clusters = new HashSet<String>();
-		switch(entity.getEntityType()) {
-		case CLUSTER:
-		    clusters.add(entity.getName());
-		    break;
-		    
-		case FEED:
-			Feed feed = (Feed) entity;
-			for(Cluster cluster:feed.getClusters().getClusters())
-				clusters.add(cluster.getName());
-			break;
-			
-		case PROCESS:
-			Process process = (Process) entity;
-			for(org.apache.falcon.entity.v0.process.Cluster cluster:process.getClusters().getClusters())
-				clusters.add(cluster.getName());
-			break;
-		}  
-		return clusters;
-	}
-	
-	public static Set<String> getClustersDefinedInColos(Entity entity) {
-		Set<String> entityClusters = EntityUtil.getClustersDefined(entity);
-		if (DeploymentUtil.isEmbeddedMode())
-			return entityClusters;
-
-		Set<String> myClusters = DeploymentUtil.getCurrentClusters();
-		Set<String> applicableClusters = new HashSet<String>();
-		for (String cluster : entityClusters)
-			if (myClusters.contains(cluster))
-				applicableClusters.add(cluster);
-		return applicableClusters;
-	}
-
-	public static Path getStagingPath(
-			org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-					throws FalconException {
-		try {
-			return new Path(ClusterHelper.getLocation(cluster, "staging"),
-					EntityUtil.getStagingPath(entity));
-		} catch (Exception e) {
-			throw new FalconException(e);
-		}
-	}
-
-	public static Retry getRetry(Entity entity) throws FalconException {
-		switch (entity.getEntityType()) {
-		case FEED:
-			if (!RuntimeProperties.get()
-					.getProperty("feed.retry.allowed", "true")
-					.equalsIgnoreCase("true")) {
-				return null;
-			}
-			Retry retry = new Retry();
-			retry.setAttempts(Integer.parseInt(RuntimeProperties.get()
-					.getProperty("feed.retry.attempts", "3")));
-			retry.setDelay(new Frequency(RuntimeProperties.get().getProperty(
-					"feed.retry.frequency", "minutes(5)")));
-			retry.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
-					.getProperty("feed.retry.policy", "exp-backoff")));
-			return retry;
-		case PROCESS:
-			Process process = (Process) entity;
-			return process.getRetry();
-		default:
-			throw new FalconException("Cannot create Retry for entity:"+entity.getName());
-		}
-	}
-
-	public static LateProcess getLateProcess(Entity entity)
-			throws FalconException {
-		switch (entity.getEntityType()) {
-		case FEED:
-			if (!RuntimeProperties.get()
-					.getProperty("feed.late.allowed", "true")
-					.equalsIgnoreCase("true")) {
-				return null;
-			}
-			LateProcess lateProcess = new LateProcess();
-			lateProcess.setDelay(new Frequency(RuntimeProperties.get()
-					.getProperty("feed.late.frequency", "hours(3)")));
-			lateProcess.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
-					.getProperty("feed.late.policy", "exp-backoff")));
-			LateInput lateInput = new LateInput();
-			lateInput.setInput(entity.getName());
-			//TODO - Assuming the late workflow is not used
-			lateInput.setWorkflowPath("ignore.xml");
-			lateProcess.getLateInputs().add(lateInput);
-			return lateProcess;
-		case PROCESS:
-			Process process = (Process) entity;
-			return process.getLateProcess();
-		default:
-			throw new FalconException("Cannot create Late Process for entity:"+entity.getName());
-		}
-	}
-	
-	public static Path getLogPath(
-			org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
-			throws FalconException {
-		Path logPath = new Path(ClusterHelper.getLocation(cluster,
-				"staging"), EntityUtil.getStagingPath(entity) + "/../logs");
-		return logPath;
-	}
-	
-	public static String UTCtoURIDate(String utc) throws FalconException {
-		DateFormat utcFormat = new SimpleDateFormat(
-				"yyyy'-'MM'-'dd'T'HH':'mm'Z'");
-		Date utcDate;
-		try {
-			utcDate = utcFormat.parse(utc);
-		} catch (ParseException e) {
-			throw new FalconException("Unable to parse utc date:", e);
-		}
-		DateFormat uriFormat = new SimpleDateFormat("yyyy'-'MM'-'dd'-'HH'-'mm");
-		return uriFormat.format(utcDate);
-	}
+        try {
+            return "falcon/workflows/" + entity.getEntityType().name().toLowerCase() + "/" + entity.getName() + "/"
+                    + md5(entity);
+        } catch (Exception e) {
+            throw new FalconException(e);
+        }
+    }
+
+    public static WorkflowName getWorkflowName(Tag tag, List<String> suffixes,
+                                               Entity entity) {
+        WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
+                entity);
+        builder.setTag(tag);
+        builder.setSuffixes(suffixes);
+        return builder.getWorkflowName();
+    }
+
+    public static WorkflowName getWorkflowName(Tag tag, Entity entity) {
+        return getWorkflowName(tag, null, entity);
+    }
+
+    public static WorkflowName getWorkflowName(Entity entity) {
+        return getWorkflowName(null, null, entity);
+    }
+
+    public static String getWorkflowNameSuffix(String workflowName,
+                                               Entity entity) throws FalconException {
+        WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
+                entity);
+        return builder.getWorkflowSuffixes(workflowName).replaceAll("_", "");
+    }
+
+    public static Tag getWorkflowNameTag(String workflowName, Entity entity) {
+        WorkflowNameBuilder<Entity> builder = new WorkflowNameBuilder<Entity>(
+                entity);
+        return builder.getWorkflowTag(workflowName);
+    }
+
+    public static <T extends Entity> T getClusterView(T entity, String clusterName) {
+        switch (entity.getEntityType()) {
+            case CLUSTER:
+                return entity;
+
+            case FEED:
+                Feed feed = (Feed) entity.clone();
+                Cluster feedCluster = FeedHelper.getCluster(feed, clusterName);
+                Iterator<Cluster> itr = feed.getClusters().getClusters().iterator();
+                while (itr.hasNext()) {
+                    Cluster cluster = itr.next();
+                    //In addition to retaining the required clster, retain the sources clusters if this is the target
+                    // cluster
+                    //1. Retain cluster if cluster n
+                    if (!(cluster.getName().equals(clusterName) ||
+                            (feedCluster.getType() == ClusterType.TARGET && cluster.getType() == ClusterType.SOURCE))) {
+                        itr.remove();
+                    }
+                }
+                return (T) feed;
+
+            case PROCESS:
+                Process process = (Process) entity.clone();
+                Iterator<org.apache.falcon.entity.v0.process.Cluster> procItr
+                        = process.getClusters().getClusters().iterator();
+                while (procItr.hasNext()) {
+                    org.apache.falcon.entity.v0.process.Cluster cluster = procItr.next();
+                    if (!cluster.getName().equals(clusterName)) {
+                        procItr.remove();
+                    }
+                }
+                return (T) process;
+        }
+        throw new UnsupportedOperationException("Not supported for entity type " + entity.getEntityType());
+    }
+
+    public static Set<String> getClustersDefined(Entity entity) {
+        Set<String> clusters = new HashSet<String>();
+        switch (entity.getEntityType()) {
+            case CLUSTER:
+                clusters.add(entity.getName());
+                break;
+
+            case FEED:
+                Feed feed = (Feed) entity;
+                for (Cluster cluster : feed.getClusters().getClusters()) {
+                    clusters.add(cluster.getName());
+                }
+                break;
+
+            case PROCESS:
+                Process process = (Process) entity;
+                for (org.apache.falcon.entity.v0.process.Cluster cluster : process.getClusters().getClusters()) {
+                    clusters.add(cluster.getName());
+                }
+                break;
+        }
+        return clusters;
+    }
+
+    public static Set<String> getClustersDefinedInColos(Entity entity) {
+        Set<String> entityClusters = EntityUtil.getClustersDefined(entity);
+        if (DeploymentUtil.isEmbeddedMode()) {
+            return entityClusters;
+        }
+
+        Set<String> myClusters = DeploymentUtil.getCurrentClusters();
+        Set<String> applicableClusters = new HashSet<String>();
+        for (String cluster : entityClusters) {
+            if (myClusters.contains(cluster)) {
+                applicableClusters.add(cluster);
+            }
+        }
+        return applicableClusters;
+    }
+
+    public static Path getStagingPath(
+            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+            throws FalconException {
+        try {
+            return new Path(ClusterHelper.getLocation(cluster, "staging"),
+                    EntityUtil.getStagingPath(entity));
+        } catch (Exception e) {
+            throw new FalconException(e);
+        }
+    }
+
+    public static Retry getRetry(Entity entity) throws FalconException {
+        switch (entity.getEntityType()) {
+            case FEED:
+                if (!RuntimeProperties.get()
+                        .getProperty("feed.retry.allowed", "true")
+                        .equalsIgnoreCase("true")) {
+                    return null;
+                }
+                Retry retry = new Retry();
+                retry.setAttempts(Integer.parseInt(RuntimeProperties.get()
+                        .getProperty("feed.retry.attempts", "3")));
+                retry.setDelay(new Frequency(RuntimeProperties.get().getProperty(
+                        "feed.retry.frequency", "minutes(5)")));
+                retry.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
+                        .getProperty("feed.retry.policy", "exp-backoff")));
+                return retry;
+            case PROCESS:
+                Process process = (Process) entity;
+                return process.getRetry();
+            default:
+                throw new FalconException("Cannot create Retry for entity:" + entity.getName());
+        }
+    }
+
+    public static LateProcess getLateProcess(Entity entity)
+            throws FalconException {
+        switch (entity.getEntityType()) {
+            case FEED:
+                if (!RuntimeProperties.get()
+                        .getProperty("feed.late.allowed", "true")
+                        .equalsIgnoreCase("true")) {
+                    return null;
+                }
+                LateProcess lateProcess = new LateProcess();
+                lateProcess.setDelay(new Frequency(RuntimeProperties.get()
+                        .getProperty("feed.late.frequency", "hours(3)")));
+                lateProcess.setPolicy(PolicyType.fromValue(RuntimeProperties.get()
+                        .getProperty("feed.late.policy", "exp-backoff")));
+                LateInput lateInput = new LateInput();
+                lateInput.setInput(entity.getName());
+                //TODO - Assuming the late workflow is not used
+                lateInput.setWorkflowPath("ignore.xml");
+                lateProcess.getLateInputs().add(lateInput);
+                return lateProcess;
+            case PROCESS:
+                Process process = (Process) entity;
+                return process.getLateProcess();
+            default:
+                throw new FalconException("Cannot create Late Process for entity:" + entity.getName());
+        }
+    }
+
+    public static Path getLogPath(
+            org.apache.falcon.entity.v0.cluster.Cluster cluster, Entity entity)
+            throws FalconException {
+        Path logPath = new Path(ClusterHelper.getLocation(cluster,
+                "staging"), EntityUtil.getStagingPath(entity) + "/../logs");
+        return logPath;
+    }
+
+    public static String UTCtoURIDate(String utc) throws FalconException {
+        DateFormat utcFormat = new SimpleDateFormat(
+                "yyyy'-'MM'-'dd'T'HH':'mm'Z'");
+        Date utcDate;
+        try {
+            utcDate = utcFormat.parse(utc);
+        } catch (ParseException e) {
+            throw new FalconException("Unable to parse utc date:", e);
+        }
+        DateFormat uriFormat = new SimpleDateFormat("yyyy'-'MM'-'dd'-'HH'-'mm");
+        return uriFormat.format(utcDate);
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/ExternalId.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ExternalId.java b/common/src/main/java/org/apache/falcon/entity/ExternalId.java
index 1b9f713..cb865d2 100644
--- a/common/src/main/java/org/apache/falcon/entity/ExternalId.java
+++ b/common/src/main/java/org/apache/falcon/entity/ExternalId.java
@@ -32,42 +32,43 @@ public class ExternalId {
     public ExternalId(String id) {
         this.id = id;
     }
-    
+
     public String getId() {
         return id;
     }
-    
+
     public ExternalId(String name, Tag tag, String elexpr) {
-        if(StringUtils.isEmpty(name) || tag == null || StringUtils.isEmpty(elexpr))
+        if (StringUtils.isEmpty(name) || tag == null || StringUtils.isEmpty(elexpr)) {
             throw new IllegalArgumentException("Empty inputs!");
-        
+        }
+
         id = name + SEPARATOR + tag.name() + SEPARATOR + elexpr;
     }
-    
+
     public ExternalId(String name, Tag tag, Date date) {
         this(name, tag, SchemaHelper.formatDateUTC(date));
     }
-    
+
     public String getName() {
         String[] parts = id.split(SEPARATOR);
         return parts[0];
     }
-    
+
     public Date getDate() throws FalconException {
-        return EntityUtil.parseDateUTC(getDateAsString());            
+        return EntityUtil.parseDateUTC(getDateAsString());
     }
-    
+
     public String getDateAsString() {
         String[] parts = id.split(SEPARATOR);
         return parts[2];
     }
-    
-	public Tag getTag() {
-		String[] parts = id.split(SEPARATOR);
-		return Tag.valueOf(parts[1]);
-	}
 
-	public String getDFSname() {
-		return id.replace(":", "-");
-	}
+    public Tag getTag() {
+        String[] parts = id.split(SEPARATOR);
+        return Tag.valueOf(parts[1]);
+    }
+
+    public String getDFSname() {
+        return id.replace(":", "-");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/FeedHelper.java b/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
index 2fcb7cc..4a39d8c 100644
--- a/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/FeedHelper.java
@@ -18,57 +18,54 @@
 
 package org.apache.falcon.entity;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.cluster.Property;
-import org.apache.falcon.entity.v0.feed.Cluster;
-import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.feed.Location;
-import org.apache.falcon.entity.v0.feed.LocationType;
-import org.apache.falcon.entity.v0.feed.Locations;
+import org.apache.falcon.entity.v0.feed.*;
 import org.apache.falcon.expression.ExpressionHelper;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
 public class FeedHelper {
     public static Cluster getCluster(Feed feed, String clusterName) {
-        for(Cluster cluster:feed.getClusters().getClusters())
-            if(cluster.getName().equals(clusterName))
+        for (Cluster cluster : feed.getClusters().getClusters()) {
+            if (cluster.getName().equals(clusterName)) {
                 return cluster;
+            }
+        }
         return null;
     }
-    
-	public static Location getLocation(Feed feed, LocationType type,
-			String clusterName) {
-		Cluster cluster = getCluster(feed, clusterName);
-		if (cluster!=null &&cluster.getLocations() != null 
-				&& cluster.getLocations() .getLocations().size() != 0) {
-			return getLocation(cluster.getLocations() , type);
-		}
-		else{
-			return getLocation(feed.getLocations(), type);
-		}
 
-	}
-	
-	public static Location getLocation(Feed feed, LocationType type) {
-		return getLocation(feed.getLocations(), type);
-	}
+    public static Location getLocation(Feed feed, LocationType type,
+                                       String clusterName) {
+        Cluster cluster = getCluster(feed, clusterName);
+        if (cluster != null && cluster.getLocations() != null
+                && cluster.getLocations().getLocations().size() != 0) {
+            return getLocation(cluster.getLocations(), type);
+        } else {
+            return getLocation(feed.getLocations(), type);
+        }
+
+    }
+
+    public static Location getLocation(Feed feed, LocationType type) {
+        return getLocation(feed.getLocations(), type);
+    }
+
+    public static Location getLocation(Locations locations, LocationType type) {
+        for (Location loc : locations.getLocations()) {
+            if (loc.getType() == type) {
+                return loc;
+            }
+        }
+        Location loc = new Location();
+        loc.setPath("/tmp");
+        loc.setType(type);
+        return loc;
+    }
 
-	public static Location getLocation(Locations locations, LocationType type) {
-		for (Location loc : locations.getLocations()) {
-			if (loc.getType() == type) {
-				return loc;
-			}
-		}
-		Location loc = new Location();
-		loc.setPath("/tmp");
-		loc.setType(type);
-		return loc;
-	}
-    
     public static String normalizePartitionExpression(String part1, String part2) {
         String partExp = StringUtils.stripToEmpty(part1) + "/" + StringUtils.stripToEmpty(part2);
         partExp = partExp.replaceAll("//+", "/");
@@ -79,16 +76,17 @@ public class FeedHelper {
 
     public static String normalizePartitionExpression(String partition) {
         return normalizePartitionExpression(partition, null);
-    }    
-    
+    }
+
     private static Properties loadClusterProperties(org.apache.falcon.entity.v0.cluster.Cluster cluster) {
         Properties properties = new Properties();
         Map<String, String> clusterVars = new HashMap<String, String>();
         clusterVars.put("colo", cluster.getColo());
         clusterVars.put("name", cluster.getName());
         if (cluster.getProperties() != null) {
-            for (Property property : cluster.getProperties().getProperties())
+            for (Property property : cluster.getProperties().getProperties()) {
                 clusterVars.put(property.getName(), property.getValue());
+            }
         }
         properties.put("cluster", clusterVars);
         return properties;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java b/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
index ca72c21..598f26e 100644
--- a/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
+++ b/common/src/main/java/org/apache/falcon/entity/ProcessHelper.java
@@ -23,9 +23,11 @@ import org.apache.falcon.entity.v0.process.Process;
 
 public class ProcessHelper {
     public static Cluster getCluster(Process process, String clusterName) {
-        for(Cluster cluster:process.getClusters().getClusters())
-            if(cluster.getName().equals(clusterName))
+        for (Cluster cluster : process.getClusters().getClusters()) {
+            if (cluster.getName().equals(clusterName)) {
                 return cluster;
+            }
+        }
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
index 93e728a..4a636cd 100644
--- a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
+++ b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
@@ -17,107 +17,107 @@
  */
 package org.apache.falcon.entity;
 
+import org.apache.falcon.Pair;
+import org.apache.falcon.Tag;
+import org.apache.falcon.entity.v0.Entity;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.falcon.Pair;
-import org.apache.falcon.Tag;
-import org.apache.falcon.entity.v0.Entity;
-
 public class WorkflowNameBuilder<T extends Entity> {
-	private static final String PREFIX = "FALCON";
-
-	T entity;
-	private Tag tag;
-	private List<String> suffixes;
-
-	public WorkflowNameBuilder(T entity) {
-		this.entity = entity;
-	}
-
-	public void setTag(Tag tag) {
-		this.tag = tag;
-	}
-
-	public void setSuffixes(List<String> suffixes) {
-		this.suffixes = suffixes;
-	}
-
-	public WorkflowName getWorkflowName() {
-		return new WorkflowName(PREFIX, entity.getEntityType().name(),
-				tag == null ? null : tag.name(), entity.getName(),
-				suffixes == null ? new ArrayList<String>() : suffixes);
-	}
-
-	public Tag getWorkflowTag(String workflowName) {
-		return WorkflowName.getTagAndSuffixes(entity, workflowName) == null ? null
-				: WorkflowName.getTagAndSuffixes(entity, workflowName).first;
-	}
-
-	public String getWorkflowSuffixes(String workflowName) {
-		return WorkflowName.getTagAndSuffixes(entity, workflowName) == null ? ""
-				: WorkflowName.getTagAndSuffixes(entity, workflowName).second;
-	}
-
-	public static class WorkflowName {
-		private static final String SEPARATOR = "_";
-
-		private String prefix;
-		private String entityType;
-		private String tag;
-		private String entityName;
-		private List<String> suffixes;
-
-		public WorkflowName(String prefix, String entityType, String tag,
-				String entityName, List<String> suffixes) {
-			this.prefix = prefix;
-			this.entityType = entityType;
-			this.tag = tag;
-			this.entityName = entityName;
-			this.suffixes = suffixes;
-		}
-
-		@Override
-		public String toString() {
-			StringBuilder builder = new StringBuilder();
-			builder.append(prefix).append(SEPARATOR).append(entityType)
-					.append(tag == null ? "" : SEPARATOR + tag)
-					.append(SEPARATOR).append(entityName);
-
-			for (String suffix : suffixes) {
-				builder.append(SEPARATOR).append(suffix);
-			}
-
-			return builder.toString();
-		}
-
-		public static Pair<Tag, String> getTagAndSuffixes(Entity entity,
-				String workflowName) {
-
-			StringBuilder namePattern = new StringBuilder(PREFIX + SEPARATOR
-					+ entity.getEntityType().name() + SEPARATOR + "(");
-			for (Tag tag : Tag.values()) {
-				namePattern.append(tag.name());
-				namePattern.append("|");
-			}
-			namePattern = namePattern.deleteCharAt(namePattern.length()-1);
-			namePattern.append(")" + SEPARATOR + entity.getName()
-					+ "([_A-Za-z0-9-.]*)");
-			
-			Pattern pattern = Pattern.compile(namePattern.toString());
-
-			Matcher matcher = pattern.matcher(workflowName);
-			if (matcher.matches()) {
-				matcher.reset();
-				if (matcher.find()) {
-					String tag = matcher.group(1);
-					String suffixes = matcher.group(2);
-					return new Pair<Tag, String>(Tag.valueOf(tag), suffixes);
-				}
-			}
-			return null;
-		}
-	}
+    private static final String PREFIX = "FALCON";
+
+    T entity;
+    private Tag tag;
+    private List<String> suffixes;
+
+    public WorkflowNameBuilder(T entity) {
+        this.entity = entity;
+    }
+
+    public void setTag(Tag tag) {
+        this.tag = tag;
+    }
+
+    public void setSuffixes(List<String> suffixes) {
+        this.suffixes = suffixes;
+    }
+
+    public WorkflowName getWorkflowName() {
+        return new WorkflowName(PREFIX, entity.getEntityType().name(),
+                tag == null ? null : tag.name(), entity.getName(),
+                suffixes == null ? new ArrayList<String>() : suffixes);
+    }
+
+    public Tag getWorkflowTag(String workflowName) {
+        return WorkflowName.getTagAndSuffixes(entity, workflowName) == null ? null
+                : WorkflowName.getTagAndSuffixes(entity, workflowName).first;
+    }
+
+    public String getWorkflowSuffixes(String workflowName) {
+        return WorkflowName.getTagAndSuffixes(entity, workflowName) == null ? ""
+                : WorkflowName.getTagAndSuffixes(entity, workflowName).second;
+    }
+
+    public static class WorkflowName {
+        private static final String SEPARATOR = "_";
+
+        private String prefix;
+        private String entityType;
+        private String tag;
+        private String entityName;
+        private List<String> suffixes;
+
+        public WorkflowName(String prefix, String entityType, String tag,
+                            String entityName, List<String> suffixes) {
+            this.prefix = prefix;
+            this.entityType = entityType;
+            this.tag = tag;
+            this.entityName = entityName;
+            this.suffixes = suffixes;
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder builder = new StringBuilder();
+            builder.append(prefix).append(SEPARATOR).append(entityType)
+                    .append(tag == null ? "" : SEPARATOR + tag)
+                    .append(SEPARATOR).append(entityName);
+
+            for (String suffix : suffixes) {
+                builder.append(SEPARATOR).append(suffix);
+            }
+
+            return builder.toString();
+        }
+
+        public static Pair<Tag, String> getTagAndSuffixes(Entity entity,
+                                                          String workflowName) {
+
+            StringBuilder namePattern = new StringBuilder(PREFIX + SEPARATOR
+                    + entity.getEntityType().name() + SEPARATOR + "(");
+            for (Tag tag : Tag.values()) {
+                namePattern.append(tag.name());
+                namePattern.append("|");
+            }
+            namePattern = namePattern.deleteCharAt(namePattern.length() - 1);
+            namePattern.append(")" + SEPARATOR + entity.getName()
+                    + "([_A-Za-z0-9-.]*)");
+
+            Pattern pattern = Pattern.compile(namePattern.toString());
+
+            Matcher matcher = pattern.matcher(workflowName);
+            if (matcher.matches()) {
+                matcher.reset();
+                if (matcher.find()) {
+                    String tag = matcher.group(1);
+                    String suffixes = matcher.group(2);
+                    return new Pair<Tag, String>(Tag.valueOf(tag), suffixes);
+                }
+            }
+            return null;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/common/Configuration.java b/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
index 1f17672..0e7e8f1 100644
--- a/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
+++ b/common/src/main/java/org/apache/falcon/entity/common/Configuration.java
@@ -25,49 +25,49 @@ import java.util.concurrent.ConcurrentHashMap;
 
 public class Configuration implements Iterable<Map.Entry<String, String>>, Cloneable {
 
-  private final Map<String, String> properties;
+    private final Map<String, String> properties;
 
-  public Configuration() {
-    properties = new ConcurrentHashMap<String, String>();
-  }
+    public Configuration() {
+        properties = new ConcurrentHashMap<String, String>();
+    }
 
-  public Configuration(Map<String, String> properties) {
-    this.properties = properties;
-  }
+    public Configuration(Map<String, String> properties) {
+        this.properties = properties;
+    }
 
-  public void addConfiguration(Configuration config) {
-    for (Entry<String, String> entry : config) {
-      properties.put(entry.getKey(), entry.getValue());
+    public void addConfiguration(Configuration config) {
+        for (Entry<String, String> entry : config) {
+            properties.put(entry.getKey(), entry.getValue());
+        }
     }
-  }
 
-  public Configuration addAndReturnNewConfiguration(Configuration config) {
-    Map<String, String> newProperties = new ConcurrentHashMap<String, String>(properties);
-    for (Entry<String, String> entry : config) {
-      newProperties.put(entry.getKey(), entry.getValue());
+    public Configuration addAndReturnNewConfiguration(Configuration config) {
+        Map<String, String> newProperties = new ConcurrentHashMap<String, String>(properties);
+        for (Entry<String, String> entry : config) {
+            newProperties.put(entry.getKey(), entry.getValue());
+        }
+        return new Configuration(newProperties);
     }
-    return new Configuration(newProperties);
-  }
 
-  public String getConf(String name) {
-    return properties.get(name);
-  }
+    public String getConf(String name) {
+        return properties.get(name);
+    }
 
-  public void setConf(String name, String value) {
-    properties.put(name, value);
-  }
+    public void setConf(String name, String value) {
+        properties.put(name, value);
+    }
 
-  public void setConf(String name, String value, String defaultValue) {
-    if (value == null) {
-      properties.put(name, defaultValue);
-    } else {
-      properties.put(name, value);
+    public void setConf(String name, String value, String defaultValue) {
+        if (value == null) {
+            properties.put(name, defaultValue);
+        } else {
+            properties.put(name, value);
+        }
     }
-  }
 
-  @Override
-  public Iterator<Entry<String, String>> iterator() {
-    return properties.entrySet().iterator();
-  }
+    @Override
+    public Iterator<Entry<String, String>> iterator() {
+        return properties.entrySet().iterator();
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java b/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
index 1d77b69..470c98a 100644
--- a/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
+++ b/common/src/main/java/org/apache/falcon/entity/common/FeedDataPath.java
@@ -21,36 +21,36 @@ import java.util.regex.Pattern;
 
 public class FeedDataPath {
 
-	public static enum VARS {
-		YEAR("yyyy"), MONTH("MM"), DAY("dd"), HOUR("HH"), MINUTE("mm");
-
-		private final Pattern pattern;
-		private final String datePattern;
-
-		private VARS(String datePattern) {
-			pattern = Pattern.compile("\\$\\{" + name() + "\\}");
-			this.datePattern = datePattern;
-		}
-
-		public String regex() {
-			return pattern.pattern();
-		}
-
-		public static VARS from(String str) {
-			for (VARS var : VARS.values()) {
-				if (var.datePattern.equals(str)) {
-					return var;
-				}
-			}
-			return null;
-		}
-	}
-
-	public static final Pattern PATTERN = Pattern.compile(VARS.YEAR.regex()
-			+ "|" + VARS.MONTH.regex() + "|" + VARS.DAY.regex() + "|"
-			+ VARS.HOUR.regex() + "|" + VARS.MINUTE.regex());
-
-	public static final Pattern DATE_FIELD_PATTERN = Pattern
-			.compile("yyyy|MM|dd|HH|mm");
+    public static enum VARS {
+        YEAR("yyyy"), MONTH("MM"), DAY("dd"), HOUR("HH"), MINUTE("mm");
+
+        private final Pattern pattern;
+        private final String datePattern;
+
+        private VARS(String datePattern) {
+            pattern = Pattern.compile("\\$\\{" + name() + "\\}");
+            this.datePattern = datePattern;
+        }
+
+        public String regex() {
+            return pattern.pattern();
+        }
+
+        public static VARS from(String str) {
+            for (VARS var : VARS.values()) {
+                if (var.datePattern.equals(str)) {
+                    return var;
+                }
+            }
+            return null;
+        }
+    }
+
+    public static final Pattern PATTERN = Pattern.compile(VARS.YEAR.regex()
+            + "|" + VARS.MONTH.regex() + "|" + VARS.DAY.regex() + "|"
+            + VARS.HOUR.regex() + "|" + VARS.MINUTE.regex());
+
+    public static final Pattern DATE_FIELD_PATTERN = Pattern
+            .compile("yyyy|MM|dd|HH|mm");
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 7fd92b6..c6ef988 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,13 +18,13 @@
 
 package org.apache.falcon.entity.parser;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.entity.ClusterHelper;
 import org.apache.falcon.entity.store.StoreAccessException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 public class ClusterEntityParser extends EntityParser<Cluster> {
@@ -36,22 +36,22 @@ public class ClusterEntityParser extends EntityParser<Cluster> {
     }
 
     @Override
-	public void validate(Cluster cluster) throws StoreAccessException,
-			ValidationException { 
-		if (new Path(ClusterHelper.getStorageUrl(cluster)).toUri().getScheme()==null) {
-			throw new ValidationException(
-					"Cannot get valid scheme for namenode from write interface of cluster: "
-							+ cluster.getName());
-		}
-		try {
-			Configuration conf = new Configuration();
-			conf.set("fs.default.name", ClusterHelper.getStorageUrl(cluster));
-			conf.setInt("ipc.client.connect.max.retries", 10);
-			FileSystem.get(conf);
-		} catch (Exception e) {
-			throw new ValidationException("Invalid HDFS server or port:"
-					+ ClusterHelper.getStorageUrl(cluster), e);
-		}
-	}
+    public void validate(Cluster cluster) throws StoreAccessException,
+                                                 ValidationException {
+        if (new Path(ClusterHelper.getStorageUrl(cluster)).toUri().getScheme() == null) {
+            throw new ValidationException(
+                    "Cannot get valid scheme for namenode from write interface of cluster: "
+                            + cluster.getName());
+        }
+        try {
+            Configuration conf = new Configuration();
+            conf.set("fs.default.name", ClusterHelper.getStorageUrl(cluster));
+            conf.setInt("ipc.client.connect.max.retries", 10);
+            FileSystem.get(conf);
+        } catch (Exception e) {
+            throw new ValidationException("Invalid HDFS server or port:"
+                    + ClusterHelper.getStorageUrl(cluster), e);
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java b/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
index 693e075..09aed24 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/CrossEntityValidations.java
@@ -18,19 +18,16 @@
 
 package org.apache.falcon.entity.parser;
 
-import java.util.Date;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.FeedHelper;
 import org.apache.falcon.entity.ProcessHelper;
 import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.process.Cluster;
-import org.apache.falcon.entity.v0.process.Input;
-import org.apache.falcon.entity.v0.process.Output;
+import org.apache.falcon.entity.v0.process.*;
 import org.apache.falcon.entity.v0.process.Process;
-import org.apache.falcon.entity.v0.process.Validity;
 import org.apache.falcon.expression.ExpressionHelper;
 
+import java.util.Date;
+
 public final class CrossEntityValidations {
 
     public static void validateInstanceRange(Process process, Input input, Feed feed) throws FalconException {
@@ -38,7 +35,8 @@ public final class CrossEntityValidations {
         try {
             for (Cluster cluster : process.getClusters().getClusters()) {
                 String clusterName = cluster.getName();
-                org.apache.falcon.entity.v0.feed.Validity feedValidity = FeedHelper.getCluster(feed, clusterName).getValidity();
+                org.apache.falcon.entity.v0.feed.Validity feedValidity = FeedHelper.getCluster(feed,
+                        clusterName).getValidity();
                 Date feedStart = feedValidity.getStart();
                 Date feedEnd = feedValidity.getEnd();
 
@@ -49,22 +47,28 @@ public final class CrossEntityValidations {
                 Validity processValidity = ProcessHelper.getCluster(process, clusterName).getValidity();
                 ExpressionHelper.setReferenceDate(processValidity.getStart());
                 Date instStart = evaluator.evaluate(instStartEL, Date.class);
-                if (instStart.before(feedStart))
+                if (instStart.before(feedStart)) {
                     throw new ValidationException("Start instance  " + instStartEL + " of feed " + feed.getName()
-                            + " is before the start of feed " + feedValidity.getStart() + " for cluster " + clusterName);
+                            + " is before the start of feed " + feedValidity.getStart() + " for cluster "
+                            + clusterName);
+                }
 
                 Date instEnd = evaluator.evaluate(instEndEL, Date.class);
-                if (instEnd.after(feedEnd))
+                if (instEnd.after(feedEnd)) {
                     throw new ValidationException("End instance  " + instEndEL + " of feed " + feed.getName()
-                            + " is before the start of feed " + feedValidity.getStart() + " for cluster " + clusterName);
+                            + " is before the start of feed " + feedValidity.getStart() + " for cluster "
+                            + clusterName);
+                }
 
-                if (instEnd.before(instStart))
+                if (instEnd.before(instStart)) {
                     throw new ValidationException("End instance " + instEndEL + " for feed " + feed.getName()
                             + " is before the start instance " + instStartEL + " for cluster " + clusterName);
+                }
 
-                if (instEnd.after(feedEnd))
+                if (instEnd.after(feedEnd)) {
                     throw new ValidationException("End instance " + instEndEL + " for feed " + feed.getName()
                             + " is after the end of feed " + feedValidity.getEnd() + " for cluster " + clusterName);
+                }
             }
         } catch (ValidationException e) {
             throw e;
@@ -73,7 +77,8 @@ public final class CrossEntityValidations {
         }
     }
 
-    public static void validateFeedRetentionPeriod(String startInstance, Feed feed, String clusterName) throws FalconException {
+    public static void validateFeedRetentionPeriod(String startInstance, Feed feed, String clusterName)
+            throws FalconException {
         String feedRetention = FeedHelper.getCluster(feed, clusterName).getRetention().getLimit().toString();
         ExpressionHelper evaluator = ExpressionHelper.get();
 
@@ -95,7 +100,8 @@ public final class CrossEntityValidations {
         try {
             for (Cluster cluster : process.getClusters().getClusters()) {
                 String clusterName = cluster.getName();
-                org.apache.falcon.entity.v0.feed.Validity feedValidity = FeedHelper.getCluster(feed, clusterName).getValidity();
+                org.apache.falcon.entity.v0.feed.Validity feedValidity = FeedHelper.getCluster(feed,
+                        clusterName).getValidity();
                 Date feedStart = feedValidity.getStart();
                 Date feedEnd = feedValidity.getEnd();
 
@@ -104,13 +110,15 @@ public final class CrossEntityValidations {
                 Validity processValidity = ProcessHelper.getCluster(process, clusterName).getValidity();
                 ExpressionHelper.setReferenceDate(processValidity.getStart());
                 Date inst = evaluator.evaluate(instEL, Date.class);
-                if (inst.before(feedStart))
+                if (inst.before(feedStart)) {
                     throw new ValidationException("Instance  " + instEL + " of feed " + feed.getName()
                             + " is before the start of feed " + feedValidity.getStart() + " for cluster" + clusterName);
+                }
 
-                if (inst.after(feedEnd))
+                if (inst.after(feedEnd)) {
                     throw new ValidationException("End instance " + instEL + " for feed " + feed.getName()
                             + " is after the end of feed " + feedValidity.getEnd() + " for cluster" + clusterName);
+                }
             }
         } catch (ValidationException e) {
             throw e;
@@ -122,12 +130,14 @@ public final class CrossEntityValidations {
     public static void validateInputPartition(Input input, Feed feed) throws ValidationException {
         String[] parts = input.getPartition().split("/");
         if (feed.getPartitions() == null || feed.getPartitions().getPartitions().isEmpty()
-                || feed.getPartitions().getPartitions().size() < parts.length)
+                || feed.getPartitions().getPartitions().size() < parts.length) {
             throw new ValidationException("Partition specification in input " + input.getName() + " is wrong");
+        }
     }
 
     public static void validateFeedDefinedForCluster(Feed feed, String clusterName) throws FalconException {
-        if (FeedHelper.getCluster(feed, clusterName) == null)
+        if (FeedHelper.getCluster(feed, clusterName) == null) {
             throw new ValidationException("Feed " + feed.getName() + " is not defined for cluster " + clusterName);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
index 52d086c..6b06cc4 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/EntityParser.java
@@ -18,12 +18,6 @@
 
 package org.apache.falcon.entity.parser;
 
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-import java.util.List;
-
-import javax.xml.bind.Unmarshaller;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.Pair;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -31,11 +25,15 @@ import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.log4j.Logger;
 
+import javax.xml.bind.Unmarshaller;
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.util.List;
+
 /**
- * 
  * Generic Abstract Entity Parser, the concrete FEED, PROCESS and CLUSTER
  * Should extend this parser to implement specific parsing.
- * 
+ *
  * @param <T>
  */
 public abstract class EntityParser<T extends Entity> {
@@ -46,9 +44,8 @@ public abstract class EntityParser<T extends Entity> {
 
     /**
      * Constructor
-     * 
-     * @param entityType
-     *            - can be FEED or PROCESS
+     *
+     * @param entityType - can be FEED or PROCESS
      */
     protected EntityParser(EntityType entityType) {
         this.entityType = entityType;
@@ -60,9 +57,8 @@ public abstract class EntityParser<T extends Entity> {
 
     /**
      * Parses a sent XML and validates it using JAXB.
-     * 
-     * @param xmlString
-     *            - Entity XML
+     *
+     * @param xmlString - Entity XML
      * @return Entity - JAVA Object
      * @throws FalconException
      */
@@ -71,10 +67,10 @@ public abstract class EntityParser<T extends Entity> {
         Entity entity = parseAndValidate(inputStream);
         return entity;
     }
-    
+
     /**
      * Parses xml stream
-     * 
+     *
      * @param xmlStream
      * @return entity
      * @throws FalconException
@@ -86,7 +82,7 @@ public abstract class EntityParser<T extends Entity> {
             T entity = (T) unmarshaller.unmarshal(xmlStream);
             LOG.info("Parsed Entity: " + entity.getName());
             return entity;
-        } catch(Exception e) {
+        } catch (Exception e) {
             throw new FalconException(e);
         }
     }
@@ -96,19 +92,20 @@ public abstract class EntityParser<T extends Entity> {
         validate(entity);
         return entity;
     }
-    
+
     protected void validateEntityExists(EntityType type, String name) throws FalconException {
-        if(ConfigurationStore.get().get(type, name) == null)
-            throw new ValidationException("Referenced " + type + " " + name + " is not registered");        
+        if (ConfigurationStore.get().get(type, name) == null) {
+            throw new ValidationException("Referenced " + type + " " + name + " is not registered");
+        }
     }
-    
+
     protected void validateEntitiesExist(List<Pair<EntityType, String>> entities) throws FalconException {
-        if(entities != null) {
-            for(Pair<EntityType, String> entity:entities) {
+        if (entities != null) {
+            for (Pair<EntityType, String> entity : entities) {
                 validateEntityExists(entity.first, entity.second);
             }
         }
     }
-    
+
     public abstract void validate(T entity) throws FalconException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java b/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
index e9369e2..43542c3 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/EntityParserFactory.java
@@ -21,35 +21,33 @@ package org.apache.falcon.entity.parser;
 import org.apache.falcon.entity.v0.EntityType;
 
 /**
- * 
  * Factory Class which returns the Parser based on the EntityType.
- * 
  */
 public final class EntityParserFactory {
 
-	private EntityParserFactory() {
-	}
+    private EntityParserFactory() {
+    }
 
-	/**
-	 * Tie EnityType with the Entity Class in one place so that it can be
-	 * unmarshalled easily by concrete classes based on the class type using
-	 * JAXB.
-	 * 
-	 * @param entityType
-	 * @return concrete parser based on entity type
-	 */
-	public static EntityParser getParser(final EntityType entityType) {
+    /**
+     * Tie EnityType with the Entity Class in one place so that it can be
+     * unmarshalled easily by concrete classes based on the class type using
+     * JAXB.
+     *
+     * @param entityType
+     * @return concrete parser based on entity type
+     */
+    public static EntityParser getParser(final EntityType entityType) {
 
-		switch (entityType) {
-		case PROCESS:
-			return new ProcessEntityParser();
-		case FEED:
-			return new FeedEntityParser();
-		case CLUSTER:
-			return new ClusterEntityParser();
-		default:
-			throw new IllegalArgumentException("Unhandled entity type: " + entityType);
-		}
-	}
+        switch (entityType) {
+            case PROCESS:
+                return new ProcessEntityParser();
+            case FEED:
+                return new FeedEntityParser();
+            case CLUSTER:
+                return new ClusterEntityParser();
+            default:
+                throw new IllegalArgumentException("Unhandled entity type: " + entityType);
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java b/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
index 52a0804..475384e 100644
--- a/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
+++ b/common/src/main/java/org/apache/falcon/entity/parser/FeedEntityParser.java
@@ -18,11 +18,6 @@
 
 package org.apache.falcon.entity.parser;
 
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.TimeZone;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.EntityUtil;
@@ -43,6 +38,11 @@ import org.apache.falcon.group.FeedGroup;
 import org.apache.falcon.group.FeedGroupMap;
 import org.apache.log4j.Logger;
 
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TimeZone;
+
 public class FeedEntityParser extends EntityParser<Feed> {
 
     private static final Logger LOG = Logger.getLogger(FeedEntityParser.class);
@@ -53,15 +53,18 @@ public class FeedEntityParser extends EntityParser<Feed> {
 
     @Override
     public void validate(Feed feed) throws FalconException {
-        if(feed.getTimezone() == null)
+        if (feed.getTimezone() == null) {
             feed.setTimezone(TimeZone.getTimeZone("UTC"));
-        
-        if (feed.getClusters() == null)
+        }
+
+        if (feed.getClusters() == null) {
             throw new ValidationException("Feed should have atleast one cluster");
+        }
 
         for (Cluster cluster : feed.getClusters().getClusters()) {
             validateEntityExists(EntityType.CLUSTER, cluster.getName());
-            validateClusterValidity(cluster.getValidity().getStart(), cluster.getValidity().getEnd(), cluster.getName());
+            validateClusterValidity(cluster.getValidity().getStart(), cluster.getValidity().getEnd(),
+                    cluster.getName());
             validateFeedCutOffPeriod(feed, cluster);
         }
 
@@ -72,8 +75,9 @@ public class FeedEntityParser extends EntityParser<Feed> {
         // But is this an update ?
 
         Feed oldFeed = ConfigurationStore.get().get(EntityType.FEED, feed.getName());
-        if (oldFeed == null)
+        if (oldFeed == null) {
             return; // Not an update case
+        }
 
         // Is actually an update. Need to iterate over all the processes
         // depending on this feed and see if they are valid with the new
@@ -81,8 +85,9 @@ public class FeedEntityParser extends EntityParser<Feed> {
         EntityGraph graph = EntityGraph.get();
         Set<Entity> referenced = graph.getDependents(oldFeed);
         Set<Process> processes = findProcesses(referenced);
-        if (processes.isEmpty())
+        if (processes.isEmpty()) {
             return;
+        }
 
         ensureValidityFor(feed, processes);
     }
@@ -98,33 +103,35 @@ public class FeedEntityParser extends EntityParser<Feed> {
     }
 
     private void validateFeedGroups(Feed feed) throws ValidationException {
-        String[] groupNames = feed.getGroups() != null ? feed.getGroups().split(",") : new String[] {};
+        String[] groupNames = feed.getGroups() != null ? feed.getGroups().split(",") : new String[]{};
         String defaultPath = FeedHelper.getLocation(feed, LocationType.DATA)
-		.getPath();
-		for (Cluster cluster : feed.getClusters().getClusters()) {
-			if (!FeedGroup.getDatePattern(
-					FeedHelper.getLocation(feed, LocationType.DATA,
-							cluster.getName()).getPath()).equals(
-					FeedGroup.getDatePattern(defaultPath))) {
-				throw new ValidationException("Feeds default path pattern: "
-						+ FeedHelper.getLocation(feed, LocationType.DATA)
-								.getPath()
-						+ ", does not match with cluster: "
-						+ cluster.getName()
-						+ " path pattern: "
-						+ FeedHelper.getLocation(feed, LocationType.DATA,
-								cluster.getName()).getPath());
-			}
-		}
+                .getPath();
+        for (Cluster cluster : feed.getClusters().getClusters()) {
+            if (!FeedGroup.getDatePattern(
+                    FeedHelper.getLocation(feed, LocationType.DATA,
+                            cluster.getName()).getPath()).equals(
+                    FeedGroup.getDatePattern(defaultPath))) {
+                throw new ValidationException("Feeds default path pattern: "
+                        + FeedHelper.getLocation(feed, LocationType.DATA)
+                        .getPath()
+                        + ", does not match with cluster: "
+                        + cluster.getName()
+                        + " path pattern: "
+                        + FeedHelper.getLocation(feed, LocationType.DATA,
+                        cluster.getName()).getPath());
+            }
+        }
         for (String groupName : groupNames) {
             FeedGroup group = FeedGroupMap.get().getGroupsMapping().get(groupName);
             if (group == null || group.canContainFeed(feed)) {
                 continue;
             } else {
-                throw new ValidationException("Feed " + feed.getName() + "'s frequency: " + feed.getFrequency().toString()
-                        + ", path pattern: " + FeedHelper.getLocation(feed, LocationType.DATA).getPath()
-                        + " does not match with group: " + group.getName() + "'s frequency: " + group.getFrequency()
-                        + ", date pattern: " + group.getDatePattern());
+                throw new ValidationException(
+                        "Feed " + feed.getName() + "'s frequency: " + feed.getFrequency().toString()
+                                + ", path pattern: " + FeedHelper.getLocation(feed, LocationType.DATA).getPath()
+                                + " does not match with group: " + group.getName() + "'s frequency: "
+                                + group.getFrequency()
+                                + ", date pattern: " + group.getDatePattern());
             }
         }
     }
@@ -134,8 +141,9 @@ public class FeedEntityParser extends EntityParser<Feed> {
             try {
                 ensureValidityFor(newFeed, process);
             } catch (FalconException e) {
-                throw new ValidationException("Process " + process.getName() + " is not compatible " + "with changes to feed "
-                        + newFeed.getName(), e);
+                throw new ValidationException(
+                        "Process " + process.getName() + " is not compatible " + "with changes to feed "
+                                + newFeed.getName(), e);
             }
         }
     }
@@ -145,8 +153,9 @@ public class FeedEntityParser extends EntityParser<Feed> {
             String clusterName = cluster.getName();
             if (process.getInputs() != null) {
                 for (Input input : process.getInputs().getInputs()) {
-                    if (!input.getFeed().equals(newFeed.getName()))
+                    if (!input.getFeed().equals(newFeed.getName())) {
                         continue;
+                    }
                     CrossEntityValidations.validateFeedDefinedForCluster(newFeed, clusterName);
                     CrossEntityValidations.validateFeedRetentionPeriod(input.getStart(), newFeed, clusterName);
                     CrossEntityValidations.validateInstanceRange(process, input, newFeed);
@@ -159,13 +168,15 @@ public class FeedEntityParser extends EntityParser<Feed> {
 
             if (process.getOutputs() != null) {
                 for (Output output : process.getOutputs().getOutputs()) {
-                    if (!output.getFeed().equals(newFeed.getName()))
+                    if (!output.getFeed().equals(newFeed.getName())) {
                         continue;
+                    }
                     CrossEntityValidations.validateFeedDefinedForCluster(newFeed, clusterName);
                     CrossEntityValidations.validateInstance(process, output, newFeed);
                 }
             }
-            LOG.debug("Verified and found " + process.getName() + " to be valid for new definition of " + newFeed.getName());
+            LOG.debug("Verified and found " + process.getName() + " to be valid for new definition of "
+                    + newFeed.getName());
         }
     }
 
@@ -188,59 +199,66 @@ public class FeedEntityParser extends EntityParser<Feed> {
         String feedRetention = cluster.getRetention().getLimit().toString();
         long retentionPeriod = evaluator.evaluate(feedRetention, Long.class);
 
-        if(feed.getLateArrival()==null){
-        	LOG.debug("Feed's late arrival cut-off not set");
-        	return;
+        if (feed.getLateArrival() == null) {
+            LOG.debug("Feed's late arrival cut-off not set");
+            return;
         }
         String feedCutoff = feed.getLateArrival().getCutOff().toString();
         long feedCutOffPeriod = evaluator.evaluate(feedCutoff, Long.class);
 
         if (retentionPeriod < feedCutOffPeriod) {
-            throw new ValidationException("Feed's retention limit: " + feedRetention + " of referenced cluster " + cluster.getName()
-                    + " should be more than feed's late arrival cut-off period: " + feedCutoff + " for feed: " + feed.getName());
+            throw new ValidationException(
+                    "Feed's retention limit: " + feedRetention + " of referenced cluster " + cluster.getName()
+                            + " should be more than feed's late arrival cut-off period: " + feedCutoff + " for feed: "
+                            + feed.getName());
         }
     }
-    
+
     private void validateFeedPartitionExpression(Feed feed) throws FalconException {
         int numSourceClusters = 0, numTrgClusters = 0;
         Set<String> clusters = new HashSet<String>();
         for (Cluster cl : feed.getClusters().getClusters()) {
-			if (!clusters.add(cl.getName())) {
-				throw new ValidationException("Cluster: " + cl.getName()
-						+ " is defined more than once for feed: "+feed.getName());
-			}
-            if (cl.getType() == ClusterType.SOURCE){
+            if (!clusters.add(cl.getName())) {
+                throw new ValidationException("Cluster: " + cl.getName()
+                        + " is defined more than once for feed: " + feed.getName());
+            }
+            if (cl.getType() == ClusterType.SOURCE) {
                 numSourceClusters++;
-            } else if(cl.getType() == ClusterType.TARGET) {
+            } else if (cl.getType() == ClusterType.TARGET) {
                 numTrgClusters++;
             }
         }
-        
-		if (numTrgClusters >= 1 && numSourceClusters == 0) {
-			throw new ValidationException("Feed: " + feed.getName()
-					+ " should have atleast one source cluster defined");
-		}
-        
+
+        if (numTrgClusters >= 1 && numSourceClusters == 0) {
+            throw new ValidationException("Feed: " + feed.getName()
+                    + " should have atleast one source cluster defined");
+        }
+
         int feedParts = feed.getPartitions() != null ? feed.getPartitions().getPartitions().size() : 0;
-        
-        for(Cluster cluster:feed.getClusters().getClusters()) {
 
-            if(cluster.getType() == ClusterType.SOURCE && numSourceClusters > 1 && numTrgClusters >= 1) {
+        for (Cluster cluster : feed.getClusters().getClusters()) {
+
+            if (cluster.getType() == ClusterType.SOURCE && numSourceClusters > 1 && numTrgClusters >= 1) {
                 String part = FeedHelper.normalizePartitionExpression(cluster.getPartition());
-                if(StringUtils.split(part, '/').length == 0)
-                    throw new ValidationException("Partition expression has to be specified for cluster " + cluster.getName() +
-                            " as there are more than one source clusters");
+                if (StringUtils.split(part, '/').length == 0) {
+                    throw new ValidationException(
+                            "Partition expression has to be specified for cluster " + cluster.getName() +
+                                    " as there are more than one source clusters");
+                }
                 validateClusterExpDefined(cluster);
 
-            } else if(cluster.getType() == ClusterType.TARGET) {
+            } else if (cluster.getType() == ClusterType.TARGET) {
 
-                for(Cluster src:feed.getClusters().getClusters()) {
-                    if(src.getType() == ClusterType.SOURCE) {
-                        String part = FeedHelper.normalizePartitionExpression(src.getPartition(), cluster.getPartition());
+                for (Cluster src : feed.getClusters().getClusters()) {
+                    if (src.getType() == ClusterType.SOURCE) {
+                        String part = FeedHelper.normalizePartitionExpression(src.getPartition(),
+                                cluster.getPartition());
                         int numParts = StringUtils.split(part, '/').length;
-                        if(numParts > feedParts)
-                            throw new ValidationException("Partition for " + src.getName() + " and " + cluster.getName() + 
-                                    "clusters is more than the number of partitions defined in feed");
+                        if (numParts > feedParts) {
+                            throw new ValidationException(
+                                    "Partition for " + src.getName() + " and " + cluster.getName() +
+                                            "clusters is more than the number of partitions defined in feed");
+                        }
                     }
                 }
 
@@ -252,12 +270,15 @@ public class FeedEntityParser extends EntityParser<Feed> {
     }
 
     private void validateClusterExpDefined(Cluster cl) throws FalconException {
-        if(cl.getPartition() == null)
+        if (cl.getPartition() == null) {
             return;
-        
+        }
+
         org.apache.falcon.entity.v0.cluster.Cluster cluster = EntityUtil.getEntity(EntityType.CLUSTER, cl.getName());
         String part = FeedHelper.normalizePartitionExpression(cl.getPartition());
-        if(FeedHelper.evaluateClusterExp(cluster, part).equals(part))
-            throw new ValidationException("Alteast one of the partition tags has to be a cluster expression for cluster " + cl.getName()); 
+        if (FeedHelper.evaluateClusterExp(cluster, part).equals(part)) {
+            throw new ValidationException(
+                    "Alteast one of the partition tags has to be a cluster expression for cluster " + cl.getName());
+        }
     }
 }


[23/47] git commit: Fixing issue with entity::clone() method

Posted by sr...@apache.org.
Fixing issue with entity::clone() method


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

Branch: refs/heads/master
Commit: 74a1c6584d6e5c9f31636e0973e4573859a89edf
Parents: e69b220
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Tue Apr 16 23:50:06 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Tue Apr 16 23:50:06 2013 +0530

----------------------------------------------------------------------
 .../java/org/apache/falcon/entity/v0/Entity.java   |    5 ++---
 1 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/74a1c658/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
index 8759976..1929429 100644
--- a/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
+++ b/client/src/main/java/org/apache/falcon/entity/v0/Entity.java
@@ -26,7 +26,7 @@ import java.io.StringWriter;
 /**
  * Base class that all entity jaxb object will extend.
  */
-public abstract class Entity implements Cloneable{
+public abstract class Entity {
     public abstract String getName();
 
     public EntityType getEntityType() {
@@ -89,8 +89,7 @@ public abstract class Entity implements Cloneable{
     }
 
     @Override
-    public Entity clone() throws CloneNotSupportedException {
-        Object ignore = super.clone();
+    public Entity clone() {
         return fromString(getEntityType(), toString());
     }
 }


[30/47] git commit: Changes to checkstyle plugin for supressing some errors

Posted by sr...@apache.org.
Changes to checkstyle plugin for supressing some errors


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

Branch: refs/heads/master
Commit: 6119bedb1e2087e565381e4cfcd5e9357fd69f37
Parents: 2a94690
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:45:47 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:45:47 2013 +0530

----------------------------------------------------------------------
 .../src/main/resources/falcon/checkstyle.xml       |    8 +++++---
 1 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/6119bedb/build-tools/src/main/resources/falcon/checkstyle.xml
----------------------------------------------------------------------
diff --git a/build-tools/src/main/resources/falcon/checkstyle.xml b/build-tools/src/main/resources/falcon/checkstyle.xml
index 641c5ff..3853c5c 100644
--- a/build-tools/src/main/resources/falcon/checkstyle.xml
+++ b/build-tools/src/main/resources/falcon/checkstyle.xml
@@ -178,6 +178,8 @@
         <module name="StringLiteralEquality"/>
         <module name="HiddenField">
             <property name="ignoreConstructorParameter" value="true"/>
+            <property name="ignoreAbstractMethods" value="true"/>
+            <property name="ignoreSetter" value="true"/>
         </module>
         <module name="IllegalInstantiation"/>
         <module name="InnerAssignment"/>
@@ -223,9 +225,9 @@
 
     <!-- allow warnings to be suppressed -->
     <module name="SuppressionCommentFilter">
-        <property name="offCommentFormat" value="SUSPEND CHECKSTYLE CHECK"/>
-        <property name="onCommentFormat" value="RESUME CHECKSTYLE CHECK"/>
-        <property name="checkFormat" value="ParameterNumberCheck|VisibilityModifierCheck|HiddenFieldCheck|EmptyBlockCheck"/>
+        <property name="offCommentFormat" value="SUSPEND CHECKSTYLE CHECK ParameterNumberCheck|VisibilityModifierCheck|HiddenFieldCheck"/>
+        <property name="onCommentFormat" value="RESUME CHECKSTYLE CHECK ParameterNumberCheck|VisibilityModifierCheck|HiddenFieldCheck"/>
+        <property name="checkFormat" value="ParameterNumberCheck|VisibilityModifierCheck|HiddenFieldCheck"/>
     </module>
 
 </module>


[25/47] Check style fixes relating to common module

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/BuildProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/BuildProperties.java b/common/src/main/java/org/apache/falcon/util/BuildProperties.java
index 898daee..339dcb5 100644
--- a/common/src/main/java/org/apache/falcon/util/BuildProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/BuildProperties.java
@@ -23,10 +23,13 @@ import org.apache.falcon.FalconException;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class BuildProperties extends ApplicationProperties {
+/**
+ * Application build info properties are exposed through this.
+ */
+public final class BuildProperties extends ApplicationProperties {
     private static final String PROPERTY_FILE = "falcon-buildinfo.properties";
 
-    private static final AtomicReference<BuildProperties> instance =
+    private static final AtomicReference<BuildProperties> INSTANCE =
             new AtomicReference<BuildProperties>();
 
     private BuildProperties() throws FalconException {
@@ -40,13 +43,13 @@ public class BuildProperties extends ApplicationProperties {
 
     public static Properties get() {
         try {
-            if (instance.get() == null) {
-                instance.compareAndSet(null, new BuildProperties());
+            if (INSTANCE.get() == null) {
+                INSTANCE.compareAndSet(null, new BuildProperties());
             }
-            return instance.get();
+            return INSTANCE.get();
         } catch (FalconException e) {
-            throw new RuntimeException("Unable to read application " +
-                    "falcon build information properties", e);
+            throw new RuntimeException("Unable to read application "
+                + "falcon build information properties", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java b/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
index 4e2f7db..715b7ba 100644
--- a/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/DeploymentProperties.java
@@ -23,10 +23,14 @@ import org.apache.falcon.FalconException;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class DeploymentProperties extends ApplicationProperties {
+/**
+ * Application deployment properties. particularly relating to
+ * whether the server is in embedded mode or distributed mode.
+ */
+public final class DeploymentProperties extends ApplicationProperties {
     private static final String PROPERTY_FILE = "deploy.properties";
 
-    private static final AtomicReference<DeploymentProperties> instance =
+    private static final AtomicReference<DeploymentProperties> INSTANCE =
             new AtomicReference<DeploymentProperties>();
 
     private DeploymentProperties() throws FalconException {
@@ -40,13 +44,12 @@ public class DeploymentProperties extends ApplicationProperties {
 
     public static Properties get() {
         try {
-            if (instance.get() == null) {
-                instance.compareAndSet(null, new DeploymentProperties());
+            if (INSTANCE.get() == null) {
+                INSTANCE.compareAndSet(null, new DeploymentProperties());
             }
-            return instance.get();
+            return INSTANCE.get();
         } catch (FalconException e) {
-            throw new RuntimeException("Unable to read application " +
-                    "startup properties", e);
+            throw new RuntimeException("Unable to read application " + "startup properties", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java b/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
index 9aeb3ab..8909c17 100644
--- a/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
+++ b/common/src/main/java/org/apache/falcon/util/DeploymentUtil.java
@@ -24,7 +24,10 @@ import org.apache.log4j.Logger;
 import java.util.HashSet;
 import java.util.Set;
 
-public class DeploymentUtil {
+/**
+ * Helper methods to deployment properties.
+ */
+public final class DeploymentUtil {
     private static final Logger LOG = Logger.getLogger(DeploymentUtil.class);
 
     protected static final String DEFAULT_COLO = "default";
@@ -32,34 +35,36 @@ public class DeploymentUtil {
     protected static final String DEPLOY_MODE = "deploy.mode";
     private static final Set<String> DEFAULT_ALL_COLOS = new HashSet<String>();
 
-    protected final static String currentColo;
-    protected final static boolean embeddedMode;
+    protected static final String CURRENT_COLO;
+    protected static final boolean EMBEDDED_MODE;
     protected static boolean prism = false;
 
     static {
         DEFAULT_ALL_COLOS.add(DEFAULT_COLO);
-        embeddedMode = DeploymentProperties.get().
+        EMBEDDED_MODE = DeploymentProperties.get().
                 getProperty(DEPLOY_MODE, EMBEDDED).equals(EMBEDDED);
-        if (embeddedMode) {
-            currentColo = DEFAULT_COLO;
+        if (EMBEDDED_MODE) {
+            CURRENT_COLO = DEFAULT_COLO;
         } else {
-            currentColo = StartupProperties.get().
+            CURRENT_COLO = StartupProperties.get().
                     getProperty("current.colo", DEFAULT_COLO);
         }
-        LOG.info("Running in embedded mode? " + embeddedMode);
-        LOG.info("Current colo: " + currentColo);
+        LOG.info("Running in embedded mode? " + EMBEDDED_MODE);
+        LOG.info("Current colo: " + CURRENT_COLO);
     }
 
+    private DeploymentUtil() {}
+
     public static void setPrismMode() {
         prism = true;
     }
 
     public static boolean isPrism() {
-        return !embeddedMode && prism;
+        return !EMBEDDED_MODE && prism;
     }
 
     public static String getCurrentColo() {
-        return currentColo;
+        return CURRENT_COLO;
     }
 
     public static Set<String> getCurrentClusters() {
@@ -68,7 +73,7 @@ public class DeploymentUtil {
     }
 
     public static boolean isEmbeddedMode() {
-        return embeddedMode;
+        return EMBEDDED_MODE;
     }
 
     public static String getDefaultColo() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java b/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
index d1bed8e..4a00fa9 100644
--- a/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
+++ b/common/src/main/java/org/apache/falcon/util/ReflectionUtils.java
@@ -22,8 +22,13 @@ import org.apache.falcon.FalconException;
 
 import java.lang.reflect.Method;
 
+/**
+ * Helper methods for class instantiation through reflection.
+ */
 public final class ReflectionUtils {
 
+    private ReflectionUtils() {}
+
     public static <T> T getInstance(String classKey) throws FalconException {
         String clazzName = StartupProperties.get().getProperty(classKey);
         try {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java b/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
index 86a54f8..cc87c8c 100644
--- a/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/RuntimeProperties.java
@@ -24,13 +24,16 @@ import org.apache.log4j.Logger;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class RuntimeProperties extends ApplicationProperties {
+/**
+ * Dynamic properties that may be modified while the server is running.
+ */
+public final class RuntimeProperties extends ApplicationProperties {
 
-    private static Logger LOG = Logger.getLogger(RuntimeProperties.class);
+    private static final Logger LOG = Logger.getLogger(RuntimeProperties.class);
 
     private static final String PROPERTY_FILE = "runtime.properties";
 
-    private static final AtomicReference<RuntimeProperties> instance =
+    private static final AtomicReference<RuntimeProperties> INSTANCE =
             new AtomicReference<RuntimeProperties>();
 
     private RuntimeProperties() throws FalconException {
@@ -46,17 +49,19 @@ public class RuntimeProperties extends ApplicationProperties {
 
     public static Properties get() {
         try {
-            if (instance.get() == null) {
-                instance.compareAndSet(null, new RuntimeProperties());
+            if (INSTANCE.get() == null) {
+                INSTANCE.compareAndSet(null, new RuntimeProperties());
             }
-            return instance.get();
+            return INSTANCE.get();
         } catch (FalconException e) {
-            throw new RuntimeException("Unable to read application " +
-                    "runtime properties", e);
+            throw new RuntimeException("Unable to read application " + "runtime properties", e);
         }
     }
 
-    private class DynamicLoader implements Runnable {
+    /**
+     * Thread for loading properties periodically.
+     */
+    private final class DynamicLoader implements Runnable {
 
         private static final long REFRESH_DELAY = 300000L;
         private static final int MAX_ITER = 20;  //1hr

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/util/StartupProperties.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/util/StartupProperties.java b/common/src/main/java/org/apache/falcon/util/StartupProperties.java
index 4a19df4..7522b0d 100644
--- a/common/src/main/java/org/apache/falcon/util/StartupProperties.java
+++ b/common/src/main/java/org/apache/falcon/util/StartupProperties.java
@@ -23,11 +23,14 @@ import org.apache.falcon.FalconException;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class StartupProperties extends ApplicationProperties {
+/**
+ * Properties read during application startup.
+ */
+public final class StartupProperties extends ApplicationProperties {
 
     private static final String PROPERTY_FILE = "startup.properties";
 
-    private static final AtomicReference<StartupProperties> instance =
+    private static final AtomicReference<StartupProperties> INSTANCE =
             new AtomicReference<StartupProperties>();
 
     private StartupProperties() throws FalconException {
@@ -41,13 +44,12 @@ public class StartupProperties extends ApplicationProperties {
 
     public static Properties get() {
         try {
-            if (instance.get() == null) {
-                instance.compareAndSet(null, new StartupProperties());
+            if (INSTANCE.get() == null) {
+                INSTANCE.compareAndSet(null, new StartupProperties());
             }
-            return instance.get();
+            return INSTANCE.get();
         } catch (FalconException e) {
-            throw new RuntimeException("Unable to read application " +
-                    "startup properties", e);
+            throw new RuntimeException("Unable to read application " + "startup properties", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
index 76a9edc..26243e7 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowBuilder.java
@@ -27,6 +27,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+/**
+ * Builder for building workflow definition for the underlying scheduler.
+ * @param <T>
+ */
 public abstract class WorkflowBuilder<T extends Entity> {
 
     public static WorkflowBuilder<Entity> getBuilder(String engine, Entity entity) throws FalconException {
@@ -37,7 +41,7 @@ public abstract class WorkflowBuilder<T extends Entity> {
     public abstract Map<String, Properties> newWorkflowSchedule(T entity, List<String> clusters) throws FalconException;
 
     public abstract Properties newWorkflowSchedule(T entity, Date startDate, String clusterName, String user)
-            throws FalconException;
+        throws FalconException;
 
     public abstract String[] getWorkflowNames(T entity);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java b/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
index a267e39..756c6b8 100644
--- a/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
+++ b/common/src/main/java/org/apache/falcon/workflow/WorkflowEngineFactory.java
@@ -22,16 +22,18 @@ import org.apache.falcon.FalconException;
 import org.apache.falcon.util.ReflectionUtils;
 import org.apache.falcon.workflow.engine.AbstractWorkflowEngine;
 
+/**
+ * Factory for providing appropriate workflow engine to the falcon service.
+ */
 @SuppressWarnings("unchecked")
-public class WorkflowEngineFactory {
+public final class WorkflowEngineFactory {
 
     private static final String WORKFLOW_ENGINE = "workflow.engine.impl";
 
     private WorkflowEngineFactory() {
     }
 
-    public static AbstractWorkflowEngine getWorkflowEngine()
-            throws FalconException {
+    public static AbstractWorkflowEngine getWorkflowEngine() throws FalconException {
         return ReflectionUtils.getInstance(WORKFLOW_ENGINE);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java b/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
index 0e0dcd0..f7526e4 100644
--- a/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
+++ b/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
@@ -29,7 +29,7 @@ import java.util.Set;
 
 /**
  * Workflow engine should minimally support the
- * following operations
+ * following operations.
  */
 public abstract class AbstractWorkflowEngine {
 
@@ -61,16 +61,16 @@ public abstract class AbstractWorkflowEngine {
     public abstract InstancesResult getRunningInstances(Entity entity) throws FalconException;
 
     public abstract InstancesResult killInstances(Entity entity, Date start, Date end, Properties props)
-            throws FalconException;
+        throws FalconException;
 
     public abstract InstancesResult reRunInstances(Entity entity, Date start, Date end, Properties props)
-            throws FalconException;
+        throws FalconException;
 
     public abstract InstancesResult suspendInstances(Entity entity, Date start, Date end, Properties props)
-            throws FalconException;
+        throws FalconException;
 
     public abstract InstancesResult resumeInstances(Entity entity, Date start, Date end, Properties props)
-            throws FalconException;
+        throws FalconException;
 
     public abstract InstancesResult getStatus(Entity entity, Date start, Date end) throws FalconException;
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/main/java/org/apache/falcon/workflow/engine/WorkflowEngineActionListener.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/engine/WorkflowEngineActionListener.java b/common/src/main/java/org/apache/falcon/workflow/engine/WorkflowEngineActionListener.java
index 11916ec..2a1cbd4 100644
--- a/common/src/main/java/org/apache/falcon/workflow/engine/WorkflowEngineActionListener.java
+++ b/common/src/main/java/org/apache/falcon/workflow/engine/WorkflowEngineActionListener.java
@@ -21,6 +21,10 @@ package org.apache.falcon.workflow.engine;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 
+/**
+ * Listener that will be notified before and after
+ * workflow life cycle operations are performed.
+ */
 public interface WorkflowEngineActionListener {
 
     void beforeSchedule(Entity entity, String cluster) throws FalconException;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 101f987..6324448 100644
--- a/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
@@ -33,34 +33,28 @@ import org.testng.annotations.Test;
 
 import java.io.IOException;
 
+/**
+ * Test for log cleanup service.
+ */
 public class LogCleanupServiceTest extends AbstractTestBase {
 
     private FileSystem fs;
     private FileSystem tfs;
     private EmbeddedCluster targetDfsCluster;
-    Path instanceLogPath = new Path(
-            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
-                    + "/logs/job-2010-01-01-01-00/000");
-    Path instanceLogPath1 = new Path(
-            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
-                    + "/logs/job-2010-01-01-01-00/001");
-    Path instanceLogPath2 = new Path(
-            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
-                    + "/logs/job-2010-01-01-02-00/001");
-    Path instanceLogPath3 = new Path(
-            "/projects/falcon/staging/falcon/workflows/process/" + "sample2"
-                    + "/logs/job-2010-01-01-01-00/000");
-    Path instanceLogPath4 = new Path(
-            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
-                    + "/logs/latedata/2010-01-01-01-00");
-    Path feedInstanceLogPath = new Path(
-            "/projects/falcon/staging/falcon/workflows/feed/"
-                    + "impressionFeed"
-                    + "/logs/job-2010-01-01-01-00/testCluster/000");
-    Path feedInstanceLogPath1 = new Path(
-            "/projects/falcon/staging/falcon/workflows/feed/"
-                    + "impressionFeed2"
-                    + "/logs/job-2010-01-01-01-00/testCluster/000");
+    private 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/"
+        + "sample" + "/logs/job-2010-01-01-01-00/001");
+    private 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/"
+        + "sample2" + "/logs/job-2010-01-01-01-00/000");
+    private 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/"
+        + "impressionFeed" + "/logs/job-2010-01-01-01-00/testCluster/000");
+    private Path feedInstanceLogPath1 = new Path("/projects/falcon/staging/falcon/workflows/feed/"
+        + "impressionFeed2" + "/logs/job-2010-01-01-01-00/testCluster/000");
 
 
     @AfterClass
@@ -76,8 +70,7 @@ public class LogCleanupServiceTest extends AbstractTestBase {
         fs = dfsCluster.getFileSystem();
 
         storeEntity(EntityType.CLUSTER, "testCluster");
-        System.setProperty("test.build.data",
-                "target/tdfs/data" + System.currentTimeMillis());
+        System.setProperty("test.build.data", "target/tdfs/data" + System.currentTimeMillis());
         this.targetDfsCluster = EmbeddedCluster.newCluster("backupCluster", false);
         conf = targetDfsCluster.getConf();
 
@@ -87,9 +80,8 @@ public class LogCleanupServiceTest extends AbstractTestBase {
         storeEntity(EntityType.FEED, "imp-click-join1");
         storeEntity(EntityType.FEED, "imp-click-join2");
         storeEntity(EntityType.PROCESS, "sample");
-        Process process = ConfigurationStore.get().get(EntityType.PROCESS,
-                "sample");
-        Process otherProcess = (Process) process.clone();
+        Process process = ConfigurationStore.get().get(EntityType.PROCESS, "sample");
+        Process otherProcess = (Process) process.copy();
         otherProcess.setName("sample2");
         otherProcess.setFrequency(new Frequency("days(1)"));
         ConfigurationStore.get().remove(EntityType.PROCESS,
@@ -115,13 +107,10 @@ public class LogCleanupServiceTest extends AbstractTestBase {
         tfs.createNewFile(new Path(feedInstanceLogPath, "oozie.log"));
 
         Thread.sleep(61000);
-
-
     }
 
     @Test
-    public void testProcessLogs() throws IOException, FalconException,
-                                         InterruptedException {
+    public void testProcessLogs() throws IOException, FalconException, InterruptedException {
 
         AbstractCleanupHandler processCleanupHandler = new ProcessCleanupHandler();
         processCleanupHandler.cleanup();
@@ -134,8 +123,7 @@ public class LogCleanupServiceTest extends AbstractTestBase {
     }
 
     @Test
-    public void testFeedLogs() throws IOException, FalconException,
-                                      InterruptedException {
+    public void testFeedLogs() throws IOException, FalconException, InterruptedException {
 
         AbstractCleanupHandler feedCleanupHandler = new FeedCleanupHandler();
         feedCleanupHandler.cleanup();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java b/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
index 394fa70..f08c6ac 100644
--- a/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
+++ b/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
@@ -40,6 +40,9 @@ import javax.xml.bind.Unmarshaller;
 import java.io.StringWriter;
 import java.util.Collection;
 
+/**
+ * Base class for config store test.
+ */
 public class AbstractTestBase {
     protected static final String PROCESS_XML = "/config/process/process-0.1.xml";
     protected static final String FEED_XML = "/config/feed/feed-0.1.xml";
@@ -71,29 +74,30 @@ public class AbstractTestBase {
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
         switch (type) {
-            case CLUSTER:
-                Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
-                cluster.setName(name);
-                ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
-                store.publish(type, cluster);
-                break;
+        case CLUSTER:
+            Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
+            cluster.setName(name);
+            ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
+            store.publish(type, cluster);
+            break;
 
-            case FEED:
-                Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
-                feed.setName(name);
-                store.publish(type, feed);
-                break;
+        case FEED:
+            Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
+            feed.setName(name);
+            store.publish(type, feed);
+            break;
 
-            case PROCESS:
-                Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
-                process.setName(name);
-                FileSystem fs = dfsCluster.getFileSystem();
-                fs.mkdirs(new Path(process.getWorkflow().getPath()));
-                if (!fs.exists(new Path(process.getWorkflow() + "/lib"))) {
-                    fs.mkdirs(new Path(process.getWorkflow() + "/lib"));
-                }
-                store.publish(type, process);
-                break;
+        case PROCESS:
+            Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
+            process.setName(name);
+            FileSystem fs = dfsCluster.getFileSystem();
+            fs.mkdirs(new Path(process.getWorkflow().getPath()));
+            if (!fs.exists(new Path(process.getWorkflow() + "/lib"))) {
+                fs.mkdirs(new Path(process.getWorkflow() + "/lib"));
+            }
+            store.publish(type, process);
+            break;
+        default:
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java b/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
index d60126e..936b478 100644
--- a/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
@@ -26,6 +26,9 @@ import org.testng.annotations.Test;
 
 import java.util.Set;
 
+/**
+ * Tests for validating relationship between cluster to data center/co-location.
+ */
 @Test
 public class ColoClusterRelationTest extends AbstractTestBase {
     private Cluster newCluster(String name, String colo) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java b/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
index 509fce8..e947f69 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
@@ -22,6 +22,9 @@ import org.apache.falcon.entity.v0.EntityType;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Test for validating entity types.
+ */
 public class EntityTypeTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
index 622fe48..d8a44ea 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -32,6 +32,9 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.TimeZone;
 
+/**
+ * Test for validating Entity util helper methods.
+ */
 public class EntityUtilTest extends AbstractTestBase {
     private static TimeZone tz = TimeZone.getTimeZone("UTC");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java b/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
index 100fb63..f6994fc 100644
--- a/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
@@ -24,6 +24,9 @@ import org.apache.falcon.entity.v0.cluster.Property;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Test for feed helper methods.
+ */
 public class FeedHelperTest {
     @Test
     public void testPartitionExpression() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java b/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
index 6486397..6060731 100644
--- a/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
+++ b/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
@@ -25,6 +25,9 @@ import org.testng.annotations.Test;
 
 import java.util.Arrays;
 
+/**
+ * Test for workflow name builder.
+ */
 public class TestWorkflowNameBuilder {
 
     @Test
@@ -60,7 +63,7 @@ public class TestWorkflowNameBuilder {
     }
 
     @Test
-    public void WorkflowNameTest() {
+    public void workflowNameTest() {
         Feed feed = new Feed();
         feed.setName("raw-logs");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 b54ca63..1b34141 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
@@ -39,6 +39,9 @@ import java.io.StringWriter;
 
 import static org.testng.AssertJUnit.assertEquals;
 
+/**
+ * Test for validating cluster entity parsing.
+ */
 public class ClusterEntityParserTest extends AbstractTestBase {
 
     private final ClusterEntityParser parser = (ClusterEntityParser) EntityParserFactory.getParser(EntityType.CLUSTER);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
index c461ee5..ff8cc46 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
@@ -18,10 +18,6 @@
 
 package org.apache.falcon.entity.parser;
 
-/**
- * Test Cases for ProcessEntityParser
- */
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.AbstractTestBase;
 import org.apache.falcon.entity.FeedHelper;
@@ -44,6 +40,9 @@ import java.io.StringWriter;
 
 import static org.testng.AssertJUnit.assertEquals;
 
+/**
+ * Test Cases for Feed entity parser.
+ */
 public class FeedEntityParserTest extends AbstractTestBase {
 
     private final FeedEntityParser parser = (FeedEntityParser) EntityParserFactory
@@ -146,6 +145,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
     }
 
 
+    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testPartitionExpression() throws FalconException {
         Feed feed = (Feed) parser.parseAndValidate(ProcessEntityParserTest.class
@@ -160,7 +160,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch (ValidationException e) {
+        } catch (ValidationException ignore) {
         }
 
         //When there are more than 1 src clusters, the partition expression should contain cluster variable
@@ -168,7 +168,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch (ValidationException e) {
+        } catch (ValidationException ignore) {
         }
 
         //When there are more than 1 target cluster, there should be partition expre
@@ -176,7 +176,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch (ValidationException e) {
+        } catch (ValidationException ignore) {
         }
 
         //When there are more than 1 target clusters, the partition expression should contain cluster variable
@@ -184,7 +184,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch (ValidationException e) {
+        } catch (ValidationException ignore) {
         }
 
         //Number of parts in partition expression < number of partitions defined for feed
@@ -192,7 +192,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch (ValidationException e) {
+        } catch (ValidationException ignore) {
         }
 
         feed.getClusters().getClusters().get(0).setPartition(null);
@@ -201,6 +201,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         feed.setPartitions(null);
         parser.validate(feed);
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Test
     public void testInvalidClusterValidityTime() {
@@ -332,6 +333,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
         ConfigurationStore.get().publish(EntityType.FEED, feed1);
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testInvalidGroupNames() throws FalconException, JAXBException {
         Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
@@ -361,6 +363,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
 
         }
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Test
     public void testClusterPartitionExp() throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 4f62431..35ca217 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
@@ -30,6 +30,9 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+/**
+ * Test for feed update helper methods.
+ */
 public class FeedUpdateTest extends AbstractTestBase {
 
     private final FeedEntityParser parser = (FeedEntityParser)
@@ -81,8 +84,7 @@ public class FeedUpdateTest extends AbstractTestBase {
             storeEntity(EntityType.PROCESS, "sample");
 
             //Try parsing the same feed xml
-            parser.parseAndValidate(this.getClass()
-                    .getResourceAsStream(FEED_XML));
+            parser.parseAndValidate(this.getClass().getResourceAsStream(FEED_XML));
         } catch (Exception e) {
             Assert.fail("Didn't expect feed parsing to fail", e);
         }
@@ -110,6 +112,7 @@ public class FeedUpdateTest extends AbstractTestBase {
         }
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testFeedUpdateWithViolations() throws Exception {
         ConfigurationStore.get().remove(EntityType.FEED, "clicks");
@@ -124,7 +127,7 @@ public class FeedUpdateTest extends AbstractTestBase {
         Process process = processParser.parseAndValidate(this.getClass()
                 .getResourceAsStream(PROCESS1_XML));
         ConfigurationStore.get().publish(EntityType.PROCESS, process);
-        Process p1 = (Process) process.clone();
+        Process p1 = (Process) process.copy();
         p1.setName("sample2");
         ConfigurationStore.get().publish(EntityType.PROCESS, p1);
 
@@ -136,4 +139,5 @@ public class FeedUpdateTest extends AbstractTestBase {
         } catch (ValidationException ignore) {
         }
     }
+    //RESUME CHECKSTYLE CHECK
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 88e9968..af9fddd 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
@@ -40,14 +40,16 @@ import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.List;
 
+/**
+ * Tests for validating process entity parser.
+ */
 public class ProcessEntityParserTest extends AbstractTestBase {
 
     private final ProcessEntityParser parser = (ProcessEntityParser) EntityParserFactory.getParser(EntityType.PROCESS);
-    private String INVALID_PROCESS_XML = "/config/process/process-invalid.xml";
 
     @Test
     public void testNotNullgetUnmarshaller() throws Exception {
-        final Unmarshaller unmarshaller = EntityType.PROCESS.getUnmarshaller();
+        Unmarshaller unmarshaller = EntityType.PROCESS.getUnmarshaller();
         Assert.assertNotNull(unmarshaller);
     }
 
@@ -115,6 +117,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
         // TODO for retry and late policy
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testELExpressions() throws Exception {
         Process process = parser.parseAndValidate(getClass().getResourceAsStream(PROCESS_XML));
@@ -140,13 +143,14 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             throw new AssertionError("Expected ValidationException!");
         } catch (ValidationException e) {
         }
-
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Test(expectedExceptions = FalconException.class)
     public void doParseInvalidXML() throws IOException, FalconException {
 
-        parser.parseAndValidate(this.getClass().getResourceAsStream(INVALID_PROCESS_XML));
+        String invalidProcessXml = "/config/process/process-invalid.xml";
+        parser.parseAndValidate(this.getClass().getResourceAsStream(invalidProcessXml));
     }
 
     @Test(expectedExceptions = ValidationException.class)
@@ -161,6 +165,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
         parser.parseAndValidate("<process></process>");
     }
 
+    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testConcurrentParsing() throws Exception {
         List<Thread> threadList = new ArrayList<Thread>();
@@ -184,6 +189,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             thread.join();
         }
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Test(expectedExceptions = ValidationException.class)
     public void testInvalidProcessValidity() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java b/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
index e58184e..86298cc 100644
--- a/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/store/ConfigurationStoreTest.java
@@ -32,9 +32,12 @@ import org.testng.annotations.Test;
 
 import java.io.IOException;
 
+/**
+ * Tests for validating configuration store.
+ */
 public class ConfigurationStoreTest {
 
-    private static Logger LOG = Logger.getLogger(ConfigurationStoreTest.class);
+    private static final Logger LOG = Logger.getLogger(ConfigurationStoreTest.class);
 
     private ConfigurationStore store = ConfigurationStore.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
index 9b4bffc..a8c5eb1 100644
--- a/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/v0/EntityGraphTest.java
@@ -30,6 +30,9 @@ import org.testng.annotations.Test;
 
 import java.util.Set;
 
+/**
+ * Entity graph tests.
+ */
 public class EntityGraphTest extends AbstractTestBase {
 
     private ConfigurationStore store = ConfigurationStore.get();
@@ -130,10 +133,10 @@ public class EntityGraphTest extends AbstractTestBase {
         if (process.getOutputs() == null) {
             process.setOutputs(new Outputs());
         }
-        Outputs Outputs = process.getOutputs();
-        Output Output = new Output();
-        Output.setFeed(feed);
-        Outputs.getOutputs().add(Output);
+        Outputs outputs = process.getOutputs();
+        Output output = new Output();
+        output.setFeed(feed);
+        outputs.getOutputs().add(output);
         Feed f1 = new Feed();
         f1.setName(feed);
         Clusters clusters = new Clusters();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java b/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
index f7812b9..a2087c0 100644
--- a/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
+++ b/common/src/test/java/org/apache/falcon/group/FeedGroupMapTest.java
@@ -34,6 +34,9 @@ import org.testng.annotations.Test;
 import javax.xml.bind.JAXBException;
 import java.util.Map;
 
+/**
+ * Feed group map tests.
+ */
 public class FeedGroupMapTest extends AbstractTestBase {
     private ConfigurationStore store = ConfigurationStore.get();
     private static Cluster cluster;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java b/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
index b31e3cc..fe7155b 100644
--- a/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
+++ b/common/src/test/java/org/apache/falcon/security/CurrentUserTest.java
@@ -21,6 +21,9 @@ package org.apache.falcon.security;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+/**
+ * Test for current user's thread safety.
+ */
 public class CurrentUserTest {
 
     @Test(threadPoolSize = 10, invocationCount = 10, timeOut = 10000)

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/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 11e5d06..23fa148 100644
--- a/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
+++ b/common/src/test/java/org/apache/falcon/update/UpdateHelperTest.java
@@ -36,6 +36,9 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+/**
+ * Test for Update helper methods.
+ */
 public class UpdateHelperTest extends AbstractTestBase {
     private final FeedEntityParser parser = (FeedEntityParser)
             EntityParserFactory.getParser(EntityType.FEED);
@@ -69,7 +72,7 @@ public class UpdateHelperTest extends AbstractTestBase {
         Feed oldFeed = parser.parseAndValidate(this.getClass()
                 .getResourceAsStream(FEED_XML));
         String cluster = "testCluster";
-        Feed newFeed = (Feed) oldFeed.clone();
+        Feed newFeed = (Feed) oldFeed.copy();
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldFeed, newFeed, cluster));
 
         newFeed.setGroups("newgroups");
@@ -81,7 +84,7 @@ public class UpdateHelperTest extends AbstractTestBase {
 
         Process oldProcess = processParser.parseAndValidate(this.getClass().
                 getResourceAsStream(PROCESS_XML));
-        Process newProcess = (Process) oldProcess.clone();
+        Process newProcess = (Process) oldProcess.copy();
 
         newProcess.getRetry().setPolicy(PolicyType.FINAL);
         Assert.assertFalse(UpdateHelper.shouldUpdate(oldProcess, newProcess, cluster));
@@ -98,7 +101,7 @@ public class UpdateHelperTest extends AbstractTestBase {
         Feed oldFeed = parser.parseAndValidate(this.getClass()
                 .getResourceAsStream(FEED_XML));
 
-        Feed newFeed = (Feed) oldFeed.clone();
+        Feed newFeed = (Feed) oldFeed.copy();
         Process process = processParser.parseAndValidate(this.getClass().
                 getResourceAsStream(PROCESS_XML));
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java b/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
index fc3d604..6b2ec06 100644
--- a/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
+++ b/common/src/test/java/org/apache/falcon/util/StartupPropertiesTest.java
@@ -23,6 +23,9 @@ import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.assertEquals;
 
+/**
+ * Test for startup properties test.
+ */
 @Test
 public class StartupPropertiesTest {
     @BeforeClass

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
----------------------------------------------------------------------
diff --git a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
index 11eeadd..8eec0d4 100644
--- a/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
+++ b/oozie/src/main/java/org/apache/falcon/logging/LogProvider.java
@@ -76,7 +76,7 @@ public final class LogProvider {
         if (StringUtils.isEmpty(runId)) {
             Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
                     EntityUtil.getLogPath(cluster, entity) + "/job-"
-                            + EntityUtil.UTCtoURIDate(instance.instance) + "/*");
+                            + EntityUtil.fromUTCtoURIDate(instance.instance) + "/*");
 
             FileStatus[] runs = fs.globStatus(jobPath);
             if (runs.length > 0) {
@@ -90,7 +90,7 @@ public final class LogProvider {
         } else {
             Path jobPath = new Path(ClusterHelper.getStorageUrl(cluster),
                     EntityUtil.getLogPath(cluster, entity) + "/job-"
-                            + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                            + EntityUtil.fromUTCtoURIDate(instance.instance) + "/"
                             + getFormatedRunId(runId));
             if (fs.exists(jobPath)) {
                 return getFormatedRunId(runId);
@@ -108,7 +108,7 @@ public final class LogProvider {
 
         Path actionPaths = new Path(ClusterHelper.getStorageUrl(cluster),
                 EntityUtil.getLogPath(cluster, entity) + "/job-"
-                        + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                        + EntityUtil.fromUTCtoURIDate(instance.instance) + "/"
                         + formatedRunId + "/*");
         FileStatus[] actions = fs.globStatus(actionPaths);
         InstanceAction[] instanceActions = new InstanceAction[actions.length - 1];
@@ -119,7 +119,7 @@ public final class LogProvider {
             String dfsBrowserUrl = getDFSbrowserUrl(
                     ClusterHelper.getStorageUrl(cluster),
                     EntityUtil.getLogPath(cluster, entity) + "/job-"
-                            + EntityUtil.UTCtoURIDate(instance.instance) + "/"
+                            + EntityUtil.fromUTCtoURIDate(instance.instance) + "/"
                             + formatedRunId, file.getPath().getName());
             if (filePath.getName().equals("oozie.log")) {
                 instance.logFile = dfsBrowserUrl;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
index bd2d6ac..4a0f7c4 100644
--- a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
+++ b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
@@ -26,6 +26,9 @@ import org.apache.falcon.entity.v0.process.Process;
 
 import javax.xml.bind.Unmarshaller;
 
+/**
+ * Base for falcon unit tests involving configuration store.
+ */
 public class AbstractTestBase {
     private static final String PROCESS_XML = "/config/process/process-0.1.xml";
     private static final String FEED_XML = "/config/feed/feed-0.1.xml";
@@ -37,23 +40,24 @@ public class AbstractTestBase {
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
         switch (type) {
-            case CLUSTER:
-                Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
-                cluster.setName(name);
-                store.publish(type, cluster);
-                break;
+        case CLUSTER:
+            Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
+            cluster.setName(name);
+            store.publish(type, cluster);
+            break;
 
-            case FEED:
-                Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
-                feed.setName(name);
-                store.publish(type, feed);
-                break;
+        case FEED:
+            Feed feed = (Feed) unmarshaller.unmarshal(this.getClass().getResource(FEED_XML));
+            feed.setName(name);
+            store.publish(type, feed);
+            break;
 
-            case PROCESS:
-                Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
-                process.setName(name);
-                store.publish(type, process);
-                break;
+        case PROCESS:
+            Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
+            process.setName(name);
+            store.publish(type, process);
+            break;
+        default:
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/bdadf2ed/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
----------------------------------------------------------------------
diff --git a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
index ad19157..e24cc69 100644
--- a/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
+++ b/rerun/src/main/java/org/apache/falcon/rerun/handler/LateRerunHandler.java
@@ -72,7 +72,7 @@ public class LateRerunHandler<M extends DelayedQueue<LaterunEvent>> extends
                 String srcClusterName = this.getWfEngine().getWorkflowProperty(
                         cluster, wfId, "srcClusterName");
                 Path lateLogPath = this.getLateLogPath(logDir,
-                        EntityUtil.UTCtoURIDate(nominalTime), srcClusterName);
+                        EntityUtil.fromUTCtoURIDate(nominalTime), srcClusterName);
                 LOG.info("Going to delete path:" + lateLogPath);
                 FileSystem fs = FileSystem.get(getConfiguration(cluster,
                         wfId));


[29/47] git commit: Check style fixes relating to metrics module

Posted by sr...@apache.org.
Check style fixes relating to metrics module


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

Branch: refs/heads/master
Commit: 2a94690df67c1df868dcb7f816fffa6cd2f6651b
Parents: 46f07b3
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 15:57:46 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 15:57:46 2013 +0530

----------------------------------------------------------------------
 .../apache/falcon/aspect/AbstractFalconAspect.java |   11 +++-----
 .../org/apache/falcon/aspect/GenericAlert.java     |   12 +++++++--
 .../org/apache/falcon/aspect/ResourceMessage.java  |    7 ++++-
 .../java/org/apache/falcon/monitors/Dimension.java |    3 ++
 .../java/org/apache/falcon/monitors/Monitored.java |    3 ++
 .../java/org/apache/falcon/monitors/TimeTaken.java |    3 ++
 .../org/apache/falcon/plugin/LoggingPlugin.java    |    3 ++
 .../org/apache/falcon/plugin/MonitoringPlugin.java |    3 ++
 .../falcon/util/ResourcesReflectionUtil.java       |   19 ++++++++-------
 9 files changed, 44 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java b/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
index 1f3f256..314e27b 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/AbstractFalconAspect.java
@@ -34,8 +34,7 @@ import java.util.Map;
 @Aspect
 public abstract class AbstractFalconAspect {
 
-    private static final Logger LOG = Logger
-            .getLogger(AbstractFalconAspect.class);
+    private static final Logger LOG = Logger.getLogger(AbstractFalconAspect.class);
 
     @Around("@annotation(org.apache.falcon.monitors.Monitored)")
     public Object logAround(ProceedingJoinPoint joinPoint) throws Throwable {
@@ -88,11 +87,9 @@ public abstract class AbstractFalconAspect {
         }
         Integer timeTakenArg = ResourcesReflectionUtil
                 .getResourceTimeTakenName(methodName);
-        return timeTakenArg == null ? new ResourceMessage(action, dimensions,
-                status, executionTime) : new ResourceMessage(action,
-                dimensions, status, Long.valueOf(args[timeTakenArg]
-                .toString()));
+        return timeTakenArg == null ? new ResourceMessage(action, dimensions, status, executionTime)
+            : new ResourceMessage(action, dimensions, status, Long.valueOf(args[timeTakenArg].toString()));
     }
 
-    abstract public void publishMessage(ResourceMessage message);
+    public abstract void publishMessage(ResourceMessage message);
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
index b792434..8099760 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/GenericAlert.java
@@ -27,8 +27,11 @@ import org.aspectj.lang.annotation.Aspect;
  * and iMon, invoke this method from code.
  */
 @Aspect
-public class GenericAlert {
+public final class GenericAlert {
 
+    private GenericAlert() {}
+
+    //SUSPEND CHECKSTYLE CHECK
     @Monitored(event = "retry-instance-failed")
     public static String alertRetryFailed(
             @Dimension(value = "entity-type") String entityType,
@@ -65,7 +68,8 @@ public class GenericAlert {
             @Dimension(value = "error-message") String errorMessage,
             @Dimension(value = "message") String message,
             @TimeTaken long timeTaken)
-            throws Exception {
+        throws Exception {
+
         return "IGNORE";
     }
 
@@ -80,9 +84,11 @@ public class GenericAlert {
             @Dimension(value = "operation") String operation,
             @Dimension(value = "start-time") String startTime,
             @TimeTaken long timeTaken)
-            throws Exception {
+        throws Exception {
+
         return "IGNORE";
     }
+    //RESUME CHECKSTYLE CHECK
 
     @Monitored(event = "rerun-queue-failed")
     public static String alertRerunConsumerFailed(

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java b/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
index c7ad8e7..d989f9e 100644
--- a/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
+++ b/metrics/src/main/java/org/apache/falcon/aspect/ResourceMessage.java
@@ -20,7 +20,9 @@ package org.apache.falcon.aspect;
 
 import java.util.Map;
 
-//Message to be sent to logging system
+/**
+ * Message to be sent to logging system.
+ */
 public class ResourceMessage {
 
     private String action;
@@ -28,6 +30,9 @@ public class ResourceMessage {
     private Status status;
     private long executionTime;
 
+    /**
+     * Convenient enum for resource message status.
+     */
     public enum Status {
         SUCCEEDED, FAILED
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/monitors/Dimension.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/monitors/Dimension.java b/metrics/src/main/java/org/apache/falcon/monitors/Dimension.java
index 233e2f7..387e43a 100644
--- a/metrics/src/main/java/org/apache/falcon/monitors/Dimension.java
+++ b/metrics/src/main/java/org/apache/falcon/monitors/Dimension.java
@@ -23,6 +23,9 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
+/**
+ * Dimension annotation for metrics collection.
+ */
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.PARAMETER)
 public @interface Dimension {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/monitors/Monitored.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/monitors/Monitored.java b/metrics/src/main/java/org/apache/falcon/monitors/Monitored.java
index 83314d4..768aa4e 100644
--- a/metrics/src/main/java/org/apache/falcon/monitors/Monitored.java
+++ b/metrics/src/main/java/org/apache/falcon/monitors/Monitored.java
@@ -23,6 +23,9 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
+/**
+ * Monitored annotation for metrics collection.
+ */
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.METHOD)
 public @interface Monitored {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/monitors/TimeTaken.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/monitors/TimeTaken.java b/metrics/src/main/java/org/apache/falcon/monitors/TimeTaken.java
index 94854b2..5b5aec4 100644
--- a/metrics/src/main/java/org/apache/falcon/monitors/TimeTaken.java
+++ b/metrics/src/main/java/org/apache/falcon/monitors/TimeTaken.java
@@ -23,6 +23,9 @@ import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
+/**
+ * Time taken  annotation for metrics collection.
+ */
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.PARAMETER)
 public @interface TimeTaken {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/plugin/LoggingPlugin.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/plugin/LoggingPlugin.java b/metrics/src/main/java/org/apache/falcon/plugin/LoggingPlugin.java
index d7e5a3a..41a52bf 100644
--- a/metrics/src/main/java/org/apache/falcon/plugin/LoggingPlugin.java
+++ b/metrics/src/main/java/org/apache/falcon/plugin/LoggingPlugin.java
@@ -21,6 +21,9 @@ package org.apache.falcon.plugin;
 import org.apache.falcon.aspect.ResourceMessage;
 import org.apache.log4j.Logger;
 
+/**
+ * Plugin for logging metrics using log4j.
+ */
 public class LoggingPlugin implements MonitoringPlugin {
     private static final Logger METRIC = Logger.getLogger("METRIC");
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/plugin/MonitoringPlugin.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/plugin/MonitoringPlugin.java b/metrics/src/main/java/org/apache/falcon/plugin/MonitoringPlugin.java
index 36a108c..d33e65a 100644
--- a/metrics/src/main/java/org/apache/falcon/plugin/MonitoringPlugin.java
+++ b/metrics/src/main/java/org/apache/falcon/plugin/MonitoringPlugin.java
@@ -19,6 +19,9 @@ package org.apache.falcon.plugin;
 
 import org.apache.falcon.aspect.ResourceMessage;
 
+/**
+ * Generic interface to receiving monitoring and metric messages.
+ */
 public interface MonitoringPlugin {
 
     void monitor(ResourceMessage message);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2a94690d/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
----------------------------------------------------------------------
diff --git a/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java b/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
index 119a7f7..8744ad8 100644
--- a/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
+++ b/metrics/src/main/java/org/apache/falcon/util/ResourcesReflectionUtil.java
@@ -32,13 +32,11 @@ import java.util.Map;
  * Builds a cached of methods annotated with Monitored and params of methods
  * annotated with Dimension.
  */
-public class ResourcesReflectionUtil {
+public final class ResourcesReflectionUtil {
 
-    private static final Map<String, MethodAnnotation> methods = new HashMap<String, MethodAnnotation>();
+    private static final Map<String, MethodAnnotation> METHODS = new HashMap<String, MethodAnnotation>();
 
-    private ResourcesReflectionUtil() {
-
-    }
+    private ResourcesReflectionUtil() { }
 
     static {
         //TODO load these classes from properties file
@@ -50,18 +48,21 @@ public class ResourcesReflectionUtil {
     }
 
     public static Map<Integer, String> getResourceDimensionsName(String methodName) {
-        return methods.get(methodName) != null ? Collections.unmodifiableMap(methods.get(methodName).params) : null;
+        return METHODS.get(methodName) != null ? Collections.unmodifiableMap(METHODS.get(methodName).params) : null;
     }
 
     public static String getResourceMonitorName(String methodName) {
-        return methods.get(methodName) != null ? methods.get(methodName).monitoredName : null;
+        return METHODS.get(methodName) != null ? METHODS.get(methodName).monitoredName : null;
     }
 
     public static Integer getResourceTimeTakenName(String methodName) {
-        return methods.get(methodName) != null ? methods.get(methodName).timeTakenArgIndex
+        return METHODS.get(methodName) != null ? METHODS.get(methodName).timeTakenArgIndex
                 : null;
     }
 
+    /**
+     * Holder class for method annotation.
+     */
     public static class MethodAnnotation {
         private String monitoredName;
         // argument <index,DimensionValue>
@@ -101,7 +102,7 @@ public class ResourcesReflectionUtil {
                             .getParameterAnnotations();
                     // scan every param
                     annotation.params = getDeclaredParamAnnots(paramAnnots, annotation);
-                    methods.put(
+                    METHODS.put(
                             clazz.getSimpleName() + "."
                                     + declMethod.getName(), annotation);
                 }


[32/47] git commit: More check style fixes relating to common module

Posted by sr...@apache.org.
More check style fixes relating to common module


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

Branch: refs/heads/master
Commit: 9bfdf3bb2236496e1f16f0722004902e8c347c23
Parents: 968f197
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 16:46:24 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 16:46:24 2013 +0530

----------------------------------------------------------------------
 .../apache/falcon/entity/WorkflowNameBuilder.java  |    2 -
 .../apache/falcon/security/FalconLoginModule.java  |    8 +++++-
 .../falcon/entity/parser/FeedEntityParserTest.java |   17 ++++++++-------
 .../falcon/entity/parser/FeedUpdateTest.java       |    3 +-
 .../entity/parser/ProcessEntityParserTest.java     |    9 ++++---
 5 files changed, 21 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9bfdf3bb/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
index a1216e7..6890594 100644
--- a/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
+++ b/common/src/main/java/org/apache/falcon/entity/WorkflowNameBuilder.java
@@ -41,7 +41,6 @@ public class WorkflowNameBuilder<T extends Entity> {
         this.entity = entity;
     }
 
-    //SUSPEND CHECKSTYLE CHECK
     public void setTag(Tag tag) {
         this.tag = tag;
     }
@@ -49,7 +48,6 @@ public class WorkflowNameBuilder<T extends Entity> {
     public void setSuffixes(List<String> suffixes) {
         this.suffixes = suffixes;
     }
-    //RESUME CHECKSTYLE CHECK
 
     public WorkflowName getWorkflowName() {
         return new WorkflowName(PREFIX, entity.getEntityType().name(),

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9bfdf3bb/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
index 4f4542c..d95e147 100644
--- a/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
+++ b/common/src/main/java/org/apache/falcon/security/FalconLoginModule.java
@@ -37,6 +37,10 @@ public class FalconLoginModule implements LoginModule {
 
     private Subject subject;
 
+    public Subject getSubject() {
+        return subject;
+    }
+
     @Override
     public boolean abort() throws LoginException {
         return true;
@@ -65,13 +69,13 @@ public class FalconLoginModule implements LoginModule {
         throw new LoginException("No such user " + subject);
     }
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK HiddenFieldCheck
     @Override
     public void initialize(Subject subject, CallbackHandler callbackHandler,
                            Map<String, ?> sharedState, Map<String, ?> options) {
         this.subject = subject;
     }
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK HiddenFieldCheck
 
     @Override
     public boolean login() throws LoginException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9bfdf3bb/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
index ff8cc46..1d24e6c 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
@@ -145,7 +145,6 @@ public class FeedEntityParserTest extends AbstractTestBase {
     }
 
 
-    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testPartitionExpression() throws FalconException {
         Feed feed = (Feed) parser.parseAndValidate(ProcessEntityParserTest.class
@@ -161,6 +160,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
         } catch (ValidationException ignore) {
+            //ignore
         }
 
         //When there are more than 1 src clusters, the partition expression should contain cluster variable
@@ -169,6 +169,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
         } catch (ValidationException ignore) {
+            //ignore
         }
 
         //When there are more than 1 target cluster, there should be partition expre
@@ -177,6 +178,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
         } catch (ValidationException ignore) {
+            //ignore
         }
 
         //When there are more than 1 target clusters, the partition expression should contain cluster variable
@@ -185,6 +187,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
         } catch (ValidationException ignore) {
+            //ignore
         }
 
         //Number of parts in partition expression < number of partitions defined for feed
@@ -193,6 +196,7 @@ public class FeedEntityParserTest extends AbstractTestBase {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
         } catch (ValidationException ignore) {
+            //ignore
         }
 
         feed.getClusters().getClusters().get(0).setPartition(null);
@@ -201,7 +205,6 @@ public class FeedEntityParserTest extends AbstractTestBase {
         feed.setPartitions(null);
         parser.validate(feed);
     }
-    //RESUME CHECKSTYLE CHECK
 
     @Test
     public void testInvalidClusterValidityTime() {
@@ -333,7 +336,6 @@ public class FeedEntityParserTest extends AbstractTestBase {
         ConfigurationStore.get().publish(EntityType.FEED, feed1);
     }
 
-    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testInvalidGroupNames() throws FalconException, JAXBException {
         Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
@@ -345,25 +347,24 @@ public class FeedEntityParserTest extends AbstractTestBase {
             feed1.setGroups("commaend,");
             parser.parseAndValidate(feed1.toString());
             Assert.fail("Expected exception");
-        } catch (FalconException e) {
-
+        } catch (FalconException ignore) {
+            //ignore
         }
         try {
             feed1.setGroups("group8,   group9");
             parser.parseAndValidate(feed1.toString());
             Assert.fail("Expected exception");
         } catch (FalconException e) {
-
+            //ignore
         }
         try {
             feed1.setGroups("space in group,group9");
             parser.parseAndValidate(feed1.toString());
             Assert.fail("Expected exception");
         } catch (FalconException e) {
-
+            //ignore
         }
     }
-    //RESUME CHECKSTYLE CHECK
 
     @Test
     public void testClusterPartitionExp() throws FalconException {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9bfdf3bb/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 35ca217..e887fee 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
@@ -112,7 +112,6 @@ public class FeedUpdateTest extends AbstractTestBase {
         }
     }
 
-    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testFeedUpdateWithViolations() throws Exception {
         ConfigurationStore.get().remove(EntityType.FEED, "clicks");
@@ -137,7 +136,7 @@ public class FeedUpdateTest extends AbstractTestBase {
                     .getResourceAsStream(FEED1_XML));
             Assert.fail("Expected feed parsing to fail");
         } catch (ValidationException ignore) {
+            //ignore
         }
     }
-    //RESUME CHECKSTYLE CHECK
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/9bfdf3bb/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 af9fddd..fd15062 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
@@ -117,7 +117,6 @@ public class ProcessEntityParserTest extends AbstractTestBase {
         // TODO for retry and late policy
     }
 
-    //SUSPEND CHECKSTYLE CHECK
     @Test
     public void testELExpressions() throws Exception {
         Process process = parser.parseAndValidate(getClass().getResourceAsStream(PROCESS_XML));
@@ -126,6 +125,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
         } catch (ValidationException e) {
+            //ignore
         }
 
         process.getInputs().getInputs().get(0).setStart("today(0,0)");
@@ -134,6 +134,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
         } catch (ValidationException e) {
+            //ignore
         }
 
         process.getInputs().getInputs().get(0).setStart("today(2,0)");
@@ -142,9 +143,9 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
         } catch (ValidationException e) {
+            //ignore
         }
     }
-    //RESUME CHECKSTYLE CHECK
 
     @Test(expectedExceptions = FalconException.class)
     public void doParseInvalidXML() throws IOException, FalconException {
@@ -165,7 +166,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
         parser.parseAndValidate("<process></process>");
     }
 
-    //SUSPEND CHECKSTYLE CHECK
+    //SUSPEND CHECKSTYLE CHECK HiddenFieldCheck
     @Test
     public void testConcurrentParsing() throws Exception {
         List<Thread> threadList = new ArrayList<Thread>();
@@ -189,7 +190,7 @@ public class ProcessEntityParserTest extends AbstractTestBase {
             thread.join();
         }
     }
-    //RESUME CHECKSTYLE CHECK
+    //RESUME CHECKSTYLE CHECK HiddenFieldCheck
 
     @Test(expectedExceptions = ValidationException.class)
     public void testInvalidProcessValidity() throws Exception {


[28/47] git commit: Check style fixes relating to messaging module

Posted by sr...@apache.org.
Check style fixes relating to messaging module


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

Branch: refs/heads/master
Commit: 46f07b342e8f0b5e2dcc670d2f1f6c949ac4f355
Parents: afbcea0
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Wed Apr 17 15:24:35 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Wed Apr 17 15:24:35 2013 +0530

----------------------------------------------------------------------
 .../falcon/messaging/EntityInstanceMessage.java    |   23 ++++++++------
 .../messaging/EntityInstanceMessageCreator.java    |    2 +-
 .../apache/falcon/messaging/MessageProducer.java   |   23 +++++++--------
 .../falcon/messaging/FalconTopicProducerTest.java  |   11 ++++--
 .../apache/falcon/messaging/FeedProducerTest.java  |   15 ++++++----
 .../falcon/messaging/ProcessProducerTest.java      |   11 ++++--
 6 files changed, 48 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
index f6b90df..a2ced24 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessage.java
@@ -36,7 +36,7 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 
 /**
- * Value Object which is stored in JMS Topic as MapMessage
+ * Value Object which is stored in JMS Topic as MapMessage.
  */
 public class EntityInstanceMessage {
 
@@ -45,10 +45,16 @@ public class EntityInstanceMessage {
             .getLogger(EntityInstanceMessage.class);
     private static final String FALCON_ENTITY_TOPIC_NAME = "FALCON.ENTITY.TOPIC";
 
+    /**
+     * Feed Entity operations supported.
+     */
     public enum EntityOps {
         GENERATE, DELETE, ARCHIVE, REPLICATE, CHMOD
     }
 
+    /**
+     * properties available in feed entity operation workflow.
+     */
     public enum ARG {
         entityName("entityName"), feedNames("feedNames"), feedInstancePaths(
                 "feedInstancePaths"), workflowId("workflowId"), runId("runId"), nominalTime(
@@ -122,7 +128,8 @@ public class EntityInstanceMessage {
     }
 
     public static EntityInstanceMessage[] getMessages(CommandLine cmd)
-            throws ParseException {
+        throws ParseException {
+
         String[] feedNames = getFeedNames(cmd);
         if (feedNames == null) {
             return null;
@@ -181,16 +188,12 @@ public class EntityInstanceMessage {
         if (topicName.equals(FALCON_ENTITY_TOPIC_NAME)) {
             LOG.debug("Returning instance paths for Falcon Topic: "
                     + cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
-            return new String[]{cmd.getOptionValue(ARG.feedInstancePaths
-                    .getArgName())};
+            return new String[]{cmd.getOptionValue(ARG.feedInstancePaths.getArgName()), };
         }
 
-        if (operation.equals(EntityOps.GENERATE.name())
-                || operation.equals(EntityOps.REPLICATE.name())) {
-            LOG.debug("Returning instance paths: "
-                    + cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
-            return cmd.getOptionValue(ARG.feedInstancePaths.getArgName())
-                    .split(",");
+        if (operation.equals(EntityOps.GENERATE.name()) || operation.equals(EntityOps.REPLICATE.name())) {
+            LOG.debug("Returning instance paths: " + cmd.getOptionValue(ARG.feedInstancePaths.getArgName()));
+            return cmd.getOptionValue(ARG.feedInstancePaths.getArgName()).split(",");
         }
         //else case of feed retention
         Path logFile = new Path(cmd.getOptionValue(ARG.logFile.getArgName()));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
index 0a8e7df..ecda5eb 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/EntityInstanceMessageCreator.java
@@ -27,7 +27,7 @@ import javax.jms.Session;
 import java.util.Map.Entry;
 
 /**
- * Falcon JMS message creator- creates JMS TextMessage
+ * Falcon JMS message creator- creates JMS TextMessage.
  */
 public class EntityInstanceMessageCreator {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
----------------------------------------------------------------------
diff --git a/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java b/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
index cb0ad8a..cf5c2d7 100644
--- a/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
+++ b/messaging/src/main/java/org/apache/falcon/messaging/MessageProducer.java
@@ -28,6 +28,9 @@ import org.apache.log4j.Logger;
 import javax.jms.*;
 import java.lang.reflect.InvocationTargetException;
 
+/**
+ * Message producer used in the workflow to send a message to the queue/topic.
+ */
 public class MessageProducer extends Configured implements Tool {
 
     private Connection connection;
@@ -35,13 +38,13 @@ public class MessageProducer extends Configured implements Tool {
     private static final long DEFAULT_TTL = 3 * 24 * 60 * 60 * 1000;
 
     /**
-     * @param arguments - Accepts a Message to be send to JMS topic, creates a new
+     * @param entityInstanceMessage - Accepts a Message to be send to JMS topic, creates a new
      *                  Topic based on topic name if it does not exist or else
      *                  existing topic with the same name is used to send the message.
      * @throws JMSException
      */
     protected void sendMessage(EntityInstanceMessage entityInstanceMessage)
-            throws JMSException {
+        throws JMSException {
 
         Session session = connection.createSession(false,
                 Session.AUTO_ACKNOWLEDGE);
@@ -69,15 +72,10 @@ public class MessageProducer extends Configured implements Tool {
         ToolRunner.run(new MessageProducer(), args);
     }
 
-    private void createAndStartConnection(String implementation,
-                                          String userName, String password, String url) throws JMSException,
-                                                                                               ClassNotFoundException,
-                                                                                               IllegalArgumentException,
-                                                                                               SecurityException,
-                                                                                               InstantiationException,
-                                                                                               IllegalAccessException,
-                                                                                               InvocationTargetException,
-                                                                                               NoSuchMethodException {
+    private void createAndStartConnection(String implementation, String userName,
+                                          String password, String url)
+        throws JMSException, ClassNotFoundException, InstantiationException,
+               IllegalAccessException, InvocationTargetException, NoSuchMethodException {
 
         Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) MessageProducer.class
                 .getClassLoader().loadClass(implementation);
@@ -91,7 +89,8 @@ public class MessageProducer extends Configured implements Tool {
     }
 
     private static CommandLine getCommand(String[] arguments)
-            throws ParseException {
+        throws ParseException {
+
         Options options = new Options();
         addOption(options, new Option(ARG.brokerImplClass.getArgName(), true,
                 "message broker Implementation class"));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
index 9a13738..9912678 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FalconTopicProducerTest.java
@@ -27,6 +27,9 @@ import org.testng.annotations.Test;
 
 import javax.jms.*;
 
+/**
+ * Test for falcon topic message producer.
+ */
 public class FalconTopicProducerTest {
 
     private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
@@ -71,7 +74,7 @@ public class FalconTopicProducerTest {
                                      "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
                                      "-" + ARG.status.getArgName(), ("SUCCEEDED"),
                                      "-" + ARG.brokerTTL.getArgName(), "10",
-                                     "-" + ARG.cluster.getArgName(), "corp"};
+                                     "-" + ARG.cluster.getArgName(), "corp", };
         testProcessMessageCreator(args);
     }
 
@@ -93,7 +96,7 @@ public class FalconTopicProducerTest {
                                      "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
                                      "-" + ARG.status.getArgName(), ("SUCCEEDED"),
                                      "-" + ARG.brokerTTL.getArgName(), "10",
-                                     "-" + ARG.cluster.getArgName(), "corp"};
+                                     "-" + ARG.cluster.getArgName(), "corp", };
         testProcessMessageCreator(args);
     }
 
@@ -107,7 +110,7 @@ public class FalconTopicProducerTest {
                 } catch (AssertionError e) {
                     error = e;
                 } catch (JMSException ignore) {
-
+                    error = null;
                 }
             }
         };
@@ -133,7 +136,7 @@ public class FalconTopicProducerTest {
 
         // wait till you get atleast one message
         MapMessage m;
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/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 45252dd..32f731f 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/FeedProducerTest.java
@@ -36,6 +36,9 @@ import javax.jms.*;
 import java.io.InputStream;
 import java.io.OutputStream;
 
+/**
+ * Test for feed message producer.
+ */
 public class FeedProducerTest {
 
     private String[] args;
@@ -76,7 +79,7 @@ public class FeedProducerTest {
                             "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
                             "-" + ARG.status.getArgName(), ("SUCCEEDED"),
                             "-" + ARG.brokerTTL.getArgName(), "10",
-                            "-" + ARG.cluster.getArgName(), "corp"};
+                            "-" + ARG.cluster.getArgName(), "corp", };
 
         broker = new BrokerService();
         broker.addConnector(BROKER_URL);
@@ -125,7 +128,7 @@ public class FeedProducerTest {
                 } catch (AssertionError e) {
                     error = e;
                 } catch (JMSException ignore) {
-
+                    error = null;
                 }
             }
         };
@@ -151,7 +154,7 @@ public class FeedProducerTest {
 
         // wait till you get atleast one message
         MapMessage m;
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());
@@ -159,7 +162,7 @@ public class FeedProducerTest {
         Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/20");
 
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());
@@ -167,7 +170,7 @@ public class FeedProducerTest {
         Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/21");
 
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());
@@ -175,7 +178,7 @@ public class FeedProducerTest {
         Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
                 "/falcon/feed/agg-logs/path1/2010/10/10/22");
 
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/46f07b34/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
----------------------------------------------------------------------
diff --git a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
index 55a5fc0..078b9c2 100644
--- a/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
+++ b/messaging/src/test/java/org/apache/falcon/messaging/ProcessProducerTest.java
@@ -28,6 +28,9 @@ import org.testng.annotations.Test;
 
 import javax.jms.*;
 
+/**
+ * Test for process message producer.
+ */
 public class ProcessProducerTest {
 
     private String[] args;
@@ -58,7 +61,7 @@ public class ProcessProducerTest {
                             "-" + ARG.topicName.getArgName(), (TOPIC_NAME),
                             "-" + ARG.status.getArgName(), ("SUCCEEDED"),
                             "-" + ARG.brokerTTL.getArgName(), "10",
-                            "-" + ARG.cluster.getArgName(), "corp"};
+                            "-" + ARG.cluster.getArgName(), "corp", };
         broker = new BrokerService();
         broker.addConnector(BROKER_URL);
         broker.setDataDirectory("target/activemq");
@@ -84,7 +87,7 @@ public class ProcessProducerTest {
                 } catch (AssertionError e) {
                     error = e;
                 } catch (JMSException ignore) {
-
+                    error = null;
                 }
             }
         };
@@ -110,7 +113,7 @@ public class ProcessProducerTest {
 
         // wait till you get atleast one message
         MapMessage m;
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());
@@ -120,7 +123,7 @@ public class ProcessProducerTest {
         Assert.assertEquals(m.getString(ARG.feedInstancePaths.getArgName()),
                 "/click-logs/10/05/05/00/20");
 
-        for (m = null; m == null; ) {
+        for (m = null; m == null;) {
             m = (MapMessage) consumer.receive();
         }
         System.out.println("Consumed: " + m.toString());


[16/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java b/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
index a91f8ad..0e0dcd0 100644
--- a/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
+++ b/common/src/main/java/org/apache/falcon/workflow/engine/AbstractWorkflowEngine.java
@@ -18,18 +18,21 @@
 
 package org.apache.falcon.workflow.engine;
 
-import java.util.*;
-
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.resource.InstancesResult;
 
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.Set;
+
 /**
  * Workflow engine should minimally support the
  * following operations
  */
 public abstract class AbstractWorkflowEngine {
-	
+
     public static final String NAME_NODE = "nameNode";
     public static final String JOB_TRACKER = "jobTracker";
 
@@ -46,9 +49,9 @@ public abstract class AbstractWorkflowEngine {
     public abstract String resume(Entity entity) throws FalconException;
 
     public abstract String delete(Entity entity) throws FalconException;
-    
+
     public abstract String delete(Entity entity, String cluster) throws FalconException;
-    
+
     public abstract void reRun(String cluster, String wfId, Properties props) throws FalconException;
 
     public abstract boolean isActive(Entity entity) throws FalconException;
@@ -56,22 +59,26 @@ public abstract class AbstractWorkflowEngine {
     public abstract boolean isSuspended(Entity entity) throws FalconException;
 
     public abstract InstancesResult getRunningInstances(Entity entity) throws FalconException;
-    
-    public abstract InstancesResult killInstances(Entity entity, Date start, Date end, Properties props) throws FalconException;
-    
-    public abstract InstancesResult reRunInstances(Entity entity, Date start, Date end, Properties props) throws FalconException;
 
-    public abstract InstancesResult suspendInstances(Entity entity, Date start, Date end, Properties props) throws FalconException;
+    public abstract InstancesResult killInstances(Entity entity, Date start, Date end, Properties props)
+            throws FalconException;
 
-    public abstract InstancesResult resumeInstances(Entity entity, Date start, Date end, Properties props) throws FalconException;
+    public abstract InstancesResult reRunInstances(Entity entity, Date start, Date end, Properties props)
+            throws FalconException;
+
+    public abstract InstancesResult suspendInstances(Entity entity, Date start, Date end, Properties props)
+            throws FalconException;
+
+    public abstract InstancesResult resumeInstances(Entity entity, Date start, Date end, Properties props)
+            throws FalconException;
 
     public abstract InstancesResult getStatus(Entity entity, Date start, Date end) throws FalconException;
 
-	public abstract void update(Entity oldEntity, Entity newEntity, String cluster) throws FalconException;
+    public abstract void update(Entity oldEntity, Entity newEntity, String cluster) throws FalconException;
+
+    public abstract String getWorkflowStatus(String cluster, String jobId) throws FalconException;
+
+    public abstract String getWorkflowProperty(String cluster, String jobId, String property) throws FalconException;
 
-	public abstract String getWorkflowStatus(String cluster, String jobId) throws FalconException;
-	
-	public abstract String getWorkflowProperty(String cluster, String jobId , String property) throws FalconException;
-	
-	public abstract InstancesResult  getJobDetails(String cluster, String jobId) throws FalconException;
+    public abstract InstancesResult getJobDetails(String cluster, String jobId) throws FalconException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/main/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/common/src/main/resources/log4j.xml b/common/src/main/resources/log4j.xml
index da50fb2..189d096 100644
--- a/common/src/main/resources/log4j.xml
+++ b/common/src/main/resources/log4j.xml
@@ -20,72 +20,72 @@
 <!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
 
 <log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-  <appender name="console" class="org.apache.log4j.ConsoleAppender">
-    <param name="Target" value="System.out"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="console" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
 
-  <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/application.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/application.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/audit.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
+
+    <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/tranlog.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
 
-  <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/audit.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
-  <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/tranlog.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
     <appender name="METRIC" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/metric.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %m%n"/>
-    </layout>
-  </appender>
+        <param name="File" value="/var/log/falcon/metric.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %m%n"/>
+        </layout>
+    </appender>
+
+    <logger name="org.apache.falcon">
+        <level value="debug"/>
+        <appender-ref ref="FILE"/>
+    </logger>
+
+    <logger name="AUDIT">
+        <level value="info"/>
+        <appender-ref ref="AUDIT"/>
+    </logger>
 
-  <logger name="org.apache.falcon">
-    <level value="debug"/>
-    <appender-ref ref="FILE" />
-  </logger>
+    <logger name="TRANSACTIONLOG">
+        <level value="info"/>
+        <appender-ref ref="TRANSACTIONLOG"/>
+    </logger>
 
-  <logger name="AUDIT">
-    <level value="info"/>
-    <appender-ref ref="AUDIT" />
-  </logger>
-  
-  <logger name="TRANSACTIONLOG">
-    <level value="info"/>
-    <appender-ref ref="TRANSACTIONLOG" />
-  </logger>
-  
     <logger name="METRIC">
-    <level value="info"/>
-    <appender-ref ref="METRIC" />
-  </logger>
+        <level value="info"/>
+        <appender-ref ref="METRIC"/>
+    </logger>
 
-  <root>
-    <priority value ="info" />
-    <appender-ref ref="console" />
-  </root>
+    <root>
+        <priority value="info"/>
+        <appender-ref ref="console"/>
+    </root>
 
 </log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 ee6b148..101f987 100644
--- a/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
+++ b/common/src/test/java/org/apache/falcon/cleanup/LogCleanupServiceTest.java
@@ -17,8 +17,6 @@
  */
 package org.apache.falcon.cleanup;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.AbstractTestBase;
@@ -26,6 +24,8 @@ import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.entity.v0.process.Process;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -35,115 +35,115 @@ import java.io.IOException;
 
 public class LogCleanupServiceTest extends AbstractTestBase {
 
-	private FileSystem fs;
-	private FileSystem tfs;
-	private EmbeddedCluster targetDfsCluster;
-	Path instanceLogPath = new Path(
-			"/projects/falcon/staging/falcon/workflows/process/" + "sample"
-					+ "/logs/job-2010-01-01-01-00/000");
-	Path instanceLogPath1 = new Path(
-			"/projects/falcon/staging/falcon/workflows/process/" + "sample"
-					+ "/logs/job-2010-01-01-01-00/001");
-	Path instanceLogPath2 = new Path(
-			"/projects/falcon/staging/falcon/workflows/process/" + "sample"
-					+ "/logs/job-2010-01-01-02-00/001");
-	Path instanceLogPath3 = new Path(
-			"/projects/falcon/staging/falcon/workflows/process/" + "sample2"
-					+ "/logs/job-2010-01-01-01-00/000");
-	Path instanceLogPath4 = new Path(
-			"/projects/falcon/staging/falcon/workflows/process/" + "sample"
-					+ "/logs/latedata/2010-01-01-01-00");
-	Path feedInstanceLogPath = new Path(
-			"/projects/falcon/staging/falcon/workflows/feed/"
-					+ "impressionFeed"
-					+ "/logs/job-2010-01-01-01-00/testCluster/000");
-	Path feedInstanceLogPath1 = new Path(
-			"/projects/falcon/staging/falcon/workflows/feed/"
-					+ "impressionFeed2"
-					+ "/logs/job-2010-01-01-01-00/testCluster/000");
-
-
-	@AfterClass
-	public void tearDown() {
-		this.dfsCluster.shutdown();
-		this.targetDfsCluster.shutdown();
-	}
-
-	@BeforeClass
-	public void setup() throws Exception {
-		this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
+    private FileSystem fs;
+    private FileSystem tfs;
+    private EmbeddedCluster targetDfsCluster;
+    Path instanceLogPath = new Path(
+            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
+                    + "/logs/job-2010-01-01-01-00/000");
+    Path instanceLogPath1 = new Path(
+            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
+                    + "/logs/job-2010-01-01-01-00/001");
+    Path instanceLogPath2 = new Path(
+            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
+                    + "/logs/job-2010-01-01-02-00/001");
+    Path instanceLogPath3 = new Path(
+            "/projects/falcon/staging/falcon/workflows/process/" + "sample2"
+                    + "/logs/job-2010-01-01-01-00/000");
+    Path instanceLogPath4 = new Path(
+            "/projects/falcon/staging/falcon/workflows/process/" + "sample"
+                    + "/logs/latedata/2010-01-01-01-00");
+    Path feedInstanceLogPath = new Path(
+            "/projects/falcon/staging/falcon/workflows/feed/"
+                    + "impressionFeed"
+                    + "/logs/job-2010-01-01-01-00/testCluster/000");
+    Path feedInstanceLogPath1 = new Path(
+            "/projects/falcon/staging/falcon/workflows/feed/"
+                    + "impressionFeed2"
+                    + "/logs/job-2010-01-01-01-00/testCluster/000");
+
+
+    @AfterClass
+    public void tearDown() {
+        this.dfsCluster.shutdown();
+        this.targetDfsCluster.shutdown();
+    }
+
+    @BeforeClass
+    public void setup() throws Exception {
+        this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
         conf = dfsCluster.getConf();
-		fs = dfsCluster.getFileSystem();
-		
-		storeEntity(EntityType.CLUSTER, "testCluster");
-		System.setProperty("test.build.data",
+        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", false);
         conf = targetDfsCluster.getConf();
 
-		storeEntity(EntityType.CLUSTER, "backupCluster");
-		storeEntity(EntityType.FEED, "impressionFeed");
-		storeEntity(EntityType.FEED, "clicksFeed");
-		storeEntity(EntityType.FEED, "imp-click-join1");
-		storeEntity(EntityType.FEED, "imp-click-join2");
-		storeEntity(EntityType.PROCESS, "sample");
-		Process process = ConfigurationStore.get().get(EntityType.PROCESS,
-				"sample");
-		Process otherProcess = (Process) process.clone();
-		otherProcess.setName("sample2");
-		otherProcess.setFrequency(new Frequency("days(1)"));
-		ConfigurationStore.get().remove(EntityType.PROCESS,
-				otherProcess.getName());
-		ConfigurationStore.get().publish(EntityType.PROCESS, otherProcess);		
-
-		fs.mkdirs(instanceLogPath);
-		fs.mkdirs(instanceLogPath1);
-		fs.mkdirs(instanceLogPath2);
-		fs.mkdirs(instanceLogPath3);
-		fs.mkdirs(instanceLogPath4);
-
-		// fs.setTimes wont work on dirs
-		fs.createNewFile(new Path(instanceLogPath, "oozie.log"));
-		fs.createNewFile(new Path(instanceLogPath, "pigAction_SUCCEEDED.log"));
-		
-		tfs = targetDfsCluster.getFileSystem();
-		fs.mkdirs(feedInstanceLogPath);
-		fs.mkdirs(feedInstanceLogPath1);
-		tfs.mkdirs(feedInstanceLogPath);
-		tfs.mkdirs(feedInstanceLogPath1);
-		fs.createNewFile(new Path(feedInstanceLogPath, "oozie.log"));
-		tfs.createNewFile(new Path(feedInstanceLogPath, "oozie.log"));
-		
-		Thread.sleep(61000);
-
-
-	}
-
-	@Test
-	public void testProcessLogs() throws IOException, FalconException,
-			InterruptedException {
-
-		AbstractCleanupHandler processCleanupHandler = new ProcessCleanupHandler();
-		processCleanupHandler.cleanup();
-
-		Assert.assertFalse(fs.exists(instanceLogPath));
-		Assert.assertFalse(fs.exists(instanceLogPath1));
-		Assert.assertFalse(fs.exists(instanceLogPath2));
-		Assert.assertTrue(fs.exists(instanceLogPath3));
-
-	}
-
-	@Test
-	public void testFeedLogs() throws IOException, FalconException,
-			InterruptedException {
-
-		AbstractCleanupHandler feedCleanupHandler = new FeedCleanupHandler();
-		feedCleanupHandler.cleanup();
-
-		Assert.assertFalse(fs.exists(feedInstanceLogPath));
-		Assert.assertFalse(tfs.exists(feedInstanceLogPath));
-		Assert.assertTrue(fs.exists(feedInstanceLogPath1));
-		Assert.assertTrue(tfs.exists(feedInstanceLogPath1));
-
-	}
+        storeEntity(EntityType.CLUSTER, "backupCluster");
+        storeEntity(EntityType.FEED, "impressionFeed");
+        storeEntity(EntityType.FEED, "clicksFeed");
+        storeEntity(EntityType.FEED, "imp-click-join1");
+        storeEntity(EntityType.FEED, "imp-click-join2");
+        storeEntity(EntityType.PROCESS, "sample");
+        Process process = ConfigurationStore.get().get(EntityType.PROCESS,
+                "sample");
+        Process otherProcess = (Process) process.clone();
+        otherProcess.setName("sample2");
+        otherProcess.setFrequency(new Frequency("days(1)"));
+        ConfigurationStore.get().remove(EntityType.PROCESS,
+                otherProcess.getName());
+        ConfigurationStore.get().publish(EntityType.PROCESS, otherProcess);
+
+        fs.mkdirs(instanceLogPath);
+        fs.mkdirs(instanceLogPath1);
+        fs.mkdirs(instanceLogPath2);
+        fs.mkdirs(instanceLogPath3);
+        fs.mkdirs(instanceLogPath4);
+
+        // fs.setTimes wont work on dirs
+        fs.createNewFile(new Path(instanceLogPath, "oozie.log"));
+        fs.createNewFile(new Path(instanceLogPath, "pigAction_SUCCEEDED.log"));
+
+        tfs = targetDfsCluster.getFileSystem();
+        fs.mkdirs(feedInstanceLogPath);
+        fs.mkdirs(feedInstanceLogPath1);
+        tfs.mkdirs(feedInstanceLogPath);
+        tfs.mkdirs(feedInstanceLogPath1);
+        fs.createNewFile(new Path(feedInstanceLogPath, "oozie.log"));
+        tfs.createNewFile(new Path(feedInstanceLogPath, "oozie.log"));
+
+        Thread.sleep(61000);
+
+
+    }
+
+    @Test
+    public void testProcessLogs() throws IOException, FalconException,
+                                         InterruptedException {
+
+        AbstractCleanupHandler processCleanupHandler = new ProcessCleanupHandler();
+        processCleanupHandler.cleanup();
+
+        Assert.assertFalse(fs.exists(instanceLogPath));
+        Assert.assertFalse(fs.exists(instanceLogPath1));
+        Assert.assertFalse(fs.exists(instanceLogPath2));
+        Assert.assertTrue(fs.exists(instanceLogPath3));
+
+    }
+
+    @Test
+    public void testFeedLogs() throws IOException, FalconException,
+                                      InterruptedException {
+
+        AbstractCleanupHandler feedCleanupHandler = new FeedCleanupHandler();
+        feedCleanupHandler.cleanup();
+
+        Assert.assertFalse(fs.exists(feedInstanceLogPath));
+        Assert.assertFalse(tfs.exists(feedInstanceLogPath));
+        Assert.assertTrue(fs.exists(feedInstanceLogPath1));
+        Assert.assertTrue(tfs.exists(feedInstanceLogPath1));
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java b/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
index b6e749e..394fa70 100644
--- a/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
+++ b/common/src/test/java/org/apache/falcon/entity/AbstractTestBase.java
@@ -18,9 +18,6 @@
 
 package org.apache.falcon.entity;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.falcon.entity.store.ConfigurationStore;
@@ -32,6 +29,9 @@ import org.apache.falcon.entity.v0.cluster.Interfacetype;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.util.StartupProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.testng.annotations.BeforeClass;
 
 import javax.xml.bind.JAXBException;
@@ -45,23 +45,24 @@ public class AbstractTestBase {
     protected static final String FEED_XML = "/config/feed/feed-0.1.xml";
     protected static final String CLUSTER_XML = "/config/cluster/cluster-0.1.xml";
     protected EmbeddedCluster dfsCluster;
-    protected Configuration conf= new Configuration();
+    protected Configuration conf = new Configuration();
 
     @BeforeClass
     public void initConfigStore() throws Exception {
         cleanupStore();
         String listeners = StartupProperties.get().getProperty("configstore.listeners");
-        StartupProperties.get().setProperty("configstore.listeners", 
+        StartupProperties.get().setProperty("configstore.listeners",
                 listeners.replace("org.apache.falcon.service.SharedLibraryHostingService", ""));
         ConfigurationStore.get().init();
     }
-    
+
     protected void cleanupStore() throws FalconException {
         ConfigurationStore store = ConfigurationStore.get();
-        for(EntityType type:EntityType.values()) {
+        for (EntityType type : EntityType.values()) {
             Collection<String> entities = store.getEntities(type);
-            for(String entity:entities)
+            for (String entity : entities) {
                 store.remove(type, entity);
+            }
         }
     }
 
@@ -69,8 +70,8 @@ public class AbstractTestBase {
         Unmarshaller unmarshaller = type.getUnmarshaller();
         ConfigurationStore store = ConfigurationStore.get();
         store.remove(type, name);
-		switch (type) {
-		case CLUSTER:
+        switch (type) {
+            case CLUSTER:
                 Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass().getResource(CLUSTER_XML));
                 cluster.setName(name);
                 ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(conf.get("fs.default.name"));
@@ -86,10 +87,10 @@ public class AbstractTestBase {
             case PROCESS:
                 Process process = (Process) unmarshaller.unmarshal(this.getClass().getResource(PROCESS_XML));
                 process.setName(name);
-                FileSystem fs =dfsCluster.getFileSystem();
+                FileSystem fs = dfsCluster.getFileSystem();
                 fs.mkdirs(new Path(process.getWorkflow().getPath()));
-                if (!fs.exists(new Path(process.getWorkflow()+"/lib"))) {
-                	fs.mkdirs(new Path(process.getWorkflow()+"/lib"));
+                if (!fs.exists(new Path(process.getWorkflow() + "/lib"))) {
+                    fs.mkdirs(new Path(process.getWorkflow() + "/lib"));
                 }
                 store.publish(type, process);
                 break;
@@ -97,12 +98,12 @@ public class AbstractTestBase {
     }
 
     public void setup() throws Exception {
-		ConfigurationStore store = ConfigurationStore.get();
-		for (EntityType type : EntityType.values()) {
-			for (String name : store.getEntities(type)) {
-				store.remove(type, name);
-			}
-		}
+        ConfigurationStore store = ConfigurationStore.get();
+        for (EntityType type : EntityType.values()) {
+            for (String name : store.getEntities(type)) {
+                store.remove(type, name);
+            }
+        }
         storeEntity(EntityType.CLUSTER, "corp");
         storeEntity(EntityType.FEED, "clicks");
         storeEntity(EntityType.FEED, "impressions");
@@ -110,20 +111,20 @@ public class AbstractTestBase {
         storeEntity(EntityType.PROCESS, "clicksummary");
     }
 
-	public String marshallEntity(final Entity entity) throws FalconException,
-			JAXBException {
-		Marshaller marshaller = entity.getEntityType().getMarshaller();
-		StringWriter stringWriter = new StringWriter();
-		marshaller.marshal(entity, stringWriter);
-		return stringWriter.toString();
-	}
-	
-	private Interface newInterface(Interfacetype type, String endPoint,
-			String version) {
-		Interface iface = new Interface();
-		iface.setType(type);
-		iface.setEndpoint(endPoint);
-		iface.setVersion(version);
-		return iface;
-	}
+    public String marshallEntity(final Entity entity) throws FalconException,
+                                                             JAXBException {
+        Marshaller marshaller = entity.getEntityType().getMarshaller();
+        StringWriter stringWriter = new StringWriter();
+        marshaller.marshal(entity, stringWriter);
+        return stringWriter.toString();
+    }
+
+    private Interface newInterface(Interfacetype type, String endPoint,
+                                   String version) {
+        Interface iface = new Interface();
+        iface.setType(type);
+        iface.setEndpoint(endPoint);
+        iface.setVersion(version);
+        return iface;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java b/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
index 1d2ffd3..d60126e 100644
--- a/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/ColoClusterRelationTest.java
@@ -18,23 +18,23 @@
 
 package org.apache.falcon.entity;
 
-import org.testng.Assert;
-import org.testng.annotations.Test;
-import java.util.Set;
-
 import org.apache.falcon.entity.store.ConfigurationStore;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Set;
 
 @Test
-public class ColoClusterRelationTest extends AbstractTestBase{
+public class ColoClusterRelationTest extends AbstractTestBase {
     private Cluster newCluster(String name, String colo) {
         Cluster cluster = new Cluster();
         cluster.setName(name);
         cluster.setColo(colo);
         return cluster;
     }
-    
+
     @Test
     public void testMapping() throws Exception {
         Cluster cluster1 = newCluster("cluster1", "colo1");
@@ -44,14 +44,14 @@ public class ColoClusterRelationTest extends AbstractTestBase{
         store.publish(EntityType.CLUSTER, cluster1);
         store.publish(EntityType.CLUSTER, cluster2);
         store.publish(EntityType.CLUSTER, cluster3);
-        
+
         ColoClusterRelation relation = ColoClusterRelation.get();
         Set<String> clusters = relation.getClusters("colo1");
         Assert.assertNotNull(clusters);
         Assert.assertEquals(2, clusters.size());
         Assert.assertTrue(clusters.contains(cluster1.getName()));
         Assert.assertTrue(clusters.contains(cluster2.getName()));
-        
+
         clusters = relation.getClusters("colo2");
         Assert.assertNotNull(clusters);
         Assert.assertEquals(1, clusters.size());
@@ -62,7 +62,7 @@ public class ColoClusterRelationTest extends AbstractTestBase{
         Assert.assertNotNull(clusters);
         Assert.assertEquals(1, clusters.size());
         Assert.assertTrue(clusters.contains(cluster2.getName()));
-        
+
         store.remove(EntityType.CLUSTER, cluster2.getName());
         clusters = relation.getClusters("colo1");
         Assert.assertNotNull(clusters);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java b/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
index a3aadba..509fce8 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityTypeTest.java
@@ -24,16 +24,16 @@ import org.testng.annotations.Test;
 
 public class EntityTypeTest {
 
-	@Test
-	public void testGetEntityClass() {
-		Assert.assertEquals(EntityType.PROCESS.getEntityClass().getName(),
-				"org.apache.falcon.entity.v0.process.Process");
-	}
+    @Test
+    public void testGetEntityClass() {
+        Assert.assertEquals(EntityType.PROCESS.getEntityClass().getName(),
+                "org.apache.falcon.entity.v0.process.Process");
+    }
 
-	@Test
-	public void testIsSchedulable() {
-		Assert.assertTrue(EntityType.PROCESS.isSchedulable());
-		Assert.assertTrue(EntityType.FEED.isSchedulable());
-		Assert.assertFalse(EntityType.CLUSTER.isSchedulable());
-	}
+    @Test
+    public void testIsSchedulable() {
+        Assert.assertTrue(EntityType.PROCESS.isSchedulable());
+        Assert.assertTrue(EntityType.FEED.isSchedulable());
+        Assert.assertFalse(EntityType.CLUSTER.isSchedulable());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
index 8a565bf..622fe48 100644
--- a/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/EntityUtilTest.java
@@ -18,11 +18,6 @@
 
 package org.apache.falcon.entity;
 
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.TimeZone;
-
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.Frequency;
 import org.apache.falcon.entity.v0.SchemaHelper;
@@ -32,7 +27,12 @@ import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-public class EntityUtilTest extends AbstractTestBase{
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+public class EntityUtilTest extends AbstractTestBase {
     private static TimeZone tz = TimeZone.getTimeZone("UTC");
 
     @Test
@@ -50,7 +50,7 @@ public class EntityUtilTest extends AbstractTestBase{
         Assert.assertEquals(newProcess.getClusters().getClusters().size(), 1);
         Assert.assertEquals(newProcess.getClusters().getClusters().get(0).getName(), currentCluster);
     }
-    
+
     @Test
     public void testFeedView() throws Exception {
         Feed feed = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
@@ -58,11 +58,11 @@ public class EntityUtilTest extends AbstractTestBase{
         Feed view = EntityUtil.getClusterView(feed, "testCluster");
         Assert.assertEquals(view.getClusters().getClusters().size(), 1);
         Assert.assertEquals(view.getClusters().getClusters().get(0).getName(), "testCluster");
-        
+
         view = EntityUtil.getClusterView(feed, "backupCluster");
         Assert.assertEquals(view.getClusters().getClusters().size(), 2);
     }
-    
+
     @Test
     public void testEquals() throws Exception {
         Process process1 = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(
@@ -72,10 +72,11 @@ public class EntityUtilTest extends AbstractTestBase{
         Assert.assertTrue(EntityUtil.equals(process1, process2));
         Assert.assertTrue(EntityUtil.md5(process1).equals(EntityUtil.md5(process2)));
 
-        process2.getClusters().getClusters().get(0).getValidity().setEnd(SchemaHelper.parseDateUTC("2013-04-21T00:00Z"));
+        process2.getClusters().getClusters().get(0).getValidity().setEnd(
+                SchemaHelper.parseDateUTC("2013-04-21T00:00Z"));
         Assert.assertFalse(EntityUtil.equals(process1, process2));
         Assert.assertFalse(EntityUtil.md5(process1).equals(EntityUtil.md5(process2)));
-        Assert.assertTrue(EntityUtil.equals(process1, process2, new String[] {"clusters.clusters[\\d+].validity.end"}));
+        Assert.assertTrue(EntityUtil.equals(process1, process2, new String[]{"clusters.clusters[\\d+].validity.end"}));
     }
 
     private static Date getDate(String date) throws Exception {
@@ -145,7 +146,7 @@ public class EntityUtilTest extends AbstractTestBase{
 
         Frequency frequency = Frequency.fromString("hours(1)");
         Assert.assertEquals(198, EntityUtil.getInstanceSequence(start,
-                frequency,tz, instance));
+                frequency, tz, instance));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java b/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
index 66a56d6..100fb63 100644
--- a/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/FeedHelperTest.java
@@ -31,7 +31,7 @@ public class FeedHelperTest {
         Assert.assertEquals(FeedHelper.normalizePartitionExpression(null, "  /b// "), "b");
         Assert.assertEquals(FeedHelper.normalizePartitionExpression(null, null), "");
     }
-    
+
     @Test
     public void testEvaluateExpression() throws Exception {
         Cluster cluster = new Cluster();
@@ -42,9 +42,10 @@ public class FeedHelperTest {
         prop.setName("pname");
         prop.setValue("pvalue");
         cluster.getProperties().getProperties().add(prop);
-        
+
         Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster, "${cluster.colo}/*/US"), "colo/*/US");
-        Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster, "${cluster.name}/*/${cluster.pname}"), "name/*/pvalue");
+        Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster, "${cluster.name}/*/${cluster.pname}"),
+                "name/*/pvalue");
         Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster, "IN"), "IN");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java b/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
index 332b122..6486397 100644
--- a/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
+++ b/common/src/test/java/org/apache/falcon/entity/TestWorkflowNameBuilder.java
@@ -17,72 +17,72 @@
  */
 package org.apache.falcon.entity;
 
-import java.util.Arrays;
-
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.v0.feed.Feed;
 import org.apache.falcon.entity.v0.process.Process;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import java.util.Arrays;
+
 public class TestWorkflowNameBuilder {
 
-	@Test
-	public void getTagTest() {
-		Feed feed = new Feed();
-		feed.setName("raw-logs");
+    @Test
+    public void getTagTest() {
+        Feed feed = new Feed();
+        feed.setName("raw-logs");
 
-		WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
-		Tag tag = builder.getWorkflowTag("FALCON_FEED_RETENTION_raw-logs");
-		Assert.assertEquals(tag, Tag.RETENTION);
+        WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
+        Tag tag = builder.getWorkflowTag("FALCON_FEED_RETENTION_raw-logs");
+        Assert.assertEquals(tag, Tag.RETENTION);
 
-		tag = builder.getWorkflowTag("FALCON_FEED_raw-logs");
-		Assert.assertNull(tag);
+        tag = builder.getWorkflowTag("FALCON_FEED_raw-logs");
+        Assert.assertNull(tag);
 
-		tag = builder.getWorkflowTag("FALCON_FEED_REPLICATION_raw-logs_corp1");
-		Assert.assertEquals(tag, Tag.REPLICATION);
+        tag = builder.getWorkflowTag("FALCON_FEED_REPLICATION_raw-logs_corp1");
+        Assert.assertEquals(tag, Tag.REPLICATION);
 
-	}
+    }
 
-	@Test
-	public void getSuffixesTest() {
-		Feed feed = new Feed();
-		feed.setName("raw-logs");
-		WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
+    @Test
+    public void getSuffixesTest() {
+        Feed feed = new Feed();
+        feed.setName("raw-logs");
+        WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
 
-		String suffixes = builder
-				.getWorkflowSuffixes("FALCON_FEED_REPLICATION_raw-logs_corp-1");
-		Assert.assertEquals(suffixes, "_corp-1");
+        String suffixes = builder
+                .getWorkflowSuffixes("FALCON_FEED_REPLICATION_raw-logs_corp-1");
+        Assert.assertEquals(suffixes, "_corp-1");
 
-		suffixes = builder
-				.getWorkflowSuffixes("FALCON_FEED_REPLICATION_raw-logs");
-		Assert.assertEquals(suffixes, "");
-	}
+        suffixes = builder
+                .getWorkflowSuffixes("FALCON_FEED_REPLICATION_raw-logs");
+        Assert.assertEquals(suffixes, "");
+    }
 
-	@Test
-	public void WorkflowNameTest() {
-		Feed feed = new Feed();
-		feed.setName("raw-logs");
+    @Test
+    public void WorkflowNameTest() {
+        Feed feed = new Feed();
+        feed.setName("raw-logs");
 
-		WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
-		Assert.assertEquals(builder.getWorkflowName().toString(),
-				"FALCON_FEED_raw-logs");
+        WorkflowNameBuilder<Feed> builder = new WorkflowNameBuilder<Feed>(feed);
+        Assert.assertEquals(builder.getWorkflowName().toString(),
+                "FALCON_FEED_raw-logs");
 
-		builder.setTag(Tag.REPLICATION);
-		Assert.assertEquals(builder.getWorkflowName().toString(),
-				"FALCON_FEED_REPLICATION_raw-logs");
+        builder.setTag(Tag.REPLICATION);
+        Assert.assertEquals(builder.getWorkflowName().toString(),
+                "FALCON_FEED_REPLICATION_raw-logs");
 
-		builder.setSuffixes(Arrays.asList("cluster1"));
-		Assert.assertEquals(builder.getWorkflowName().toString(),
-				"FALCON_FEED_REPLICATION_raw-logs_cluster1");
+        builder.setSuffixes(Arrays.asList("cluster1"));
+        Assert.assertEquals(builder.getWorkflowName().toString(),
+                "FALCON_FEED_REPLICATION_raw-logs_cluster1");
 
-		Process process = new Process();
-		process.setName("agg-logs");
-		WorkflowNameBuilder<Process> processBuilder = new WorkflowNameBuilder<Process>(
-				process);
-		processBuilder.setTag(Tag.DEFAULT);
-		Assert.assertEquals(processBuilder.getWorkflowName().toString(),
-				"FALCON_PROCESS_DEFAULT_agg-logs");
+        Process process = new Process();
+        process.setName("agg-logs");
+        WorkflowNameBuilder<Process> processBuilder = new WorkflowNameBuilder<Process>(
+                process);
+        processBuilder.setTag(Tag.DEFAULT);
+        Assert.assertEquals(processBuilder.getWorkflowName().toString(),
+                "FALCON_PROCESS_DEFAULT_agg-logs");
 
-	}
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 ae4a8fa..b54ca63 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
@@ -79,15 +79,15 @@ public class ClusterEntityParserTest extends AbstractTestBase {
         System.out.println(stringWriter.toString());
         parser.parseAndValidate(stringWriter.toString());
     }
-    
+
     @BeforeClass
     public void init() throws Exception {
         this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
         this.conf = dfsCluster.getConf();
     }
-    
-	@AfterClass
-	public void tearDown() {
-		this.dfsCluster.shutdown();
-	}
+
+    @AfterClass
+    public void tearDown() {
+        this.dfsCluster.shutdown();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
index cfdc1b4..c461ee5 100644
--- a/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
+++ b/common/src/test/java/org/apache/falcon/entity/parser/FeedEntityParserTest.java
@@ -21,14 +21,6 @@ package org.apache.falcon.entity.parser;
 /**
  * Test Cases for ProcessEntityParser
  */
-import static org.testng.AssertJUnit.assertEquals;
-
-import java.io.IOException;
-import java.io.StringWriter;
-
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
 
 import org.apache.falcon.FalconException;
 import org.apache.falcon.entity.AbstractTestBase;
@@ -38,126 +30,127 @@ 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.cluster.Cluster;
-import org.apache.falcon.entity.v0.feed.ActionType;
-import org.apache.falcon.entity.v0.feed.ClusterType;
-import org.apache.falcon.entity.v0.feed.Feed;
-import org.apache.falcon.entity.v0.feed.Location;
-import org.apache.falcon.entity.v0.feed.LocationType;
-import org.apache.falcon.entity.v0.feed.Locations;
-import org.apache.falcon.entity.v0.feed.Validity;
-import org.apache.falcon.group.FeedGroup;
+import org.apache.falcon.entity.v0.feed.*;
 import org.apache.falcon.group.FeedGroupMapTest;
 import org.testng.Assert;
 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 static org.testng.AssertJUnit.assertEquals;
+
 public class FeedEntityParserTest extends AbstractTestBase {
 
-	private final FeedEntityParser parser = (FeedEntityParser) EntityParserFactory
-			.getParser(EntityType.FEED);
-
-	private Feed modifiableFeed;
-
-	@BeforeMethod
-	public void setUp() throws Exception {
-	    cleanupStore();
-	    ConfigurationStore store = ConfigurationStore.get();
-	    
-		Unmarshaller unmarshaller = EntityType.CLUSTER.getUnmarshaller();
-		Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass()
-				.getResourceAsStream(CLUSTER_XML));
-		cluster.setName("testCluster");
-		store.publish(EntityType.CLUSTER, cluster);
-
-		cluster = (Cluster) unmarshaller.unmarshal(this.getClass()
-				.getResourceAsStream(CLUSTER_XML));
-		cluster.setName("backupCluster");
-		store.publish(EntityType.CLUSTER, cluster);
-
-		modifiableFeed = (Feed) parser.parseAndValidate(this.getClass()
-				.getResourceAsStream(FEED_XML));
-	}
-
-	@Test(expectedExceptions = ValidationException.class)
-	public void testValidations() throws Exception {
-		ConfigurationStore.get().remove(EntityType.CLUSTER, "backupCluster");
-		parser.parseAndValidate(this.getClass().getResourceAsStream(FEED_XML));
-	}
-
-	@Test
-	public void testParse() throws IOException, FalconException, JAXBException {
-
-		Feed feed = parser.parseAndValidate(this.getClass()
-				.getResourceAsStream(FEED_XML));
-
-		Assert.assertNotNull(feed);
-		assertEquals(feed.getName(), "clicks");
-		assertEquals(feed.getDescription(), "clicks log");
-		assertEquals(feed.getFrequency().toString(), "hours(1)");
-		assertEquals(feed.getGroups(), "online,bi");
-
-		assertEquals(feed.getClusters().getClusters().get(0).getName(),
-				"testCluster");
-		assertEquals(feed.getClusters().getClusters().get(0).getType(),
-				ClusterType.SOURCE);
-		assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity()
-				.getStart()), "2011-11-01T00:00Z");
-		assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity()
-				.getEnd()), "2011-12-31T00:00Z");
-		assertEquals(feed.getTimezone().getID(), "UTC");
-		assertEquals(feed.getClusters().getClusters().get(0).getRetention()
-				.getAction(), ActionType.DELETE);
-		assertEquals(feed.getClusters().getClusters().get(0).getRetention()
-				.getLimit().toString(), "hours(48)");
-
-		assertEquals(feed.getClusters().getClusters().get(1).getName(),
-				"backupCluster");
-		assertEquals(feed.getClusters().getClusters().get(1).getType(),
-				ClusterType.TARGET);
-		assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(1).getValidity()
-				.getStart()), "2011-11-01T00:00Z");
-		assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(1).getValidity()
-				.getEnd()), "2011-12-31T00:00Z");
-		assertEquals(feed.getClusters().getClusters().get(1).getRetention()
-				.getAction(), ActionType.ARCHIVE);
-		assertEquals(feed.getClusters().getClusters().get(1).getRetention()
-				.getLimit().toString(), "hours(6)");
-
-		assertEquals(FeedHelper.getLocation(feed, LocationType.DATA).getPath(),
-				"/projects/falcon/clicks");
-		assertEquals(FeedHelper.getLocation(feed, LocationType.META).getPath(),
-				"/projects/falcon/clicksMetaData");
-		assertEquals(FeedHelper.getLocation(feed, LocationType.STATS).getPath(),
-				"/projects/falcon/clicksStats");
-
-		assertEquals(feed.getACL().getGroup(), "group");
-		assertEquals(feed.getACL().getOwner(), "testuser");
-		assertEquals(feed.getACL().getPermission(), "0x755");
-
-		assertEquals(feed.getSchema().getLocation(), "/schema/clicks");
-		assertEquals(feed.getSchema().getProvider(), "protobuf");
-
-		StringWriter stringWriter = new StringWriter();
-		Marshaller marshaller = EntityType.FEED.getMarshaller();
-		marshaller.marshal(feed, stringWriter);
-		System.out.println(stringWriter.toString());
-	}
-
-	@Test(expectedExceptions = ValidationException.class)
-	public void applyValidationInvalidFeed() throws Exception {
-		Feed feed = (Feed) parser
-				.parseAndValidate(ProcessEntityParserTest.class
-						.getResourceAsStream(FEED_XML));
-		feed.getClusters().getClusters().get(0).setName("invalid cluster");
-		parser.validate(feed);
-	}
-
-	
-	@Test
-	public void testPartitionExpression() throws FalconException {
+    private final FeedEntityParser parser = (FeedEntityParser) EntityParserFactory
+            .getParser(EntityType.FEED);
+
+    private Feed modifiableFeed;
+
+    @BeforeMethod
+    public void setUp() throws Exception {
+        cleanupStore();
+        ConfigurationStore store = ConfigurationStore.get();
+
+        Unmarshaller unmarshaller = EntityType.CLUSTER.getUnmarshaller();
+        Cluster cluster = (Cluster) unmarshaller.unmarshal(this.getClass()
+                .getResourceAsStream(CLUSTER_XML));
+        cluster.setName("testCluster");
+        store.publish(EntityType.CLUSTER, cluster);
+
+        cluster = (Cluster) unmarshaller.unmarshal(this.getClass()
+                .getResourceAsStream(CLUSTER_XML));
+        cluster.setName("backupCluster");
+        store.publish(EntityType.CLUSTER, cluster);
+
+        modifiableFeed = (Feed) parser.parseAndValidate(this.getClass()
+                .getResourceAsStream(FEED_XML));
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testValidations() throws Exception {
+        ConfigurationStore.get().remove(EntityType.CLUSTER, "backupCluster");
+        parser.parseAndValidate(this.getClass().getResourceAsStream(FEED_XML));
+    }
+
+    @Test
+    public void testParse() throws IOException, FalconException, JAXBException {
+
+        Feed feed = parser.parseAndValidate(this.getClass()
+                .getResourceAsStream(FEED_XML));
+
+        Assert.assertNotNull(feed);
+        assertEquals(feed.getName(), "clicks");
+        assertEquals(feed.getDescription(), "clicks log");
+        assertEquals(feed.getFrequency().toString(), "hours(1)");
+        assertEquals(feed.getGroups(), "online,bi");
+
+        assertEquals(feed.getClusters().getClusters().get(0).getName(),
+                "testCluster");
+        assertEquals(feed.getClusters().getClusters().get(0).getType(),
+                ClusterType.SOURCE);
+        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity()
+                .getStart()), "2011-11-01T00:00Z");
+        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(0).getValidity()
+                .getEnd()), "2011-12-31T00:00Z");
+        assertEquals(feed.getTimezone().getID(), "UTC");
+        assertEquals(feed.getClusters().getClusters().get(0).getRetention()
+                .getAction(), ActionType.DELETE);
+        assertEquals(feed.getClusters().getClusters().get(0).getRetention()
+                .getLimit().toString(), "hours(48)");
+
+        assertEquals(feed.getClusters().getClusters().get(1).getName(),
+                "backupCluster");
+        assertEquals(feed.getClusters().getClusters().get(1).getType(),
+                ClusterType.TARGET);
+        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(1).getValidity()
+                .getStart()), "2011-11-01T00:00Z");
+        assertEquals(SchemaHelper.formatDateUTC(feed.getClusters().getClusters().get(1).getValidity()
+                .getEnd()), "2011-12-31T00:00Z");
+        assertEquals(feed.getClusters().getClusters().get(1).getRetention()
+                .getAction(), ActionType.ARCHIVE);
+        assertEquals(feed.getClusters().getClusters().get(1).getRetention()
+                .getLimit().toString(), "hours(6)");
+
+        assertEquals(FeedHelper.getLocation(feed, LocationType.DATA).getPath(),
+                "/projects/falcon/clicks");
+        assertEquals(FeedHelper.getLocation(feed, LocationType.META).getPath(),
+                "/projects/falcon/clicksMetaData");
+        assertEquals(FeedHelper.getLocation(feed, LocationType.STATS).getPath(),
+                "/projects/falcon/clicksStats");
+
+        assertEquals(feed.getACL().getGroup(), "group");
+        assertEquals(feed.getACL().getOwner(), "testuser");
+        assertEquals(feed.getACL().getPermission(), "0x755");
+
+        assertEquals(feed.getSchema().getLocation(), "/schema/clicks");
+        assertEquals(feed.getSchema().getProvider(), "protobuf");
+
+        StringWriter stringWriter = new StringWriter();
+        Marshaller marshaller = EntityType.FEED.getMarshaller();
+        marshaller.marshal(feed, stringWriter);
+        System.out.println(stringWriter.toString());
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void applyValidationInvalidFeed() throws Exception {
+        Feed feed = (Feed) parser
+                .parseAndValidate(ProcessEntityParserTest.class
+                        .getResourceAsStream(FEED_XML));
+        feed.getClusters().getClusters().get(0).setName("invalid cluster");
+        parser.validate(feed);
+    }
+
+
+    @Test
+    public void testPartitionExpression() throws FalconException {
         Feed feed = (Feed) parser.parseAndValidate(ProcessEntityParserTest.class
                 .getResourceAsStream(FEED_XML));
-        
+
         //When there are more than 1 src clusters, there should be partition expression
         org.apache.falcon.entity.v0.feed.Cluster newCluster = new org.apache.falcon.entity.v0.feed.Cluster();
         newCluster.setName("newCluster");
@@ -167,264 +160,269 @@ public class FeedEntityParserTest extends AbstractTestBase {
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch(ValidationException e) { }
-        
+        } catch (ValidationException e) {
+        }
+
         //When there are more than 1 src clusters, the partition expression should contain cluster variable
         feed.getClusters().getClusters().get(0).setPartition("*");
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch(ValidationException e) { }
-        
+        } catch (ValidationException e) {
+        }
+
         //When there are more than 1 target cluster, there should be partition expre
         newCluster.setType(ClusterType.TARGET);
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch(ValidationException e) { }
-        
+        } catch (ValidationException e) {
+        }
+
         //When there are more than 1 target clusters, the partition expression should contain cluster variable
         feed.getClusters().getClusters().get(1).setPartition("*");
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch(ValidationException e) { }
-        
+        } catch (ValidationException e) {
+        }
+
         //Number of parts in partition expression < number of partitions defined for feed
         feed.getClusters().getClusters().get(1).setPartition("*/*");
         try {
             parser.validate(feed);
             Assert.fail("Expected ValidationException");
-        } catch(ValidationException e) { }
+        } catch (ValidationException e) {
+        }
 
         feed.getClusters().getClusters().get(0).setPartition(null);
         feed.getClusters().getClusters().get(1).setPartition(null);
         feed.getClusters().getClusters().remove(2);
         feed.setPartitions(null);
         parser.validate(feed);
-	}
-
-	@Test
-	public void testInvalidClusterValidityTime() {
-		Validity validity = modifiableFeed.getClusters().getClusters().get(0)
-				.getValidity();
-		try {
-			validity.setStart(SchemaHelper.parseDateUTC("2007-02-29T00:00Z"));
-			modifiableFeed.getClusters().getClusters().get(0)
-					.setValidity(validity);
-			parser.parseAndValidate(marshallEntity(modifiableFeed));
-			Assert.fail("Cluster validity failed");
-		} catch (Exception e) {
-			System.out.println(e.getMessage());
-			validity.setStart(SchemaHelper.parseDateUTC("2011-11-01T00:00Z"));
-			modifiableFeed.getClusters().getClusters().get(0)
-					.setValidity(validity);
-		}
-
-		try {
-			validity.setEnd(SchemaHelper.parseDateUTC("2010-04-31T00:00Z"));
-			modifiableFeed.getClusters().getClusters().get(0)
-					.setValidity(validity);
-			parser.parseAndValidate(marshallEntity(modifiableFeed));
-			Assert.fail("Cluster validity failed");
-		} catch (Exception e) {
-			System.out.println(e.getMessage());
-			validity.setEnd(SchemaHelper.parseDateUTC("2011-12-31T00:00Z"));
-			modifiableFeed.getClusters().getClusters().get(0)
-					.setValidity(validity);
-		}
-	}
-
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidProcessValidity() throws Exception {
-		Feed feed = parser.parseAndValidate((FeedEntityParserTest.class
-				.getResourceAsStream(FEED_XML)));
-		feed.getClusters().getClusters().get(0).getValidity()
-				.setStart(SchemaHelper.parseDateUTC("2012-11-01T00:00Z"));
-		parser.validate(feed);
-	}
-
-	@Test
-	public void testValidFeedGroup() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				(FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
-		feed1.setName("f1" + System.currentTimeMillis());
-		feed1.setGroups("group1,group2,group3");
-		feed1.setLocations(new Locations());
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
-		location.setType(LocationType.DATA);
-		feed1.getLocations().getLocations().add(location);
-		feed1.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
-		parser.parseAndValidate(feed1.toString());
-		ConfigurationStore.get().publish(EntityType.FEED, feed1);
-
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				(FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
-		feed2.setName("f2" + System.currentTimeMillis());
-		feed2.setGroups("group1,group2,group5");
-		feed2.setLocations(new Locations());
-		Location location2 = new Location();
-		location2
-				.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
-		location2.setType(LocationType.DATA);
-		feed2.getLocations().getLocations().add(location2);
-		feed2.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
-		parser.parseAndValidate(feed2.toString());
-	}
-	
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidFeedClusterDataLocation() throws JAXBException, FalconException{
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				(FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
-		feed1.setName("f1" + System.currentTimeMillis());
-		feed1.setGroups("group1,group2,group3");
-		feed1.setLocations(new Locations());
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
-		location.setType(LocationType.DATA);
-		feed1.getLocations().getLocations().add(location);
-		parser.parseAndValidate(feed1.toString());
-	}
-
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidFeedGroup() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				(FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
-		feed1.setName("f1" + System.currentTimeMillis());
-		feed1.setGroups("group1,group2,group3");
-		feed1.setLocations(new Locations());
-		Location location = new Location();
-		location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
-		location.setType(LocationType.DATA);
-		feed1.getLocations().getLocations().add(location);
-		parser.parseAndValidate(feed1.toString());
-		
-		feed1.getClusters().getClusters().get(0).getLocations().getLocations().set(0,location);
-		ConfigurationStore.get().publish(EntityType.FEED, feed1);
-
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				(FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
-		feed2.setName("f2" + System.currentTimeMillis());
-		feed2.setGroups("group1,group2,group5");
-		feed2.setLocations(new Locations());
-		Location location2 = new Location();
-		location2
-				.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}/${HOUR}/ad");
-		location2.setType(LocationType.DATA);
-		feed2.getLocations().getLocations().add(location2);
-		feed2.getClusters().getClusters().get(0).getLocations().getLocations().set(0,location);
-		parser.parseAndValidate(feed2.toString());
-	}
-
-	@Test
-	public void testValidGroupNames() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("f1" + System.currentTimeMillis());
-		feed1.setGroups("group7,group8");
-		parser.parseAndValidate(feed1.toString());
-
-		feed1.setGroups("group7");
-		parser.parseAndValidate(feed1.toString());
-
-		feed1.setGroups(null);
-		parser.parseAndValidate(feed1.toString());
-		ConfigurationStore.get().publish(EntityType.FEED, feed1);
-	}
-
-	@Test
-	public void testInvalidGroupNames() throws FalconException, JAXBException {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("f1" + System.currentTimeMillis());
-
-		try {
-			feed1.setGroups("commaend,");
-			parser.parseAndValidate(feed1.toString());
-			Assert.fail("Expected exception");
-		} catch (FalconException e) {
-
-		}
-		try {
-			feed1.setGroups("group8,   group9");
-			parser.parseAndValidate(feed1.toString());
-			Assert.fail("Expected exception");
-		} catch (FalconException e) {
-
-		}
-		try {
-			feed1.setGroups("space in group,group9");
-			parser.parseAndValidate(feed1.toString());
-			Assert.fail("Expected exception");
-		} catch (FalconException e) {
-
-		}
-	}
-	
-	@Test
-	public void testClusterPartitionExp() throws FalconException {
-		Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER,
-				"testCluster");
-		Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
-				"/*/${cluster.colo}"), "/*/" + cluster.getColo());
-		Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
-				"/*/${cluster.name}/Local"), "/*/" + cluster.getName()+"/Local");
-		Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
-				"/*/${cluster.field1}/Local"), "/*/value1/Local");
-	}
-	
-	@Test(expectedExceptions=FalconException.class)
-	public void testInvalidFeedName() throws JAXBException, FalconException  {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("Feed_name");
-		parser.parseAndValidate(feed1.toString());
-	}
-	
-	@Test(expectedExceptions=FalconException.class)
-	public void testInvalidFeedGroupName() throws JAXBException, FalconException  {
-		Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed1.setName("feed1");
-		feed1.getLocations().getLocations().get(0)
-				.setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
-		feed1.getClusters().getClusters().get(0).getLocations().getLocations()
-				.get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
-		ConfigurationStore.get().publish(EntityType.FEED, feed1);
-		
-		Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		feed2.setName("feed2");
-		feed2.getLocations().getLocations().get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
-		feed2.getClusters().getClusters().get(0).getLocations().getLocations()
-		.get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
-		feed2.setFrequency(new Frequency("hours(1)"));
-		try{
-			parser.parseAndValidate(feed2.toString());
-		}catch(FalconException e){
-			e.printStackTrace();
-			Assert.fail("Not expecting exception for same frequency");
-		}
-		feed2.setFrequency(new Frequency("hours(2)"));
-		//expecting exception
-		parser.parseAndValidate(feed2.toString());
-	}
-	
-	@Test
-	public void testNullFeedLateArrival() throws JAXBException, FalconException  {
-		Feed feed = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
-				FeedGroupMapTest.class
-						.getResourceAsStream("/config/feed/feed-0.1.xml"));
-		
-		feed.setLateArrival(null);
-		parser.parseAndValidate(feed.toString());
-		
-	}
+    }
+
+    @Test
+    public void testInvalidClusterValidityTime() {
+        Validity validity = modifiableFeed.getClusters().getClusters().get(0)
+                .getValidity();
+        try {
+            validity.setStart(SchemaHelper.parseDateUTC("2007-02-29T00:00Z"));
+            modifiableFeed.getClusters().getClusters().get(0)
+                    .setValidity(validity);
+            parser.parseAndValidate(marshallEntity(modifiableFeed));
+            Assert.fail("Cluster validity failed");
+        } catch (Exception e) {
+            System.out.println(e.getMessage());
+            validity.setStart(SchemaHelper.parseDateUTC("2011-11-01T00:00Z"));
+            modifiableFeed.getClusters().getClusters().get(0)
+                    .setValidity(validity);
+        }
+
+        try {
+            validity.setEnd(SchemaHelper.parseDateUTC("2010-04-31T00:00Z"));
+            modifiableFeed.getClusters().getClusters().get(0)
+                    .setValidity(validity);
+            parser.parseAndValidate(marshallEntity(modifiableFeed));
+            Assert.fail("Cluster validity failed");
+        } catch (Exception e) {
+            System.out.println(e.getMessage());
+            validity.setEnd(SchemaHelper.parseDateUTC("2011-12-31T00:00Z"));
+            modifiableFeed.getClusters().getClusters().get(0)
+                    .setValidity(validity);
+        }
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidProcessValidity() throws Exception {
+        Feed feed = parser.parseAndValidate((FeedEntityParserTest.class
+                .getResourceAsStream(FEED_XML)));
+        feed.getClusters().getClusters().get(0).getValidity()
+                .setStart(SchemaHelper.parseDateUTC("2012-11-01T00:00Z"));
+        parser.validate(feed);
+    }
+
+    @Test
+    public void testValidFeedGroup() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                (FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
+        feed1.setName("f1" + System.currentTimeMillis());
+        feed1.setGroups("group1,group2,group3");
+        feed1.setLocations(new Locations());
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        location.setType(LocationType.DATA);
+        feed1.getLocations().getLocations().add(location);
+        feed1.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
+        parser.parseAndValidate(feed1.toString());
+        ConfigurationStore.get().publish(EntityType.FEED, feed1);
+
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                (FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
+        feed2.setName("f2" + System.currentTimeMillis());
+        feed2.setGroups("group1,group2,group5");
+        feed2.setLocations(new Locations());
+        Location location2 = new Location();
+        location2
+                .setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        location2.setType(LocationType.DATA);
+        feed2.getLocations().getLocations().add(location2);
+        feed2.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
+        parser.parseAndValidate(feed2.toString());
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidFeedClusterDataLocation() throws JAXBException, FalconException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                (FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
+        feed1.setName("f1" + System.currentTimeMillis());
+        feed1.setGroups("group1,group2,group3");
+        feed1.setLocations(new Locations());
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        location.setType(LocationType.DATA);
+        feed1.getLocations().getLocations().add(location);
+        parser.parseAndValidate(feed1.toString());
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidFeedGroup() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                (FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
+        feed1.setName("f1" + System.currentTimeMillis());
+        feed1.setGroups("group1,group2,group3");
+        feed1.setLocations(new Locations());
+        Location location = new Location();
+        location.setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}-${DAY}/ad");
+        location.setType(LocationType.DATA);
+        feed1.getLocations().getLocations().add(location);
+        parser.parseAndValidate(feed1.toString());
+
+        feed1.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
+        ConfigurationStore.get().publish(EntityType.FEED, feed1);
+
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                (FeedEntityParserTest.class.getResourceAsStream(FEED_XML)));
+        feed2.setName("f2" + System.currentTimeMillis());
+        feed2.setGroups("group1,group2,group5");
+        feed2.setLocations(new Locations());
+        Location location2 = new Location();
+        location2
+                .setPath("/projects/bi/rmc/daily/ad/${YEAR}/fraud/${MONTH}/${HOUR}/ad");
+        location2.setType(LocationType.DATA);
+        feed2.getLocations().getLocations().add(location2);
+        feed2.getClusters().getClusters().get(0).getLocations().getLocations().set(0, location);
+        parser.parseAndValidate(feed2.toString());
+    }
+
+    @Test
+    public void testValidGroupNames() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("f1" + System.currentTimeMillis());
+        feed1.setGroups("group7,group8");
+        parser.parseAndValidate(feed1.toString());
+
+        feed1.setGroups("group7");
+        parser.parseAndValidate(feed1.toString());
+
+        feed1.setGroups(null);
+        parser.parseAndValidate(feed1.toString());
+        ConfigurationStore.get().publish(EntityType.FEED, feed1);
+    }
+
+    @Test
+    public void testInvalidGroupNames() throws FalconException, JAXBException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("f1" + System.currentTimeMillis());
+
+        try {
+            feed1.setGroups("commaend,");
+            parser.parseAndValidate(feed1.toString());
+            Assert.fail("Expected exception");
+        } catch (FalconException e) {
+
+        }
+        try {
+            feed1.setGroups("group8,   group9");
+            parser.parseAndValidate(feed1.toString());
+            Assert.fail("Expected exception");
+        } catch (FalconException e) {
+
+        }
+        try {
+            feed1.setGroups("space in group,group9");
+            parser.parseAndValidate(feed1.toString());
+            Assert.fail("Expected exception");
+        } catch (FalconException e) {
+
+        }
+    }
+
+    @Test
+    public void testClusterPartitionExp() throws FalconException {
+        Cluster cluster = ConfigurationStore.get().get(EntityType.CLUSTER,
+                "testCluster");
+        Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
+                "/*/${cluster.colo}"), "/*/" + cluster.getColo());
+        Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
+                "/*/${cluster.name}/Local"), "/*/" + cluster.getName() + "/Local");
+        Assert.assertEquals(FeedHelper.evaluateClusterExp(cluster,
+                "/*/${cluster.field1}/Local"), "/*/value1/Local");
+    }
+
+    @Test(expectedExceptions = FalconException.class)
+    public void testInvalidFeedName() throws JAXBException, FalconException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("Feed_name");
+        parser.parseAndValidate(feed1.toString());
+    }
+
+    @Test(expectedExceptions = FalconException.class)
+    public void testInvalidFeedGroupName() throws JAXBException, FalconException {
+        Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed1.setName("feed1");
+        feed1.getLocations().getLocations().get(0)
+                .setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
+        feed1.getClusters().getClusters().get(0).getLocations().getLocations()
+                .get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
+        ConfigurationStore.get().publish(EntityType.FEED, feed1);
+
+        Feed feed2 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+        feed2.setName("feed2");
+        feed2.getLocations().getLocations().get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
+        feed2.getClusters().getClusters().get(0).getLocations().getLocations()
+                .get(0).setPath("/data/clicks/${YEAR}/${MONTH}/${DAY}/${HOUR}");
+        feed2.setFrequency(new Frequency("hours(1)"));
+        try {
+            parser.parseAndValidate(feed2.toString());
+        } catch (FalconException e) {
+            e.printStackTrace();
+            Assert.fail("Not expecting exception for same frequency");
+        }
+        feed2.setFrequency(new Frequency("hours(2)"));
+        //expecting exception
+        parser.parseAndValidate(feed2.toString());
+    }
+
+    @Test
+    public void testNullFeedLateArrival() throws JAXBException, FalconException {
+        Feed feed = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
+                FeedGroupMapTest.class
+                        .getResourceAsStream("/config/feed/feed-0.1.xml"));
+
+        feed.setLateArrival(null);
+        parser.parseAndValidate(feed.toString());
+
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 7d5ef5f..4f62431 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
@@ -46,10 +46,10 @@ public class FeedUpdateTest extends AbstractTestBase {
         setup();
     }
 
-	@AfterClass
-	public void tearDown() {
-		this.dfsCluster.shutdown();
-	}
+    @AfterClass
+    public void tearDown() {
+        this.dfsCluster.shutdown();
+    }
 
     @Override
     public void setup() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 9118bbd..88e9968 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,15 +18,6 @@
 
 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;
@@ -41,7 +32,15 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-public class ProcessEntityParserTest extends AbstractTestBase{
+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;
+
+public class ProcessEntityParserTest extends AbstractTestBase {
 
     private final ProcessEntityParser parser = (ProcessEntityParser) EntityParserFactory.getParser(EntityType.PROCESS);
     private String INVALID_PROCESS_XML = "/config/process/process-invalid.xml";
@@ -57,15 +56,15 @@ public class ProcessEntityParserTest extends AbstractTestBase{
         this.dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
         this.conf = dfsCluster.getConf();
     }
-    
-	@AfterClass
-	public void tearDown() {
-		this.dfsCluster.shutdown();
-	}
-    
+
+    @AfterClass
+    public void tearDown() {
+        this.dfsCluster.shutdown();
+    }
+
     @BeforeMethod
     public void setup() throws Exception {
-        storeEntity(EntityType.CLUSTER, "testCluster");        
+        storeEntity(EntityType.CLUSTER, "testCluster");
         storeEntity(EntityType.FEED, "impressionFeed");
         storeEntity(EntityType.FEED, "clicksFeed");
         storeEntity(EntityType.FEED, "imp-click-join1");
@@ -77,7 +76,7 @@ public class ProcessEntityParserTest extends AbstractTestBase{
     public void testParse() throws FalconException, JAXBException {
 
         Process process = parser.parseAndValidate(getClass().getResourceAsStream(PROCESS_XML));
-        
+
         Assert.assertNotNull(process);
         Assert.assertEquals(process.getName(), "sample");
 
@@ -123,28 +122,31 @@ public class ProcessEntityParserTest extends AbstractTestBase{
         try {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
-        } catch (ValidationException e) { }
+        } catch (ValidationException e) {
+        }
 
         process.getInputs().getInputs().get(0).setStart("today(0,0)");
         process.getInputs().getInputs().get(0).setEnd("lastMonth(0,0,0)");
         try {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
-        } catch (ValidationException e) { }
+        } catch (ValidationException e) {
+        }
 
         process.getInputs().getInputs().get(0).setStart("today(2,0)");
         process.getInputs().getInputs().get(0).setEnd("today(0,0)");
         try {
             parser.validate(process);
             throw new AssertionError("Expected ValidationException!");
-        } catch (ValidationException e) { }
+        } catch (ValidationException e) {
+        }
 
     }
-    
+
     @Test(expectedExceptions = FalconException.class)
     public void doParseInvalidXML() throws IOException, FalconException {
 
-        parser.parseAndValidate(this.getClass().getResourceAsStream(INVALID_PROCESS_XML ));
+        parser.parseAndValidate(this.getClass().getResourceAsStream(INVALID_PROCESS_XML));
     }
 
     @Test(expectedExceptions = ValidationException.class)
@@ -162,7 +164,7 @@ public class ProcessEntityParserTest extends AbstractTestBase{
     @Test
     public void testConcurrentParsing() throws Exception {
         List<Thread> threadList = new ArrayList<Thread>();
-        
+
         for (int i = 0; i < 3; i++) {
             threadList.add(new Thread() {
                 public void run() {
@@ -175,102 +177,103 @@ public class ProcessEntityParserTest extends AbstractTestBase{
                 }
             });
         }
-        for(Thread thread:threadList) {
+        for (Thread thread : threadList) {
             thread.start();
         }
-        for(Thread thread:threadList) {
+        for (Thread thread : threadList) {
             thread.join();
         }
     }
 
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidProcessValidity() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getClusters().getClusters().get(0).getValidity().setStart(SchemaHelper.parseDateUTC("2011-12-31T00:00Z"));
-		parser.validate(process);
-	}
-	
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidDependentFeedsRetentionLimit() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getInputs().getInputs().get(0).setStart("today(-48,0)");
-		parser.validate(process);
-	}
-	
-	@Test(expectedExceptions = ValidationException.class)
-	public void testDuplicateInputOutputNames() throws FalconException {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getInputs().getInputs().get(0).setName("duplicateName");
-		process.getOutputs().getOutputs().get(0).setName("duplicateName");
-		parser.validate(process);
-	}
-	
-	@Test(expectedExceptions = FalconException.class)
-	public void testInvalidRetryAttempt() throws FalconException {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getRetry().setAttempts(-1);
-		parser.parseAndValidate(process.toString());
-	}
-
-	@Test(expectedExceptions = FalconException.class)
-	public void testInvalidRetryDelay() throws FalconException {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getRetry().setDelay(Frequency.fromString("hours(0)"));
-		parser.parseAndValidate(process.toString());
-	}
-	
-	@Test(expectedExceptions = ValidationException.class)
-	public void testInvalidLateInputs() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getLateProcess().getLateInputs().get(0).setInput("invalidInput");
-		parser.parseAndValidate(process.toString());
-	}
-	
-	@Test(expectedExceptions = FalconException.class)
-	public void testInvalidProcessName() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.setName("name_with_underscore");
-		parser.parseAndValidate(process.toString());
-	}
-	
-	@Test
-	public void testOozieFutureExpression() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getInputs().getInputs().get(0).setStart("future(1,2)");
-		parser.parseAndValidate(process.toString());
-	}
-	
-	@Test
-	public void testOozieLatestExpression() throws Exception {
-		Process process = parser
-				.parseAndValidate((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getInputs().getInputs().get(0).setStart("latest(-1)");
-		parser.parseAndValidate(process.toString());
-	}
-	
-	@Test(expectedExceptions=ValidationException.class)
-	public void testDuplicateClusterName() throws Exception {
-		Process process = parser
-				.parse((ProcessEntityParserTest.class
-						.getResourceAsStream(PROCESS_XML)));
-		process.getClusters().getClusters().add(1, process.getClusters().getClusters().get(0));
-		parser.validate(process);
-	}
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidProcessValidity() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getClusters().getClusters().get(0).getValidity().setStart(
+                SchemaHelper.parseDateUTC("2011-12-31T00:00Z"));
+        parser.validate(process);
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidDependentFeedsRetentionLimit() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getInputs().getInputs().get(0).setStart("today(-48,0)");
+        parser.validate(process);
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testDuplicateInputOutputNames() throws FalconException {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getInputs().getInputs().get(0).setName("duplicateName");
+        process.getOutputs().getOutputs().get(0).setName("duplicateName");
+        parser.validate(process);
+    }
+
+    @Test(expectedExceptions = FalconException.class)
+    public void testInvalidRetryAttempt() throws FalconException {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getRetry().setAttempts(-1);
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test(expectedExceptions = FalconException.class)
+    public void testInvalidRetryDelay() throws FalconException {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getRetry().setDelay(Frequency.fromString("hours(0)"));
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testInvalidLateInputs() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getLateProcess().getLateInputs().get(0).setInput("invalidInput");
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test(expectedExceptions = FalconException.class)
+    public void testInvalidProcessName() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.setName("name_with_underscore");
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test
+    public void testOozieFutureExpression() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getInputs().getInputs().get(0).setStart("future(1,2)");
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test
+    public void testOozieLatestExpression() throws Exception {
+        Process process = parser
+                .parseAndValidate((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getInputs().getInputs().get(0).setStart("latest(-1)");
+        parser.parseAndValidate(process.toString());
+    }
+
+    @Test(expectedExceptions = ValidationException.class)
+    public void testDuplicateClusterName() throws Exception {
+        Process process = parser
+                .parse((ProcessEntityParserTest.class
+                        .getResourceAsStream(PROCESS_XML)));
+        process.getClusters().getClusters().add(1, process.getClusters().getClusters().get(0));
+        parser.validate(process);
+    }
 }


[41/47] git commit: checkstyle related fixes for prism module.

Posted by sr...@apache.org.
checkstyle related fixes for prism module.


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

Branch: refs/heads/master
Commit: f8b9122243af317905fc60c8555f1514851f96f3
Parents: 12889e2
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:26:07 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:26:07 2013 -0700

----------------------------------------------------------------------
 .../java/org/apache/falcon/FalconWebException.java |    7 +-
 prism/src/main/java/org/apache/falcon/Prism.java   |   26 +++++++-
 .../falcon/listener/ContextStartupListener.java    |    5 +-
 .../falcon/plugin/ChainableMonitoringPlugin.java   |    4 +
 .../falcon/resource/AbstractEntityManager.java     |   17 +++--
 .../falcon/resource/AbstractInstanceManager.java   |    8 ++-
 .../resource/AbstractSchedulableEntityManager.java |   12 ++--
 .../falcon/resource/admin/AdminResource.java       |   11 +++-
 .../falcon/resource/channel/AbstractChannel.java   |   12 ++--
 .../apache/falcon/resource/channel/Channel.java    |    3 +
 .../falcon/resource/channel/ChannelFactory.java    |   18 ++++--
 .../falcon/resource/channel/HTTPChannel.java       |   49 ++++++---------
 .../apache/falcon/resource/channel/IPCChannel.java |   14 +++--
 .../apache/falcon/resource/channel/MethodKey.java  |   22 ++++---
 .../resource/channel/NullServletRequest.java       |    3 +
 .../resource/provider/JAXBContextResolver.java     |   11 +++-
 .../falcon/resource/proxy/BufferedRequest.java     |    5 +-
 .../resource/proxy/InstanceManagerProxy.java       |    3 +
 .../proxy/SchedulableEntityManagerProxy.java       |    5 +-
 .../apache/falcon/security/BasicAuthFilter.java    |   29 ++++-----
 .../falcon/service/FalconTopicSubscriber.java      |   30 +++------
 .../falcon/service/ProcessSubscriberService.java   |   22 +++---
 .../falcon/service/SLAMonitoringService.java       |   31 +++++-----
 .../org/apache/falcon/util/EmbeddedServer.java     |    3 +
 .../org/apache/falcon/aspect/GenericAlertTest.java |    3 +
 .../apache/falcon/aspect/LoggingAspectTest.java    |   12 ++--
 .../apache/falcon/resource/EntityManagerTest.java  |   12 ++--
 .../falcon/service/FalconTopicSubscriberTest.java  |    6 +-
 28 files changed, 219 insertions(+), 164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/FalconWebException.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/FalconWebException.java b/prism/src/main/java/org/apache/falcon/FalconWebException.java
index c0d41ca..39340b6 100644
--- a/prism/src/main/java/org/apache/falcon/FalconWebException.java
+++ b/prism/src/main/java/org/apache/falcon/FalconWebException.java
@@ -27,6 +27,9 @@ import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 
+/**
+ * Exception for REST APIs.
+ */
 public class FalconWebException extends WebApplicationException {
 
     private static final Logger LOG = Logger.getLogger(FalconWebException.class);
@@ -74,8 +77,7 @@ public class FalconWebException extends WebApplicationException {
     private static String getAddnInfo(Throwable e) {
         String addnInfo = "";
         Throwable cause = e.getCause();
-        if (cause != null && cause.getMessage() != null &&
-                !getMessage(e).contains(cause.getMessage())) {
+        if (cause != null && cause.getMessage() != null && !getMessage(e).contains(cause.getMessage())) {
             addnInfo = cause.getMessage();
         }
         return addnInfo;
@@ -84,5 +86,4 @@ public class FalconWebException extends WebApplicationException {
     public FalconWebException(Response response) {
         super(response);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/Prism.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/Prism.java b/prism/src/main/java/org/apache/falcon/Prism.java
index 9cb2cc9..4c8b43c 100644
--- a/prism/src/main/java/org/apache/falcon/Prism.java
+++ b/prism/src/main/java/org/apache/falcon/Prism.java
@@ -20,10 +20,30 @@ package org.apache.falcon;
 
 import org.apache.falcon.util.EmbeddedServer;
 
-public class Prism {
+/**
+ * Driver for running Prism as a standalone server.
+ */
+public final class Prism {
+
+    /**
+     * Default port number for embedded server.
+     */
+    private static final int PORT = 16000;
+
+    /**
+     * Prevent users from constructing this.
+     */
+    private Prism() {
+    }
 
-    public static void main(String[] args) throws Exception {
-        EmbeddedServer server = new EmbeddedServer(16000,
+    /**
+     * Main method.
+     *
+     * @param args argument array
+     * @throws Exception exception
+     */
+    public static void main(final String[] args) throws Exception {
+        EmbeddedServer server = new EmbeddedServer(PORT,
                 "prism/target/falcon-prism-0.2-SNAPSHOT");
         server.start();
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java b/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
index 54a9b4a..f22d831 100644
--- a/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
+++ b/prism/src/main/java/org/apache/falcon/listener/ContextStartupListener.java
@@ -31,9 +31,12 @@ import javax.servlet.ServletContextListener;
 import java.util.Map;
 import java.util.Properties;
 
+/**
+ * Listener for bootstrapping Falcon Services and configuration properties.
+ */
 public class ContextStartupListener implements ServletContextListener {
 
-    private static Logger LOG = Logger.getLogger(ContextStartupListener.class);
+    private static final Logger LOG = Logger.getLogger(ContextStartupListener.class);
 
     private final ServiceInitializer startupServices = new ServiceInitializer();
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java b/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
index 2b99e70..72e0302 100644
--- a/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
+++ b/prism/src/main/java/org/apache/falcon/plugin/ChainableMonitoringPlugin.java
@@ -30,6 +30,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+/**
+ * This class implements the chain of responsibility for configured implementations
+ * of {@link MonitoringPlugin}. {@link LoggingPlugin} is the default.
+ */
 @Aspect
 public class ChainableMonitoringPlugin extends AbstractFalconAspect implements MonitoringPlugin {
     private static final Logger LOG = Logger.getLogger(ChainableMonitoringPlugin.class);

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
index a272957..cad0d68 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractEntityManager.java
@@ -51,6 +51,9 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.*;
 
+/**
+ * A base class for managing Entity operations.
+ */
 public abstract class AbstractEntityManager {
     private static final Logger LOG = Logger.getLogger(AbstractEntityManager.class);
     private static final Logger AUDIT = Logger.getLogger("AUDIT");
@@ -67,7 +70,7 @@ public abstract class AbstractEntityManager {
         }
     }
 
-    protected void checkColo(String colo) throws FalconWebException {
+    protected void checkColo(String colo) {
         if (!DeploymentUtil.getCurrentColo().equals(colo)) {
             throw FalconWebException.newException(
                     "Current colo (" + DeploymentUtil.getCurrentColo() + ") is not " + colo,
@@ -93,7 +96,7 @@ public abstract class AbstractEntityManager {
         return colos;
     }
 
-    protected Set<String> getApplicableColos(String type, String name) throws FalconWebException {
+    protected Set<String> getApplicableColos(String type, String name) {
         try {
             if (DeploymentUtil.isEmbeddedMode()) {
                 return DeploymentUtil.getDefaultColos();
@@ -109,7 +112,7 @@ public abstract class AbstractEntityManager {
         }
     }
 
-    protected Set<String> getApplicableColos(String type, Entity entity) throws FalconWebException {
+    protected Set<String> getApplicableColos(String type, Entity entity) {
         try {
             if (DeploymentUtil.isEmbeddedMode()) {
                 return DeploymentUtil.getDefaultColos();
@@ -295,7 +298,7 @@ public abstract class AbstractEntityManager {
     }
 
     protected synchronized Entity submitInternal(HttpServletRequest request, String type)
-            throws IOException, FalconException {
+        throws IOException, FalconException {
 
         EntityType entityType = EntityType.valueOf(type.toUpperCase());
         Entity entity = deserializeEntity(request, entityType);
@@ -318,7 +321,7 @@ public abstract class AbstractEntityManager {
     }
 
     protected Entity deserializeEntity(HttpServletRequest request, EntityType entityType)
-            throws IOException, FalconException {
+        throws IOException, FalconException {
 
         EntityParser<?> entityParser = EntityParserFactory.getParser(entityType);
         InputStream xmlStream = request.getInputStream();
@@ -334,6 +337,7 @@ public abstract class AbstractEntityManager {
                     String xmlData = getAsString(xmlStream);
                     LOG.debug("XML DUMP for (" + entityType + "): " + xmlData, e);
                 } catch (IOException ignore) {
+                    // ignore
                 }
             }
             throw e;
@@ -449,7 +453,7 @@ public abstract class AbstractEntityManager {
     }
 
     /**
-     * Returns the entity definition as an XML based on name
+     * Returns the entity definition as an XML based on name.
      *
      * @param type
      * @param entityName
@@ -473,5 +477,4 @@ public abstract class AbstractEntityManager {
     protected AbstractWorkflowEngine getWorkflowEngine() {
         return this.workflowEngine;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
index 8097c43..d94e8c5 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractInstanceManager.java
@@ -40,6 +40,9 @@ import java.util.Date;
 import java.util.Properties;
 import java.util.Set;
 
+/**
+ * A base class for managing Entity's Instance operations.
+ */
 public abstract class AbstractInstanceManager extends AbstractEntityManager {
     private static final Logger LOG = Logger.getLogger(AbstractInstanceManager.class);
 
@@ -232,8 +235,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
         validateDateRange(entityObject, startStr, endStr);
     }
 
-    private void validateDateRange(Entity entity, String start, String end)
-            throws FalconException {
+    private void validateDateRange(Entity entity, String start, String end) throws FalconException {
         Set<String> clusters = EntityUtil.getClustersDefined(entity);
         Pair<Date, String> clusterMinStartDate = null;
         Pair<Date, String> clusterMaxEndDate = null;
@@ -253,7 +255,7 @@ public abstract class AbstractInstanceManager extends AbstractEntityManager {
 
     private void validateDateRangeFor(Entity entity, Pair<Date, String> clusterMinStart,
                                       Pair<Date, String> clusterMaxEnd, String start, String end)
-            throws FalconException {
+        throws FalconException {
 
         Date instStart = EntityUtil.parseDateUTC(start);
         if (instStart.before(clusterMinStart.first)) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
index 9a41d33..cc4446c 100644
--- a/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
+++ b/prism/src/main/java/org/apache/falcon/resource/AbstractSchedulableEntityManager.java
@@ -33,15 +33,15 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
 
 /**
- * REST resource of allowed actions on Schedulable Entities Only Process and
- * Feed can have schedulable actions
+ * REST resource of allowed actions on Schedulable Entities, Only Process and
+ * Feed can have schedulable actions.
  */
 public abstract class AbstractSchedulableEntityManager extends AbstractEntityManager {
 
     private static final Logger LOG = Logger.getLogger(AbstractSchedulableEntityManager.class);
 
     /**
-     * Schedules an submitted entity immediately
+     * Schedules an submitted entity immediately.
      *
      * @param type
      * @param entity
@@ -69,7 +69,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
     }
 
     /**
-     * Submits a new entity and schedules it immediately
+     * Submits a new entity and schedules it immediately.
      *
      * @param type
      * @return
@@ -92,7 +92,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
     }
 
     /**
-     * Suspends a running entity
+     * Suspends a running entity.
      *
      * @param type
      * @param entity
@@ -120,7 +120,7 @@ public abstract class AbstractSchedulableEntityManager extends AbstractEntityMan
     }
 
     /**
-     * Resumes a suspended entity
+     * Resumes a suspended entity.
      *
      * @param type
      * @param entity

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/admin/AdminResource.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/admin/AdminResource.java b/prism/src/main/java/org/apache/falcon/resource/admin/AdminResource.java
index 93db427..de14547 100644
--- a/prism/src/main/java/org/apache/falcon/resource/admin/AdminResource.java
+++ b/prism/src/main/java/org/apache/falcon/resource/admin/AdminResource.java
@@ -36,6 +36,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+/**
+ * Jersey Resource for admin operations.
+ */
 @Path("admin")
 public class AdminResource {
 
@@ -68,8 +71,8 @@ public class AdminResource {
     @Produces(MediaType.TEXT_PLAIN)
     public String getVersion() {
         if (version == null) {
-            version = "{Version:\"" + BuildProperties.get().getProperty("build.version") +
-                    "\",Mode:\"" + DeploymentProperties.get().getProperty("deploy.mode") + "\"}";
+            version = "{Version:\"" + BuildProperties.get().getProperty("build.version")
+                    + "\",Mode:\"" + DeploymentProperties.get().getProperty("deploy.mode") + "\"}";
         }
         return version;
     }
@@ -105,16 +108,20 @@ public class AdminResource {
         return propertyList;
     }
 
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     @XmlRootElement(name = "property")
     @XmlAccessorType(XmlAccessType.FIELD)
     private static class Property {
         public String key;
         public String value;
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     @XmlRootElement(name = "properties")
     @XmlAccessorType(XmlAccessType.FIELD)
     private static class PropertyList {
         public List<Property> properties;
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
index 7b15b96..d9ed6af 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/AbstractChannel.java
@@ -23,13 +23,15 @@ import org.apache.falcon.FalconException;
 import java.lang.reflect.Method;
 import java.util.concurrent.ConcurrentHashMap;
 
+/**
+ * A base class for Channel.
+ */
 public abstract class AbstractChannel implements Channel {
 
-    private final ConcurrentHashMap<MethodKey, Method> methods
-            = new ConcurrentHashMap<MethodKey, Method>();
+    private final ConcurrentHashMap<MethodKey, Method> methods = new ConcurrentHashMap<MethodKey, Method>();
 
     protected Method getMethod(Class service, String methodName, Object... args)
-            throws FalconException {
+        throws FalconException {
         MethodKey methodKey = new MethodKey(methodName, args);
         Method method = methods.get(methodKey);
         if (method == null) {
@@ -41,8 +43,8 @@ public abstract class AbstractChannel implements Channel {
                     return item;
                 }
             }
-            throw new FalconException("Lookup for " + methodKey +
-                    " in service " + service.getName() + " found no match");
+            throw new FalconException("Lookup for " + methodKey
+                    + " in service " + service.getName() + " found no match");
         }
         return method;
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/Channel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/Channel.java b/prism/src/main/java/org/apache/falcon/resource/channel/Channel.java
index cafbd88..9dff680 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/Channel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/Channel.java
@@ -20,6 +20,9 @@ package org.apache.falcon.resource.channel;
 
 import org.apache.falcon.FalconException;
 
+/**
+ * Interface for a Channel.
+ */
 public interface Channel {
 
     void init(String colo, String serviceName) throws FalconException;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/ChannelFactory.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/ChannelFactory.java b/prism/src/main/java/org/apache/falcon/resource/channel/ChannelFactory.java
index 24e0b1e..06e234e 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/ChannelFactory.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/ChannelFactory.java
@@ -24,21 +24,28 @@ import org.apache.falcon.util.DeploymentProperties;
 import java.util.HashMap;
 import java.util.Map;
 
-public class ChannelFactory {
+/**
+ * A factory implementation for doling out {@link Channel} objects.
+ */
+public final class ChannelFactory {
 
     private static Map<String, Channel> channels = new HashMap<String, Channel>();
 
     private static final String EMBEDDED = "embedded";
     private static final String MODE = "deploy.mode";
 
-    public synchronized static Channel get(String serviceName, String colo)
-            throws FalconException {
-        Channel channel;
-        if ((channel = channels.get(colo + "/" + serviceName)) == null) {
+    private ChannelFactory() {
+    }
+
+    public static synchronized Channel get(String serviceName, String colo)
+        throws FalconException {
+        Channel channel = channels.get(colo + "/" + serviceName);
+        if (channel == null) {
             channel = getChannel(DeploymentProperties.get().getProperty(MODE));
             channel.init(colo, serviceName);
             channels.put(colo + "/" + serviceName, channel);
         }
+
         return channel;
     }
 
@@ -49,6 +56,7 @@ public class ChannelFactory {
         } else {
             channel = new HTTPChannel();
         }
+
         return channel;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
index 9873a96..2ba76a7 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/HTTPChannel.java
@@ -38,42 +38,38 @@ import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.util.Properties;
 
+/**
+ * A Channel implementation for HTTP.
+ */
 public class HTTPChannel extends AbstractChannel {
     private static final Logger LOG = Logger.getLogger(HTTPChannel.class);
 
     private static final String REMOTE_USER = "Remote-User";
 
-    private static final HttpServletRequest DEFAULT_NULL_REQUEST =
-            new NullServletRequest();
+    private static final HttpServletRequest DEFAULT_NULL_REQUEST = new NullServletRequest();
 
-    private static final Properties deploymentProperties = DeploymentProperties.get();
+    private static final Properties DEPLOYMENT_PROPERTIES = DeploymentProperties.get();
 
     private Class service;
     private String urlPrefix;
 
     public void init(String colo, String serviceName) throws FalconException {
-        String prefixPath = deploymentProperties.
-                getProperty(serviceName + ".path");
-        String falconEndPoint = RuntimeProperties.get().
-                getProperty("falcon." + colo + ".endpoint");
+        String prefixPath = DEPLOYMENT_PROPERTIES.getProperty(serviceName + ".path");
+        String falconEndPoint = RuntimeProperties.get().getProperty("falcon." + colo + ".endpoint");
         urlPrefix = falconEndPoint + "/" + prefixPath;
 
         try {
-            String proxyClassName = deploymentProperties.
-                    getProperty(serviceName + ".proxy");
+            String proxyClassName = DEPLOYMENT_PROPERTIES.getProperty(serviceName + ".proxy");
             service = Class.forName(proxyClassName);
             LOG.info("Service: " + serviceName + ", url = " + urlPrefix);
         } catch (Exception e) {
-            throw new FalconException("Unable to initialize channel for "
-                    + serviceName, e);
+            throw new FalconException("Unable to initialize channel for " + serviceName, e);
         }
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T> T invoke(String methodName, Object... args)
-            throws FalconException {
-
+    public <T> T invoke(String methodName, Object... args) throws FalconException {
         try {
             Method method = getMethod(service, methodName, args);
             String url = urlPrefix + "/" + pathValue(method, args);
@@ -96,8 +92,8 @@ public class HTTPChannel extends AbstractChannel {
             Family status = response.getClientResponseStatus().getFamily();
             if (status == Family.INFORMATIONAL || status == Family.SUCCESSFUL) {
                 return (T) response.getEntity(method.getReturnType());
-            } else if (response.getClientResponseStatus().getStatusCode() ==
-                    Response.Status.BAD_REQUEST.getStatusCode()) {
+            } else if (response.getClientResponseStatus().getStatusCode()
+                    == Response.Status.BAD_REQUEST.getStatusCode()) {
                 LOG.error("Request failed: " + response.getClientResponseStatus().getStatusCode());
                 return (T) response.getEntity(method.getReturnType());
             } else {
@@ -111,14 +107,10 @@ public class HTTPChannel extends AbstractChannel {
     }
 
     private boolean isPost(String httpMethod) {
-        if (httpMethod.equals("POST") || httpMethod.equals("PUT")) {
-            return true;
-        }
-        return false;
+        return httpMethod.equals("POST") || httpMethod.equals("PUT");
     }
 
-    private String pathValue(Method method, Object... args)
-            throws FalconException {
+    private String pathValue(Method method, Object... args) throws FalconException {
 
         Path pathParam = method.getAnnotation(Path.class);
         if (pathParam == null) {
@@ -139,8 +131,8 @@ public class HTTPChannel extends AbstractChannel {
                         queryString.append(getAnnotationValue(paramAnnotation, "value")).
                                 append('=').append(arg).append("&");
                     } else if (annotationClass.equals(PathParam.class.getName())) {
-                        pathValue = pathValue.replace("{" +
-                                getAnnotationValue(paramAnnotation, "value") + "}", arg);
+                        pathValue = pathValue.replace("{"
+                                + getAnnotationValue(paramAnnotation, "value") + "}", arg);
                     }
                 }
             }
@@ -149,15 +141,13 @@ public class HTTPChannel extends AbstractChannel {
     }
 
     private String getAnnotationValue(Annotation paramAnnotation,
-                                      String annotationAttribute)
-            throws FalconException {
-
+                                      String annotationAttribute) throws FalconException {
         try {
             return String.valueOf(paramAnnotation.annotationType().
                     getMethod(annotationAttribute).invoke(paramAnnotation));
         } catch (Exception e) {
-            throw new FalconException("Unable to get attribute value for " +
-                    paramAnnotation + "[" + annotationAttribute + "]");
+            throw new FalconException("Unable to get attribute value for "
+                    + paramAnnotation + "[" + annotationAttribute + "]");
         }
     }
 
@@ -204,5 +194,4 @@ public class HTTPChannel extends AbstractChannel {
         }
         return produces.value()[0];
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java b/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
index b4d88f9..bbdaf40 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/IPCChannel.java
@@ -27,6 +27,9 @@ import org.apache.log4j.Logger;
 
 import java.lang.reflect.Method;
 
+/**
+ * Inter-process implementation of a Channel.
+ */
 public class IPCChannel extends AbstractChannel {
     private static final Logger LOG = Logger.getLogger(IPCChannel.class);
     private AbstractEntityManager service;
@@ -37,11 +40,10 @@ public class IPCChannel extends AbstractChannel {
 
     @SuppressWarnings("unchecked")
     @Override
-    public <T> T invoke(String methodName, Object... args)
-            throws FalconException {
-        LOG.debug("Invoking method " + methodName + " on service " +
-                service.getClass().getName());
+    public <T> T invoke(String methodName, Object... args) throws FalconException {
+        LOG.debug("Invoking method " + methodName + " on service " + service.getClass().getName());
         Method method = getMethod(service.getClass(), methodName, args);
+
         try {
             return (T) method.invoke(service, args);
         } catch (Exception e) {
@@ -57,8 +59,8 @@ public class IPCChannel extends AbstractChannel {
                     throw (FalconException) cause;
                 }
             }
-            throw new FalconException("Unable to invoke on the channel " + methodName +
-                    " on service : " + service.getClass().getName() + cause);
+            throw new FalconException("Unable to invoke on the channel " + methodName
+                    + " on service : " + service.getClass().getName() + cause);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java b/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
index e4fddc2..446bcc6 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/MethodKey.java
@@ -20,6 +20,9 @@ package org.apache.falcon.resource.channel;
 
 import java.util.Arrays;
 
+/**
+ * This class represents a tuple of method name and class objects.
+ */
 public class MethodKey {
 
     private final String name;
@@ -45,24 +48,25 @@ public class MethodKey {
         if (this == methodRHS) {
             return true;
         }
-        if (methodRHS == null ||
-                getClass() != methodRHS.getClass()) {
+
+        if (methodRHS == null || getClass() != methodRHS.getClass()) {
             return false;
         }
 
         MethodKey methodKey = (MethodKey) methodRHS;
 
-        if (name != null ? !name.equals(methodKey.name) :
-                methodKey.name != null) {
+        if (name != null ? !name.equals(methodKey.name) : methodKey.name != null) {
             return false;
         }
+
         boolean matching = true;
         for (int index = 0; index < argClasses.length; index++) {
-            if (argClasses[index] != null && methodKey.argClasses[index] != null &&
-                    !methodKey.argClasses[index].isAssignableFrom(argClasses[index])) {
+            if (argClasses[index] != null && methodKey.argClasses[index] != null
+                    && !methodKey.argClasses[index].isAssignableFrom(argClasses[index])) {
                 matching = false;
             }
         }
+
         return matching;
     }
 
@@ -75,9 +79,7 @@ public class MethodKey {
 
     @Override
     public String toString() {
-        return "MethodKey{" +
-                "name='" + name + '\'' +
-                ", argClasses=" + (argClasses == null ? null : Arrays.asList(argClasses)) +
-                '}';
+        return "MethodKey{name='" + name + '\'' + ", argClasses="
+                + (argClasses == null ? null : Arrays.asList(argClasses)) + '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/channel/NullServletRequest.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/channel/NullServletRequest.java b/prism/src/main/java/org/apache/falcon/resource/channel/NullServletRequest.java
index 8586644..d90926d 100644
--- a/prism/src/main/java/org/apache/falcon/resource/channel/NullServletRequest.java
+++ b/prism/src/main/java/org/apache/falcon/resource/channel/NullServletRequest.java
@@ -31,6 +31,9 @@ import java.util.Enumeration;
 import java.util.Locale;
 import java.util.Map;
 
+/**
+ * A null implementation of a ServletRequest.
+ */
 public class NullServletRequest implements HttpServletRequest {
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java b/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
index 11d0882..a58eade 100644
--- a/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
+++ b/prism/src/main/java/org/apache/falcon/resource/provider/JAXBContextResolver.java
@@ -30,12 +30,19 @@ import javax.ws.rs.ext.Provider;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
+/**
+ * An implementation of Context Resolver for JAXB.
+ */
 @Provider
 @Produces(MediaType.APPLICATION_JSON)
 public class JAXBContextResolver implements ContextResolver<JAXBContext> {
+
     private static JAXBContext context;
-    private static Class<?>[] types = {InstancesResult.class, APIResult.class, InstancesResult.Instance.class,
-                                       InstancesResult.WorkflowStatus.class};
+    private static Class<?>[] types = {
+        InstancesResult.class,
+        APIResult.class,
+        InstancesResult.Instance.class,
+        InstancesResult.WorkflowStatus.class, };
 
     static {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java b/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
index e797fb6..c667114 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/BufferedRequest.java
@@ -33,6 +33,9 @@ import java.util.Enumeration;
 import java.util.Locale;
 import java.util.Map;
 
+/**
+ * A buffered implementation of HttpServletRequest.
+ */
 public class BufferedRequest implements HttpServletRequest {
 
     private final HttpServletRequest request;
@@ -47,7 +50,7 @@ public class BufferedRequest implements HttpServletRequest {
                 }
             };
 
-    public BufferedRequest(HttpServletRequest request) throws FalconWebException {
+    public BufferedRequest(HttpServletRequest request) {
         try {
             this.request = request;
             ByteArrayOutputStream copyBuffer = streams.get();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
index c637984..dfc02a1 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/InstanceManagerProxy.java
@@ -37,6 +37,9 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.util.*;
 
+/**
+ * A proxy implementation of the entity instance operations.
+ */
 @Path("instance")
 public class InstanceManagerProxy extends AbstractInstanceManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
index aa1ef6d..986291e 100644
--- a/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
+++ b/prism/src/main/java/org/apache/falcon/resource/proxy/SchedulableEntityManagerProxy.java
@@ -45,6 +45,9 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 
+/**
+ * A proxy implementation of the schedulable entity operations.
+ */
 @Path("entities")
 public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityManager {
     private static final String PRISM_TAG = "prism";
@@ -252,7 +255,7 @@ public class SchedulableEntityManagerProxy extends AbstractSchedulableEntityMana
     @Override
     public APIResult getStatus(@Dimension("entityType") @PathParam("type") final String type,
                                @Dimension("entityName") @PathParam("entity") final String entity,
-                               @Dimension("colo") @QueryParam("colo") final String coloExpr) throws FalconWebException {
+                               @Dimension("colo") @QueryParam("colo") final String coloExpr) {
         return new EntityProxy(type, entity) {
             @Override
             protected Set<String> getColosToApply() {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java b/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
index e3aedb1..f172e82 100644
--- a/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
+++ b/prism/src/main/java/org/apache/falcon/security/BasicAuthFilter.java
@@ -32,6 +32,9 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.UUID;
 
+/**
+ * This enforces authentication as part of the filter before processing the request.
+ */
 public class BasicAuthFilter implements Filter {
 
     private static final Logger LOG = Logger.getLogger(BasicAuthFilter.class);
@@ -41,27 +44,20 @@ public class BasicAuthFilter implements Filter {
     private static final Set<String> BLACK_LISTED_USER = new HashSet<String>(
             Arrays.asList(new String[]{"hdfs", "mapred", "oozie", "falcon"}));
 
-    private boolean secure;
-
-    private String colo;
+    private boolean isSecure;
 
     @Override
-    public void init(FilterConfig filterConfig)
-            throws ServletException {
-        String secure = StartupProperties.get().getProperty("security.enabled",
-                "true");
-        colo = StartupProperties.get().getProperty("current.colo", "default");
-        this.secure = Boolean.parseBoolean(secure);
+    public void init(FilterConfig filterConfig) throws ServletException {
+        String secure = StartupProperties.get().getProperty("security.enabled", "true");
+        this.isSecure = Boolean.parseBoolean(secure);
     }
 
     @Override
     public void doFilter(ServletRequest request,
                          ServletResponse response,
-                         FilterChain chain)
-            throws IOException, ServletException {
+                         FilterChain chain) throws IOException, ServletException {
 
-        if (!(request instanceof HttpServletRequest) ||
-                !(response instanceof HttpServletResponse)) {
+        if (!(request instanceof HttpServletRequest) || !(response instanceof HttpServletResponse)) {
             throw new IllegalStateException("Invalid request/response object");
         }
         HttpServletRequest httpRequest = (HttpServletRequest) request;
@@ -70,7 +66,7 @@ public class BasicAuthFilter implements Filter {
         String user;
         String requestId = UUID.randomUUID().toString();
 
-        if (!secure) {
+        if (!isSecure) {
             user = GUEST;
         } else {
             user = httpRequest.getHeader("Remote-User");
@@ -87,9 +83,8 @@ public class BasicAuthFilter implements Filter {
             try {
                 NDC.push(user + ":" + httpRequest.getMethod() + "/" + httpRequest.getPathInfo());
                 NDC.push(requestId);
-                LOG.info("Request from user: " + user + ", path=" +
-                        httpRequest.getPathInfo() + ", query=" +
-                        httpRequest.getQueryString());
+                LOG.info("Request from user: " + user + ", path=" + httpRequest.getPathInfo()
+                        + ", query=" + httpRequest.getQueryString());
                 chain.doFilter(request, response);
             } finally {
                 NDC.pop();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java b/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
index 4e48488..e20f091 100644
--- a/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
+++ b/prism/src/main/java/org/apache/falcon/service/FalconTopicSubscriber.java
@@ -34,9 +34,11 @@ import javax.jms.*;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Date;
 
+/**
+ * Subscribes to the falcon topic for handling retries and alerts.
+ */
 public class FalconTopicSubscriber implements MessageListener, ExceptionListener {
-    private static final Logger LOG = Logger
-            .getLogger(FalconTopicSubscriber.class);
+    private static final Logger LOG = Logger.getLogger(FalconTopicSubscriber.class);
 
     private TopicSubscriber subscriber;
     private String implementation;
@@ -46,10 +48,8 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
     private String topicName;
     private Connection connection;
 
-    private AbstractRerunHandler retryHandler = RerunHandlerFactory
-            .getRerunHandler(RerunType.RETRY);
-    private AbstractRerunHandler latedataHandler = RerunHandlerFactory
-            .getRerunHandler(RerunType.LATE);
+    private AbstractRerunHandler retryHandler = RerunHandlerFactory.getRerunHandler(RerunType.RETRY);
+    private AbstractRerunHandler latedataHandler = RerunHandlerFactory.getRerunHandler(RerunType.LATE);
 
     public FalconTopicSubscriber(String implementation, String userName,
                                  String password, String url, String topicName) {
@@ -140,7 +140,7 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
     }
 
     private SLAMonitoringService getSLAMonitoringService() {
-        return (SLAMonitoringService) Services.get().getService(SLAMonitoringService.SERVICE_NAME);
+        return Services.get().getService(SLAMonitoringService.SERVICE_NAME);
     }
 
     private void debug(MapMessage mapMessage) throws JMSException {
@@ -159,9 +159,7 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
 
     @Override
     public void onException(JMSException ignore) {
-        LOG.info(
-                "Error in onException for subscriber of topic: "
-                        + this.toString(), ignore);
+        LOG.info("Error in onException for subscriber of topic: " + this.toString(), ignore);
     }
 
     public void closeSubscriber() throws FalconException {
@@ -177,14 +175,9 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
     }
 
     private static Connection createAndGetConnection(String implementation,
-                                                     String userName, String password, String url) throws JMSException,
-                                                                                                          ClassNotFoundException,
-                                                                                                          IllegalArgumentException,
-                                                                                                          SecurityException,
-                                                                                                          InstantiationException,
-                                                                                                          IllegalAccessException,
-                                                                                                          InvocationTargetException,
-                                                                                                          NoSuchMethodException {
+                                                     String userName, String password, String url)
+        throws JMSException, ClassNotFoundException, InstantiationException,
+            IllegalAccessException, InvocationTargetException, NoSuchMethodException {
 
         @SuppressWarnings("unchecked")
         Class<ConnectionFactory> clazz = (Class<ConnectionFactory>) FalconTopicSubscriber.class
@@ -201,5 +194,4 @@ public class FalconTopicSubscriber implements MessageListener, ExceptionListener
     public String toString() {
         return topicName;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java b/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
index 9d5e78f..63f5a3c 100644
--- a/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
+++ b/prism/src/main/java/org/apache/falcon/service/ProcessSubscriberService.java
@@ -20,11 +20,14 @@ package org.apache.falcon.service;
 import org.apache.falcon.FalconException;
 import org.apache.falcon.util.StartupProperties;
 
+/**
+ * An Ivory Service that initializes and starts a topic subscriber.
+ */
 public class ProcessSubscriberService implements FalconService {
 
     private FalconTopicSubscriber subscriber;
 
-    private static enum JMSprops {
+    private static enum JMSProps {
         FalconBrokerImplClass("broker.impl.class", "org.apache.activemq.ActiveMQConnectionFactory"),
         FalconBrokerUrl("broker.url", "tcp://localhost:61616?daemon=true"),
         FalconEntityTopic("entity.topic", "FALCON.ENTITY.TOPIC");
@@ -32,11 +35,10 @@ public class ProcessSubscriberService implements FalconService {
         private String propName;
         private String defaultPropValue;
 
-        private JMSprops(String propName, String defaultPropValue) {
+        private JMSProps(String propName, String defaultPropValue) {
             this.propName = propName;
             this.defaultPropValue = defaultPropValue;
         }
-
     }
 
     @Override
@@ -46,18 +48,16 @@ public class ProcessSubscriberService implements FalconService {
 
     @Override
     public void init() throws FalconException {
-        String falconBrokerImplClass = getPropertyValue(JMSprops.FalconBrokerImplClass);
-        String falconBrokerUrl = getPropertyValue(JMSprops.FalconBrokerUrl);
-        String falconEntityTopic = getPropertyValue(JMSprops.FalconEntityTopic);
+        String falconBrokerImplClass = getPropertyValue(JMSProps.FalconBrokerImplClass);
+        String falconBrokerUrl = getPropertyValue(JMSProps.FalconBrokerUrl);
+        String falconEntityTopic = getPropertyValue(JMSProps.FalconEntityTopic);
 
-        subscriber = new FalconTopicSubscriber(falconBrokerImplClass, "", "",
-                falconBrokerUrl, falconEntityTopic);
+        subscriber = new FalconTopicSubscriber(falconBrokerImplClass, "", "", falconBrokerUrl, falconEntityTopic);
         subscriber.startSubscriber();
     }
 
-    private String getPropertyValue(JMSprops prop) {
-        return StartupProperties.get().getProperty(prop.propName,
-                prop.defaultPropValue);
+    private String getPropertyValue(JMSProps prop) {
+        return StartupProperties.get().getProperty(prop.propName, prop.defaultPropValue);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java b/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
index da7887e..947edd8 100644
--- a/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
+++ b/prism/src/main/java/org/apache/falcon/service/SLAMonitoringService.java
@@ -35,15 +35,17 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+/**
+ * A service implementation for SLA Monitoring.
+ */
 public class SLAMonitoringService implements FalconService, WorkflowEngineActionListener {
     private static final Logger LOG = Logger.getLogger(SLAMonitoringService.class);
     public static final String SERVICE_NAME = "SLAMonitor";
 
-    private ConcurrentMap<String, Long> monitoredEntities =
-            new ConcurrentHashMap<String, Long>();
+    private ConcurrentMap<String, Long> monitoredEntities = new ConcurrentHashMap<String, Long>();
 
-    private ConcurrentMap<String, ConcurrentMap<Date, Date>> pendingJobs =
-            new ConcurrentHashMap<String, ConcurrentMap<Date, Date>>();
+    private ConcurrentMap<String, ConcurrentMap<Date, Date>> pendingJobs
+        = new ConcurrentHashMap<String, ConcurrentMap<Date, Date>>();
 
     private static final long INITIAL_LATENCY_SECS = 12 * 3600;
 
@@ -118,8 +120,8 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
     private void removeFromPendingList(Entity entity, String cluster, Date nominalTime) {
         ConcurrentMap<Date, Date> pendingInstances = pendingJobs.get(getKey(entity, cluster));
         if (pendingInstances != null) {
-            LOG.debug("Removing from pending jobs: " + getKey(entity, cluster) + " ---> " +
-                    SchemaHelper.formatDateUTC(nominalTime));
+            LOG.debug("Removing from pending jobs: " + getKey(entity, cluster) + " ---> "
+                    + SchemaHelper.formatDateUTC(nominalTime));
             pendingInstances.remove(nominalTime);
         }
     }
@@ -155,8 +157,8 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
                         String cluster = getCluster(key);
                         GenericAlert.alertOnLikelySLAMiss(cluster, entity.getEntityType().name(),
                                 entity.getName(), SchemaHelper.formatDateUTC(entry.getKey()));
-                        LOG.debug("Removing from pending jobs: " + key + " ---> " +
-                                SchemaHelper.formatDateUTC(entry.getKey()));
+                        LOG.debug("Removing from pending jobs: " + key + " ---> "
+                                + SchemaHelper.formatDateUTC(entry.getKey()));
                         pendingInstances.remove(entry.getKey());
                     }
                 }
@@ -191,10 +193,9 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
                     if (latency == null) {
                         break;
                     }
-                    pendingInstances.putIfAbsent(nextStart, new Date(nextStart.getTime() +
-                            latency * 1500));  //1.5 times latency is when it is supposed to have breached
-                    LOG.debug("Adding to pending jobs: " + key + " ---> " +
-                            SchemaHelper.formatDateUTC(nextStart));
+                    // 1.5 times latency is when it is supposed to have breached
+                    pendingInstances.putIfAbsent(nextStart, new Date(nextStart.getTime() + latency * 1500));
+                    LOG.debug("Adding to pending jobs: " + key + " ---> " + SchemaHelper.formatDateUTC(nextStart));
                     Calendar startCal = Calendar.getInstance(timeZone);
                     startCal.setTime(nextStart);
                     startCal.add(frequency.getTimeUnit().getCalendarUnit(), frequency.getFrequency());
@@ -204,17 +205,17 @@ public class SLAMonitoringService implements FalconService, WorkflowEngineAction
         }
     }
 
-    private static final Pattern regex = Pattern.compile("[()\\s/]");
+    private static final Pattern PATTERN = Pattern.compile("[()\\s/]");
 
     private Entity getEntity(String key) throws FalconException {
-        String[] parts = regex.split(key);
+        String[] parts = PATTERN.split(key);
         String name = parts[3];
         String type = parts[1];
         return EntityUtil.getEntity(type, name);
     }
 
     private String getCluster(String key) throws FalconException {
-        String[] parts = regex.split(key);
+        String[] parts = PATTERN.split(key);
         return parts[4];
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
----------------------------------------------------------------------
diff --git a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
index 6baeb13..80c6d80 100644
--- a/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
+++ b/prism/src/main/java/org/apache/falcon/util/EmbeddedServer.java
@@ -24,6 +24,9 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.bio.SocketConnector;
 import org.mortbay.jetty.webapp.WebAppContext;
 
+/**
+ * This class embeds a Jetty server and a connector.
+ */
 public class EmbeddedServer {
     private Server server = new Server();
     private Connector connector = new SocketConnector();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java b/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
index 979c349..1db71fd 100644
--- a/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
+++ b/prism/src/test/java/org/apache/falcon/aspect/GenericAlertTest.java
@@ -20,6 +20,9 @@ package org.apache.falcon.aspect;
 import org.aspectj.lang.annotation.Aspect;
 import org.testng.annotations.Test;
 
+/**
+ * Test class for Alerts.
+ */
 @Aspect
 public class GenericAlertTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java b/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
index 2e04904..3b5bd87 100644
--- a/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
+++ b/prism/src/test/java/org/apache/falcon/aspect/LoggingAspectTest.java
@@ -20,25 +20,24 @@ package org.apache.falcon.aspect;
 
 import junit.framework.Assert;
 import org.apache.falcon.FalconWebException;
-import org.apache.falcon.resource.APIResult;
 import org.apache.falcon.resource.AbstractEntityManager;
 import org.testng.annotations.Test;
 
 import java.util.ArrayList;
 import java.util.List;
 
+/**
+ * Test class for LoggingAspect.
+ */
 public class LoggingAspectTest {
 
-
-    private AbstractEntityManager em = new AbstractEntityManager() {
-    };
+    private AbstractEntityManager em = new AbstractEntityManager() {};
 
     private volatile Exception threadException;
 
     @Test(expectedExceptions = FalconWebException.class)
     public void testBeanLoading() {
-
-        APIResult result = em.getStatus("type", "entity", "colo");
+        em.getStatus("type", "entity", "colo");
     }
 
     @Test
@@ -79,5 +78,4 @@ public class LoggingAspectTest {
             throw threadException;
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
index 24e5a8a..73cf493 100644
--- a/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
+++ b/prism/src/test/java/org/apache/falcon/resource/EntityManagerTest.java
@@ -60,8 +60,7 @@ public class EntityManagerTest extends AbstractEntityManager {
     }
 
     /**
-     * Run this testcase for different types of VALID entity xmls like process,
-     * feed, dataEndPoint
+     * Run this testcase for different types of VALID entity xmls like process, feed, dataEndPoint.
      *
      * @param stream
      * @throws IOException
@@ -84,6 +83,7 @@ public class EntityManagerTest extends AbstractEntityManager {
                     EntityType.PROCESS.name());
             Assert.fail("Invalid entity type was accepted by the system");
         } catch (FalconWebException ignore) {
+            // ignore
         }
     }
 
@@ -98,19 +98,18 @@ public class EntityManagerTest extends AbstractEntityManager {
                     "InvalidEntityType");
             Assert.fail("Invalid entity type was accepted by the system");
         } catch (FalconWebException ignore) {
+            // ignore
         }
     }
 
     /**
-     * Converts a InputStream into ServletInputStream
+     * Converts a InputStream into ServletInputStream.
      *
      * @param resourceName
      * @return ServletInputStream
      */
     private ServletInputStream getServletInputStream(String resourceName) {
-        final InputStream stream = this.getClass().getResourceAsStream(
-                resourceName);
-
+        final InputStream stream = this.getClass().getResourceAsStream(resourceName);
         return new ServletInputStream() {
 
             @Override
@@ -119,5 +118,4 @@ public class EntityManagerTest extends AbstractEntityManager {
             }
         };
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/f8b91222/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
----------------------------------------------------------------------
diff --git a/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java b/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
index 50122fe..e390aad 100644
--- a/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
+++ b/prism/src/test/java/org/apache/falcon/service/FalconTopicSubscriberTest.java
@@ -29,6 +29,9 @@ import org.testng.annotations.Test;
 
 import javax.jms.*;
 
+/**
+ * Test for FalconTopicSubscriber.
+ */
 public class FalconTopicSubscriberTest {
 
     private static final String BROKER_URL = "vm://localhost?broker.useJmx=false&broker.persistent=true";
@@ -73,7 +76,6 @@ public class FalconTopicSubscriberTest {
         message.getKeyValueMap().put(ARG.status, "FAILED");
         TextMessage textMessage = session.createTextMessage(message.toString());
         producer.send(textMessage);
-
     }
 
     private EntityInstanceMessage getMockFalconMessage(int i) {
@@ -104,12 +106,10 @@ public class FalconTopicSubscriberTest {
         subscriber1.startSubscriber();
         sendMessages();
         subscriber1.closeSubscriber();
-
     }
 
     @AfterClass
     public void tearDown() throws Exception {
         broker.stop();
     }
-
 }


[38/47] git commit: Adding back java-test.jar

Posted by sr...@apache.org.
Adding back java-test.jar


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

Branch: refs/heads/master
Commit: 5c3dd0e40eb15ce230f304d2f6e1c8f05a15a815
Parents: f739a7e
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Fri Apr 19 18:27:35 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Fri Apr 19 18:27:35 2013 +0530

----------------------------------------------------------------------
 .../org/apache/falcon/logging/java-test.jar        |  Bin 0 -> 15728 bytes
 1 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/5c3dd0e4/webapp/src/test/resources/org/apache/falcon/logging/java-test.jar
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/org/apache/falcon/logging/java-test.jar b/webapp/src/test/resources/org/apache/falcon/logging/java-test.jar
new file mode 100644
index 0000000..d743543
Binary files /dev/null and b/webapp/src/test/resources/org/apache/falcon/logging/java-test.jar differ


[46/47] git commit: checkstyle related fixes for test-util module.

Posted by sr...@apache.org.
checkstyle related fixes for test-util module.


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

Branch: refs/heads/master
Commit: e9979317f2a7ecddd7079baf3f1a7cfa9b7d29f5
Parents: 4e39c7a
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:27:55 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:27:55 2013 -0700

----------------------------------------------------------------------
 .../falcon/cluster/util/EmbeddedCluster.java       |   16 ++++++--------
 .../falcon/cluster/util/StandAloneCluster.java     |    7 +++--
 2 files changed, 11 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e9979317/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 41f2d95..22a3191 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
@@ -33,9 +33,12 @@ 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.
+ */
 public class EmbeddedCluster {
 
-    private static Logger LOG = Logger.getLogger(EmbeddedCluster.class);
+    private static final Logger LOG = Logger.getLogger(EmbeddedCluster.class);
 
     protected EmbeddedCluster() {
     }
@@ -49,16 +52,13 @@ public class EmbeddedCluster {
         return conf;
     }
 
-    public static EmbeddedCluster newCluster(final String name, final boolean withMR)
-            throws Exception {
+    public static EmbeddedCluster newCluster(final String name, final boolean withMR) throws Exception {
         return createClusterAsUser(name, withMR);
     }
 
     public static EmbeddedCluster newCluster(final String name,
                                              final boolean withMR,
-                                             final String user)
-            throws Exception {
-
+                                             final String user) throws Exception {
         UserGroupInformation hdfsUser = UserGroupInformation.createRemoteUser(user);
         return hdfsUser.doAs(new PrivilegedExceptionAction<EmbeddedCluster>() {
             @Override
@@ -69,9 +69,7 @@ public class EmbeddedCluster {
     }
 
     private static EmbeddedCluster createClusterAsUser(String name,
-                                                       boolean withMR)
-            throws IOException {
-
+                                                       boolean withMR) throws IOException {
         EmbeddedCluster cluster = new EmbeddedCluster();
         File target = new File("webapp/target");
         if (!target.exists()) {

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/e9979317/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
----------------------------------------------------------------------
diff --git a/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java b/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
index baf6347..6864651 100644
--- a/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
+++ b/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
@@ -26,8 +26,10 @@ import org.apache.log4j.Logger;
 
 import java.io.File;
 
-
-public class StandAloneCluster extends EmbeddedCluster {
+/**
+ * A standalone cluster.
+ */
+public final class StandAloneCluster extends EmbeddedCluster {
     private static final Logger LOG = Logger.getLogger(StandAloneCluster.class);
 
     private StandAloneCluster() {
@@ -50,6 +52,5 @@ public class StandAloneCluster extends EmbeddedCluster {
 
     @Override
     public void shutdown() {
-
     }
 }


[07/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 fe329b5..66e99a6 100644
--- a/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
+++ b/retention/src/test/java/org/apache/falcon/retention/FeedEvictorTest.java
@@ -18,12 +18,12 @@
 
 package org.apache.falcon.retention;
 
+import org.apache.falcon.Pair;
+import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.falcon.Pair;
-import org.apache.falcon.cluster.util.EmbeddedCluster;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -37,380 +37,388 @@ import java.util.concurrent.TimeUnit;
 
 public class FeedEvictorTest {
 
-	private EmbeddedCluster cluster;
-	private InMemoryWriter stream = new InMemoryWriter(System.out);
-	private Map<String, String> map = new HashMap<String, String>();
-
-	@BeforeClass
-	public void start() throws Exception{
-		cluster = EmbeddedCluster.newCluster("test", false);
-		FeedEvictor.stream = stream;
-	}
-
-	@AfterClass
-	public void close() throws Exception {
-		cluster.shutdown();
-	}
-
-	@Test
-	public void testBadArgs() throws Exception {
-		try {
-			FeedEvictor.main(null);
-			Assert.fail("Expected an exception to be thrown");
-		} catch (Exception ignore) { }
-
-		try {
-			FeedEvictor.main(new String[] {"1","2"});
-			Assert.fail("Expected an exception to be thrown");
-		} catch (Exception ignore) { }
-	}
-
-	@Test
-	public void testEviction1() throws Exception {
-		try {
-			FeedEvictor.main(new String[]{"1", "2", "3", "4", "5","6","7"});
-		} catch (Exception e) {
-		}
-	}
-
-	@Test
-	public void testEviction2() throws Exception {
-		try {
-			Configuration conf = cluster.getConf();
-			FileSystem fs = FileSystem.get(conf);
-			fs.delete(new Path("/"), true);
-			stream.clear();
-
-			Pair<List<String>, List<String>>  pair;
-			pair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS,"/data");
-			String dataPath = "/data/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}";
-			String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
-
-			FeedEvictor.main(new String[] {
-					"-feedBasePath",cluster.getConf().get("fs.default.name")
-					+ dataPath,
-					"-retentionType","instance", "-retentionLimit","days(10)", "-timeZone","UTC", "-frequency","daily",
-					"-logFile",logFile});
-
-			assertFailures(fs, pair);
-			compare(map.get("feed1"), stream.getBuffer());
-
-			Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
-
-
-		} catch (Exception e) {
-			Assert.fail("Unknown exception", e);
-		}
-	}
-
-	private String getExpectedInstancePaths(String dataPath){
-		StringBuffer newBuffer = new StringBuffer("instancePaths=");
-		DateFormat format = new SimpleDateFormat("yyyyMMddHHmm");
-		format.setTimeZone(TimeZone.getTimeZone("UTC"));
-		String[] locs = dataPath.split("#");
-		String [] instances = stream.getBuffer().split("instances=")[1].split(",");
-		if(instances[0].equals("NULL")){
-			return "instancePaths=";
-		}
-
-		for(int i=0;i<locs.length;i++){
-			for (int j=0, k=i*instances.length/locs.length;j<instances.length/locs.length;j++) {
-				String instancePath = locs[i].replaceAll("\\?\\{YEAR\\}", instances[j+k].substring(0,4));
-				instancePath = instancePath.replaceAll("\\?\\{MONTH\\}", instances[j+k].substring(4,6));
-				instancePath = instancePath.replaceAll("\\?\\{DAY\\}", instances[j+k].substring(6,8));
-				instancePath = instancePath.replaceAll("\\?\\{HOUR\\}", instances[j+k].substring(8,10));
-				instancePath = instancePath.replaceAll("\\?\\{MINUTE\\}", instances[j+k].substring(10,12));
-				newBuffer.append(instancePath).append(',');
-			}
-		}
-		return newBuffer.toString();
-	}
-
-	private String readLogFile(Path logFile) throws IOException {
-		Configuration conf = cluster.getConf();
-		FileSystem fs = FileSystem.get(conf);
-		ByteArrayOutputStream writer = new ByteArrayOutputStream();
-		InputStream date = fs.open(logFile);
-		IOUtils.copyBytes(date, writer, 4096, true);
-		return writer.toString();
-	}
-
-	private void compare(String str1, String str2) {
-		String[] instances1 = str1.split("=")[1].split(",");
-		String[] instances2 = str2.split("instances=")[1].split(",");
-
-		Arrays.sort(instances1);
-		Arrays.sort(instances2);
-		Assert.assertEquals(instances1, instances2);
-	}
-
-	private void assertFailures(FileSystem fs, Pair<List<String>, List<String>> pair) throws IOException {
-		for (String path : pair.second) {
-			if (!fs.exists(new Path(path))) {
-				Assert.fail("Expecting " + path + " to be present");
-			}
-		}
-		for (String path : pair.first) {
-			if (fs.exists(new Path(path))) {
-				Assert.fail("Expecting " + path + " to be deleted");
-			}
-		}
-	}
-
-	@Test
-	public void testEviction3() throws Exception {
-		try {
-			Configuration conf = cluster.getConf();
-			FileSystem fs = FileSystem.get(conf);
-			fs.delete(new Path("/"), true);
-			stream.clear();
-
-			Pair<List<String>, List<String>>  pair;
-			pair = createTestData("feed2", "yyyyMMddHH/'more'/yyyy", 5, TimeUnit.HOURS,"/data");
-			String dataPath = "/data/YYYY/feed2/mmHH/dd/MM/" +
-					"?{YEAR}?{MONTH}?{DAY}?{HOUR}/more/?{YEAR}";
-			String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
-			FeedEvictor.main(new String[] {
-					"-feedBasePath",cluster.getConf().get("fs.default.name")
-					+ dataPath,
-					"-retentionType","instance", "-retentionLimit","hours(5)", "-timeZone","UTC", "-frequency","hourly",
-					"-logFile",logFile});
-			assertFailures(fs, pair);
-
-			compare(map.get("feed2"), stream.getBuffer());
-
-			Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
-
-		} catch (Exception e) {
-			Assert.fail("Unknown exception", e);
-		}
-	}
-
-
-	@Test
-	public void testEviction4() throws Exception {
-		try {
-			Configuration conf = cluster.getConf();
-			FileSystem fs = FileSystem.get(conf);
-			fs.delete(new Path("/"), true);
-			stream.clear();
-
-			Pair<List<String>, List<String>>  pair;
-			pair = createTestData("/data");
-			FeedEvictor.main(new String[] {
-					"-feedBasePath",
-					cluster.getConf().get("fs.default.name")
-							+ "/data/YYYY/feed3/dd/MM/"
-							+ "?{MONTH}/more/?{HOUR}", "-retentionType",
-					"instance", "-retentionLimit", "months(5)", "-timeZone",
-					"UTC", "-frequency", "hourly", "-logFile",
-					"/falcon/staging/feed/2012-01-01-04-00" });
-			Assert.assertEquals("instances=NULL", stream.getBuffer());
-
-			stream.clear();
-			String dataPath="/data/YYYY/feed4/dd/MM/" +
-					"02/more/hello";
-			String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
-			FeedEvictor.main(new String[] { "-feedBasePath",
-					cluster.getConf().get("fs.default.name") + dataPath,
-					"-retentionType", "instance", "-retentionLimit",
-					"hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
-					"-logFile", logFile });
-			Assert.assertEquals("instances=NULL", stream.getBuffer());     
-			
-			Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
-
-			assertFailures(fs, pair);
-		} catch (Exception e) {
-			Assert.fail("Unknown exception", e);
-		}
-	}
-	
-	@Test
-	public void testEviction5() throws Exception {
-		try {
-			Configuration conf = cluster.getConf();
-			FileSystem fs = FileSystem.get(conf);
-			fs.delete(new Path("/"), true);
-			stream.clear();
-
-			Pair<List<String>, List<String>>  pair,statsPair,metaPair,tmpPair;
-			pair = createTestData("/data");
-			statsPair = createTestData("/stats");
-			metaPair = createTestData("/meta");
-			tmpPair = createTestData("/tmp");
-			FeedEvictor.main(new String[] {
-					"-feedBasePath",
-					getFeedBasePath(cluster, "/data") + "#"
-							+ getFeedBasePath(cluster, "/stats") + "#"
-							+ getFeedBasePath(cluster, "/meta") + "#"
-							+ getFeedBasePath(cluster, "/tmp"),
-					"-retentionType", "instance", "-retentionLimit",
-					"months(5)", "-timeZone", "UTC", "-frequency", "hourly",
-					"-logFile", "/falcon/staging/feed/2012-01-01-04-00" });
-			Assert.assertEquals("instances=NULL", stream.getBuffer());
-
-			stream.clear();
-			String dataPath="/data/YYYY/feed4/dd/MM/" +
-					"02/more/hello";
-			String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
-			FeedEvictor.main(new String[] { "-feedBasePath",
-					cluster.getConf().get("fs.default.name") + dataPath,
-					"-retentionType", "instance", "-retentionLimit",
-					"hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
-					"-logFile", logFile });
-			Assert.assertEquals("instances=NULL", stream.getBuffer());     
-			
-			Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
-
-			assertFailures(fs, pair);
-		} catch (Exception e) {
-			Assert.fail("Unknown exception", e);
-		}
-	}
-	
-	@Test
-	public void testEviction6() throws Exception {
-		try {
-			Configuration conf = cluster.getConf();
-			FileSystem fs = FileSystem.get(conf);
-			fs.delete(new Path("/"), true);
-			stream.clear();
-
-			Pair<List<String>, List<String>>  pair, statsPair,metaPair;
-			pair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS,"/data");
-			statsPair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS,"/stats");
-			metaPair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS,"/meta");
-			String dataPath = cluster.getConf().get("fs.default.name")+"/data/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}"
-					+ "#"
-					+ cluster.getConf().get("fs.default.name")+"/stats/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}"
-					+ "#"
-					+ cluster.getConf().get("fs.default.name")+"/meta/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}";
-			String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
-
-			FeedEvictor.main(new String[] {
-					"-feedBasePath",
-					dataPath,
-					"-retentionType","instance", "-retentionLimit","days(10)", "-timeZone","UTC", "-frequency","daily",
-					"-logFile",logFile});
-
-			assertFailures(fs, pair);
-
-			Assert.assertEquals(readLogFile(new Path(logFile)),
-					getExpectedInstancePaths(dataPath.replaceAll(cluster
-							.getConf().get("fs.default.name"), "")));
-
-
-		} catch (Exception e) {
-			Assert.fail("Unknown exception", e);
-		}
-	}
-
-	private Pair<List<String>, List<String>> createTestData(String locationType) throws Exception {
-		Configuration conf = cluster.getConf();
-		FileSystem fs = FileSystem.get(conf);
-
-		List<String> outOfRange = new ArrayList<String>();
-		List<String> inRange = new ArrayList<String>();
-
-		touch(fs, locationType+"/YYYY/feed3/dd/MM/02/more/hello");
-		touch(fs, locationType+"/YYYY/feed4/dd/MM/02/more/hello");
-		touch(fs, locationType+"/YYYY/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
-		touch(fs, locationType+"/somedir/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
-		outOfRange.add(locationType+"/YYYY/feed3/dd/MM/02/more/hello");
-		outOfRange.add(locationType+"/YYYY/feed4/dd/MM/02/more/hello");
-		outOfRange.add(locationType+"/YYYY/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
-		outOfRange.add(locationType+"/somedir/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
-
-		return Pair.of(inRange, outOfRange);
-	}
-
-	private Pair<List<String>, List<String>> createTestData(String feed,
-			String mask,
-			int period,
-			TimeUnit timeUnit, String locationType)
-					throws Exception {
-
-		Configuration conf = cluster.getConf();
-		FileSystem fs = FileSystem.get(conf);
-
-		List<String> outOfRange = new ArrayList<String>();
-		List<String> inRange = new ArrayList<String>();
-
-		Pair<List<String>, List<String>> pair = createTestData(locationType);
-		outOfRange.addAll(pair.second);
-		inRange.addAll(pair.first);
-
-		pair = generateInstances(fs, feed, mask, period, timeUnit,locationType);
-		outOfRange.addAll(pair.second);
-		inRange.addAll(pair.first);
-		return Pair.of(inRange,  outOfRange);
-	}
-
-	private Pair<List<String>, List<String>> generateInstances(
-			FileSystem fs, String feed, String formatString,
-			int range, TimeUnit timeUnit, String locationType) throws Exception {
-
-		List<String> outOfRange = new ArrayList<String>();
-		List<String> inRange = new ArrayList<String>();
-
-		DateFormat format = new SimpleDateFormat(formatString);
-		format.setTimeZone(TimeZone.getTimeZone("UTC"));
-		long now = System.currentTimeMillis();
-
-		DateFormat displayFormat = new
-				SimpleDateFormat(timeUnit == TimeUnit.HOURS ? "yyyyMMddHH" : "yyyyMMdd");
-		displayFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
-
-		StringBuffer buffer = new StringBuffer();
-		for (long date = now;
-				date > now - timeUnit.toMillis(range + 6);
-				date -= timeUnit.toMillis(1)) {
-			String path = locationType+"/YYYY/" + feed + "/mmHH/dd/MM/" +
-					format.format(date);
-			touch(fs, path);
-			if (date <= now && date > now - timeUnit.toMillis(range)) {
-				outOfRange.add(path);
-			} else {
-				inRange.add(path);
-				buffer.append((displayFormat.format(date) + "0000").
-						substring(0, 12)).append(',');
-			}
-		}
-		map.put(feed, "instances=" + buffer.substring(0, buffer.length() -1));
-		return Pair.of(inRange, outOfRange);
-	}
-
-	private void touch(FileSystem fs, String path) throws Exception {
-		fs.create(new Path(path)).close();
-	}
-	
-	private String getFeedBasePath(EmbeddedCluster cluster, String locationType){
-		return cluster.getConf().get("fs.default.name")
-		+ "/data/YYYY/feed3/dd/MM/"
-		+ "?{MONTH}/more/?{HOUR}";
-	}
-
-	private static class InMemoryWriter extends PrintStream {
-
-		private StringBuffer buffer = new StringBuffer();
-
-		public InMemoryWriter(OutputStream out) {
-			super(out);
-		}
-
-		@Override
-		public void println(String x) {
-			buffer.append(x);
-			super.println(x);
-		}
-
-		public String getBuffer() {
-			return buffer.toString();
-		}
-
-		public void clear() {
-			buffer.delete(0, buffer.length());
-		}
-	}
+    private EmbeddedCluster cluster;
+    private InMemoryWriter stream = new InMemoryWriter(System.out);
+    private Map<String, String> map = new HashMap<String, String>();
+
+    @BeforeClass
+    public void start() throws Exception {
+        cluster = EmbeddedCluster.newCluster("test", false);
+        FeedEvictor.stream = stream;
+    }
+
+    @AfterClass
+    public void close() throws Exception {
+        cluster.shutdown();
+    }
+
+    @Test
+    public void testBadArgs() throws Exception {
+        try {
+            FeedEvictor.main(null);
+            Assert.fail("Expected an exception to be thrown");
+        } catch (Exception ignore) {
+        }
+
+        try {
+            FeedEvictor.main(new String[]{"1", "2"});
+            Assert.fail("Expected an exception to be thrown");
+        } catch (Exception ignore) {
+        }
+    }
+
+    @Test
+    public void testEviction1() throws Exception {
+        try {
+            FeedEvictor.main(new String[]{"1", "2", "3", "4", "5", "6", "7"});
+        } catch (Exception e) {
+        }
+    }
+
+    @Test
+    public void testEviction2() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair;
+            pair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS, "/data");
+            String dataPath = "/data/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}";
+            String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
+
+            FeedEvictor.main(new String[]{
+                    "-feedBasePath", cluster.getConf().get("fs.default.name")
+                    + dataPath,
+                    "-retentionType", "instance", "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency",
+                    "daily",
+                    "-logFile", logFile});
+
+            assertFailures(fs, pair);
+            compare(map.get("feed1"), stream.getBuffer());
+
+            Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
+
+
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
+    private String getExpectedInstancePaths(String dataPath) {
+        StringBuffer newBuffer = new StringBuffer("instancePaths=");
+        DateFormat format = new SimpleDateFormat("yyyyMMddHHmm");
+        format.setTimeZone(TimeZone.getTimeZone("UTC"));
+        String[] locs = dataPath.split("#");
+        String[] instances = stream.getBuffer().split("instances=")[1].split(",");
+        if (instances[0].equals("NULL")) {
+            return "instancePaths=";
+        }
+
+        for (int i = 0; i < locs.length; i++) {
+            for (int j = 0, k = i * instances.length / locs.length; j < instances.length / locs.length; j++) {
+                String instancePath = locs[i].replaceAll("\\?\\{YEAR\\}", instances[j + k].substring(0, 4));
+                instancePath = instancePath.replaceAll("\\?\\{MONTH\\}", instances[j + k].substring(4, 6));
+                instancePath = instancePath.replaceAll("\\?\\{DAY\\}", instances[j + k].substring(6, 8));
+                instancePath = instancePath.replaceAll("\\?\\{HOUR\\}", instances[j + k].substring(8, 10));
+                instancePath = instancePath.replaceAll("\\?\\{MINUTE\\}", instances[j + k].substring(10, 12));
+                newBuffer.append(instancePath).append(',');
+            }
+        }
+        return newBuffer.toString();
+    }
+
+    private String readLogFile(Path logFile) throws IOException {
+        Configuration conf = cluster.getConf();
+        FileSystem fs = FileSystem.get(conf);
+        ByteArrayOutputStream writer = new ByteArrayOutputStream();
+        InputStream date = fs.open(logFile);
+        IOUtils.copyBytes(date, writer, 4096, true);
+        return writer.toString();
+    }
+
+    private void compare(String str1, String str2) {
+        String[] instances1 = str1.split("=")[1].split(",");
+        String[] instances2 = str2.split("instances=")[1].split(",");
+
+        Arrays.sort(instances1);
+        Arrays.sort(instances2);
+        Assert.assertEquals(instances1, instances2);
+    }
+
+    private void assertFailures(FileSystem fs, Pair<List<String>, List<String>> pair) throws IOException {
+        for (String path : pair.second) {
+            if (!fs.exists(new Path(path))) {
+                Assert.fail("Expecting " + path + " to be present");
+            }
+        }
+        for (String path : pair.first) {
+            if (fs.exists(new Path(path))) {
+                Assert.fail("Expecting " + path + " to be deleted");
+            }
+        }
+    }
+
+    @Test
+    public void testEviction3() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair;
+            pair = createTestData("feed2", "yyyyMMddHH/'more'/yyyy", 5, TimeUnit.HOURS, "/data");
+            String dataPath = "/data/YYYY/feed2/mmHH/dd/MM/" +
+                    "?{YEAR}?{MONTH}?{DAY}?{HOUR}/more/?{YEAR}";
+            String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
+            FeedEvictor.main(new String[]{
+                    "-feedBasePath", cluster.getConf().get("fs.default.name")
+                    + dataPath,
+                    "-retentionType", "instance", "-retentionLimit", "hours(5)", "-timeZone", "UTC", "-frequency",
+                    "hourly",
+                    "-logFile", logFile});
+            assertFailures(fs, pair);
+
+            compare(map.get("feed2"), stream.getBuffer());
+
+            Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
+
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
+
+    @Test
+    public void testEviction4() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair;
+            pair = createTestData("/data");
+            FeedEvictor.main(new String[]{
+                    "-feedBasePath",
+                    cluster.getConf().get("fs.default.name")
+                            + "/data/YYYY/feed3/dd/MM/"
+                            + "?{MONTH}/more/?{HOUR}", "-retentionType",
+                    "instance", "-retentionLimit", "months(5)", "-timeZone",
+                    "UTC", "-frequency", "hourly", "-logFile",
+                    "/falcon/staging/feed/2012-01-01-04-00"});
+            Assert.assertEquals("instances=NULL", stream.getBuffer());
+
+            stream.clear();
+            String dataPath = "/data/YYYY/feed4/dd/MM/" +
+                    "02/more/hello";
+            String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
+            FeedEvictor.main(new String[]{"-feedBasePath",
+                                          cluster.getConf().get("fs.default.name") + dataPath,
+                                          "-retentionType", "instance", "-retentionLimit",
+                                          "hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
+                                          "-logFile", logFile});
+            Assert.assertEquals("instances=NULL", stream.getBuffer());
+
+            Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
+
+            assertFailures(fs, pair);
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
+    @Test
+    public void testEviction5() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair, statsPair, metaPair, tmpPair;
+            pair = createTestData("/data");
+            statsPair = createTestData("/stats");
+            metaPair = createTestData("/meta");
+            tmpPair = createTestData("/tmp");
+            FeedEvictor.main(new String[]{
+                    "-feedBasePath",
+                    getFeedBasePath(cluster, "/data") + "#"
+                            + getFeedBasePath(cluster, "/stats") + "#"
+                            + getFeedBasePath(cluster, "/meta") + "#"
+                            + getFeedBasePath(cluster, "/tmp"),
+                    "-retentionType", "instance", "-retentionLimit",
+                    "months(5)", "-timeZone", "UTC", "-frequency", "hourly",
+                    "-logFile", "/falcon/staging/feed/2012-01-01-04-00"});
+            Assert.assertEquals("instances=NULL", stream.getBuffer());
+
+            stream.clear();
+            String dataPath = "/data/YYYY/feed4/dd/MM/" +
+                    "02/more/hello";
+            String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-02-00.csv";
+            FeedEvictor.main(new String[]{"-feedBasePath",
+                                          cluster.getConf().get("fs.default.name") + dataPath,
+                                          "-retentionType", "instance", "-retentionLimit",
+                                          "hours(5)", "-timeZone", "UTC", "-frequency", "hourly",
+                                          "-logFile", logFile});
+            Assert.assertEquals("instances=NULL", stream.getBuffer());
+
+            Assert.assertEquals(readLogFile(new Path(logFile)), getExpectedInstancePaths(dataPath));
+
+            assertFailures(fs, pair);
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
+    @Test
+    public void testEviction6() throws Exception {
+        try {
+            Configuration conf = cluster.getConf();
+            FileSystem fs = FileSystem.get(conf);
+            fs.delete(new Path("/"), true);
+            stream.clear();
+
+            Pair<List<String>, List<String>> pair, statsPair, metaPair;
+            pair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS, "/data");
+            statsPair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS, "/stats");
+            metaPair = createTestData("feed1", "yyyy-MM-dd/'more'/yyyy", 10, TimeUnit.DAYS, "/meta");
+            String dataPath = cluster.getConf().get("fs.default.name")
+                    + "/data/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}"
+                    + "#"
+                    + cluster.getConf().get("fs.default.name")
+                    + "/stats/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}"
+                    + "#"
+                    + cluster.getConf().get("fs.default.name")
+                    + "/meta/YYYY/feed1/mmHH/dd/MM/?{YEAR}-?{MONTH}-?{DAY}/more/?{YEAR}";
+            String logFile = "/falcon/staging/feed/instancePaths-2012-01-01-01-00.csv";
+
+            FeedEvictor.main(new String[]{
+                    "-feedBasePath",
+                    dataPath,
+                    "-retentionType", "instance", "-retentionLimit", "days(10)", "-timeZone", "UTC", "-frequency",
+                    "daily",
+                    "-logFile", logFile});
+
+            assertFailures(fs, pair);
+
+            Assert.assertEquals(readLogFile(new Path(logFile)),
+                    getExpectedInstancePaths(dataPath.replaceAll(cluster
+                            .getConf().get("fs.default.name"), "")));
+
+
+        } catch (Exception e) {
+            Assert.fail("Unknown exception", e);
+        }
+    }
+
+    private Pair<List<String>, List<String>> createTestData(String locationType) throws Exception {
+        Configuration conf = cluster.getConf();
+        FileSystem fs = FileSystem.get(conf);
+
+        List<String> outOfRange = new ArrayList<String>();
+        List<String> inRange = new ArrayList<String>();
+
+        touch(fs, locationType + "/YYYY/feed3/dd/MM/02/more/hello");
+        touch(fs, locationType + "/YYYY/feed4/dd/MM/02/more/hello");
+        touch(fs, locationType + "/YYYY/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
+        touch(fs, locationType + "/somedir/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
+        outOfRange.add(locationType + "/YYYY/feed3/dd/MM/02/more/hello");
+        outOfRange.add(locationType + "/YYYY/feed4/dd/MM/02/more/hello");
+        outOfRange.add(locationType + "/YYYY/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
+        outOfRange.add(locationType + "/somedir/feed1/mmHH/dd/MM/bad-va-lue/more/hello");
+
+        return Pair.of(inRange, outOfRange);
+    }
+
+    private Pair<List<String>, List<String>> createTestData(String feed,
+                                                            String mask,
+                                                            int period,
+                                                            TimeUnit timeUnit, String locationType)
+            throws Exception {
+
+        Configuration conf = cluster.getConf();
+        FileSystem fs = FileSystem.get(conf);
+
+        List<String> outOfRange = new ArrayList<String>();
+        List<String> inRange = new ArrayList<String>();
+
+        Pair<List<String>, List<String>> pair = createTestData(locationType);
+        outOfRange.addAll(pair.second);
+        inRange.addAll(pair.first);
+
+        pair = generateInstances(fs, feed, mask, period, timeUnit, locationType);
+        outOfRange.addAll(pair.second);
+        inRange.addAll(pair.first);
+        return Pair.of(inRange, outOfRange);
+    }
+
+    private Pair<List<String>, List<String>> generateInstances(
+            FileSystem fs, String feed, String formatString,
+            int range, TimeUnit timeUnit, String locationType) throws Exception {
+
+        List<String> outOfRange = new ArrayList<String>();
+        List<String> inRange = new ArrayList<String>();
+
+        DateFormat format = new SimpleDateFormat(formatString);
+        format.setTimeZone(TimeZone.getTimeZone("UTC"));
+        long now = System.currentTimeMillis();
+
+        DateFormat displayFormat = new
+                SimpleDateFormat(timeUnit == TimeUnit.HOURS ? "yyyyMMddHH" : "yyyyMMdd");
+        displayFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+
+        StringBuffer buffer = new StringBuffer();
+        for (long date = now;
+             date > now - timeUnit.toMillis(range + 6);
+             date -= timeUnit.toMillis(1)) {
+            String path = locationType + "/YYYY/" + feed + "/mmHH/dd/MM/" +
+                    format.format(date);
+            touch(fs, path);
+            if (date <= now && date > now - timeUnit.toMillis(range)) {
+                outOfRange.add(path);
+            } else {
+                inRange.add(path);
+                buffer.append((displayFormat.format(date) + "0000").
+                        substring(0, 12)).append(',');
+            }
+        }
+        map.put(feed, "instances=" + buffer.substring(0, buffer.length() - 1));
+        return Pair.of(inRange, outOfRange);
+    }
+
+    private void touch(FileSystem fs, String path) throws Exception {
+        fs.create(new Path(path)).close();
+    }
+
+    private String getFeedBasePath(EmbeddedCluster cluster, String locationType) {
+        return cluster.getConf().get("fs.default.name")
+                + "/data/YYYY/feed3/dd/MM/"
+                + "?{MONTH}/more/?{HOUR}";
+    }
+
+    private static class InMemoryWriter extends PrintStream {
+
+        private StringBuffer buffer = new StringBuffer();
+
+        public InMemoryWriter(OutputStream out) {
+            super(out);
+        }
+
+        @Override
+        public void println(String x) {
+            buffer.append(x);
+            super.println(x);
+        }
+
+        public String getBuffer() {
+            return buffer.toString();
+        }
+
+        public void clear() {
+            buffer.delete(0, buffer.length());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 4b81006..296decb 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,10 +18,7 @@
 
 package org.apache.falcon.cluster.util;
 
-import java.io.File;
-import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
-
+import org.apache.falcon.entity.v0.cluster.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -29,15 +26,13 @@ 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.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.security.authorize.ProxyUsers;
 import org.apache.log4j.Logger;
 
+import java.io.File;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
 public class EmbeddedCluster {
 
     private static Logger LOG = Logger.getLogger(EmbeddedCluster.class);
@@ -105,7 +100,7 @@ public class EmbeddedCluster {
             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));
+            FileSystem.get(cluster.conf).setPermission(path, new FsPermission((short) 511));
             cluster.mrCluster = new MiniMRCluster(1,
                     hdfsUrl, 1);
             Configuration mrConf = cluster.mrCluster.createJobConf();
@@ -137,14 +132,14 @@ public class EmbeddedCluster {
         interfaces.getInterfaces().add(newInterface(Interfacetype.READONLY, fsUrl, "0.1"));
         interfaces.getInterfaces().add(newInterface(Interfacetype.WRITE, fsUrl, "0.1"));
         interfaces.getInterfaces().add(newInterface(Interfacetype.EXECUTE,
-                        conf.get("mapred.job.tracker"), "0.1"));
-		interfaces
-				.getInterfaces()
-				.add(newInterface(
-						Interfacetype.MESSAGING,
-						"vm://localhost",
-						"0.1"));
-      clusterEntity.setInterfaces(interfaces);
+                conf.get("mapred.job.tracker"), "0.1"));
+        interfaces
+                .getInterfaces()
+                .add(newInterface(
+                        Interfacetype.MESSAGING,
+                        "vm://localhost",
+                        "0.1"));
+        clusterEntity.setInterfaces(interfaces);
 
         Location location = new Location();
         location.setName("staging");
@@ -168,7 +163,9 @@ public class EmbeddedCluster {
     }
 
     public void shutdown() {
-        if (mrCluster != null) mrCluster.shutdown();
+        if (mrCluster != null) {
+            mrCluster.shutdown();
+        }
         dfsCluster.shutdown();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
----------------------------------------------------------------------
diff --git a/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java b/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
index b65c0e3..baf6347 100644
--- a/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
+++ b/test-util/src/main/java/org/apache/falcon/cluster/util/StandAloneCluster.java
@@ -18,36 +18,38 @@
 
 package org.apache.falcon.cluster.util;
 
-import java.io.File;
-
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.cluster.Cluster;
 import org.apache.falcon.entity.v0.cluster.Interface;
 import org.apache.falcon.entity.v0.cluster.Interfacetype;
 import org.apache.log4j.Logger;
 
+import java.io.File;
 
-public class StandAloneCluster extends EmbeddedCluster{
+
+public class StandAloneCluster extends EmbeddedCluster {
     private static final Logger LOG = Logger.getLogger(StandAloneCluster.class);
-    
+
     private StandAloneCluster() {
     }
-    
+
     public static StandAloneCluster newCluster(String clusterFile) throws Exception {
         LOG.debug("Initialising standalone cluster");
         StandAloneCluster cluster = new StandAloneCluster();
         cluster.clusterEntity = (Cluster) EntityType.CLUSTER.getUnmarshaller().unmarshal(new File(clusterFile));
-        
-        for(Interface inter:cluster.getCluster().getInterfaces().getInterfaces())
-            if(inter.getType() == Interfacetype.WRITE)
+
+        for (Interface inter : cluster.getCluster().getInterfaces().getInterfaces()) {
+            if (inter.getType() == Interfacetype.WRITE) {
                 cluster.getConf().set("fs.default.name", inter.getEndpoint());
-                
+            }
+        }
+
         LOG.info("Cluster Namenode = " + cluster.getConf().get("fs.default.name"));
         return cluster;
     }
-    
+
     @Override
     public void shutdown() {
-        
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/java/org/apache/falcon/Debug.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/Debug.java b/webapp/src/main/java/org/apache/falcon/Debug.java
index dde4f92..73c99ed 100644
--- a/webapp/src/main/java/org/apache/falcon/Debug.java
+++ b/webapp/src/main/java/org/apache/falcon/Debug.java
@@ -21,11 +21,10 @@ package org.apache.falcon;
 import org.apache.falcon.client.FalconClient;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.store.ConfigurationStore;
-import org.apache.falcon.entity.v0.Frequency;
-import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.entity.v0.Entity;
 import org.apache.falcon.entity.v0.EntityType;
-import org.apache.falcon.entity.v0.feed.Feed;
+import org.apache.falcon.entity.v0.Frequency;
+import org.apache.falcon.entity.v0.process.Process;
 import org.apache.falcon.security.CurrentUser;
 import org.apache.falcon.service.Services;
 import org.apache.falcon.util.DeploymentProperties;
@@ -58,7 +57,9 @@ public class Debug {
             for (String line : deps) {
                 String[] fields = line.replace("(", "").replace(")", "").split(" ");
                 EntityType eType = EntityType.valueOf(fields[0].toUpperCase());
-                if (ConfigurationStore.get().get(eType, fields[1]) != null) continue;
+                if (ConfigurationStore.get().get(eType, fields[1]) != null) {
+                    continue;
+                }
                 String xml = client.getDefinition(eType.name().toLowerCase(), fields[1]);
                 System.out.println(xml);
                 store(eType, xml);
@@ -70,7 +71,7 @@ public class Debug {
 
         entity = args[2];
         Entity obj = EntityUtil.getEntity(type, entity);
-        Process newEntity = (Process)obj.clone();
+        Process newEntity = (Process) obj.clone();
         newEntity.setFrequency(Frequency.fromString("minutes(5)"));
         System.out.println("##############OLD ENTITY " + EntityUtil.md5(obj));
         System.out.println("##############NEW ENTITY " + EntityUtil.md5(newEntity));

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/java/org/apache/falcon/Main.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/Main.java b/webapp/src/main/java/org/apache/falcon/Main.java
index 8cbabc6..3726559 100644
--- a/webapp/src/main/java/org/apache/falcon/Main.java
+++ b/webapp/src/main/java/org/apache/falcon/Main.java
@@ -28,11 +28,11 @@ public class Main {
         EmbeddedServer server = new EmbeddedServer(15000,
                 "webapp/target/falcon-webapp-0.2-SNAPSHOT");
         server.start();
-        
+
         BrokerService broker = new BrokerService();
         broker.setUseJmx(false);
-		broker.setDataDirectory("target/");
-		broker.addConnector("vm://localhost");
+        broker.setDataDirectory("target/");
+        broker.addConnector("vm://localhost");
         broker.start();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java b/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
index 6b55911..248c2a0 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/ConfigSyncService.java
@@ -31,8 +31,8 @@ public class ConfigSyncService extends AbstractEntityManager {
 
     @POST
     @Path("submit/{type}")
-    @Consumes({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Consumes({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "submit")
     @Override
     public APIResult submit(@Context HttpServletRequest request,
@@ -43,7 +43,7 @@ public class ConfigSyncService extends AbstractEntityManager {
 
     @DELETE
     @Path("delete/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "delete")
     @Override
     public APIResult delete(@Context HttpServletRequest request,
@@ -55,7 +55,7 @@ public class ConfigSyncService extends AbstractEntityManager {
 
     @POST
     @Path("update/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "update")
     @Override
     public APIResult update(@Context HttpServletRequest request,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
index 094fe75..9f789a8 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/InstanceManager.java
@@ -32,47 +32,47 @@ public class InstanceManager extends AbstractInstanceManager {
     @GET
     @Path("running/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="running")
+    @Monitored(event = "running")
     @Override
     public InstancesResult getRunningInstances(@PathParam("type") String type,
-                                                      @PathParam("entity") String entity,
-                                                      @QueryParam("colo") String colo) {
+                                               @PathParam("entity") String entity,
+                                               @QueryParam("colo") String colo) {
         return super.getRunningInstances(type, entity, colo);
     }
 
     @GET
     @Path("status/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="instance-status")
+    @Monitored(event = "instance-status")
     @Override
     public InstancesResult getStatus(@Dimension("type") @PathParam("type") String type,
-                                            @Dimension("entity") @PathParam("entity") String entity,
-                                            @Dimension("start-time") @QueryParam("start") String startStr,
-                                            @Dimension("end-time") @QueryParam("end") String endStr,
-                                            @Dimension("colo") @QueryParam("colo") String colo) {
+                                     @Dimension("entity") @PathParam("entity") String entity,
+                                     @Dimension("start-time") @QueryParam("start") String startStr,
+                                     @Dimension("end-time") @QueryParam("end") String endStr,
+                                     @Dimension("colo") @QueryParam("colo") String colo) {
         return super.getStatus(type, entity, startStr, endStr, colo);
     }
 
-	@GET
-	@Path("logs/{type}/{entity}")
-	@Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event = "instance-logs")
-	@Override
-	public InstancesResult getLogs(
-			@Dimension("type") @PathParam("type") String type,
-			@Dimension("entity") @PathParam("entity") String entity,
-			@Dimension("start-time") @QueryParam("start") String startStr,
-			@Dimension("end-time") @QueryParam("end") String endStr,
-			@Dimension("colo") @QueryParam("colo") String colo,
-			@Dimension("run-id") @QueryParam("runid") String runId) {
-		return super.getLogs(type, entity, startStr, endStr, colo, runId);
-	}
-    
+    @GET
+    @Path("logs/{type}/{entity}")
+    @Produces(MediaType.APPLICATION_JSON)
+    @Monitored(event = "instance-logs")
+    @Override
+    public InstancesResult getLogs(
+            @Dimension("type") @PathParam("type") String type,
+            @Dimension("entity") @PathParam("entity") String entity,
+            @Dimension("start-time") @QueryParam("start") String startStr,
+            @Dimension("end-time") @QueryParam("end") String endStr,
+            @Dimension("colo") @QueryParam("colo") String colo,
+            @Dimension("run-id") @QueryParam("runid") String runId) {
+        return super.getLogs(type, entity, startStr, endStr, colo, runId);
+    }
+
 
     @POST
     @Path("kill/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-    @Monitored(event="kill-instance")
+    @Monitored(event = "kill-instance")
     @Override
     public InstancesResult killInstance(@Context HttpServletRequest request,
                                         @Dimension("type") @PathParam("type") String type,
@@ -86,7 +86,7 @@ public class InstanceManager extends AbstractInstanceManager {
     @POST
     @Path("suspend/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="suspend-instance")
+    @Monitored(event = "suspend-instance")
     @Override
     public InstancesResult suspendInstance(@Context HttpServletRequest request,
                                            @Dimension("type") @PathParam("type") String type,
@@ -100,7 +100,7 @@ public class InstanceManager extends AbstractInstanceManager {
     @POST
     @Path("resume/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="resume-instance")
+    @Monitored(event = "resume-instance")
     @Override
     public InstancesResult resumeInstance(@Context HttpServletRequest request,
                                           @Dimension("type") @PathParam("type") String type,
@@ -114,14 +114,14 @@ public class InstanceManager extends AbstractInstanceManager {
     @POST
     @Path("rerun/{type}/{entity}")
     @Produces(MediaType.APPLICATION_JSON)
-	@Monitored(event="re-run-instance")
+    @Monitored(event = "re-run-instance")
     @Override
     public InstancesResult reRunInstance(@Dimension("type") @PathParam("type") String type,
-                                                @Dimension("entity") @PathParam("entity") String entity,
-                                                @Dimension("start-time") @QueryParam("start") String startStr,
-                                                @Dimension("end-time") @QueryParam("end") String endStr,
-                                                @Context HttpServletRequest request,
-                                                @Dimension("colo") @QueryParam("colo") String colo) {
+                                         @Dimension("entity") @PathParam("entity") String entity,
+                                         @Dimension("start-time") @QueryParam("start") String startStr,
+                                         @Dimension("end-time") @QueryParam("end") String endStr,
+                                         @Context HttpServletRequest request,
+                                         @Dimension("colo") @QueryParam("colo") String colo) {
         return super.reRunInstance(type, entity, startStr, endStr, request, colo);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
index 099fc5a..02e6614 100644
--- a/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
+++ b/webapp/src/main/java/org/apache/falcon/resource/SchedulableEntityManager.java
@@ -32,12 +32,12 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
     @GET
     @Path("status/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "status")
     @Override
     public APIResult getStatus(@Dimension("entityType") @PathParam("type") String type,
-                            @Dimension("entityName") @PathParam("entity") String entity,
-                            @Dimension("colo") @QueryParam("colo") final String colo)
+                               @Dimension("entityName") @PathParam("entity") String entity,
+                               @Dimension("colo") @QueryParam("colo") final String colo)
             throws FalconWebException {
         return super.getStatus(type, entity, colo);
     }
@@ -63,7 +63,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
     @GET
     @Path("definition/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "definition")
     @Override
     public String getEntityDefinition(@Dimension("type") @PathParam("type") String type,
@@ -73,7 +73,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
     @POST
     @Path("schedule/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "schedule")
     @Override
     public APIResult schedule(@Context HttpServletRequest request,
@@ -85,7 +85,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
     @POST
     @Path("suspend/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "suspend")
     @Override
     public APIResult suspend(@Context HttpServletRequest request,
@@ -97,7 +97,7 @@ public class SchedulableEntityManager extends AbstractSchedulableEntityManager {
 
     @POST
     @Path("resume/{type}/{entity}")
-    @Produces({ MediaType.TEXT_XML, MediaType.TEXT_PLAIN })
+    @Produces({MediaType.TEXT_XML, MediaType.TEXT_PLAIN})
     @Monitored(event = "resume")
     @Override
     public APIResult resume(@Context HttpServletRequest request,

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/resources/log4j.xml b/webapp/src/main/resources/log4j.xml
index da50fb2..189d096 100644
--- a/webapp/src/main/resources/log4j.xml
+++ b/webapp/src/main/resources/log4j.xml
@@ -20,72 +20,72 @@
 <!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
 
 <log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
-  <appender name="console" class="org.apache.log4j.ConsoleAppender">
-    <param name="Target" value="System.out"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="console" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
 
-  <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/application.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
-    </layout>
-  </appender>
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/application.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p - [%t:%x] ~ %m (%c{1}:%L)%n"/>
+        </layout>
+    </appender>
+
+    <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/audit.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
+
+    <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="/var/log/falcon/tranlog.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %x %m%n"/>
+        </layout>
+    </appender>
 
-  <appender name="AUDIT" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/audit.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
-  <appender name="TRANSACTIONLOG" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/tranlog.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %x %m%n"/>
-    </layout>
-  </appender>
-  
     <appender name="METRIC" class="org.apache.log4j.DailyRollingFileAppender">
-    <param name="File" value="/var/log/falcon/metric.log"/>
-    <param name="Append" value="true"/>
-    <param name="Threshold" value="debug"/>
-    <layout class="org.apache.log4j.PatternLayout">
-      <param name="ConversionPattern" value="%d %m%n"/>
-    </layout>
-  </appender>
+        <param name="File" value="/var/log/falcon/metric.log"/>
+        <param name="Append" value="true"/>
+        <param name="Threshold" value="debug"/>
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %m%n"/>
+        </layout>
+    </appender>
+
+    <logger name="org.apache.falcon">
+        <level value="debug"/>
+        <appender-ref ref="FILE"/>
+    </logger>
+
+    <logger name="AUDIT">
+        <level value="info"/>
+        <appender-ref ref="AUDIT"/>
+    </logger>
 
-  <logger name="org.apache.falcon">
-    <level value="debug"/>
-    <appender-ref ref="FILE" />
-  </logger>
+    <logger name="TRANSACTIONLOG">
+        <level value="info"/>
+        <appender-ref ref="TRANSACTIONLOG"/>
+    </logger>
 
-  <logger name="AUDIT">
-    <level value="info"/>
-    <appender-ref ref="AUDIT" />
-  </logger>
-  
-  <logger name="TRANSACTIONLOG">
-    <level value="info"/>
-    <appender-ref ref="TRANSACTIONLOG" />
-  </logger>
-  
     <logger name="METRIC">
-    <level value="info"/>
-    <appender-ref ref="METRIC" />
-  </logger>
+        <level value="info"/>
+        <appender-ref ref="METRIC"/>
+    </logger>
 
-  <root>
-    <priority value ="info" />
-    <appender-ref ref="console" />
-  </root>
+    <root>
+        <priority value="info"/>
+        <appender-ref ref="console"/>
+    </root>
 
 </log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/webapp/WEB-INF/distributed/web.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/webapp/WEB-INF/distributed/web.xml b/webapp/src/main/webapp/WEB-INF/distributed/web.xml
index 68d6248..86fc8d2 100644
--- a/webapp/src/main/webapp/WEB-INF/distributed/web.xml
+++ b/webapp/src/main/webapp/WEB-INF/distributed/web.xml
@@ -53,7 +53,9 @@
         </init-param>
         <init-param>
             <param-name>com.sun.jersey.config.property.classnames</param-name>
-            <param-value>org.apache.falcon.resource.admin.AdminResource,org.apache.falcon.resource.provider.JAXBContextResolver,org.apache.falcon.resource.SchedulableEntityManager,org.apache.falcon.resource.InstanceManager</param-value>
+            <param-value>
+                org.apache.falcon.resource.admin.AdminResource,org.apache.falcon.resource.provider.JAXBContextResolver,org.apache.falcon.resource.SchedulableEntityManager,org.apache.falcon.resource.InstanceManager
+            </param-value>
         </init-param>
         <load-on-startup>1</load-on-startup>
     </servlet>
@@ -67,7 +69,9 @@
         </init-param>
         <init-param>
             <param-name>com.sun.jersey.config.property.classnames</param-name>
-            <param-value>org.apache.falcon.resource.provider.JAXBContextResolver,org.apache.falcon.resource.ConfigSyncService</param-value>
+            <param-value>
+                org.apache.falcon.resource.provider.JAXBContextResolver,org.apache.falcon.resource.ConfigSyncService
+            </param-value>
         </init-param>
         <load-on-startup>1</load-on-startup>
     </servlet>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/webapp/WEB-INF/embedded/web.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/webapp/WEB-INF/embedded/web.xml b/webapp/src/main/webapp/WEB-INF/embedded/web.xml
index e5e634b..bc74d91 100644
--- a/webapp/src/main/webapp/WEB-INF/embedded/web.xml
+++ b/webapp/src/main/webapp/WEB-INF/embedded/web.xml
@@ -25,37 +25,39 @@
     <display-name>Apache Falcon Embedded Server</display-name>
     <description>Apache Falcon Embedded Server</description>
 
-  <filter>
-    <filter-name>auth</filter-name>
-    <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
-  </filter>
-
-  <filter-mapping>
-    <filter-name>auth</filter-name>
-    <servlet-name>FalconRESTApi</servlet-name>
-  </filter-mapping>
-
-  <listener>
-    <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
-  </listener>
-
-  <servlet>
-      <servlet-name>FalconRESTApi</servlet-name>
-      <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
-          <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
-      </init-param>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.packages</param-name>
-          <param-value>org.apache.falcon.resource.admin,org.apache.falcon.resource.provider,org.apache.falcon.resource.proxy</param-value>
-      </init-param>
-      <load-on-startup>1</load-on-startup>
-  </servlet>
-
-  <servlet-mapping>
-      <servlet-name>FalconRESTApi</servlet-name>
-      <url-pattern>/api/*</url-pattern>
-  </servlet-mapping>
+    <filter>
+        <filter-name>auth</filter-name>
+        <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
+    </filter>
+
+    <filter-mapping>
+        <filter-name>auth</filter-name>
+        <servlet-name>FalconRESTApi</servlet-name>
+    </filter-mapping>
+
+    <listener>
+        <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
+    </listener>
+
+    <servlet>
+        <servlet-name>FalconRESTApi</servlet-name>
+        <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
+            <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
+        </init-param>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.packages</param-name>
+            <param-value>
+                org.apache.falcon.resource.admin,org.apache.falcon.resource.provider,org.apache.falcon.resource.proxy
+            </param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
+    <servlet-mapping>
+        <servlet-name>FalconRESTApi</servlet-name>
+        <url-pattern>/api/*</url-pattern>
+    </servlet-mapping>
 
 </web-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/webapp/src/main/webapp/WEB-INF/web.xml b/webapp/src/main/webapp/WEB-INF/web.xml
index b0ffa39..3864789 100644
--- a/webapp/src/main/webapp/WEB-INF/web.xml
+++ b/webapp/src/main/webapp/WEB-INF/web.xml
@@ -25,37 +25,39 @@
     <display-name>Apache Falcon Placeholder</display-name>
     <description>Apache Falcon Placeholder</description>
 
-  <filter>
-    <filter-name>auth</filter-name>
-    <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
-  </filter>
-
-  <filter-mapping>
-    <filter-name>auth</filter-name>
-    <servlet-name>FalconRESTApi</servlet-name>
-  </filter-mapping>
-
-  <listener>
-    <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
-  </listener>
-
-  <servlet>
-      <servlet-name>FalconRESTApi</servlet-name>
-      <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
-          <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
-      </init-param>
-      <init-param>
-          <param-name>com.sun.jersey.config.property.packages</param-name>
-          <param-value>org.apache.falcon.resource.admin,org.apache.falcon.resource.proxy,org.apache.falcon.resource.provider</param-value>
-      </init-param>
-      <load-on-startup>1</load-on-startup>
-  </servlet>
-
-  <servlet-mapping>
-      <servlet-name>FalconRESTApi</servlet-name>
-      <url-pattern>/api/*</url-pattern>
-  </servlet-mapping>
+    <filter>
+        <filter-name>auth</filter-name>
+        <filter-class>org.apache.falcon.security.BasicAuthFilter</filter-class>
+    </filter>
+
+    <filter-mapping>
+        <filter-name>auth</filter-name>
+        <servlet-name>FalconRESTApi</servlet-name>
+    </filter-mapping>
+
+    <listener>
+        <listener-class>org.apache.falcon.listener.ContextStartupListener</listener-class>
+    </listener>
+
+    <servlet>
+        <servlet-name>FalconRESTApi</servlet-name>
+        <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.resourceConfigClass</param-name>
+            <param-value>com.sun.jersey.api.core.PackagesResourceConfig</param-value>
+        </init-param>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.packages</param-name>
+            <param-value>
+                org.apache.falcon.resource.admin,org.apache.falcon.resource.proxy,org.apache.falcon.resource.provider
+            </param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+
+    <servlet-mapping>
+        <servlet-name>FalconRESTApi</servlet-name>
+        <url-pattern>/api/*</url-pattern>
+    </servlet-mapping>
 
 </web-app>


[03/47] git commit: Excluding activemq test data from git inclusion

Posted by sr...@apache.org.
Excluding activemq test data from git inclusion


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

Branch: refs/heads/master
Commit: 2e147a6dd3c9a80591f7c2caa25e689900213590
Parents: b09c5ee
Author: srikanth.sundarrajan <sr...@inmobi.com>
Authored: Mon Apr 15 11:20:24 2013 +0530
Committer: srikanth.sundarrajan <sr...@inmobi.com>
Committed: Mon Apr 15 11:20:24 2013 +0530

----------------------------------------------------------------------
 .gitignore |    2 ++
 1 files changed, 2 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/2e147a6d/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b1174c7..6c05272 100644
--- a/.gitignore
+++ b/.gitignore
@@ -28,3 +28,5 @@ target
 .project
 .settings
 
+#ActiveMQ
+activemq-data


[05/47] Fixes for Checkstyle

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
index 99b330e..ca2a968 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/EntityManagerJerseyTest.java
@@ -17,27 +17,7 @@
  */
 package org.apache.falcon.resource;
 
-import java.io.File;
-import java.io.InputStream;
-import java.io.StringReader;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-
-import javax.servlet.ServletInputStream;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.xml.bind.JAXBException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
-import org.apache.hadoop.fs.Path;
+import com.sun.jersey.api.client.ClientResponse;
 import org.apache.falcon.FalconWebException;
 import org.apache.falcon.entity.v0.EntityType;
 import org.apache.falcon.entity.v0.feed.Feed;
@@ -47,20 +27,33 @@ import org.apache.falcon.entity.v0.process.Property;
 import org.apache.falcon.entity.v0.process.Validity;
 import org.apache.falcon.util.BuildProperties;
 import org.apache.falcon.util.DeploymentProperties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
 import org.apache.oozie.client.BundleJob;
 import org.apache.oozie.client.Job;
 import org.apache.oozie.client.Job.Status;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-import com.sun.jersey.api.client.ClientResponse;
+import javax.servlet.ServletInputStream;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.xml.bind.JAXBException;
+import java.io.File;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.*;
 
-public class EntityManagerJerseyTest extends AbstractTestBase{
+public class EntityManagerJerseyTest extends AbstractTestBase {
     /**
      * Tests should be enabled only in local environments as they need running
      * instance of webserver
      */
-    
+
     @Test
     public void testUpdateCheckUser() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
@@ -72,37 +65,39 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
         scheduleProcess(tmpFile.getAbsolutePath(), overlay);
         waitForBundleStart(Status.RUNNING);
-        
+
         List<BundleJob> bundles = getBundles();
         Assert.assertEquals(bundles.size(), 1);
         Assert.assertEquals(bundles.get(0).getUser(), REMOTE_USER);
-        
-        ClientResponse response = this.service.path("api/entities/definition/feed/" + outputFeedName).header("Remote-User", REMOTE_USER)
+
+        ClientResponse response = this.service.path("api/entities/definition/feed/" + outputFeedName).header(
+                "Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
         Feed feed = (Feed) EntityType.FEED.getUnmarshaller()
                 .unmarshal(new StringReader(response.getEntity(String.class)));
-        
+
         //change output feed path and update feed as another user
         feed.getLocations().getLocations().get(0).setPath("/falcon/test/output2/${YEAR}/${MONTH}/${DAY}");
         tmpFile = getTempFile();
         EntityType.FEED.getMarshaller().marshal(feed, tmpFile);
-        response = this.service.path("api/entities/update/feed/" + outputFeedName).header("Remote-User", "testuser").accept(MediaType.TEXT_XML)
+        response = this.service.path("api/entities/update/feed/" + outputFeedName).header("Remote-User",
+                "testuser").accept(MediaType.TEXT_XML)
                 .post(ClientResponse.class, getServletInputStream(tmpFile.getAbsolutePath()));
-        assertSuccessful(response);    
-        
+        assertSuccessful(response);
+
         bundles = getBundles();
         Assert.assertEquals(bundles.size(), 2);
         Assert.assertEquals(bundles.get(0).getUser(), REMOTE_USER);
         Assert.assertEquals(bundles.get(1).getUser(), REMOTE_USER);
     }
-    
-    
+
+
     @Test(enabled = false)
     public void testOptionalInput() throws Exception {
         Map<String, String> overlay = getUniqueOverlay();
         String tmpFileName = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
         Process process = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(new File(tmpFileName));
-        
+
         Input in1 = process.getInputs().getInputs().get(0);
         Input in2 = new Input();
         in2.setFeed(in1.getFeed());
@@ -112,51 +107,53 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         in2.setStart("now(-1,0)");
         in2.setEnd("now(0,0)");
         process.getInputs().getInputs().add(in2);
-        
+
         File tmpFile = getTempFile();
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
         scheduleProcess(tmpFile.getAbsolutePath(), overlay);
         waitForWorkflowStart(processName);
     }
-    
-	@Test
-	public void testProcessDeleteAndSchedule() throws Exception {
-		//Submit process with invalid property so that coord submit fails and bundle goes to failed state
-		Map<String, String> overlay = getUniqueOverlay();
+
+    @Test
+    public void testProcessDeleteAndSchedule() throws Exception {
+        //Submit process with invalid property so that coord submit fails and bundle goes to failed state
+        Map<String, String> overlay = getUniqueOverlay();
         String tmpFileName = overlayParametersOverTemplate(PROCESS_TEMPLATE, overlay);
-		Process process = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(new File(tmpFileName));
-		Property prop = new Property();
-		prop.setName("newProp");
-		prop.setValue("${formatTim()}");
+        Process process = (Process) EntityType.PROCESS.getUnmarshaller().unmarshal(new File(tmpFileName));
+        Property prop = new Property();
+        prop.setName("newProp");
+        prop.setValue("${formatTim()}");
         process.getProperties().getProperties().add(prop);
         File tmpFile = getTempFile();
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
         scheduleProcess(tmpFile.getAbsolutePath(), overlay);
         waitForBundleStart(Status.FAILED);
-        
+
         //Delete and re-submit the process with correct workflow
-        ClientResponse clientRepsonse = this.service.path("api/entities/delete/process/" + processName).header("Remote-User", REMOTE_USER)
+        ClientResponse clientRepsonse = this.service.path("api/entities/delete/process/" + processName).header(
+                "Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).delete(ClientResponse.class);
-		assertSuccessful(clientRepsonse);
-		process.getWorkflow().setPath("/falcon/test/workflow");
+        assertSuccessful(clientRepsonse);
+        process.getWorkflow().setPath("/falcon/test/workflow");
         tmpFile = getTempFile();
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
         clientRepsonse = this.service.path("api/entities/submitAndSchedule/process").header("Remote-User", REMOTE_USER)
-        		.accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
+                .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
                 .post(ClientResponse.class, getServletInputStream(tmpFile.getAbsolutePath()));
-        assertSuccessful(clientRepsonse);    
-        
+        assertSuccessful(clientRepsonse);
+
         //Assert that new schedule creates new bundle
         List<BundleJob> bundles = getBundles();
         Assert.assertEquals(bundles.size(), 2);
-	}
-	
+    }
+
     @Test
     public void testProcessInputUpdate() throws Exception {
         scheduleProcess();
         waitForBundleStart(Job.Status.RUNNING);
-        
-        ClientResponse response = this.service.path("api/entities/definition/process/" + processName).header("Remote-User", REMOTE_USER)
+
+        ClientResponse response = this.service.path("api/entities/definition/process/" + processName).header(
+                "Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
         Process process = (Process) EntityType.PROCESS.getUnmarshaller()
                 .unmarshal(new StringReader(response.getEntity(String.class)));
@@ -179,21 +176,23 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         processValidity.setEnd(new Date(new Date().getTime() + 2 * 24 * 60 * 60 * 1000));
         File tmpFile = getTempFile();
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
-        response = this.service.path("api/entities/update/process/" + processName).header("Remote-User", REMOTE_USER).accept(MediaType.TEXT_XML)
+        response = this.service.path("api/entities/update/process/" + processName).header("Remote-User",
+                REMOTE_USER).accept(MediaType.TEXT_XML)
                 .post(ClientResponse.class, getServletInputStream(tmpFile.getAbsolutePath()));
-        assertSuccessful(response);    
-        
+        assertSuccessful(response);
+
         //Assert that update creates new bundle
         List<BundleJob> bundles = getBundles();
         Assert.assertEquals(bundles.size(), 2);
     }
-    
+
     @Test
     public void testProcessEndtimeUpdate() throws Exception {
         scheduleProcess();
         waitForBundleStart(Job.Status.RUNNING);
-        
-        ClientResponse response = this.service.path("api/entities/definition/process/" + processName).header("Remote-User", REMOTE_USER)
+
+        ClientResponse response = this.service.path("api/entities/definition/process/" + processName).header(
+                "Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
         Process process = (Process) EntityType.PROCESS.getUnmarshaller()
                 .unmarshal(new StringReader(response.getEntity(String.class)));
@@ -202,15 +201,16 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         processValidity.setEnd(new Date(new Date().getTime() + 60 * 60 * 1000));
         File tmpFile = getTempFile();
         EntityType.PROCESS.getMarshaller().marshal(process, tmpFile);
-        response = this.service.path("api/entities/update/process/" + processName).header("Remote-User", REMOTE_USER).accept(MediaType.TEXT_XML)
+        response = this.service.path("api/entities/update/process/" + processName).header("Remote-User",
+                REMOTE_USER).accept(MediaType.TEXT_XML)
                 .post(ClientResponse.class, getServletInputStream(tmpFile.getAbsolutePath()));
-        assertSuccessful(response);    
-        
+        assertSuccessful(response);
+
         //Assert that update does not create new bundle
         List<BundleJob> bundles = getBundles();
         Assert.assertEquals(bundles.size(), 1);
     }
-    
+
     @Test
     public void testStatus() throws Exception {
         ClientResponse response;
@@ -227,12 +227,12 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
                 .header("Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
 
-        APIResult result = (APIResult)unmarshaller.
+        APIResult result = (APIResult) unmarshaller.
                 unmarshal(new StringReader(response.getEntity(String.class)));
         Assert.assertTrue(result.getMessage().contains("SUBMITTED"));
-        
+
     }
-    
+
     @Test
     public void testIdempotentSubmit() throws Exception {
         ClientResponse response;
@@ -242,33 +242,31 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         assertSuccessful(response);
 
         response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
-        assertSuccessful(response);        
+        assertSuccessful(response);
     }
-        
+
     @Test
-    public void testNotFoundStatus() throws FalconWebException
-    {
-    	ClientResponse response;
-    	String feed1 = "f1" + System.currentTimeMillis();
-    	response = this.service
+    public void testNotFoundStatus() throws FalconWebException {
+        ClientResponse response;
+        String feed1 = "f1" + System.currentTimeMillis();
+        response = this.service
                 .path("api/entities/status/feed/" + feed1)
                 .header("Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
         String status = response.getEntity(String.class);
-    	Assert.assertEquals(response.getStatus(), Response.Status.BAD_REQUEST.getStatusCode());
-  
+        Assert.assertEquals(response.getStatus(), Response.Status.BAD_REQUEST.getStatusCode());
+
     }
 
     @Test
-    public void testVersion() throws FalconWebException
-    {
-    	ClientResponse response;
-    	response = this.service
+    public void testVersion() throws FalconWebException {
+        ClientResponse response;
+        response = this.service
                 .path("api/admin/version")
                 .header("Remote-User", REMOTE_USER)
                 .accept(MediaType.TEXT_PLAIN).get(ClientResponse.class);
         String status = response.getEntity(String.class);
-    	Assert.assertEquals(status, "{Version:\"" +
+        Assert.assertEquals(status, "{Version:\"" +
                 BuildProperties.get().getProperty("build.version") + "\",Mode:\"" +
                 DeploymentProperties.get().getProperty("deploy.mode") + "\"}");
 
@@ -304,42 +302,42 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         assertSuccessful(clientRepsonse);
     }
 
-	@Test
-	public void testClusterSubmitScheduleSuspendResumeDelete() throws Exception {
-		ClientResponse clientRepsonse;
+    @Test
+    public void testClusterSubmitScheduleSuspendResumeDelete() throws Exception {
+        ClientResponse clientRepsonse;
         Map<String, String> overlay = getUniqueOverlay();
 
-		clientRepsonse = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay,
-				EntityType.CLUSTER);
-		assertSuccessful(clientRepsonse);
+        clientRepsonse = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay,
+                EntityType.CLUSTER);
+        assertSuccessful(clientRepsonse);
 
-		clientRepsonse = this.service
-				.path("api/entities/schedule/cluster/" + clusterName)
+        clientRepsonse = this.service
+                .path("api/entities/schedule/cluster/" + clusterName)
                 .header("Remote-User", REMOTE_USER)
-				.accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
-				.post(ClientResponse.class);
-		assertFailure(clientRepsonse);
+                .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
+                .post(ClientResponse.class);
+        assertFailure(clientRepsonse);
 
-		clientRepsonse = this.service
-				.path("api/entities/suspend/cluster/" + clusterName)
+        clientRepsonse = this.service
+                .path("api/entities/suspend/cluster/" + clusterName)
                 .header("Remote-User", REMOTE_USER)
-				.accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
-				.post(ClientResponse.class);
-		assertFailure(clientRepsonse);
+                .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
+                .post(ClientResponse.class);
+        assertFailure(clientRepsonse);
 
-		clientRepsonse = this.service
-				.path("api/entities/resume/cluster/" + clusterName)
+        clientRepsonse = this.service
+                .path("api/entities/resume/cluster/" + clusterName)
                 .header("Remote-User", REMOTE_USER)
-				.accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
-				.post(ClientResponse.class);
-		assertFailure(clientRepsonse);
+                .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
+                .post(ClientResponse.class);
+        assertFailure(clientRepsonse);
 
-		clientRepsonse = this.service
-				.path("api/entities/delete/cluster/" + clusterName)
+        clientRepsonse = this.service
+                .path("api/entities/delete/cluster/" + clusterName)
                 .header("Remote-User", REMOTE_USER)
-				.accept(MediaType.TEXT_XML).delete(ClientResponse.class);
-		assertSuccessful(clientRepsonse);
-	}
+                .accept(MediaType.TEXT_XML).delete(ClientResponse.class);
+        assertSuccessful(clientRepsonse);
+    }
 
     @Test
     public void testSubmit() throws Exception {
@@ -378,7 +376,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
 
         String feedXML = response.getEntity(String.class);
         try {
-            Feed result = (Feed)unmarshaller.
+            Feed result = (Feed) unmarshaller.
                     unmarshal(new StringReader(feedXML));
             Assert.assertEquals(result.getName(), overlay.get("inputFeedName"));
         } catch (JAXBException e) {
@@ -398,7 +396,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
     @Test
     public void testScheduleSuspendResume() throws Exception {
         scheduleProcess();
-        
+
         ClientResponse clientRepsonse = this.service
                 .path("api/entities/suspend/process/" + processName)
                 .header("Remote-User", REMOTE_USER)
@@ -412,7 +410,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         assertSuccessful(clientRepsonse);
     }
 
-    @Test  (enabled = true)
+    @Test(enabled = true)
     public void testFeedSchedule() throws Exception {
         ClientResponse response;
         Map<String, String> overlay = getUniqueOverlay();
@@ -425,10 +423,10 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
 
         createTestData();
         ClientResponse clientRepsonse = this.service
-        		.path("api/entities/schedule/feed/" + overlay.get("inputFeedName"))
+                .path("api/entities/schedule/feed/" + overlay.get("inputFeedName"))
                 .header("Remote-User", REMOTE_USER)
-        		.accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
-        		.post(ClientResponse.class);
+                .accept(MediaType.TEXT_XML).type(MediaType.TEXT_XML)
+                .post(ClientResponse.class);
         assertSuccessful(clientRepsonse);
     }
 
@@ -479,7 +477,7 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         path = new Path("/examples/input-data/rawLogs/" + formatter.format(date) + "/file");
         list.add(path);
         fs.create(path).close();
-        new FsShell(conf).run(new String[] {"-chown", "-R", "guest:users", "/examples/input-data/rawLogs"});
+        new FsShell(conf).run(new String[]{"-chown", "-R", "guest:users", "/examples/input-data/rawLogs"});
         return list;
     }
 
@@ -557,26 +555,26 @@ public class EntityManagerJerseyTest extends AbstractTestBase{
         assertSuccessful(response);
 
     }
-    
+
     @Test
     public void testGetDependencies() throws Exception {
-    	ClientResponse response;
-    	response = this.service
+        ClientResponse response;
+        response = this.service
                 .path("api/entities/list/process/")
                 .header("Remote-User", REMOTE_USER).type(MediaType.TEXT_XML)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
-    	Assert.assertEquals(response.getStatus(), 200);
-    	
-    	Map<String, String> overlay = getUniqueOverlay();
+        Assert.assertEquals(response.getStatus(), 200);
+
+        Map<String, String> overlay = getUniqueOverlay();
 
         response = submitToFalcon(CLUSTER_FILE_TEMPLATE, overlay, EntityType.CLUSTER);
         assertSuccessful(response);
-        
+
         response = this.service
                 .path("api/entities/list/cluster/")
                 .header("Remote-User", REMOTE_USER).type(MediaType.TEXT_XML)
                 .accept(MediaType.TEXT_XML).get(ClientResponse.class);
         Assert.assertEquals(response.getStatus(), 200);
-        
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
index d5e861e..6ecc063 100644
--- a/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
+++ b/webapp/src/test/java/org/apache/falcon/resource/ProcessInstanceManagerTest.java
@@ -18,8 +18,6 @@
 
 package org.apache.falcon.resource;
 
-import javax.ws.rs.core.MediaType;
-
 import org.apache.falcon.Tag;
 import org.apache.falcon.entity.EntityUtil;
 import org.apache.falcon.entity.ExternalId;
@@ -34,9 +32,12 @@ import org.apache.oozie.client.WorkflowJob;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-@Test(enabled=false)
+import javax.ws.rs.core.MediaType;
+
+@Test(enabled = false)
 public class ProcessInstanceManagerTest extends AbstractTestBase {
     private static final String START_INSTANCE = "2012-04-20T00:00Z";
+
     protected void schedule() throws Exception {
         scheduleProcess();
         waitForProcessWFtoStart();
@@ -113,7 +114,7 @@ public class ProcessInstanceManagerTest extends AbstractTestBase {
 
     public void testResumesInstances() throws Exception {
         testSuspendInstances();
-        
+
         InstancesResult response = this.service.path("api/instance/resume/process/" + processName)
                 .queryParam("start", START_INSTANCE).header("Remote-User", "guest").accept(MediaType.APPLICATION_JSON)
                 .post(InstancesResult.class);
@@ -124,16 +125,18 @@ public class ProcessInstanceManagerTest extends AbstractTestBase {
 
         waitForWorkflow(START_INSTANCE, WorkflowJob.Status.RUNNING);
     }
-    
+
     private void waitForWorkflow(String instance, WorkflowJob.Status status) throws Exception {
         ExternalId extId = new ExternalId(processName, Tag.DEFAULT, EntityUtil.parseDateUTC(instance));
-        OozieClient ozClient = OozieClientFactory.get((Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, clusterName));
+        OozieClient ozClient = OozieClientFactory.get(
+                (Cluster) ConfigurationStore.get().get(EntityType.CLUSTER, clusterName));
         String jobId = ozClient.getJobId(extId.getId());
         WorkflowJob jobInfo = null;
         for (int i = 0; i < 15; i++) {
             jobInfo = ozClient.getJobInfo(jobId);
-            if (jobInfo.getStatus() == status)
+            if (jobInfo.getStatus() == status) {
                 break;
+            }
             System.out.println("Waiting for workflow job " + jobId + " status " + status);
             Thread.sleep((i + 1) * 1000);
         }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java b/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
index 3533aa2..238bfc4 100644
--- a/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
+++ b/webapp/src/test/java/org/apache/falcon/security/BasicAuthFilterTest.java
@@ -32,9 +32,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.core.Response;
 
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.*;
 
 public class BasicAuthFilterTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java b/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
index 61c9c3f..96d5c53 100644
--- a/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
+++ b/webapp/src/test/java/org/apache/falcon/util/ResourcesReflectionUtilTest.java
@@ -19,24 +19,32 @@
 package org.apache.falcon.util;
 
 import junit.framework.Assert;
-
 import org.testng.annotations.Test;
 
 public class ResourcesReflectionUtilTest {
-	
-	@Test
-	public void testMonitoredResources(){
-		Assert.assertEquals("submit",ResourcesReflectionUtil.getResourceMonitorName("SchedulableEntityManagerProxy.submit"));
-		Assert.assertEquals("entityType", ResourcesReflectionUtil.getResourceDimensionsName("SchedulableEntityManagerProxy.submit").get(1));
-		
-		Assert.assertEquals("submitAndSchedule",ResourcesReflectionUtil.getResourceMonitorName("SchedulableEntityManagerProxy.submitAndSchedule"));
-		Assert.assertEquals("entityType", ResourcesReflectionUtil.getResourceDimensionsName("SchedulableEntityManagerProxy.submit").get(1));
-		
-		Assert.assertEquals("kill-instance",ResourcesReflectionUtil.getResourceMonitorName("InstanceManagerProxy.killInstance"));
-		Assert.assertEquals("entityType", ResourcesReflectionUtil.getResourceDimensionsName("InstanceManagerProxy.killInstance").get(1));
-		
-		Assert.assertEquals("wf-instance-failed",ResourcesReflectionUtil.getResourceMonitorName("GenericAlert.instrumentFailedInstance"));
-		Assert.assertEquals(new Integer(10), ResourcesReflectionUtil.getResourceTimeTakenName("GenericAlert.instrumentFailedInstance"));
-		Assert.assertEquals(null, ResourcesReflectionUtil.getResourceTimeTakenName("GenericAlert.alertLateRerunFailed"));
-	}
+
+    @Test
+    public void testMonitoredResources() {
+        Assert.assertEquals("submit",
+                ResourcesReflectionUtil.getResourceMonitorName("SchedulableEntityManagerProxy.submit"));
+        Assert.assertEquals("entityType",
+                ResourcesReflectionUtil.getResourceDimensionsName("SchedulableEntityManagerProxy.submit").get(1));
+
+        Assert.assertEquals("submitAndSchedule",
+                ResourcesReflectionUtil.getResourceMonitorName("SchedulableEntityManagerProxy.submitAndSchedule"));
+        Assert.assertEquals("entityType",
+                ResourcesReflectionUtil.getResourceDimensionsName("SchedulableEntityManagerProxy.submit").get(1));
+
+        Assert.assertEquals("kill-instance",
+                ResourcesReflectionUtil.getResourceMonitorName("InstanceManagerProxy.killInstance"));
+        Assert.assertEquals("entityType",
+                ResourcesReflectionUtil.getResourceDimensionsName("InstanceManagerProxy.killInstance").get(1));
+
+        Assert.assertEquals("wf-instance-failed",
+                ResourcesReflectionUtil.getResourceMonitorName("GenericAlert.instrumentFailedInstance"));
+        Assert.assertEquals(new Integer(10),
+                ResourcesReflectionUtil.getResourceTimeTakenName("GenericAlert.instrumentFailedInstance"));
+        Assert.assertEquals(null,
+                ResourcesReflectionUtil.getResourceTimeTakenName("GenericAlert.alertLateRerunFailed"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/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 4bcad41..b3ba11b 100644
--- a/webapp/src/test/resources/cluster-template.xml
+++ b/webapp/src/test/resources/cluster-template.xml
@@ -17,24 +17,25 @@
   limitations under the License.
   -->
 
-<cluster colo="gs" description="" name="##cluster##" xmlns="uri:falcon:cluster:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-	<interfaces>
-		<interface type="readonly" endpoint="hftp://localhost:50010"
-			version="0.20.2" />
-		<interface type="write" endpoint="hdfs://localhost:8020"
-			version="0.20.2" />
-		<interface type="execute" endpoint="localhost:8021" version="0.20.2" />
-		<interface type="workflow" endpoint="http://localhost:11000/oozie/"
-			version="3.1" />
-		<interface type="messaging" endpoint="tcp://10.14.117.33:61616?daemon=true"
-			version="5.1.6" />
-		<interface type="registry" endpoint="Hcat" version="1" />
-	</interfaces>
-	<locations>
-		<location name="staging" path="/projects/falcon/staging" />
-		<location name="temp" path="/tmp" />
-		<location name="working" path="/project/falcon/working" />
-	</locations>
-	<properties>
-	</properties>
+<cluster colo="gs" description="" name="##cluster##" xmlns="uri:falcon:cluster:0.1"
+        >
+    <interfaces>
+        <interface type="readonly" endpoint="hftp://localhost:50010"
+                   version="0.20.2"/>
+        <interface type="write" endpoint="hdfs://localhost:8020"
+                   version="0.20.2"/>
+        <interface type="execute" endpoint="localhost:8021" version="0.20.2"/>
+        <interface type="workflow" endpoint="http://localhost:11000/oozie/"
+                   version="3.1"/>
+        <interface type="messaging" endpoint="tcp://10.14.117.33:61616?daemon=true"
+                   version="5.1.6"/>
+        <interface type="registry" endpoint="Hcat" version="1"/>
+    </interfaces>
+    <locations>
+        <location name="staging" path="/projects/falcon/staging"/>
+        <location name="temp" path="/tmp"/>
+        <location name="working" path="/project/falcon/working"/>
+    </locations>
+    <properties>
+    </properties>
 </cluster>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/feed-template1.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/feed-template1.xml b/webapp/src/test/resources/feed-template1.xml
index 3797df4..8d0c5e1 100644
--- a/webapp/src/test/resources/feed-template1.xml
+++ b/webapp/src/test/resources/feed-template1.xml
@@ -16,7 +16,8 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<feed description="clicks log" name="##inputFeedName##" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<feed description="clicks log" name="##inputFeedName##" xmlns="uri:falcon:feed:0.1"
+        >
     <partitions>
         <partition name="timestamp"/>
     </partitions>
@@ -24,21 +25,22 @@
 
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    <late-arrival cut-off="hours(6)" />
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="##cluster##" type="source">
             <validity start="2010-01-01T00:00Z" end="2012-04-21T00:00Z"/>
-            <retention limit="hours(24)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(24)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/falcon/test/input/${YEAR}/${MONTH}/${DAY}/${HOUR}" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/falcon/test/input/${YEAR}/${MONTH}/${DAY}/${HOUR}"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/feed-template2.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/feed-template2.xml b/webapp/src/test/resources/feed-template2.xml
index aa1f4ac..80a62ed 100644
--- a/webapp/src/test/resources/feed-template2.xml
+++ b/webapp/src/test/resources/feed-template2.xml
@@ -16,26 +16,28 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<feed description="clicks log" name="##outputFeedName##" xmlns="uri:falcon:feed:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<feed description="clicks log" name="##outputFeedName##" xmlns="uri:falcon:feed:0.1"
+        >
     <groups>output</groups>
 
     <frequency>days(1)</frequency>
     <timezone>UTC</timezone>
-    <late-arrival cut-off="hours(6)" />
+    <late-arrival cut-off="hours(6)"/>
 
     <clusters>
         <cluster name="##cluster##" type="source">
             <validity start="2010-01-01T01:00Z" end="##feedEndDate##"/>
-            <retention limit="hours(6)" action="delete" /> <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
+            <retention limit="hours(6)" action="delete"/>
+            <!-- Limit can be in Time or Instances 100, Action ENUM DELETE,ARCHIVE -->
         </cluster>
     </clusters>
 
     <locations>
-        <location type="data" path="/falcon/test/output/${YEAR}/${MONTH}/${DAY}" />
-        <location type="stats" path="/projects/falcon/clicksStats" />
-        <location type="meta" path="/projects/falcon/clicksMetaData" />
+        <location type="data" path="/falcon/test/output/${YEAR}/${MONTH}/${DAY}"/>
+        <location type="stats" path="/projects/falcon/clicksStats"/>
+        <location type="meta" path="/projects/falcon/clicksMetaData"/>
     </locations>
-    
-    <ACL owner="testuser" group="group" permission="0x755" />
-    <schema location="/schema/clicks" provider="protobuf" />
+
+    <ACL owner="testuser" group="group" permission="0x755"/>
+    <schema location="/schema/clicks" provider="protobuf"/>
 </feed>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/org/apache/falcon/logging/process.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/org/apache/falcon/logging/process.xml b/webapp/src/test/resources/org/apache/falcon/logging/process.xml
index 9b65800..ae8ba16 100644
--- a/webapp/src/test/resources/org/apache/falcon/logging/process.xml
+++ b/webapp/src/test/resources/org/apache/falcon/logging/process.xml
@@ -17,11 +17,11 @@
   limitations under the License.
   -->
 
-<process name="process-name" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="process-name" xmlns="uri:falcon:process:0.1">
     <!-- where -->
     <clusters>
         <cluster name="testCluster">
-            <validity start="2010-01-01T01:00Z" end="2010-01-01T01:01Z" />
+            <validity start="2010-01-01T01:00Z" end="2010-01-01T01:01Z"/>
         </cluster>
     </clusters>
 
@@ -30,13 +30,13 @@
     <order>LIFO</order>
     <frequency>minutes(1)</frequency>
     <timezone>UTC</timezone>
-   
+
     <properties>
-        <property name="queueName" value="default" />
+        <property name="queueName" value="default"/>
     </properties>
 
     <workflow path="/workflow/workflow.xml"/>
 
-    <retry policy="periodic" delay="minutes(2)" attempts="3" />
-    
+    <retry policy="periodic" delay="minutes(2)" attempts="3"/>
+
 </process>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/org/apache/falcon/logging/workflow.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/org/apache/falcon/logging/workflow.xml b/webapp/src/test/resources/org/apache/falcon/logging/workflow.xml
index 3ab3d29..4fc73d1 100644
--- a/webapp/src/test/resources/org/apache/falcon/logging/workflow.xml
+++ b/webapp/src/test/resources/org/apache/falcon/logging/workflow.xml
@@ -17,31 +17,31 @@
   -->
 
 <workflow-app xmlns="uri:oozie:workflow:0.2" name="test-wf">
-	<start to="fsAction" />
-	<action name="fsAction">
-		<fs>
-			<delete path="${nameNode}/test/dir1" />
-			<mkdir path="${nameNode}/test/dir1" />
-		</fs>
-		<ok to="javaAction" />
-		<error to="fail" />
-	</action>
-	<action name="javaAction">
-		<java>
-			<job-tracker>${jobTracker}</job-tracker>
-			<name-node>${nameNode}</name-node>
-			<main-class>org.apache.oozie.example.DemoJavaMain</main-class>
-			<arg>Hello</arg>
-			<arg>Oozie!</arg>
-		</java>
+    <start to="fsAction"/>
+    <action name="fsAction">
+        <fs>
+            <delete path="${nameNode}/test/dir1"/>
+            <mkdir path="${nameNode}/test/dir1"/>
+        </fs>
+        <ok to="javaAction"/>
+        <error to="fail"/>
+    </action>
+    <action name="javaAction">
+        <java>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <main-class>org.apache.oozie.example.DemoJavaMain</main-class>
+            <arg>Hello</arg>
+            <arg>Oozie!</arg>
+        </java>
         <ok to="end"/>
         <error to="fail"/>
-	</action>
-	<kill name="fail">
-		<message>Job failed, error
-			message[${wf:errorMessage(wf:lastErrorNode())}]
-		</message>
-	</kill>
-	<end name="end" />
+    </action>
+    <kill name="fail">
+        <message>Job failed, error
+            message[${wf:errorMessage(wf:lastErrorNode())}]
+        </message>
+    </kill>
+    <end name="end"/>
 
 </workflow-app>

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/process-template.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/process-template.xml b/webapp/src/test/resources/process-template.xml
index ff10889..716a458 100644
--- a/webapp/src/test/resources/process-template.xml
+++ b/webapp/src/test/resources/process-template.xml
@@ -17,10 +17,10 @@
   limitations under the License.
   -->
 
-<process name="##processName##" xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="##processName##" xmlns="uri:falcon:process:0.1">
     <clusters>
         <cluster name="##cluster##">
-            <validity end="2012-04-21T00:00Z" start="2012-04-20T00:00Z" />
+            <validity end="2012-04-21T00:00Z" start="2012-04-20T00:00Z"/>
         </cluster>
     </clusters>
 
@@ -28,22 +28,22 @@
     <order>FIFO</order>
     <frequency>days(1)</frequency>
     <timezone>UTC</timezone>
-    
+
     <inputs>
         <input end="today(0,0)" start="today(0,0)" feed="##inputFeedName##" name="input" partition="${fileTime}"/>
     </inputs>
     <outputs>
-        <output instance="now(0,0)" feed="##outputFeedName##" name="output" />
+        <output instance="now(0,0)" feed="##outputFeedName##" name="output"/>
     </outputs>
     <properties>
         <property name="fileTime" value="${formatTime(dateOffset(instanceTime(), 1, 'DAY'), 'yyyy-MMM-dd')}"/>
         <property name="user" value="${user()}"/>
         <property name="baseTime" value="${today(0,0)}"/>
     </properties>
-    <workflow engine="oozie" path="/falcon/test/workflow" />
-    <retry policy="periodic" delay="minutes(10)" attempts="3" />
+    <workflow engine="oozie" path="/falcon/test/workflow"/>
+    <retry policy="periodic" delay="minutes(10)" attempts="3"/>
 
     <late-process policy="exp-backoff" delay="hours(1)">
-        <late-input input="input" workflow-path="/falcon/test/workflow" />
+        <late-input input="input" workflow-path="/falcon/test/workflow"/>
     </late-process>
 </process>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/a4d79f0c/webapp/src/test/resources/process-version-0.xml
----------------------------------------------------------------------
diff --git a/webapp/src/test/resources/process-version-0.xml b/webapp/src/test/resources/process-version-0.xml
index a9b5557..3ac7ac8 100644
--- a/webapp/src/test/resources/process-version-0.xml
+++ b/webapp/src/test/resources/process-version-0.xml
@@ -16,12 +16,12 @@
   See the License for the specific language governing permissions and
   limitations under the License.
   -->
-<process name="sample"  xmlns="uri:falcon:process:0.1" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+<process name="sample" xmlns="uri:falcon:process:0.1">
     <!-- where -->
     <clusters>
-    <cluster name="testCluster">
-    <validity start="2011-11-01T00:00Z" end="2011-12-31T23:59Z"/>
-    </cluster>
+        <cluster name="testCluster">
+            <validity start="2011-11-01T00:00Z" end="2011-12-31T23:59Z"/>
+        </cluster>
     </clusters>
 
     <!-- when -->
@@ -29,30 +29,30 @@
     <order>LIFO</order>
     <frequency>hours(1)</frequency>
     <timezone>UTC</timezone>
-    
+
     <!-- what -->
     <inputs>
-        <input name="input1" feed="inputFeed1" start="$ptime-6" end="$ptime" /> 
-        <input name="input2" feed="inputFeed2" start="$ptime" end="$ptime" />
+        <input name="input1" feed="inputFeed1" start="$ptime-6" end="$ptime"/>
+        <input name="input2" feed="inputFeed2" start="$ptime" end="$ptime"/>
     </inputs>
 
     <outputs>
-        <output name="output1" feed="outputFeed1" instance="$ptime" />
-        <output name="output2" feed="outputFeed2" instance="$ptime" />
+        <output name="output1" feed="outputFeed1" instance="$ptime"/>
+        <output name="output2" feed="outputFeed2" instance="$ptime"/>
     </outputs>
 
     <!-- how -->
     <properties>
-        <property name="name" value="value" />
-        <property name="name" value="value" />
+        <property name="name" value="value"/>
+        <property name="name" value="value"/>
     </properties>
 
-    <workflow engine="oozie" path="hdfs://path/to/workflow" />
+    <workflow engine="oozie" path="hdfs://path/to/workflow"/>
 
-    <retry policy="periodic" delay="minutes(10)" attempts="3" />
+    <retry policy="periodic" delay="minutes(10)" attempts="3"/>
 
     <late-process policy="exp-backoff" delay="hours(1)">
-        <late-input input="impression" workflow-path="/impression/late/workflow" />
-        <late-input input="clicks" workflow-path="/clicks/late/workflow" />
+        <late-input input="impression" workflow-path="/impression/late/workflow"/>
+        <late-input input="clicks" workflow-path="/clicks/late/workflow"/>
     </late-process>
 </process>


[43/47] git commit: checkstyle related fixes for replication module.

Posted by sr...@apache.org.
checkstyle related fixes for replication module.


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

Branch: refs/heads/master
Commit: dbd980ea4f97fce2182689d12667cdb04915388e
Parents: fcfd9b3
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:26:55 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:26:55 2013 -0700

----------------------------------------------------------------------
 .../falcon/replication/CustomReplicator.java       |    7 ++-
 .../apache/falcon/replication/FeedReplicator.java  |    8 ++-
 .../falcon/replication/FilteredCopyListing.java    |   15 +++--
 .../falcon/repliation/FeedReplicatorTest.java      |   49 --------------
 .../falcon/replication/FeedReplicatorTest.java     |   52 +++++++++++++++
 .../replication/FilteredCopyListingTest.java       |   11 ++-
 6 files changed, 80 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java b/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
index 7dfd406..ba01c80 100644
--- a/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
+++ b/replication/src/main/java/org/apache/falcon/replication/CustomReplicator.java
@@ -27,9 +27,14 @@ import org.apache.log4j.Logger;
 
 import java.io.IOException;
 
+/**
+ * A custom implementation of DistCp that overrides the behavior of CopyListing
+ * interface to copy FileOutputCommitter.SUCCEEDED_FILE_NAME last so downstream apps
+ * depending on data availability will work correctly.
+ */
 public class CustomReplicator extends DistCp {
 
-    private static Logger LOG = Logger.getLogger(CustomReplicator.class);
+    private static final Logger LOG = Logger.getLogger(CustomReplicator.class);
 
     /**
      * Public Constructor. Creates DistCp object with specified input-parameters.

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
index fc0b5ac..efb5691 100644
--- a/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
+++ b/replication/src/main/java/org/apache/falcon/replication/FeedReplicator.java
@@ -35,9 +35,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Pattern;
 
+/**
+ * A tool for feed replication that uses DistCp tool to replicate.
+ */
 public class FeedReplicator extends Configured implements Tool {
 
-    private static Logger LOG = Logger.getLogger(FeedReplicator.class);
+    private static final Logger LOG = Logger.getLogger(FeedReplicator.class);
 
     public static void main(String[] args) throws Exception {
         ToolRunner.run(new Configuration(), new FeedReplicator(), args);
@@ -82,8 +85,7 @@ public class FeedReplicator extends Configured implements Tool {
             }
         } else {
             LOG.info("No files present in path: "
-                    + new Path(targetPath.toString() + "/" + fixedPath)
-                    .toString());
+                    + new Path(targetPath.toString() + "/" + fixedPath).toString());
         }
         LOG.info("Completed DistCp");
         return 0;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
----------------------------------------------------------------------
diff --git a/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java b/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
index c1698e3..9c97a44 100644
--- a/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
+++ b/replication/src/main/java/org/apache/falcon/replication/FilteredCopyListing.java
@@ -29,6 +29,11 @@ import org.apache.log4j.Logger;
 import java.io.IOException;
 import java.util.regex.Pattern;
 
+/**
+ * An implementation of CopyListing that overrides the default behavior by suppressing file,
+ * FileOutputCommitter.SUCCEEDED_FILE_NAME and copies that in the last so downstream apps
+ * depending on data availability will work correctly.
+ */
 public class FilteredCopyListing extends SimpleCopyListing {
     private static final Logger LOG = Logger.getLogger(FilteredCopyListing.class);
 
@@ -54,8 +59,8 @@ public class FilteredCopyListing extends SimpleCopyListing {
             LOG.info("Inclusion pattern = " + configuration.get("falcon.include.path"));
             LOG.info("Regex pattern = " + regex);
         } catch (IOException e) {
-            throw new IllegalArgumentException("Unable to build regex for " +
-                    configuration.get("falcon.include.path", ""));
+            throw new IllegalArgumentException("Unable to build regex for "
+                    + configuration.get("falcon.include.path", ""));
         }
     }
 
@@ -68,8 +73,8 @@ public class FilteredCopyListing extends SimpleCopyListing {
     }
 
     private static boolean isJavaRegexSpecialChar(char pChar) {
-        return pChar == '.' || pChar == '$' || pChar == '(' || pChar == ')' ||
-                pChar == '|' || pChar == '+';
+        return pChar == '.' || pChar == '$' || pChar == '(' || pChar == ')'
+                || pChar == '|' || pChar == '+';
     }
 
     public static Pattern getRegEx(String filePattern) throws IOException {
@@ -122,7 +127,7 @@ public class FilteredCopyListing extends SimpleCopyListing {
                 pCh = ')';
             } else if (pCh == '[' && setOpen == 0) {
                 setOpen++;
-            } else if (pCh == '^' && setOpen > 0) {
+            // } else if (pCh == '^' && setOpen > 0) {
             } else if (pCh == '-' && setOpen > 0) {
                 // Character set range
                 setRange = true;

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
----------------------------------------------------------------------
diff --git a/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java b/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
deleted file mode 100644
index 1935e51..0000000
--- a/replication/src/test/java/org/apache/falcon/repliation/FeedReplicatorTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.falcon.repliation;
-
-import org.apache.falcon.replication.FeedReplicator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.tools.DistCpOptions;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class FeedReplicatorTest {
-
-    @Test
-    public void testArguments() throws Exception {
-        /*
-         * <arg>-update</arg>
-         * <arg>-blocking</arg><arg>true</arg> <arg>-maxMaps</arg><arg>20</arg>
-         * <arg>-sourcePaths</arg><arg>${distcpSourcePaths}</arg>
-         * <arg>-targetPath</arg><arg>${distcpTargetPaths}</arg>
-         */
-        FeedReplicator replicator = new FeedReplicator();
-        DistCpOptions options = replicator.getDistCpOptions(new String[]{"true", "-maxMaps", "5", "-sourcePaths",
-                                                                         "hdfs://localhost:8020/tmp/", "-targetPath",
-                                                                         "hdfs://localhost1:8020/tmp/"});
-
-        List<Path> srcPaths = new ArrayList<Path>();
-        srcPaths.add(new Path("hdfs://localhost:8020/tmp/"));
-        Assert.assertEquals(options.getSourcePaths(), srcPaths);
-        Assert.assertEquals(options.getTargetPath(), new Path("hdfs://localhost1:8020/tmp/"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
----------------------------------------------------------------------
diff --git a/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java b/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
new file mode 100644
index 0000000..67795b5
--- /dev/null
+++ b/replication/src/test/java/org/apache/falcon/replication/FeedReplicatorTest.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.falcon.replication;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test class for FeedReplicator.
+ */
+public class FeedReplicatorTest {
+
+    @Test
+    public void testArguments() throws Exception {
+        /*
+         * <arg>-update</arg>
+         * <arg>-blocking</arg><arg>true</arg> <arg>-maxMaps</arg><arg>20</arg>
+         * <arg>-sourcePaths</arg><arg>${distcpSourcePaths}</arg>
+         * <arg>-targetPath</arg><arg>${distcpTargetPaths}</arg>
+         */
+        FeedReplicator replicator = new FeedReplicator();
+        DistCpOptions options = replicator.getDistCpOptions(new String[] {
+            "true", "-maxMaps", "5", "-sourcePaths",
+            "hdfs://localhost:8020/tmp/", "-targetPath",
+            "hdfs://localhost1:8020/tmp/", });
+
+        List<Path> srcPaths = new ArrayList<Path>();
+        srcPaths.add(new Path("hdfs://localhost:8020/tmp/"));
+        Assert.assertEquals(options.getSourcePaths(), srcPaths);
+        Assert.assertEquals(options.getTargetPath(), new Path("hdfs://localhost1:8020/tmp/"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/dbd980ea/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
----------------------------------------------------------------------
diff --git a/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java b/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
index 5054bf8..e308866 100644
--- a/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
+++ b/replication/src/test/java/org/apache/falcon/replication/FilteredCopyListingTest.java
@@ -38,11 +38,14 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
+/**
+ * Test class for org.apache.falcon.replication.FilteredCopyListing.
+ */
 public class FilteredCopyListingTest {
 
     private static final Credentials CREDENTIALS = new Credentials();
 
-    public static Map<String, String> expectedValues = new HashMap<String, String>();
+    public static final Map<String, String> EXPECTED_VALUES = new HashMap<String, String>();
 
     @BeforeClass
     public static void setup() throws Exception {
@@ -101,7 +104,7 @@ public class FilteredCopyListingTest {
     private static void recordInExpectedValues(String path) throws Exception {
         FileSystem fileSystem = FileSystem.getLocal(new Configuration());
         Path sourcePath = new Path(fileSystem.getUri().toString() + path);
-        expectedValues.put(sourcePath.toString(), DistCpUtils.getRelativePath(
+        EXPECTED_VALUES.put(sourcePath.toString(), DistCpUtils.getRelativePath(
                 new Path("/tmp/source"), sourcePath));
     }
 
@@ -215,9 +218,9 @@ public class FilteredCopyListingTest {
             actualValues.put(value.getPath().toString(), key.toString());
         }
 
-        Assert.assertEquals(expected == -1 ? expectedValues.size() : expected, actualValues.size());
+        Assert.assertEquals(expected == -1 ? EXPECTED_VALUES.size() : expected, actualValues.size());
         for (Map.Entry<String, String> entry : actualValues.entrySet()) {
-            Assert.assertEquals(entry.getValue(), expectedValues.get(entry.getKey()));
+            Assert.assertEquals(entry.getValue(), EXPECTED_VALUES.get(entry.getKey()));
         }
     }
 }


[42/47] git commit: checkstyle related fixes for process module.

Posted by sr...@apache.org.
checkstyle related fixes for process module.


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

Branch: refs/heads/master
Commit: fcfd9b36c88bf375ba8c9d25fa85ea686be8839b
Parents: f8b9122
Author: venkatesh <ve...@hortonworks.com>
Authored: Tue Apr 23 13:26:33 2013 -0700
Committer: venkatesh <ve...@hortonworks.com>
Committed: Tue Apr 23 13:26:33 2013 -0700

----------------------------------------------------------------------
 .../falcon/converter/OozieProcessMapper.java       |   35 +++++++------
 .../workflow/OozieProcessWorkflowBuilder.java      |   33 ++++++-----
 .../apache/falcon/converter/AbstractTestBase.java  |    2 +-
 .../OozieProcessMapperLateProcessTest.java         |   42 +++++++--------
 .../falcon/converter/OozieProcessMapperTest.java   |    5 ++-
 5 files changed, 62 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fcfd9b36/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java b/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
index 4e90792..592f089 100644
--- a/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
+++ b/process/src/main/java/org/apache/falcon/converter/OozieProcessMapper.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -44,6 +44,9 @@ import org.apache.hadoop.fs.Path;
 
 import java.util.*;
 
+/**
+ * This class maps the Falcon entities into Oozie artifacts.
+ */
 public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
 
     private static final String DEFAULT_WF_TEMPLATE = "/config/workflow/process-parent-workflow.xml";
@@ -76,7 +79,7 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
     }
 
     /**
-     * Creates default oozie coordinator
+     * Creates default oozie coordinator.
      *
      * @param cluster    - Cluster for which the coordiantor app need to be created
      * @param bundlePath - bundle path
@@ -108,21 +111,20 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         controls.setExecution(process.getOrder().name());
 
         Frequency timeout = process.getTimeout();
-        long frequency_ms = ExpressionHelper.get().
-                evaluate(process.getFrequency().toString(), Long.class);
-        long timeout_ms;
+        long frequencyInMillis = ExpressionHelper.get().evaluate(process.getFrequency().toString(), Long.class);
+        long timeoutInMillis;
         if (timeout != null) {
-            timeout_ms = ExpressionHelper.get().
+            timeoutInMillis = ExpressionHelper.get().
                     evaluate(process.getTimeout().toString(), Long.class);
         } else {
-            timeout_ms = frequency_ms * 6;
-            if (timeout_ms < THIRTY_MINUTES) {
-                timeout_ms = THIRTY_MINUTES;
+            timeoutInMillis = frequencyInMillis * 6;
+            if (timeoutInMillis < THIRTY_MINUTES) {
+                timeoutInMillis = THIRTY_MINUTES;
             }
         }
-        controls.setTimeout(String.valueOf(timeout_ms / (1000 * 60)));
-        if (timeout_ms / frequency_ms * 2 > 0) {
-            controls.setThrottle(String.valueOf(timeout_ms / frequency_ms * 2));
+        controls.setTimeout(String.valueOf(timeoutInMillis / (1000 * 60)));
+        if (timeoutInMillis / frequencyInMillis * 2 > 0) {
+            controls.setThrottle(String.valueOf(timeoutInMillis / frequencyInMillis * 2));
         }
         coord.setControls(controls);
 
@@ -229,10 +231,10 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         return datain;
     }
 
+    //SUSPEND CHECKSTYLE CHECK VisibilityModifierCheck
     private void createOutputEvent(String feed, String name, Cluster cluster,
                                    String type, LocationType locType, COORDINATORAPP coord,
-                                   Map<String, String> props, String instance)
-            throws FalconException {
+                                   Map<String, String> props, String instance) throws FalconException {
         SYNCDATASET dataset = createDataSet(feed, cluster, name + type,
                 locType);
         coord.getDatasets().getDatasetOrAsyncDataset().add(dataset);
@@ -247,6 +249,7 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         String outputExpr = "${coord:dataOut('" + name + type + "')}";
         props.put(name + "." + type, outputExpr);
     }
+    //RESUME CHECKSTYLE CHECK VisibilityModifierCheck
 
     private String join(Iterator<String> itr, char sep) {
         String joinedStr = StringUtils.join(itr, sep);
@@ -256,8 +259,8 @@ public class OozieProcessMapper extends AbstractOozieEntityMapper<Process> {
         return joinedStr;
     }
 
-    private SYNCDATASET createDataSet(String feedName, Cluster cluster, String datasetName, LocationType locationType)
-            throws FalconException {
+    private SYNCDATASET createDataSet(String feedName, Cluster cluster, String datasetName,
+                                      LocationType locationType) throws FalconException {
         Feed feed = (Feed) EntityUtil.getEntity(EntityType.FEED, feedName);
 
         SYNCDATASET syncdataset = new SYNCDATASET();

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fcfd9b36/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
index e422ec4..3f70557 100644
--- a/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
+++ b/process/src/main/java/org/apache/falcon/workflow/OozieProcessWorkflowBuilder.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -41,6 +41,9 @@ import org.apache.oozie.client.OozieClient;
 
 import java.util.*;
 
+/**
+ * Oozie workflow builder for falcon entities.
+ */
 public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
 
     @Override
@@ -59,7 +62,7 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
     }
 
     private void addOptionalInputProperties(Properties properties, Input in, String clusterName)
-            throws FalconException {
+        throws FalconException {
         Feed feed = EntityUtil.getEntity(EntityType.FEED, in.getFeed());
         org.apache.falcon.entity.v0.feed.Cluster cluster = FeedHelper.getCluster(feed, clusterName);
         String inName = in.getName();
@@ -80,28 +83,28 @@ public class OozieProcessWorkflowBuilder extends OozieWorkflowBuilder<Process> {
 
     private Timeunit mapToCoordTimeUnit(TimeUnit tu) {
         switch (tu) {
-            case days:
-                return Timeunit.DAY;
+        case days:
+            return Timeunit.DAY;
+
+        case hours:
+            return Timeunit.HOUR;
 
-            case hours:
-                return Timeunit.HOUR;
+        case minutes:
+            return Timeunit.MINUTE;
 
-            case minutes:
-                return Timeunit.MINUTE;
+        case months:
+            return Timeunit.MONTH;
 
-            case months:
-                return Timeunit.MONTH;
+        default:
+            throw new IllegalArgumentException("Unhandled time unit " + tu);
         }
-        throw new IllegalArgumentException("Unhandled time unit " + tu);
     }
 
     @Override
     public Properties newWorkflowSchedule(Process process, Date startDate, String clusterName, String user)
-            throws FalconException {
+        throws FalconException {
         org.apache.falcon.entity.v0.process.Cluster processCluster = ProcessHelper.getCluster(process, clusterName);
-        if (!startDate.before(processCluster.getValidity().getEnd()))
-        // start time >= end time
-        {
+        if (!startDate.before(processCluster.getValidity().getEnd())) {// start time >= end time
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fcfd9b36/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
index 4a0f7c4..9cc02df 100644
--- a/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
+++ b/process/src/test/java/org/apache/falcon/converter/AbstractTestBase.java
@@ -34,7 +34,6 @@ public class AbstractTestBase {
     private static final String FEED_XML = "/config/feed/feed-0.1.xml";
     private static final String CLUSTER_XML = "/config/cluster/cluster-0.1.xml";
 
-
     private void storeEntity(EntityType type, String name) throws Exception {
         Unmarshaller unmarshaller = type.getUnmarshaller();
         ConfigurationStore store = ConfigurationStore.get();
@@ -57,6 +56,7 @@ public class AbstractTestBase {
             process.setName(name);
             store.publish(type, process);
             break;
+
         default:
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fcfd9b36/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 d7fb4b0..34bfe7f 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperLateProcessTest.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -30,18 +30,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 
+/**
+ * Test class for late data processing.
+ */
 public class OozieProcessMapperLateProcessTest {
 
-    private static String hdfsUrl;
     private static final String CLUSTER_XML = "/config/late/late-cluster.xml";
     private static final String FEED1_XML = "/config/late/late-feed1.xml";
     private static final String FEED2_XML = "/config/late/late-feed2.xml";
     private static final String FEED3_XML = "/config/late/late-feed3.xml";
     private static final String PROCESS1_XML = "/config/late/late-process1.xml";
     private static final String PROCESS2_XML = "/config/late/late-process2.xml";
-    private static final ConfigurationStore store = ConfigurationStore.get();
+    private static final ConfigurationStore STORE = ConfigurationStore.get();
+
     private static EmbeddedCluster dfsCluster;
-    private static Configuration conf;
 
     @BeforeClass
     public void setUpDFS() throws Exception {
@@ -49,14 +51,14 @@ public class OozieProcessMapperLateProcessTest {
         cleanupStore();
 
         dfsCluster = EmbeddedCluster.newCluster("testCluster", false);
-        conf = dfsCluster.getConf();
-        hdfsUrl = conf.get("fs.default.name");
+        Configuration conf = dfsCluster.getConf();
+        String hdfsUrl = conf.get("fs.default.name");
 
         Cluster cluster = (Cluster) EntityType.CLUSTER.getUnmarshaller()
                 .unmarshal(this.getClass().getResource(CLUSTER_XML));
         ClusterHelper.getInterface(cluster, Interfacetype.WRITE).setEndpoint(hdfsUrl);
 
-        store.publish(EntityType.CLUSTER, cluster);
+        STORE.publish(EntityType.CLUSTER, cluster);
 
         Feed feed1 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
                 this.getClass().getResource(FEED1_XML));
@@ -65,34 +67,30 @@ public class OozieProcessMapperLateProcessTest {
         Feed feed3 = (Feed) EntityType.FEED.getUnmarshaller().unmarshal(
                 this.getClass().getResource(FEED3_XML));
 
-        store.publish(EntityType.FEED, feed1);
-        store.publish(EntityType.FEED, feed2);
-        store.publish(EntityType.FEED, feed3);
+        STORE.publish(EntityType.FEED, feed1);
+        STORE.publish(EntityType.FEED, feed2);
+        STORE.publish(EntityType.FEED, feed3);
 
         Process process1 = (Process) EntityType.PROCESS.getUnmarshaller()
                 .unmarshal(this.getClass().getResource(PROCESS1_XML));
-        store.publish(EntityType.PROCESS, process1);
+        STORE.publish(EntityType.PROCESS, process1);
         Process process2 = (Process) EntityType.PROCESS.getUnmarshaller()
                 .unmarshal(this.getClass().getResource(PROCESS2_XML));
-        store.publish(EntityType.PROCESS, process2);
-
+        STORE.publish(EntityType.PROCESS, process2);
     }
 
     private void cleanupStore() throws FalconException {
-        store.remove(EntityType.PROCESS, "late-process1");
-        store.remove(EntityType.PROCESS, "late-process2");
-        store.remove(EntityType.FEED, "late-feed1");
-        store.remove(EntityType.FEED, "late-feed2");
-        store.remove(EntityType.FEED, "late-feed3");
-        store.remove(EntityType.CLUSTER, "late-cluster");
-
+        STORE.remove(EntityType.PROCESS, "late-process1");
+        STORE.remove(EntityType.PROCESS, "late-process2");
+        STORE.remove(EntityType.FEED, "late-feed1");
+        STORE.remove(EntityType.FEED, "late-feed2");
+        STORE.remove(EntityType.FEED, "late-feed3");
+        STORE.remove(EntityType.CLUSTER, "late-cluster");
     }
 
-
     @AfterClass
     public void tearDown() throws Exception {
         cleanupStore();
         dfsCluster.shutdown();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-falcon/blob/fcfd9b36/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 3106920..9ed7baf 100644
--- a/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
+++ b/process/src/test/java/org/apache/falcon/converter/OozieProcessMapperTest.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -62,6 +62,9 @@ import java.io.InputStreamReader;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertTrue;
 
+/**
+ * Test for the Falcon entities mapping into Oozie artifacts.
+ */
 public class OozieProcessMapperTest extends AbstractTestBase {
 
     private String hdfsUrl;