You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by vi...@apache.org on 2012/07/31 04:26:36 UTC
svn commit: r1367394 - in /incubator/oozie/trunk: ./
core/src/main/java/org/apache/oozie/command/wf/
core/src/main/java/org/apache/oozie/executor/jpa/
core/src/test/java/org/apache/oozie/executor/jpa/
Author: virag
Date: Tue Jul 31 02:26:36 2012
New Revision: 1367394
URL: http://svn.apache.org/viewvc?rev=1367394&view=rev
Log:
OOZIE-903 Workflow action status Ok but the workflow job remains in RUNNING (virag)
Added:
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java
incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java
Modified:
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
incubator/oozie/trunk/release-log.txt
Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java?rev=1367394&r1=1367393&r2=1367394&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java Tue Jul 31 02:26:36 2012
@@ -6,9 +6,9 @@
* 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.
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configurat
import org.apache.oozie.client.WorkflowJob;
import org.apache.oozie.client.SLAEvent.SlaAppType;
import org.apache.oozie.client.SLAEvent.Status;
+import org.apache.oozie.client.rest.JsonBean;
import org.apache.oozie.WorkflowActionBean;
import org.apache.oozie.WorkflowJobBean;
import org.apache.oozie.ErrorCode;
@@ -29,12 +30,10 @@ import org.apache.oozie.command.CommandE
import org.apache.oozie.command.PreconditionException;
import org.apache.oozie.command.coord.CoordActionUpdateXCommand;
import org.apache.oozie.command.wf.ActionXCommand.ActionExecutorContext;
+import org.apache.oozie.executor.jpa.BulkUpdateInsertJPAExecutor;
import org.apache.oozie.executor.jpa.JPAExecutorException;
import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowActionInsertJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowActionUpdateJPAExecutor;
import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobUpdateJPAExecutor;
import org.apache.oozie.service.ELService;
import org.apache.oozie.service.JPAService;
import org.apache.oozie.service.SchemaService;
@@ -56,6 +55,7 @@ import org.jdom.Element;
import org.jdom.Namespace;
import java.io.StringReader;
+import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
@@ -69,6 +69,9 @@ public class SignalXCommand extends Work
private String actionId;
private WorkflowJobBean wfJob;
private WorkflowActionBean wfAction;
+ private List<JsonBean> updateList = new ArrayList<JsonBean>();
+ private List<JsonBean> insertList = new ArrayList<JsonBean>();
+
public SignalXCommand(String name, int priority, String jobId) {
super(name, name, priority);
@@ -170,12 +173,7 @@ public class SignalXCommand extends Work
wfAction.setTransition(workflowInstance.getTransition(wfAction.getName()));
queue(new NotificationXCommand(wfJob, wfAction));
}
- try {
- jpaService.execute(new WorkflowActionUpdateJPAExecutor(wfAction));
- }
- catch (JPAExecutorException je) {
- throw new CommandException(je);
- }
+ updateList.add(wfAction);
}
if (completed) {
@@ -187,7 +185,7 @@ public class SignalXCommand extends Work
actionToKill.setPending();
actionToKill.setStatus(WorkflowActionBean.Status.KILLED);
- jpaService.execute(new WorkflowActionUpdateJPAExecutor(actionToKill));
+ updateList.add(actionToKill);
queue(new ActionKillXCommand(actionToKill.getId(), actionToKill.getType()));
}
@@ -199,7 +197,7 @@ public class SignalXCommand extends Work
queue(new NotificationXCommand(wfJob, actionToFail));
SLADbXOperations.writeStausEvent(wfAction.getSlaXml(), wfAction.getId(), Status.FAILED,
SlaAppType.WORKFLOW_ACTION);
- jpaService.execute(new WorkflowActionUpdateJPAExecutor(actionToFail));
+ updateList.add(actionToFail);
}
}
catch (JPAExecutorException je) {
@@ -248,10 +246,7 @@ public class SignalXCommand extends Work
else {
wfAction.setErrorInfo(ErrorCode.E0729.toString(), actionConf);
}
- jpaService.execute(new WorkflowActionUpdateJPAExecutor(wfAction));
- }
- catch (JPAExecutorException je) {
- throw new CommandException(je);
+ updateList.add(wfAction);
}
catch (Exception ex) {
LOG.warn("Exception in SignalXCommand ", ex.getMessage(), ex);
@@ -276,7 +271,7 @@ public class SignalXCommand extends Work
oldAction = jpaService.execute(new WorkflowActionGetJPAExecutor(newAction.getId()));
oldAction.setPending();
- jpaService.execute(new WorkflowActionUpdateJPAExecutor(oldAction));
+ updateList.add(oldAction);
queue(new SignalXCommand(jobId, oldAction.getId()));
}
@@ -285,7 +280,7 @@ public class SignalXCommand extends Work
String actionSlaXml = getActionSLAXml(newAction.getName(), workflowInstance.getApp()
.getDefinition(), wfJob.getConf());
newAction.setSlaXml(actionSlaXml);
- jpaService.execute(new WorkflowActionInsertJPAExecutor(newAction));
+ insertList.add(newAction);
LOG.debug("SignalXCommand: Name: "+ newAction.getName() + ", Id: " +newAction.getId() + ", Authcode:" + newAction.getCred());
queue(new ActionStartXCommand(newAction.getId(), newAction.getType()));
}
@@ -297,7 +292,10 @@ public class SignalXCommand extends Work
}
try {
- jpaService.execute(new WorkflowJobUpdateJPAExecutor(wfJob));
+ wfJob.setLastModifiedTime(new Date());
+ updateList.add(wfJob);
+ // call JPAExecutor to do the bulk writes
+ jpaService.execute(new BulkUpdateInsertJPAExecutor(updateList, insertList));
}
catch (JPAExecutorException je) {
throw new CommandException(je);
Added: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java?rev=1367394&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java (added)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java Tue Jul 31 02:26:36 2012
@@ -0,0 +1,103 @@
+/**
+ * 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.oozie.executor.jpa;
+
+import java.util.Collection;
+
+import javax.persistence.EntityManager;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.FaultInjection;
+import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.util.ParamChecker;
+
+/**
+ * Class for inserting and updating beans in bulk
+ * @param <T>
+*/
+public class BulkUpdateInsertJPAExecutor implements JPAExecutor<Void> {
+
+ private Collection<JsonBean> updateList;
+ private Collection<JsonBean> insertList;
+
+ /**
+ * Initialize the JPAExecutor using the insert and update list of JSON beans
+ * @param updateList
+ * @param insertList
+ */
+ public BulkUpdateInsertJPAExecutor(Collection<JsonBean> updateList, Collection<JsonBean> insertList) {
+ this.updateList = updateList;
+ this.insertList = insertList;
+ }
+
+ public BulkUpdateInsertJPAExecutor() {
+ }
+
+ /**
+ * Sets the update list for JSON bean
+ * @param updateList
+ */
+ public void setUpdateList(Collection<JsonBean> updateList) {
+ this.updateList = updateList;
+ }
+
+ /**
+ * Sets the insert list for JSON bean
+ * @param insertList
+ */
+ public void setInsertList(Collection<JsonBean> insertList) {
+ this.insertList = insertList;
+ }
+
+
+ /* (non-Javadoc)
+ * @see org.apache.oozie.executor.jpa.JPAExecutor#getName()
+ */
+ @Override
+ public String getName() {
+ return "BulkUpdateInsertJPAExecutor";
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.oozie.executor.jpa.JPAExecutor#execute(javax.persistence.EntityManager)
+ */
+ @Override
+ public Void execute(EntityManager em) throws JPAExecutorException {
+ try {
+ if (updateList!= null){
+ for (JsonBean entity: updateList){
+ ParamChecker.notNull(entity, "JsonBean");
+ em.merge(entity);
+ }
+ }
+ // Only used by test cases to check for rollback of transaction
+ FaultInjection.activate("org.apache.oozie.command.SkipCommitFaultInjection");
+ if (insertList!= null){
+ for (JsonBean entity: insertList){
+ ParamChecker.notNull(entity, "JsonBean");
+ em.persist(entity);
+ }
+ }
+ return null;
+ }
+ catch (Exception e) {
+ throw new JPAExecutorException(ErrorCode.E0603, e);
+ }
+ }
+
+
+}
Added: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java?rev=1367394&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java (added)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java Tue Jul 31 02:26:36 2012
@@ -0,0 +1,254 @@
+/**
+ * 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.oozie.executor.jpa;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.CoordinatorJobBean;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.FaultInjection;
+import org.apache.oozie.WorkflowActionBean;
+import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.client.CoordinatorJob;
+import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.client.WorkflowJob;
+import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.command.SkipCommitFaultInjection;
+import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.workflow.WorkflowApp;
+import org.apache.oozie.workflow.WorkflowInstance;
+import org.apache.oozie.workflow.lite.EndNodeDef;
+import org.apache.oozie.workflow.lite.LiteWorkflowApp;
+import org.apache.oozie.workflow.lite.StartNodeDef;
+
+public class TestBulkUpdateInsertJPAExecutor extends XDataTestCase {
+ Services services;
+
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ services = new Services();
+ services.init();
+ cleanUpDBTables();
+ }
+
+ @Override
+ protected void tearDown() throws Exception {
+ services.destroy();
+ super.tearDown();
+ }
+
+ /**
+ * Test bulk updates by updating coordinator job, workflow job and workflow action
+ * @throws Exception
+ */
+ public void testUpdates() throws Exception {
+ CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, true, true);
+ WorkflowJobBean wfJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+ WorkflowActionBean action = addRecordToWfActionTable(wfJob.getId(), "1", WorkflowAction.Status.PREP);
+ JPAService jpaService = Services.get().get(JPAService.class);
+ assertNotNull(jpaService);
+
+ // update the status
+ coordJob.setStatus(CoordinatorJob.Status.RUNNING);
+ wfJob.setStatus(WorkflowJob.Status.SUCCEEDED);
+ action.setStatus(WorkflowAction.Status.RUNNING);
+
+ List<JsonBean> updateList = new ArrayList<JsonBean>();
+ // update the list for doing bulk writes
+ updateList.add(coordJob);
+ updateList.add(wfJob);
+ updateList.add(action);
+ BulkUpdateInsertJPAExecutor bulkUpdateCmd = new BulkUpdateInsertJPAExecutor();
+ bulkUpdateCmd.setUpdateList(updateList);
+ jpaService.execute(bulkUpdateCmd);
+
+ // check for expected status after running bulkUpdateJPA
+ coordJob = jpaService.execute(new CoordJobGetJPAExecutor(coordJob.getId()));
+ assertEquals("RUNNING", coordJob.getStatusStr());
+
+ wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(wfJob.getId()));
+ assertEquals("SUCCEEDED", wfJob.getStatusStr());
+
+ WorkflowActionBean action2 = jpaService.execute(new WorkflowActionGetJPAExecutor(action.getId()));
+ assertEquals(WorkflowAction.Status.RUNNING, action2.getStatus());
+
+ }
+
+ /**
+ * Test bulk inserts by inserting a workflow job and two workflow actions
+ * @throws Exception
+ */
+ public void testInserts() throws Exception{
+ WorkflowApp app = new LiteWorkflowApp("testApp", "<workflow-app/>", new StartNodeDef("end"))
+ .addNode(new EndNodeDef("end"));
+ Configuration conf = new Configuration();
+ Path appUri = new Path(getAppPath(), "workflow.xml");
+ conf.set(OozieClient.APP_PATH, appUri.toString());
+ conf.set(OozieClient.LOG_TOKEN, "testToken");
+ conf.set(OozieClient.USER_NAME, getTestUser());
+
+ WorkflowJobBean job = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+ WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+ WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+ List<JsonBean> insertList = new ArrayList<JsonBean>();
+ // insert one workflow job and two actions
+ insertList.add(action1);
+ insertList.add(action2);
+ insertList.add(job);
+
+ JPAService jpaService = Services.get().get(JPAService.class);
+ assertNotNull(jpaService);
+ BulkUpdateInsertJPAExecutor bulkInsertCmd = new BulkUpdateInsertJPAExecutor();
+ bulkInsertCmd.setInsertList(insertList);
+ jpaService.execute(bulkInsertCmd);
+
+ // check for expected status after running bulkUpdateJPA
+ WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+ action1 = jpaService.execute(actionGetCmd);
+ assertEquals("PREP", action1.getStatusStr());
+
+ actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+ action2 = jpaService.execute(actionGetCmd);
+ assertEquals("PREP", action2.getStatusStr());
+
+ WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+ job = jpaService.execute(wfGetCmd);
+ assertEquals("PREP", job.getStatusStr());
+
+ }
+
+ /**
+ * Test bulk inserts and updates by inserting wf actions and updating
+ * coordinator and workflow jobs
+ *
+ * @throws Exception
+ */
+ public void testBulkInsertUpdates() throws Exception{
+ CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, true, true);
+ WorkflowJobBean job = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+ WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+ WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+ job.setStatus(WorkflowJob.Status.RUNNING);
+ coordJob.setStatus(Job.Status.SUCCEEDED);
+ List<JsonBean> insertList = new ArrayList<JsonBean>();
+ // Add two actions to insert list
+ insertList.add(action1);
+ insertList.add(action2);
+
+ List<JsonBean> updateList = new ArrayList<JsonBean>();
+ //Add two jobs to update list
+ updateList.add(coordJob);
+ updateList.add(job);
+
+ JPAService jpaService = Services.get().get(JPAService.class);
+ assertNotNull(jpaService);
+
+ BulkUpdateInsertJPAExecutor bulkUpdateCmd = new BulkUpdateInsertJPAExecutor(updateList, insertList);
+ jpaService.execute(bulkUpdateCmd);
+
+ coordJob = jpaService.execute(new CoordJobGetJPAExecutor(coordJob.getId()));
+ assertEquals("SUCCEEDED", coordJob.getStatusStr());
+
+ WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+ WorkflowJobBean wfBean = jpaService.execute(wfGetCmd);
+ assertEquals("RUNNING", wfBean.getStatusStr());
+
+ WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+ action1 = jpaService.execute(actionGetCmd);
+ assertEquals("PREP", action1.getStatusStr());
+
+ actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+ action2 = jpaService.execute(actionGetCmd);
+ assertEquals("PREP", action2.getStatusStr());
+ }
+
+ /**
+ * Test bulk inserts and updates rollback
+ *
+ * @throws Exception
+ */
+ public void testBulkInsertUpdatesRollback() throws Exception{
+ WorkflowJobBean job = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+ WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+ WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+ job.setStatus(WorkflowJob.Status.RUNNING);
+ List<JsonBean> insertList = new ArrayList<JsonBean>();
+ // Add two actions to insert list
+ insertList.add(action1);
+ insertList.add(action2);
+
+ List<JsonBean> updateList = new ArrayList<JsonBean>();
+ // Add to update list
+ updateList.add(job);
+
+ JPAService jpaService = Services.get().get(JPAService.class);
+ assertNotNull(jpaService);
+
+ BulkUpdateInsertJPAExecutor wfUpdateCmd1 = new BulkUpdateInsertJPAExecutor(updateList, insertList);
+
+ // set fault injection to true, so transaction is roll backed
+ setSystemProperty(FaultInjection.FAULT_INJECTION, "true");
+ setSystemProperty(SkipCommitFaultInjection.ACTION_FAILOVER_FAULT_INJECTION, "true");
+ try {
+ jpaService.execute(wfUpdateCmd1);
+ fail("Expected exception due to commit failure but didn't get any");
+ }
+ catch (Exception e) {
+ }
+ FaultInjection.deactivate("org.apache.oozie.command.SkipCommitFaultInjection");
+
+ // Check whether transactions are rolled back or not
+ WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+ WorkflowJobBean wfBean = jpaService.execute(wfGetCmd);
+ // status should not be RUNNING
+ assertEquals("PREP", wfBean.getStatusStr());
+
+ WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+ try {
+ action1 = jpaService.execute(actionGetCmd);
+ fail("Expected exception but didnt get any");
+ }
+ catch (JPAExecutorException jpaee) {
+ assertEquals(ErrorCode.E0605, jpaee.getErrorCode());
+ }
+
+
+ actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+ try {
+ action2 = jpaService.execute(actionGetCmd);
+ fail("Expected exception but didnt get any");
+ }
+ catch (JPAExecutorException jpaee) {
+ assertEquals(ErrorCode.E0605, jpaee.getErrorCode());
+ }
+
+ }
+
+}
Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1367394&r1=1367393&r2=1367394&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Tue Jul 31 02:26:36 2012
@@ -1,5 +1,6 @@
-- Oozie 3.3.0 release (trunk - unreleased)
+OOZIE-903 Workflow action status 'Ok' but the workflow job remains in 'RUNNING' (virag)
OOZIE-928 Clarify the documentation for submitting coordinator jobs using web services API (rkanter via virag)
OOZIE-920 Incorrect error message for multiple start instances in coordinator xml (bcyr via tucu)
OOZIE-925 Change default logging level for oozie to INFO (rkanter via tucu)