You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by mo...@apache.org on 2012/11/29 02:38:15 UTC

svn commit: r1415025 - in /oozie/branches/hcat-intre: ./ core/src/main/java/org/apache/oozie/action/hadoop/ core/src/main/java/org/apache/oozie/command/wf/ core/src/main/java/org/apache/oozie/service/ core/src/test/java/org/apache/oozie/command/wf/ wor...

Author: mona
Date: Thu Nov 29 01:38:13 2012
New Revision: 1415025

URL: http://svn.apache.org/viewvc?rev=1415025&view=rev
Log:
sync patches with trunk

Modified:
    oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
    oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
    oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
    oozie/branches/hcat-intre/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
    oozie/branches/hcat-intre/release-log.txt
    oozie/branches/hcat-intre/workflowgenerator/README.txt
    oozie/branches/hcat-intre/workflowgenerator/src/main/java/org/apache/oozie/tools/workflowgenerator/client/OozieWorkflowGenerator.java

Modified: oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java (original)
+++ oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/action/hadoop/JavaActionExecutor.java Thu Nov 29 01:38:13 2012
@@ -79,10 +79,10 @@ import org.apache.hadoop.security.token.
 public class JavaActionExecutor extends ActionExecutor {
 
     private static final String HADOOP_USER = "user.name";
-    private static final String HADOOP_JOB_TRACKER = "mapred.job.tracker";
-    private static final String HADOOP_JOB_TRACKER_2 = "mapreduce.jobtracker.address";
-    private static final String HADOOP_YARN_RM = "yarn.resourcemanager.address";
-    private static final String HADOOP_NAME_NODE = "fs.default.name";
+    public static final String HADOOP_JOB_TRACKER = "mapred.job.tracker";
+    public static final String HADOOP_JOB_TRACKER_2 = "mapreduce.jobtracker.address";
+    public static final String HADOOP_YARN_RM = "yarn.resourcemanager.address";
+    public static final String HADOOP_NAME_NODE = "fs.default.name";
     private static final String HADOOP_JOB_NAME = "mapred.job.name";
     public static final String OOZIE_COMMON_LIBDIR = "oozie";
     public static final int MAX_EXTERNAL_STATS_SIZE_DEFAULT = Integer.MAX_VALUE;
@@ -711,7 +711,7 @@ public class JavaActionExecutor extends 
             if (alreadyRunning && !isUserRetry) {
                 runningJob = jobClient.getJob(JobID.forName(launcherId));
                 if (runningJob == null) {
-                    String jobTracker = launcherJobConf.get("mapred.job.tracker");
+                    String jobTracker = launcherJobConf.get(HADOOP_JOB_TRACKER);
                     throw new ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "JA017",
                             "unknown job [{0}@{1}], cannot recover", launcherId, jobTracker);
                 }
@@ -720,8 +720,8 @@ public class JavaActionExecutor extends 
                 XLog.getLog(getClass()).debug("Submitting the job through Job Client for action " + action.getId());
 
                 // setting up propagation of the delegation token.
-                Token<DelegationTokenIdentifier> mrdt = jobClient.getDelegationToken(new Text("mr token"));
-                launcherJobConf.getCredentials().addToken(new Text("mr token"), mrdt);
+                Token<DelegationTokenIdentifier> mrdt = jobClient.getDelegationToken(new Text(HadoopAccessorService.MR_DELEGATION_TOKEN));
+                launcherJobConf.getCredentials().addToken(new Text(HadoopAccessorService.MR_DELEGATION_TOKEN), mrdt);
 
                 // insert credentials tokens to launcher job conf if needed
                 if (needInjectCredentials()) {

Modified: oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java (original)
+++ oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/command/wf/ActionCheckXCommand.java Thu Nov 29 01:38:13 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/branches/hcat-intre/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java (original)
+++ oozie/branches/hcat-intre/core/src/main/java/org/apache/oozie/service/HadoopAccessorService.java Thu Nov 29 01:38:13 2012
@@ -28,6 +28,7 @@ import org.apache.hadoop.security.UserGr
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.security.token.Token;
 import org.apache.oozie.ErrorCode;
+import org.apache.oozie.action.hadoop.JavaActionExecutor;
 import org.apache.oozie.util.ParamChecker;
 import org.apache.oozie.util.XConfiguration;
 import org.apache.oozie.util.XLog;
@@ -62,6 +63,7 @@ public class HadoopAccessorService imple
     public static final String KERBEROS_AUTH_ENABLED = CONF_PREFIX + "kerberos.enabled";
     public static final String KERBEROS_KEYTAB = CONF_PREFIX + "keytab.file";
     public static final String KERBEROS_PRINCIPAL = CONF_PREFIX + "kerberos.principal";
+    public static final String MR_DELEGATION_TOKEN = "oozie mr token";
 
     private static final String OOZIE_HADOOP_ACCESSOR_SERVICE_CREATED = "oozie.HadoopAccessorService.created";
 
@@ -351,7 +353,7 @@ public class HadoopAccessorService imple
         if (!conf.getBoolean(OOZIE_HADOOP_ACCESSOR_SERVICE_CREATED, false)) {
             throw new HadoopAccessorException(ErrorCode.E0903);
         }
-        String jobTracker = conf.get("mapred.job.tracker");
+        String jobTracker = conf.get(JavaActionExecutor.HADOOP_JOB_TRACKER);
         validateJobTracker(jobTracker);
         try {
             UserGroupInformation ugi = getUGI(user);
@@ -360,8 +362,8 @@ public class HadoopAccessorService imple
                     return new JobClient(conf);
                 }
             });
-            Token<DelegationTokenIdentifier> mrdt = jobClient.getDelegationToken(new Text("mr token"));
-            conf.getCredentials().addToken(new Text("mr token"), mrdt);
+            Token<DelegationTokenIdentifier> mrdt = jobClient.getDelegationToken(new Text(MR_DELEGATION_TOKEN));
+            conf.getCredentials().addToken(new Text(MR_DELEGATION_TOKEN), mrdt);
             return jobClient;
         }
         catch (InterruptedException ex) {

Modified: oozie/branches/hcat-intre/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java (original)
+++ oozie/branches/hcat-intre/core/src/test/java/org/apache/oozie/command/wf/TestActionCheckXCommand.java Thu Nov 29 01:38:13 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/branches/hcat-intre/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/release-log.txt?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/release-log.txt (original)
+++ oozie/branches/hcat-intre/release-log.txt Thu Nov 29 01:38:13 2012
@@ -1,5 +1,9 @@
 -- Oozie 3.4.0 release (trunk - unreleased)
 
+OOZIE-1096 Update wfgen README.txt to have the TLP mailing list (jun aoki via rkanter)
+OOZIE-1015 HadoopAccessorService jobtracker validation should not have hardcoded conf key (mona)
+OOZIE-1078 Help -> Documentation and Help -> Online Help should link to oozie.apache.org/ (jun via mohammad)
+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-1086 Command to check the missing partitions directly against HCatalog server (mohammad)
 OOZIE-1050 Implement logic to update dependencies via push JMS message(mona via mohammad)

Modified: oozie/branches/hcat-intre/workflowgenerator/README.txt
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/workflowgenerator/README.txt?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/workflowgenerator/README.txt (original)
+++ oozie/branches/hcat-intre/workflowgenerator/README.txt Thu Nov 29 01:38:13 2012
@@ -131,8 +131,8 @@ if you want to kill process, Ctrl-C in c
 
 If you have any questions/issues, please send an email to:
 
-oozie-users@incubator.apache.org
+user@oozie.apache.org
 
 Subscribe using the link:
 
-http://incubator.apache.org/oozie/MailingLists.html
+http://oozie.apache.org/mail-lists.html

Modified: oozie/branches/hcat-intre/workflowgenerator/src/main/java/org/apache/oozie/tools/workflowgenerator/client/OozieWorkflowGenerator.java
URL: http://svn.apache.org/viewvc/oozie/branches/hcat-intre/workflowgenerator/src/main/java/org/apache/oozie/tools/workflowgenerator/client/OozieWorkflowGenerator.java?rev=1415025&r1=1415024&r2=1415025&view=diff
==============================================================================
--- oozie/branches/hcat-intre/workflowgenerator/src/main/java/org/apache/oozie/tools/workflowgenerator/client/OozieWorkflowGenerator.java (original)
+++ oozie/branches/hcat-intre/workflowgenerator/src/main/java/org/apache/oozie/tools/workflowgenerator/client/OozieWorkflowGenerator.java Thu Nov 29 01:38:13 2012
@@ -537,8 +537,20 @@ public class OozieWorkflowGenerator impl
         MenuBar helpMenu = new MenuBar(true);
         helpMenu.setAutoOpen(true);
         helpMenu.setAnimationEnabled(true);
-        helpMenu.addItem("Documentation", cmd);
-        helpMenu.addItem("Online Help", cmd);
+
+        // TODO this should point to a workflowgenerator's maven site, however there is no maven site available. (Not even in
+        // Workspace of the jenkins job at https://builds.apache.org/job/oozie-trunk-precommit-build/ws/workflowgenerator/target/)
+        // where client, for example, has target/site/apidocs
+        // The ideal place is somewhere under http://oozie.apache.org/docs/ once it is generated.
+        Command openOozieTopPageComman = new Command() {
+            @Override
+            public void execute() {
+                Window.open("http://oozie.apache.org/", "_blank", "");
+            }
+        };
+        helpMenu.addItem("Documentation", openOozieTopPageComman);
+        helpMenu.addItem("Online Help", openOozieTopPageComman);
+
         helpMenu.addItem("About", cmd);
 
         MenuBar menu = new MenuBar();