You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by ry...@apache.org on 2013/06/13 01:17:37 UTC

svn commit: r1492478 [2/3] - in /oozie/trunk: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/action/oozie/ core/src/main/java/org/apache/oozie/client/rest/ core/src/main/java/org/apache/oozie/client/rest/sla/ core/src/main/...

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java Wed Jun 12 23:17:31 2013
@@ -175,7 +175,7 @@ public class V1JobsServlet extends BaseJ
             }
             boolean startJob = (action != null);
             String user = conf.get(OozieClient.USER_NAME);
-            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user, getAuthToken(request));
+            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user);
             String id;
             boolean dryrun = false;
             if (action != null) {
@@ -214,7 +214,7 @@ public class V1JobsServlet extends BaseJ
             boolean startJob = (action != null);
             String user = conf.get(OozieClient.USER_NAME);
             CoordinatorEngine coordEngine = Services.get().get(CoordinatorEngineService.class).getCoordinatorEngine(
-                    user, getAuthToken(request));
+                    user);
             String id = null;
             boolean dryrun = false;
             if (action != null) {
@@ -251,8 +251,7 @@ public class V1JobsServlet extends BaseJ
             }
             boolean startJob = (action != null);
             String user = conf.get(OozieClient.USER_NAME);
-            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(user,
-                    getAuthToken(request));
+            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(user);
             String id = null;
             boolean dryrun = false;
             if (action != null) {
@@ -281,8 +280,7 @@ public class V1JobsServlet extends BaseJ
             throws XServletException {
         JSONObject json = new JSONObject();
         try {
-            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request),
-                    getAuthToken(request));
+            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request));
             String jobId = dagEngine.getJobIdForExternalId(externalId);
             json.put(JsonTags.JOB_ID, jobId);
         }
@@ -317,8 +315,7 @@ public class V1JobsServlet extends BaseJ
             start = (start < 1) ? 1 : start;
             int len = (lenStr != null) ? Integer.parseInt(lenStr) : 50;
             len = (len < 1) ? 50 : len;
-            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request),
-                    getAuthToken(request));
+            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request));
             WorkflowsInfo jobs = dagEngine.getJobs(filter, start, len);
             List<WorkflowJobBean> jsonWorkflows = jobs.getWorkflows();
             json.put(JsonTags.WORKFLOWS_JOBS, WorkflowJobBean.toJSONArray(jsonWorkflows, timeZoneId));
@@ -352,7 +349,7 @@ public class V1JobsServlet extends BaseJ
             int len = (lenStr != null) ? Integer.parseInt(lenStr) : 50;
             len = (len < 1) ? 50 : len;
             CoordinatorEngine coordEngine = Services.get().get(CoordinatorEngineService.class).getCoordinatorEngine(
-                    getUser(request), getAuthToken(request));
+                    getUser(request));
             CoordinatorJobInfo jobs = coordEngine.getCoordJobs(filter, start, len);
             List<CoordinatorJobBean> jsonJobs = jobs.getCoordJobs();
             json.put(JsonTags.COORDINATOR_JOBS, CoordinatorJobBean.toJSONArray(jsonJobs, timeZoneId));
@@ -381,8 +378,7 @@ public class V1JobsServlet extends BaseJ
             int len = (lenStr != null) ? Integer.parseInt(lenStr) : 50;
             len = (len < 1) ? 50 : len;
 
-            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(getUser(request),
-                    getAuthToken(request));
+            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(getUser(request));
             BundleJobInfo jobs = bundleEngine.getBundleJobs(filter, start, len);
             List<BundleJobBean> jsonJobs = jobs.getBundleJobs();
 
@@ -412,8 +408,7 @@ public class V1JobsServlet extends BaseJ
             int len = (lenStr != null) ? Integer.parseInt(lenStr) : 50;
             len = (len < 1) ? 50 : len;
 
-            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(getUser(request),
-                    getAuthToken(request));
+            BundleEngine bundleEngine = Services.get().get(BundleEngineService.class).getBundleEngine(getUser(request));
             BulkResponseInfo bulkResponse = bundleEngine.getBulkJobs(bulkFilter, start, len);
             List<BulkResponseImpl> responsesToJson = bulkResponse.getResponses();
 
@@ -438,7 +433,7 @@ public class V1JobsServlet extends BaseJ
 
         try {
             String user = conf.get(OozieClient.USER_NAME);
-            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user, getAuthToken(request));
+            DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(user);
             String id = dagEngine.submitHttpJob(conf, jobType);
             json.put(JsonTags.JOB_ID, id);
         }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V2JobServlet.java Wed Jun 12 23:17:31 2013
@@ -55,8 +55,7 @@ public class V2JobServlet extends V1JobS
             IOException {
         String topicName;
         String jobId = getResourceName(request);
-        DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request),
-                getAuthToken(request));
+        DagEngine dagEngine = Services.get().get(DagEngineService.class).getDagEngine(getUser(request));
         try {
             topicName = dagEngine.getJMSTopicName(jobId);
         }

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/sla/SLASummaryBean.java Wed Jun 12 23:17:31 2013
@@ -76,7 +76,7 @@ public class SLASummaryBean implements J
     private String appType;
 
     @Basic
-    @Column(name = "user")
+    @Column(name = "user_name")
     private String user;
 
     @Basic

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/store/CoordinatorStore.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/store/CoordinatorStore.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/store/CoordinatorStore.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/store/CoordinatorStore.java Wed Jun 12 23:17:31 2013
@@ -454,7 +454,6 @@ public class CoordinatorStore extends St
         q.setParameter("lastActionNumber", jBean.getLastActionNumber());
         q.setParameter("timeOut", jBean.getTimeout());
         q.setParameter("timeZone", jBean.getTimeZone());
-        q.setParameter("authToken", jBean.getAuthToken());
         q.setParameter("createdTime", jBean.getCreatedTimestamp());
         q.setParameter("endTime", jBean.getEndTimestamp());
         q.setParameter("execution", jBean.getExecution());

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/store/WorkflowStore.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/store/WorkflowStore.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/store/WorkflowStore.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/store/WorkflowStore.java Wed Jun 12 23:17:31 2013
@@ -866,7 +866,6 @@ public class WorkflowStore extends Store
         wfBean.setGroup(w.getGroup());
         wfBean.setRun(w.getRun());
         wfBean.setUser(w.getUser());
-        wfBean.setAuthToken(w.getAuthToken());
         wfBean.setCreatedTime(w.getCreatedTime());
         wfBean.setEndTime(w.getEndTime());
         wfBean.setExternalId(w.getExternalId());
@@ -960,7 +959,6 @@ public class WorkflowStore extends Store
         q.setParameter("groupName", wfBean.getGroup());
         q.setParameter("run", wfBean.getRun());
         q.setParameter("user", wfBean.getUser());
-        q.setParameter("authToken", wfBean.getAuthToken());
         q.setParameter("createdTime", wfBean.getCreatedTimestamp());
         q.setParameter("endTime", wfBean.getEndTimestamp());
         q.setParameter("externalId", wfBean.getExternalId());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleJobBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleJobBean.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleJobBean.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestBundleJobBean.java Wed Jun 12 23:17:31 2013
@@ -42,7 +42,6 @@ public class TestBundleJobBean extends X
     private void set(BundleJobBean bean) {
         bean.setAppName("an");
         bean.setAppPath("ap");
-        bean.setAuthToken("at");
         bean.setConf("c");
         bean.setConsoleUrl("cu");
         bean.setCreatedTime(ct);
@@ -82,9 +81,6 @@ public class TestBundleJobBean extends X
     private void _testGet(BundleJobBean bean, boolean checkAllFields) {
         assertEquals("an", bean.getAppName());
         assertEquals("ap", bean.getAppPath());
-        if (checkAllFields) {
-            assertEquals("at", bean.getAuthToken());
-        }
         assertEquals("c", bean.getConf());
         if (checkAllFields) {
             assertEquals("cu", bean.getConsoleUrl());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngine.java Wed Jun 12 23:17:31 2013
@@ -87,7 +87,7 @@ public class TestCoordinatorEngine exten
         conf.set(OozieClient.USER_NAME, getTestUser());
 
 
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         final String jobId = ce.submitJob(conf, true);
         waitFor(5000, new Predicate() {
             public boolean evaluate() throws Exception {
@@ -141,7 +141,7 @@ public class TestCoordinatorEngine exten
         conf.set(OozieClient.USER_NAME, getTestUser());
 
 
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         final String jobId = ce.submitJob(conf, true);
 
         waitFor(5000, new Predicate() {
@@ -195,7 +195,7 @@ public class TestCoordinatorEngine exten
         conf.set(OozieClient.USER_NAME, getTestUser());
 
 
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         final String jobId = ce.submitJob(conf, true);
 
         waitFor(5000, new Predicate() {
@@ -249,7 +249,7 @@ public class TestCoordinatorEngine exten
         conf.set(OozieClient.USER_NAME, getTestUser());
 
 
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         final String jobId = ce.submitJob(conf, true);
 
         //create done flag
@@ -318,7 +318,7 @@ public class TestCoordinatorEngine exten
         conf.set(OozieClient.USER_NAME, getTestUser());
 
 
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         final String jobId = ce.submitJob(conf, true);
         waitFor(5000, new Predicate() {
             public boolean evaluate() throws Exception {
@@ -337,7 +337,7 @@ public class TestCoordinatorEngine exten
     }
 
     private void _testGetJob(String jobId, String appPath) throws Exception {
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         CoordinatorJob job = ce.getCoordJob(jobId);
         assertEquals(jobId, job.getId());
         assertEquals(job.getAppPath(), appPath);
@@ -349,7 +349,7 @@ public class TestCoordinatorEngine exten
      * @throws Exception
      */
     public void _testGetJobs(String jobId) throws Exception {
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         // Test with no job filter specified
         CoordinatorJobInfo jobInfo = ce.getCoordJobs("", 1, 10);
         assertEquals(1, jobInfo.getCoordJobs().size());
@@ -386,7 +386,7 @@ public class TestCoordinatorEngine exten
     }
 
     private void _testGetDefinition(String jobId) throws Exception {
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         CoordinatorJobBean job = ce.getCoordJob(jobId);
         System.out.println("JOBXML=" + job.getOrigJobXml());
         assertNotNull(job.getOrigJobXml());
@@ -430,19 +430,19 @@ public class TestCoordinatorEngine exten
     private void _testStatus(final String jobId) throws Exception {
         waitFor(6000, new Predicate() {
             public boolean evaluate() throws Exception {
-                CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+                CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
                 CoordinatorJob job = ce.getCoordJob(jobId);
                 return !job.getStatus().equals(CoordinatorJob.Status.PREP);
             }
         });
 
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         CoordinatorJob job = ce.getCoordJob(jobId);
         assertFalse(job.getStatus().equals(CoordinatorJob.Status.PREP));
     }
 
     private void _testSubsetActions(final String jobId) throws Exception {
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         // Check for WAITING filter
         CoordinatorJob job = ce.getCoordJob(jobId, "status=WAITING", 1, 2, false);
         // As both actions are waiting, expected result size is 2

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngineStreamLog.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngineStreamLog.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngineStreamLog.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorEngineStreamLog.java Wed Jun 12 23:17:31 2013
@@ -87,7 +87,7 @@ public class TestCoordinatorEngineStream
     }
 
     private CoordinatorEngine createCoordinatorEngine() {
-        return new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        return new CoordinatorEngine(getTestUser());
     }
 
     /**

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorJobBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorJobBean.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorJobBean.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestCoordinatorJobBean.java Wed Jun 12 23:17:31 2013
@@ -51,7 +51,6 @@ public class TestCoordinatorJobBean exte
         bean.setAppName("an");
         bean.setAppNamespace("ans");
         bean.setAppPath("ap");
-        bean.setAuthToken("at");
         bean.setBundleId("bi");
         bean.setConcurrency(2);
         bean.setConf("c");
@@ -107,7 +106,6 @@ public class TestCoordinatorJobBean exte
         assertEquals("ans", bean.getAppNamespace());
         assertEquals("ap", bean.getAppPath());
         if (checkDeserialization) {
-            assertEquals("at", bean.getAuthToken());
             assertEquals("bi", bean.getBundleId());
         }
         assertEquals(2, bean.getConcurrency());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java Wed Jun 12 23:17:31 2013
@@ -100,7 +100,7 @@ public class TestDagEngine extends XTest
         defaultConf.writeXml(os);
         os.close();
 
-        final DagEngine engine = new DagEngine(getTestUser(), "a");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -149,7 +149,7 @@ public class TestDagEngine extends XTest
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine(getTestUser(), "a");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -170,7 +170,7 @@ public class TestDagEngine extends XTest
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine(getTestUser(), "a");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/TestWorkflowBean.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/TestWorkflowBean.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/TestWorkflowBean.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/TestWorkflowBean.java Wed Jun 12 23:17:31 2013
@@ -109,11 +109,9 @@ public class TestWorkflowBean extends XT
 
     public void testWorkflow() {
         WorkflowJobBean workflow = new WorkflowJobBean();
-        workflow.setAuthToken("authToken");
         workflow.setLogToken("logToken");
         // workflow.setWorkflowInstance(new MyWorkflowInstance());
         workflow.setProtoActionConf("proto");
-        assertEquals("authToken", workflow.getAuthToken());
         assertEquals("logToken", workflow.getLogToken());
         // assertNotNull(workflow.getWorkflowInstance());
         assertEquals("proto", workflow.getProtoActionConf());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/ActionExecutorTestCase.java Wed Jun 12 23:17:31 2013
@@ -236,7 +236,7 @@ public abstract class ActionExecutorTest
         wfConf.set(OozieClient.APP_PATH, appUri.toString());
 
 
-        WorkflowJobBean workflow = createWorkflow(app, wfConf, protoConf, "auth");
+        WorkflowJobBean workflow = createWorkflow(app, wfConf, protoConf);
 
         WorkflowActionBean action = new WorkflowActionBean();
         action.setName(actionName);
@@ -270,7 +270,7 @@ public abstract class ActionExecutorTest
         wfConf.set(OozieClient.APP_PATH, appUri.toString());
 
 
-        WorkflowJobBean workflow = createWorkflow(app, wfConf, protoConf, "auth");
+        WorkflowJobBean workflow = createWorkflow(app, wfConf, protoConf);
 
         WorkflowActionBean action = new WorkflowActionBean();
         action.setName(actionName);
@@ -280,8 +280,8 @@ public abstract class ActionExecutorTest
         return workflow;
     }
 
-    private WorkflowJobBean createWorkflow(WorkflowApp app, Configuration conf, XConfiguration protoConf,
-            String authToken) throws Exception {
+    private WorkflowJobBean createWorkflow(WorkflowApp app, Configuration conf, XConfiguration protoConf)
+            throws Exception {
         WorkflowLib workflowLib = Services.get().get(WorkflowStoreService.class).getWorkflowLibWithNoDB();
         WorkflowInstance wfInstance;
         wfInstance = workflowLib.createInstance(app, conf);
@@ -297,7 +297,6 @@ public abstract class ActionExecutorTest
         workflow.setRun(0);
         workflow.setUser(conf.get(OozieClient.USER_NAME));
         workflow.setGroup(conf.get(OozieClient.GROUP_NAME));
-        workflow.setAuthToken(authToken);
         workflow.setWorkflowInstance(wfInstance);
         return workflow;
     }

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/action/hadoop/TestJavaActionExecutor.java Wed Jun 12 23:17:31 2013
@@ -989,7 +989,7 @@ public class TestJavaActionExecutor exte
 
     private WorkflowJobBean createWorkflow(WorkflowApp app, Configuration conf, String authToken) throws Exception {
         WorkflowAppService wps = Services.get().get(WorkflowAppService.class);
-        Configuration protoActionConf = wps.createProtoActionConf(conf, authToken, true);
+        Configuration protoActionConf = wps.createProtoActionConf(conf, true);
         WorkflowLib workflowLib = Services.get().get(WorkflowStoreService.class).getWorkflowLibWithNoDB();
         WorkflowInstance wfInstance;
         wfInstance = workflowLib.createInstance(app, conf);
@@ -1005,7 +1005,6 @@ public class TestJavaActionExecutor exte
         workflow.setRun(0);
         workflow.setUser(conf.get(OozieClient.USER_NAME));
         workflow.setGroup(conf.get(OozieClient.GROUP_NAME));
-        workflow.setAuthToken(authToken);
         workflow.setWorkflowInstance(wfInstance);
         return workflow;
     }

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/action/oozie/TestSubWorkflowActionExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/action/oozie/TestSubWorkflowActionExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/action/oozie/TestSubWorkflowActionExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/action/oozie/TestSubWorkflowActionExecutor.java Wed Jun 12 23:17:31 2013
@@ -398,7 +398,7 @@ public class TestSubWorkflowActionExecut
         WorkflowAppService wps = Services.get().get(WorkflowAppService.class);
         WorkflowJob wf = oozieClient.getJobInfo(action.getExternalId());
         Configuration childConf = new XConfiguration(new StringReader(wf.getConf()));
-        childConf = wps.createProtoActionConf(childConf, "authToken", true);
+        childConf = wps.createProtoActionConf(childConf, true);
         assertEquals(childConf.get(WorkflowAppService.APP_LIB_PATH_LIST), subwfLibJar.toString());
     }
 }

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/TestPurgeXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/TestPurgeXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/TestPurgeXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/TestPurgeXCommand.java Wed Jun 12 23:17:31 2013
@@ -3726,7 +3726,7 @@ public class TestPurgeXCommand extends X
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, jobStatus, instanceStatus);
         wfBean.setStartTime(DateUtils.parseDateOozieTZ("2015-12-18T01:00Z"));
         wfBean.setEndTime(DateUtils.parseDateOozieTZ("2015-12-18T03:00Z"));
 
@@ -3745,10 +3745,10 @@ public class TestPurgeXCommand extends X
     }
 
     @Override
-    protected WorkflowJobBean createWorkflow(WorkflowApp app, Configuration conf, String authToken,
+    protected WorkflowJobBean createWorkflow(WorkflowApp app, Configuration conf,
             WorkflowJob.Status jobStatus, WorkflowInstance.Status instanceStatus) throws Exception {
         WorkflowAppService wps = Services.get().get(WorkflowAppService.class);
-        Configuration protoActionConf = wps.createProtoActionConf(conf, authToken, true);
+        Configuration protoActionConf = wps.createProtoActionConf(conf, true);
         WorkflowLib workflowLib = Services.get().get(WorkflowStoreService.class).getWorkflowLibWithNoDB();
         WorkflowInstance wfInstance = workflowLib.createInstance(app, conf);
         ((LiteWorkflowInstance) wfInstance).setStatus(instanceStatus);
@@ -3764,7 +3764,6 @@ public class TestPurgeXCommand extends X
         workflow.setRun(0);
         workflow.setUser(conf.get(OozieClient.USER_NAME));
         workflow.setGroup(conf.get(OozieClient.GROUP_NAME));
-        workflow.setAuthToken(authToken);
         workflow.setWorkflowInstance(wfInstance);
         workflow.setStartTime(DateUtils.parseDateOozieTZ("2009-12-18T01:00Z"));
         workflow.setEndTime(DateUtils.parseDateOozieTZ("2009-12-18T03:00Z"));

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleJobSuspendXCommand.java Wed Jun 12 23:17:31 2013
@@ -139,7 +139,7 @@ public class TestBundleJobSuspendXComman
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf, job.getAuthToken());
+        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf);
         submitCmd.call();
 
         BundleJobGetJPAExecutor bundleJobGetCmd = new BundleJobGetJPAExecutor(submitCmd.getJob().getId());
@@ -217,7 +217,7 @@ public class TestBundleJobSuspendXComman
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf, job.getAuthToken());
+        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf);
         submitCmd.call();
 
         BundleJobGetJPAExecutor bundleJobGetCmd = new BundleJobGetJPAExecutor(submitCmd.getJob().getId());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleKillXCommand.java Wed Jun 12 23:17:31 2013
@@ -99,7 +99,7 @@ public class TestBundleKillXCommand exte
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf, job.getAuthToken());
+        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf);
         submitCmd.call();
 
         BundleJobGetJPAExecutor bundleJobGetCmd = new BundleJobGetJPAExecutor(submitCmd.getJob().getId());
@@ -179,7 +179,7 @@ public class TestBundleKillXCommand exte
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf, job.getAuthToken());
+        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf);
         submitCmd.call();
 
         BundleJobGetJPAExecutor bundleJobGetCmd = new BundleJobGetJPAExecutor(submitCmd.getJob().getId());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleStartXCommand.java Wed Jun 12 23:17:31 2013
@@ -108,7 +108,7 @@ public class TestBundleStartXCommand ext
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf, job.getAuthToken());
+        BundleSubmitXCommand submitCmd = new BundleSubmitXCommand(jobConf);
         submitCmd.call();
 
         BundleJobGetJPAExecutor bundleJobGetExecutor = new BundleJobGetJPAExecutor(submitCmd.getJob().getId());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/bundle/TestBundleSubmitXCommand.java Wed Jun 12 23:17:31 2013
@@ -75,7 +75,7 @@ public class TestBundleSubmitXCommand ex
         Path appPath = new Path(jobConf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
         jobConf.set(OozieClient.BUNDLE_APP_PATH, appPath.toString());
 
-        BundleSubmitXCommand command = new BundleSubmitXCommand(true, jobConf, CREATE_TIME);
+        BundleSubmitXCommand command = new BundleSubmitXCommand(true, jobConf);
         BundleJobBean bundleBean = (BundleJobBean)command.getJob();
         bundleBean.setStartTime(new Date());
         bundleBean.setEndTime(new Date());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java Wed Jun 12 23:17:31 2013
@@ -622,7 +622,6 @@ public class TestCoordActionInputCheckXC
         coordJob.setLastModifiedTime(new Date());
         coordJob.setUser("testUser");
         coordJob.setGroup("testGroup");
-        coordJob.setAuthToken("notoken");
         coordJob.setTimeZone("UTC");
         coordJob.setTimeUnit(Timeunit.DAY);
         coordJob.setMatThrottling(2);
@@ -803,7 +802,6 @@ public class TestCoordActionInputCheckXC
         coordJob.setLastModifiedTime(new Date());
         coordJob.setUser("testUser");
         coordJob.setGroup("testGroup");
-        coordJob.setAuthToken("notoken");
         coordJob.setTimeZone("UTC");
         coordJob.setTimeUnit(Timeunit.DAY);
         coordJob.setMatThrottling(2);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java Wed Jun 12 23:17:31 2013
@@ -87,7 +87,7 @@ public class TestCoordActionStartXComman
     public void testActionStartCommand() throws IOException, JPAExecutorException, CommandException {
         String actionId = new Date().getTime() + "-COORD-ActionStartCommand-C@1";
         addRecordToActionTable(actionId, 1, null);
-        new CoordActionStartXCommand(actionId, "me", "myapp", "mytoken", "myjob").call();
+        new CoordActionStartXCommand(actionId, "me", "myapp", "myjob").call();
         checkCoordAction(actionId);
     }
 
@@ -104,7 +104,7 @@ public class TestCoordActionStartXComman
         String actionId = new Date().getTime() + "-COORD-ActionStartCommand-C@1";
         String wfApp = "<start to='${someParam}' />";
         addRecordToActionTable(actionId, 1, wfApp);
-        new CoordActionStartXCommand(actionId, "me", "myapp", "mytoken", "myjob").call();
+        new CoordActionStartXCommand(actionId, "me", "myapp", "myjob").call();
         final JPAService jpaService = Services.get().get(JPAService.class);
         CoordinatorActionBean action = jpaService.execute(new CoordActionGetForStartJPAExecutor(actionId));
         if (action.getStatus() == CoordinatorAction.Status.SUBMITTED) {
@@ -132,7 +132,7 @@ public class TestCoordActionStartXComman
                 CoordinatorAction.Status.SUBMITTED, "coord-action-start-escape-strings.xml", 0);
 
         String actionId = action.getId();
-        new CoordActionStartXCommand(actionId, getTestUser(), "myapp", "undef", "myjob").call();
+        new CoordActionStartXCommand(actionId, getTestUser(), "myapp", "myjob").call();
 
         final JPAService jpaService = Services.get().get(JPAService.class);
         action = jpaService.execute(new CoordActionGetJPAExecutor(actionId));

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordChangeXCommand.java Wed Jun 12 23:17:31 2013
@@ -401,7 +401,6 @@ public class TestCoordChangeXCommand ext
         coordJob.setTimeUnit(Timeunit.MINUTE);
         coordJob.setUser("testUser");
         coordJob.setGroup("testGroup");
-        coordJob.setAuthToken("notoken");
 
         String confStr = "<configuration></configuration>";
         coordJob.setConf(confStr);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java Wed Jun 12 23:17:31 2013
@@ -1012,7 +1012,6 @@ public class TestCoordRerunXCommand exte
         coordJob.setLastModifiedTime(new Date());
         coordJob.setUser(getTestUser());
         coordJob.setGroup(getTestGroup());
-        coordJob.setAuthToken("notoken");
 
         Properties conf = getCoordProp(appPath);
         String confStr = XmlUtils.writePropToString(conf);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordSubmitXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordSubmitXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordSubmitXCommand.java Wed Jun 12 23:17:31 2013
@@ -89,7 +89,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("appName", "var-app-name");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -123,7 +123,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -152,7 +152,7 @@ public class TestCoordSubmitXCommand ext
         IOUtils.copyCharStream(reader, writer);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -168,7 +168,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-input-instance2.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -184,7 +184,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-input-instance3.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -197,7 +197,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-input-instance4.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -223,7 +223,7 @@ public class TestCoordSubmitXCommand ext
         IOUtils.copyCharStream(reader, writer);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -240,7 +240,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-input-start-instance2.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -266,7 +266,7 @@ public class TestCoordSubmitXCommand ext
         IOUtils.copyCharStream(reader, writer);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -283,7 +283,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-input-end-instance2.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -310,7 +310,7 @@ public class TestCoordSubmitXCommand ext
 
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -326,7 +326,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-output-instance2.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -342,7 +342,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-output-instance3.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -358,7 +358,7 @@ public class TestCoordSubmitXCommand ext
         reader = IOUtils.getResourceAsReader("coord-multiple-output-instance4.xml", -1);
         writer = new FileWriter(new URI(appPath).getPath());
         IOUtils.copyCharStream(reader, writer);
-        sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -398,7 +398,7 @@ public class TestCoordSubmitXCommand ext
 
         this.addRecordToBundleActionTable(coordJob.getId(), "COORD-NAME", 0, Job.Status.PREP);
 
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING", coordJob.getId(), "COORD-NAME");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, coordJob.getId(), "COORD-NAME");
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -441,7 +441,7 @@ public class TestCoordSubmitXCommand ext
 
         this.addRecordToBundleActionTable("OOZIE-B", "COORD-NAME", 0, Job.Status.PREP);
 
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING", "OOZIE-B", "COORD-NAME");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "OOZIE-B", "COORD-NAME");
         try {
             sc.call();
             fail("Exception expected because namespace is too old when submit coordinator through bundle!");
@@ -493,7 +493,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("SLA_OFFSET", "10");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -527,7 +527,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -560,7 +560,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         try {
             sc.call();
             fail("Exception expected if schema has errors!");
@@ -588,7 +588,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -613,7 +613,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         // conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         try {
             sc.call();
             fail("Exception expected if user.name is not set!");
@@ -639,7 +639,7 @@ public class TestCoordSubmitXCommand ext
         writeToFile(appXml, appPath);
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -679,7 +679,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("MY_DONE_FLAG", "complete");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -719,7 +719,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("MY_DONE_FLAG", "complete");
         conf.set("NAME", "test_app_name");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -746,7 +746,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("MINUTES", "1");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         try {
             sc.call();
@@ -770,7 +770,7 @@ public class TestCoordSubmitXCommand ext
 
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         try {
             sc.call();
             fail("Expected to catch errors due to invalid dataset initial instance");
@@ -831,7 +831,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("appName", "var-app-name");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -901,7 +901,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("appName", "var-app-name");
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
         String jobId = sc.call();
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -942,7 +942,7 @@ public class TestCoordSubmitXCommand ext
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf, "UNIT_TESTING");
+        CoordSubmitXCommand sc = new CoordSubmitXCommand(conf);
 
         if (withDefaults) {
             String defaults = "<configuration><property><name>startTime</name>" +

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestFutureActionsTimeOut.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestFutureActionsTimeOut.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestFutureActionsTimeOut.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestFutureActionsTimeOut.java Wed Jun 12 23:17:31 2013
@@ -118,7 +118,7 @@ public class TestFutureActionsTimeOut ex
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         String jobId = ce.submitJob(conf, true);
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
         checkCoordJob(jobId);
@@ -168,7 +168,7 @@ public class TestFutureActionsTimeOut ex
      * @throws Exception thrown if failed
      */
     private void _testTimeout(final String jobId, Date createDate) throws Exception {
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
 
         waitFor(12000, new Predicate() {
             public boolean evaluate() throws Exception {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestPastActionsTimeOut.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestPastActionsTimeOut.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestPastActionsTimeOut.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestPastActionsTimeOut.java Wed Jun 12 23:17:31 2013
@@ -82,7 +82,7 @@ public class TestPastActionsTimeOut exte
         conf.set(OozieClient.COORDINATOR_APP_PATH, appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         String jobId = ce.submitJob(conf, true);
 
         assertEquals(jobId.substring(jobId.length() - 2), "-C");
@@ -131,7 +131,7 @@ public class TestPastActionsTimeOut exte
      * @throws Exception thrown if failed
      */
     private void _testTimeout(final String jobId) throws Exception {
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
 
         waitFor(6000, new Predicate() {
             public boolean evaluate() throws Exception {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java Wed Jun 12 23:17:31 2013
@@ -248,7 +248,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -303,7 +303,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -374,7 +374,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -443,7 +443,7 @@ public class TestActionErrors extends XD
         final int maxRetries = 2;
         final int retryInterval = 10;
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -528,7 +528,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -566,7 +566,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -620,7 +620,7 @@ public class TestActionErrors extends XD
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionStartXCommand.java Wed Jun 12 23:17:31 2013
@@ -258,7 +258,7 @@ public class TestActionStartXCommand ext
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, jobStatus, instanceStatus);
 
         try {
             JPAService jpaService = Services.get().get(JPAService.class);
@@ -439,7 +439,7 @@ public class TestActionStartXCommand ext
         conf.set("testCDATA",
                         "<![CDATA[?redirect=http%3A%2F%2Ftest.apache.com%2Fa-webservices%2Fv1%2FurlSigner%2FsignUrl&amp;namespace=nova.proxy&amp;keyDBHash=Vsy6n_C7K6NG0z4R2eBlKg--]]>");
 
-        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, jobStatus, instanceStatus);
 
         try {
             JPAService jpaService = Services.get().get(JPAService.class);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitHiveXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitHiveXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitHiveXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitHiveXCommand.java Wed Jun 12 23:17:31 2013
@@ -57,7 +57,7 @@ public class TestSubmitHiveXCommand exte
         String[] params = new String[]{"INPUT=/some/path", "OUTPUT=/some/other/path", "abc=xyz"};
         MapReduceMain.setStrings(conf, XOozieClient.HIVE_SCRIPT_PARAMS, params);
 
-        SubmitHiveXCommand submitHiveCmd = new SubmitHiveXCommand(conf, "token");
+        SubmitHiveXCommand submitHiveCmd = new SubmitHiveXCommand(conf);
         String xml = submitHiveCmd.getWorkflowXml(conf);
 
         XLog.getLog(getClass()).info("xml = " + xml);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitMRXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitMRXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitMRXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitMRXCommand.java Wed Jun 12 23:17:31 2013
@@ -60,7 +60,7 @@ public class TestSubmitMRXCommand extend
         conf.set(XOozieClient.FILES, "/user/oozie/input1.txt,/user/oozie/input2.txt#my.txt");
         conf.set(XOozieClient.ARCHIVES, "/user/oozie/udf1.jar,/user/oozie/udf2.jar#my.jar");
 
-        SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf, "token");
+        SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf);
         String xml = submitMRCmd.getWorkflowXml(conf);
 
         XLog.getLog(getClass()).info("xml = " + xml);
@@ -104,7 +104,7 @@ public class TestSubmitMRXCommand extend
         conf.set("name_b", "value_b");
         conf.set("name_c", "value_c");
 
-        SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf, "token");
+        SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf);
         try {
             submitMRCmd.getWorkflowXml(conf);
             fail("shoud have already failed - missing libpath def");
@@ -126,7 +126,7 @@ public class TestSubmitMRXCommand extend
             conf.set(OozieClient.LIBPATH, "libpath");
             conf.set("mapreduce.job.user.name", "test_user");
 
-            SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf, "token");
+            SubmitMRXCommand submitMRCmd = new SubmitMRXCommand(conf);
             String xml = submitMRCmd.getWorkflowXml(conf);
 
             //verifying is a valid WF

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitPigXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitPigXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitPigXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitPigXCommand.java Wed Jun 12 23:17:31 2013
@@ -58,7 +58,7 @@ public class TestSubmitPigXCommand exten
         String[] params = new String[]{"INPUT=/some/path", "OUTPUT=/some/other/path", "abc=xyz"};
         MapReduceMain.setStrings(conf, XOozieClient.PIG_SCRIPT_PARAMS, params);
 
-        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf, "token");
+        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf);
         String xml = submitPigCmd.getWorkflowXml(conf);
 
         XLog.getLog(getClass()).info("xml = " + xml);
@@ -129,7 +129,7 @@ public class TestSubmitPigXCommand exten
         args[1] = "aaa bbb";
         MapReduceMain.setStrings(conf, XOozieClient.PIG_OPTIONS, args);
 
-        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf, "token");
+        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf);
         String xml = submitPigCmd.getWorkflowXml(conf);
 
         XLog.getLog(getClass()).info("xml = " + xml);
@@ -175,7 +175,7 @@ public class TestSubmitPigXCommand exten
         String pigArgsStr = "-a aaa -b bbb -c ccc -M -Da=aaa -Db=bbb -param input=abc";
         conf.set(XOozieClient.PIG_OPTIONS, pigArgsStr);
 
-        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf, "token");
+        SubmitPigXCommand submitPigCmd = new SubmitPigXCommand(conf);
         try {
             submitPigCmd.getWorkflowXml(conf);
             fail("shoud have already failed - missing libpath def");

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSubmitXCommand.java Wed Jun 12 23:17:31 2013
@@ -60,7 +60,7 @@ public class TestSubmitXCommand extends 
       conf.set(OozieClient.APP_PATH, "file://" + appPath + "/workflow.xml");
       conf.set(OozieClient.USER_NAME, getTestUser());
       conf.set("appName", "var-app-name");
-      SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+      SubmitXCommand sc = new SubmitXCommand(conf);
       String jobId = sc.call();
       WorkflowStoreService wss = Services.get().get(WorkflowStoreService.class);
       WorkflowStore ws = wss.create();
@@ -78,7 +78,7 @@ public class TestSubmitXCommand extends 
         conf.set(OozieClient.APP_PATH, "file://" + appPath + "/workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
         conf.set("GB", "5");
-        SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(conf);
 
         try {
             sc.call();
@@ -99,7 +99,7 @@ public class TestSubmitXCommand extends 
         conf.set(OozieClient.APP_PATH, "file://" + appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(conf);
 
         try {
             sc.call();
@@ -119,7 +119,7 @@ public class TestSubmitXCommand extends 
         conf.set(OozieClient.APP_PATH, "file://" + appPath + "/workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(conf);
 
         try {
             sc.call();
@@ -139,7 +139,7 @@ public class TestSubmitXCommand extends 
         conf.set(OozieClient.APP_PATH, "file://" + appPath + "/test.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(conf);
 
         try {
             sc.call();
@@ -158,7 +158,7 @@ public class TestSubmitXCommand extends 
         writeToFile(appXml, appPath + "/test.xml");
         conf.set(OozieClient.APP_PATH, "file://" + appPath + "/does_not_exist.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
-        SubmitXCommand sc = new SubmitXCommand(conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(conf);
 
         try {
             sc.call();
@@ -176,7 +176,7 @@ public class TestSubmitXCommand extends 
         writeToFile(appXml, appPath + "/workflow.xml");
         conf.set(OozieClient.APP_PATH, "file://" + appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        SubmitXCommand sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(true, conf);
         assertEquals("OK", sc.call());
     }
 
@@ -187,7 +187,7 @@ public class TestSubmitXCommand extends 
         writeToFile(appXml, appPath + "/workflow.xml");
         conf.set(OozieClient.APP_PATH, "file://" + appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        SubmitXCommand sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        SubmitXCommand sc = new SubmitXCommand(true, conf);
         try {
             sc.call();
             fail("Should have gotten E0707 because the XML has a loop");
@@ -202,7 +202,7 @@ public class TestSubmitXCommand extends 
         writeToFile(appXml, appPath + "/workflow.xml");
         conf.set(OozieClient.APP_PATH, "file://" + appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        sc = new SubmitXCommand(true, conf);
         try {
             sc.call();
             fail("Should have gotten E0708 because the XML has an invalid transition");
@@ -217,7 +217,7 @@ public class TestSubmitXCommand extends 
         writeToFile(appXml, appPath + "/workflow.xml");
         conf.set(OozieClient.APP_PATH, "file://" + appPath);
         conf.set(OozieClient.USER_NAME, getTestUser());
-        sc = new SubmitXCommand(true, conf, "UNIT_TESTING");
+        sc = new SubmitXCommand(true, conf);
         try {
             sc.call();
             fail("Should have gotten E0701 because the XML has an invalid element");

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/event/TestEventGeneration.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/event/TestEventGeneration.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/event/TestEventGeneration.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/event/TestEventGeneration.java Wed Jun 12 23:17:31 2013
@@ -467,7 +467,7 @@ public class TestEventGeneration extends
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine(getTestUser(), "authtoken");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(OozieClient.USER_NAME, getTestUser());
@@ -550,7 +550,7 @@ public class TestEventGeneration extends
         conf.set(OozieClient.APP_PATH, appUri.toString());
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
-        WorkflowJobBean workflow = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP,
+        WorkflowJobBean workflow = createWorkflow(app, conf, WorkflowJob.Status.PREP,
                 WorkflowInstance.Status.PREP);
         String executionPath = "/";
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStartJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStartJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStartJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStartJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -117,7 +117,7 @@ public class TestBulkUpdateInsertForCoor
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean job = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowJobBean job = createWorkflow(app, conf, WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
         WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
         WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStatusJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStatusJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStatusJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertForCoordActionStatusJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -117,7 +117,7 @@ public class TestBulkUpdateInsertForCoor
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean job = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowJobBean job = createWorkflow(app, conf, WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
         WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
         WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -116,7 +116,7 @@ public class TestBulkUpdateInsertJPAExec
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean job = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowJobBean job = createWorkflow(app, conf, WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
         WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
         WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
 

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsGetForPurgeJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsGetForPurgeJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsGetForPurgeJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsGetForPurgeJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -121,7 +121,6 @@ public class TestCoordJobsGetForPurgeJPA
         coordJob.setLastModifiedTime(d);
         coordJob.setUser(getTestUser());
         coordJob.setGroup(getTestGroup());
-        coordJob.setAuthToken("notoken");
 
         Properties conf = getCoordConf(appPath);
         String confStr = XmlUtils.writePropToString(conf);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsToBeMaterializedJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsToBeMaterializedJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsToBeMaterializedJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestCoordJobsToBeMaterializedJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -109,7 +109,6 @@ public class TestCoordJobsToBeMaterializ
         coordJob.setLastModifiedTime(new Date());
         coordJob.setUser(getTestUser());
         coordJob.setGroup(getTestGroup());
-        coordJob.setAuthToken("notoken");
 
         Properties conf = getCoordConf(appPath);
         String confStr = XmlUtils.writePropToString(conf);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowIdGetForExternalIdJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowIdGetForExternalIdJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowIdGetForExternalIdJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowIdGetForExternalIdJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -75,7 +75,7 @@ public class TestWorkflowIdGetForExterna
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, jobStatus, instanceStatus);
         wfBean.setExternalId("external-id");
 
         try {

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobsGetForPurgeJPAExecutor.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobsGetForPurgeJPAExecutor.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobsGetForPurgeJPAExecutor.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowJobsGetForPurgeJPAExecutor.java Wed Jun 12 23:17:31 2013
@@ -105,7 +105,7 @@ public class TestWorkflowJobsGetForPurge
         conf.set(OozieClient.LOG_TOKEN, "testToken");
         conf.set(OozieClient.USER_NAME, getTestUser());
 
-        WorkflowJobBean wfBean = createWorkflow(app, conf, "auth", jobStatus, instanceStatus);
+        WorkflowJobBean wfBean = createWorkflow(app, conf, jobStatus, instanceStatus);
         Timestamp startTS = new Timestamp(System.currentTimeMillis() - (3 * DAY_IN_MS));
         Timestamp endTS = new Timestamp(System.currentTimeMillis() - (2 * DAY_IN_MS));
         wfBean.setStartTime(DateUtils.toDate(startTS));

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/service/TestActionCheckerService.java Wed Jun 12 23:17:31 2013
@@ -90,7 +90,7 @@ public class TestActionCheckerService ex
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine(getTestUser(), "a");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.set(WorkflowAppService.HADOOP_USER, getTestUser());
@@ -153,7 +153,7 @@ public class TestActionCheckerService ex
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine("u", "a");
+        final DagEngine engine = new DagEngine("u");
         Configuration conf = new XConfiguration();
         conf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         conf.setStrings(WorkflowAppService.HADOOP_USER, getTestUser());
@@ -223,7 +223,7 @@ public class TestActionCheckerService ex
      */
     public void testActionCheckerServiceCoord() throws Exception {
         final int actionNum = 1;
-        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser(), "UNIT_TESTING");
+        final CoordinatorEngine ce = new CoordinatorEngine(getTestUser());
         String currentDatePlusMonth = XDataTestCase.getCurrentDateafterIncrementingInMonths(1);
         Date start = DateUtils.parseDateOozieTZ(currentDatePlusMonth);
         Date end = DateUtils.parseDateOozieTZ(currentDatePlusMonth);

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/service/TestAuthorizationService.java Wed Jun 12 23:17:31 2013
@@ -120,7 +120,7 @@ public class TestAuthorizationService ex
         Writer writer = new FileWriter(getTestCaseDir() + "/workflow.xml");
         IOUtils.copyCharStream(reader, writer);
 
-        final DagEngine engine = new DagEngine(getTestUser(), "a");
+        final DagEngine engine = new DagEngine(getTestUser());
         Configuration jobConf = new XConfiguration();
         jobConf.set(OozieClient.APP_PATH, "file://" + getTestCaseDir() + File.separator + "workflow.xml");
         jobConf.set(OozieClient.USER_NAME, getTestUser());

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java?rev=1492478&r1=1492477&r2=1492478&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/service/TestCoordMaterializeTriggerService.java Wed Jun 12 23:17:31 2013
@@ -130,7 +130,6 @@ public class TestCoordMaterializeTrigger
         coordJob.setLastModifiedTime(new Date());
         coordJob.setUser(getTestUser());
         coordJob.setGroup(getTestGroup());
-        coordJob.setAuthToken("notoken");
 
         Configuration conf = getCoordConf(appPath);
         coordJob.setConf(XmlUtils.prettyPrint(conf).toString());