Author: rkanter
Date: Sun Nov 25 16:46:47 2012
New Revision: 1413370

URL: http://svn.apache.org/viewvc?rev=1413370&view=rev
Log:
OOZIE-1057 Log message for retrying to connect to the JT always says 60,000 
milliseconds (jiezhou via rkanter)

Modified:
    
oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
    
oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
    oozie/trunk/release-log.txt

Modified: 
oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
URL: 
http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java?rev=1413370&r1=1413369&r2=1413370&view=diff
==============================================================================
--- 
oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
 (original)
+++ 
oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
 Sun Nov 25 16:46:47 2012
@@ -39,6 +39,7 @@ import org.apache.oozie.executor.jpa.JPA
 import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionUpdateJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.service.ActionCheckerService;
 import org.apache.oozie.service.ActionService;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.Services;
@@ -156,6 +157,10 @@ public class ActionCheckXCommand extends
     protected Void execute() throws CommandException {
         LOG.debug("STARTED ActionCheckXCommand for wf actionId=" + actionId + 
" priority =" + getPriority());
 
+        long retryInterval = 
Services.get().getConf().getLong(ActionCheckerService.CONF_ACTION_CHECK_INTERVAL,
 executor
+                .getRetryInterval());
+        executor.setRetryInterval(retryInterval);
+
         ActionExecutorContext context = null;
         try {
             boolean isRetry = false;
@@ -239,4 +244,9 @@ public class ActionCheckXCommand extends
             InstrumentUtils.incrJobCounter(INSTR_FAILED_JOBS_COUNTER, 1, 
getInstrumentation());
         }
     }
+
+    protected long getRetryInterval() {
+        return (executor != null) ? executor.getRetryInterval() : 
ActionExecutor.RETRY_INTERVAL;
+    }
+
 }

Modified: 
oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
URL: 
http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java?rev=1413370&r1=1413369&r2=1413370&view=diff
==============================================================================
--- 
oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
 (original)
+++ 
oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
 Sun Nov 25 16:46:47 2012
@@ -40,6 +40,7 @@ import org.apache.oozie.executor.jpa.JPA
 import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowActionInsertJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.service.ActionCheckerService;
 import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.InstrumentationService;
 import org.apache.oozie.service.JPAService;
@@ -479,6 +480,28 @@ public class TestActionCheckXCommand ext
         assertEquals("SUCCEEDED", action5.getExternalStatus());
     }
 
+    /**
+     * This test case verifies if getRetryInterval picks up the
+     * overridden value.
+     *
+     * @throws Exception
+     */
+    public void testCheckInterval() throws Exception {
+        long testedValue = 10;
+        
Services.get().getConf().setLong(ActionCheckerService.CONF_ACTION_CHECK_INTERVAL,
+                testedValue);
+
+        WorkflowJobBean job0 = 
this.addRecordToWfJobTable(WorkflowJob.Status.RUNNING, 
WorkflowInstance.Status.RUNNING);
+        final String jobId = job0.getId();
+        WorkflowActionBean action0 = this.addRecordToWfActionTable(jobId, "1", 
WorkflowAction.Status.RUNNING);
+        final String actionId = action0.getId();
+
+        ActionCheckXCommand checkCommand = new ActionCheckXCommand(actionId);
+        checkCommand.call();
+        long effectiveValue = checkCommand.getRetryInterval();
+        assertEquals(testedValue, effectiveValue);
+    }
+
     @Override
     protected WorkflowActionBean addRecordToWfActionTable(String wfId, String 
actionName, WorkflowAction.Status status) throws Exception {
         WorkflowActionBean action = createWorkflowActionSetPending(wfId, 
status);

Modified: oozie/trunk/release-log.txt
URL: 
http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1413370&r1=1413369&r2=1413370&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Sun Nov 25 16:46:47 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.4.0 release (trunk - unreleased)
 
+OOZIE-1057 Log message for retrying to connect to the JT always says 60,000 
milliseconds (jiezhou via rkanter)
 OOZIE-1080 Add a dryrun option for workflows (rkanter)
 OOZIE-1062 Create a shell example
 OOZIE-1034 Allow disabling forkjoin validation just for a specific workflow


Reply via email to